From 79430bf10b553f62f13e50a36c67cf9cef915ac0 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Thu, 16 Nov 2017 22:03:32 +0300 Subject: [PATCH 001/509] Fixed SummingMergeTree. [#CLICKHOUSE-2] --- .../SummingSortedBlockInputStream.cpp | 95 +++++++++++-------- .../SummingSortedBlockInputStream.h | 18 ++-- .../00084_summing_merge_tree.reference | 2 + .../0_stateless/00084_summing_merge_tree.sql | 10 ++ 4 files changed, 75 insertions(+), 50 deletions(-) diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index 7d0f94e997e..bc5c5dfc7dc 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -36,7 +36,7 @@ String SummingSortedBlockInputStream::getID() const } -void SummingSortedBlockInputStream::insertCurrentRow(ColumnPlainPtrs & merged_columns) +void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(ColumnPlainPtrs & merged_columns, bool force_insertion) { for (auto & desc : columns_to_aggregate) { @@ -46,6 +46,19 @@ void SummingSortedBlockInputStream::insertCurrentRow(ColumnPlainPtrs & merged_co try { desc.function->insertResultInto(desc.state.data(), *desc.merged_column); + + /// Update zero status of current row + if (desc.column_numbers.size() == 1) + { + // Flag row as non-empty if at least one column number if non-zero + current_row_is_zero = current_row_is_zero && desc.merged_column->get64(desc.merged_column->size() - 1) == 0; + } + else + { + /// It is sumMap aggregate function. + /// Assume that the row isn't empty in this case (just because it is compatible with previous version) + current_row_is_zero = false; + } } catch (...) { @@ -60,8 +73,22 @@ void SummingSortedBlockInputStream::insertCurrentRow(ColumnPlainPtrs & merged_co desc.merged_column->insertDefault(); } + /// If it is "zero" row and it is not the last row of the result block, then + /// rollback the insertion (at this moment we need rollback only cols from columns_to_aggregate) + if (!force_insertion && current_row_is_zero) + { + for (auto & desc : columns_to_aggregate) + desc.merged_column->popBack(1); + + return; + } + for (auto i : column_numbers_not_to_aggregate) merged_columns[i]->insert(current_row[i]); + + /// Update per-block and per-group flags + ++merged_rows; + output_is_non_empty = true; } @@ -155,6 +182,7 @@ Block SummingSortedBlockInputStream::readImpl() desc.column_numbers = {i}; desc.function = factory.get("sumWithOverflow", {column.type}); desc.function->setArguments({column.type}); + desc.add_function = desc.function->getAddressOfAddFunction(); desc.state.resize(desc.function->sizeOfData()); columns_to_aggregate.emplace_back(std::move(desc)); } @@ -237,6 +265,7 @@ Block SummingSortedBlockInputStream::readImpl() // Create summation for all value columns in the map desc.function = factory.get("sumMap", argument_types); desc.function->setArguments(argument_types); + desc.add_function = desc.function->getAddressOfAddFunction(); desc.state.resize(desc.function->sizeOfData()); columns_to_aggregate.emplace_back(std::move(desc)); } @@ -279,7 +308,7 @@ Block SummingSortedBlockInputStream::readImpl() template void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std::priority_queue & queue) { - size_t merged_rows = 0; + merged_rows = 0; /// Take the rows in needed order and put them in `merged_block` until rows no more than `max_block_size` while (!queue.empty()) @@ -308,12 +337,7 @@ void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std: if (key_differs) { /// Write the data for the previous group. - if (!current_row_is_zero) - { - ++merged_rows; - output_is_non_empty = true; - insertCurrentRow(merged_columns); - } + insertCurrentRowIfNeeded(merged_columns, false); current_key.swap(next_key); @@ -327,11 +351,12 @@ void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std: } // Start aggregations with current row - current_row_is_zero = !addRow(current_row, current); + addRow(current_row, current); + current_row_is_zero = true; } else { - current_row_is_zero = !addRow(current_row, current); + addRow(current_row, current); // Merge maps only for same rows for (auto & desc : maps_to_sum) @@ -355,12 +380,7 @@ void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std: /// We will write the data for the last group, if it is non-zero. /// If it is zero, and without it the output stream will be empty, we will write it anyway. - if (!current_row_is_zero || !output_is_non_empty) - { - ++merged_rows; /// Dead store (result is unused). Left for clarity. - insertCurrentRow(merged_columns); - } - + insertCurrentRowIfNeeded(merged_columns, !output_is_non_empty); finished = true; } @@ -449,38 +469,29 @@ bool SummingSortedBlockInputStream::mergeMap(const MapDescription & desc, Row & template -bool SummingSortedBlockInputStream::addRow(Row & row, TSortCursor & cursor) +void SummingSortedBlockInputStream::addRow(Row & row, TSortCursor & cursor) { - bool res = false; for (auto & desc : columns_to_aggregate) { - if (desc.created) - { - // Specialized case for unary functions - if (desc.column_numbers.size() == 1) - { - auto & col = cursor->all_columns[desc.column_numbers[0]]; - desc.function->add(desc.state.data(), &col, cursor->pos, nullptr); - // Flag row as non-empty if at least one column number if non-zero - // Note: This defers compaction of signed type rows that sum to zero by one merge - if (!res) - res = col->get64(cursor->pos) != 0; - } - else - { - // Gather all source columns into a vector - ConstColumnPlainPtrs columns(desc.column_numbers.size()); - for (size_t i = 0; i < desc.column_numbers.size(); ++i) - columns[i] = cursor->all_columns[desc.column_numbers[i]]; + if (!desc.created) + throw Exception("Logical error in SummingSortedBlockInputStream, there are no description", ErrorCodes::LOGICAL_ERROR); - desc.function->add(desc.state.data(), columns.data(), cursor->pos, nullptr); - // Note: we can't detect whether the aggregation result is non-empty here yet - res = true; - } + // Specialized case for unary functions + if (desc.column_numbers.size() == 1) + { + auto & col = cursor->all_columns[desc.column_numbers[0]]; + desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, nullptr); + } + else + { + // Gather all source columns into a vector + ConstColumnPlainPtrs columns(desc.column_numbers.size()); + for (size_t i = 0; i < desc.column_numbers.size(); ++i) + columns[i] = cursor->all_columns[desc.column_numbers[i]]; + + desc.add_function(desc.function.get(),desc.state.data(), columns.data(), cursor->pos, nullptr); } } - - return res; } } diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.h b/dbms/src/DataStreams/SummingSortedBlockInputStream.h index f21d79d01a6..493e32d5326 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.h @@ -73,6 +73,7 @@ private: struct AggregateDescription { AggregateFunctionPtr function; + IAggregateFunction::AddFunc add_function = nullptr; std::vector column_numbers; ColumnPtr merged_column; std::vector state; @@ -100,9 +101,10 @@ private: RowRef next_key; /// The primary key of the next row. Row current_row; - bool current_row_is_zero = true; /// The current row is summed to zero, and it should be deleted. + bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. - bool output_is_non_empty = false; /// Have we given out at least one row as a result. + bool output_is_non_empty = false; /// Have we given out at least one row as a result. + size_t merged_rows = 0; /// Number of rows merged into current result block /** We support two different cursors - with Collation and without. * Templates are used instead of polymorphic SortCursor and calls to virtual functions. @@ -110,17 +112,17 @@ private: template void merge(ColumnPlainPtrs & merged_columns, std::priority_queue & queue); - /// Insert the summed row for the current group into the result. - void insertCurrentRow(ColumnPlainPtrs & merged_columns); + /// Insert the summed row for the current group into the result and updates some of per-block flags if the row is not "zero". + /// If force_insertion=true, then the row will be inserted even if it is "zero" + void insertCurrentRowIfNeeded(ColumnPlainPtrs & merged_columns, bool force_insertion); + /// Returns true is merge result is not empty template bool mergeMap(const MapDescription & map, Row & row, TSortCursor & cursor); - /** Add the row under the cursor to the `row`. - * Returns false if the result is zero. - */ + // Add the row under the cursor to the `row`. template - bool addRow(Row & row, TSortCursor & cursor); + void addRow(Row & row, TSortCursor & cursor); }; } diff --git a/dbms/tests/queries/0_stateless/00084_summing_merge_tree.reference b/dbms/tests/queries/0_stateless/00084_summing_merge_tree.reference index 7142e1ca063..72f24941378 100644 --- a/dbms/tests/queries/0_stateless/00084_summing_merge_tree.reference +++ b/dbms/tests/queries/0_stateless/00084_summing_merge_tree.reference @@ -2,3 +2,5 @@ 2000-01-01 Hello 5 7 9 2000-01-01 Goodbye 1 2 3 2000-01-01 Hello 1 7 9 +0 2 +666 1 diff --git a/dbms/tests/queries/0_stateless/00084_summing_merge_tree.sql b/dbms/tests/queries/0_stateless/00084_summing_merge_tree.sql index 82b5117625f..c03092d0277 100644 --- a/dbms/tests/queries/0_stateless/00084_summing_merge_tree.sql +++ b/dbms/tests/queries/0_stateless/00084_summing_merge_tree.sql @@ -29,3 +29,13 @@ SELECT * FROM test.summing_merge_tree ORDER BY d, a, x, y, z; DROP TABLE test.summing_merge_tree; + +-- +DROP TABLE IF EXISTS test.summing; +CREATE TABLE test.summing (p Date, k UInt64, s UInt64) ENGINE = SummingMergeTree(p, k, 1); + +INSERT INTO test.summing (k, s) VALUES (0, 1); +INSERT INTO test.summing (k, s) VALUES (0, 1), (666, 1), (666, 0); +OPTIMIZE TABLE test.summing PARTITION 197001; + +SELECT k, s FROM test.summing ORDER BY k; From 1b50717c7543ce2fef5273491c392ab6ef07195d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Nov 2017 23:29:30 +0300 Subject: [PATCH 002/509] Update SummingSortedBlockInputStream.h --- dbms/src/DataStreams/SummingSortedBlockInputStream.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.h b/dbms/src/DataStreams/SummingSortedBlockInputStream.h index 493e32d5326..4e12bf45827 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.h @@ -116,7 +116,7 @@ private: /// If force_insertion=true, then the row will be inserted even if it is "zero" void insertCurrentRowIfNeeded(ColumnPlainPtrs & merged_columns, bool force_insertion); - /// Returns true is merge result is not empty + /// Returns true if merge result is not empty template bool mergeMap(const MapDescription & map, Row & row, TSortCursor & cursor); From 40b5fb292e29d38de7beb2af3657632d6ef9d560 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Nov 2017 23:37:45 +0300 Subject: [PATCH 003/509] Update SummingSortedBlockInputStream.h --- dbms/src/DataStreams/SummingSortedBlockInputStream.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.h b/dbms/src/DataStreams/SummingSortedBlockInputStream.h index 4e12bf45827..e4207daf88d 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.h @@ -45,8 +45,8 @@ private: /// Read up to the end. bool finished = false; - /// Columns with which numbers should be summed. - Names column_names_to_sum; /// If set, it is converted to column_numbers_to_sum when initialized. + /// Columns with which values should be summed. + Names column_names_to_sum; /// If set, it is converted to column_numbers_to_aggregate when initialized. ColumnNumbers column_numbers_not_to_aggregate; /** A table can have nested tables that are treated in a special way. From 9229961721a2bcb521617577f02f9aca2422239c Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Mon, 20 Nov 2017 22:33:12 +0300 Subject: [PATCH 004/509] Add multi index for data_parts storage. And fixed bugs. [#CLICKHOUSE-3452] Fixed handling of obsolete parts. Fixed conflict resolution between simultaneous PreCommitted covering parts. Fixed memory leak caused by ordinary MergeTree parts stucked in Deleting state. Added hidden _state column into system.parts. --- contrib/poco | 2 +- dbms/src/Core/ErrorCodes.cpp | 1 + dbms/src/Storages/MergeTree/MergeTreeData.cpp | 483 +++++++++++------- dbms/src/Storages/MergeTree/MergeTreeData.h | 149 +++++- .../MergeTree/MergeTreeDataMerger.cpp | 2 +- .../Storages/MergeTree/MergeTreeDataPart.cpp | 13 + .../Storages/MergeTree/MergeTreeDataPart.h | 15 +- dbms/src/Storages/StorageMergeTree.cpp | 6 +- .../Storages/StorageReplicatedMergeTree.cpp | 49 +- .../Storages/System/StorageSystemParts.cpp | 40 +- dbms/src/Storages/System/StorageSystemParts.h | 4 + .../Storages/tests/gtest_range_filtered.cpp | 44 -- .../integration/test_random_inserts/test.py | 5 +- .../integration/test_random_inserts/test.sh | 5 +- libs/libcommon/include/common/RangeFiltered.h | 127 ----- 15 files changed, 524 insertions(+), 421 deletions(-) delete mode 100644 dbms/src/Storages/tests/gtest_range_filtered.cpp delete mode 100644 libs/libcommon/include/common/RangeFiltered.h diff --git a/contrib/poco b/contrib/poco index 1366df1c7e0..bcf9ebad48b 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 1366df1c7e068bb2efd846bc8dc8e286b090904e +Subproject commit bcf9ebad48b2162d25f5fc432b176d74a09f498d diff --git a/dbms/src/Core/ErrorCodes.cpp b/dbms/src/Core/ErrorCodes.cpp index ffeda42047b..bd4914ea08b 100644 --- a/dbms/src/Core/ErrorCodes.cpp +++ b/dbms/src/Core/ErrorCodes.cpp @@ -386,6 +386,7 @@ namespace ErrorCodes extern const int HTTP_LENGTH_REQUIRED = 381; extern const int CANNOT_LOAD_CATBOOST_MODEL = 382; extern const int CANNOT_APPLY_CATBOOST_MODEL = 383; + extern const int PART_IS_TEMPORARILY_LOCKED = 384; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 8e7146e4bfa..0a63dc2348b 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -74,6 +74,7 @@ namespace ErrorCodes extern const int CORRUPTED_DATA; extern const int INVALID_PARTITION_VALUE; extern const int METADATA_MISMATCH; + extern const int PART_IS_TEMPORARILY_LOCKED; } @@ -106,7 +107,9 @@ MergeTreeData::MergeTreeData( database_name(database_), table_name(table_), full_path(full_path_), columns(columns_), broken_part_callback(broken_part_callback_), - log_name(log_name_), log(&Logger::get(log_name + " (Data)")) + log_name(log_name_), log(&Logger::get(log_name + " (Data)")), + data_parts_by_name(data_parts_indexes.get()), + data_parts_by_state_and_name(data_parts_indexes.get()) { merging_params.check(*columns); @@ -381,7 +384,7 @@ Int64 MergeTreeData::getMaxDataPartIndex() std::lock_guard lock_all(data_parts_mutex); Int64 max_block_id = 0; - for (const auto & part : data_parts) + for (const DataPartPtr & part : data_parts_by_name) max_block_id = std::max(max_block_id, part->info.max_block); return max_block_id; @@ -392,9 +395,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) { LOG_DEBUG(log, "Loading data parts"); - std::lock_guard lock(data_parts_mutex); - data_parts.clear(); - Strings part_file_names; Poco::DirectoryIterator end; for (Poco::DirectoryIterator it(full_path); it != end; ++it) @@ -410,6 +410,9 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) DataPartsVector broken_parts_to_detach; size_t suspicious_broken_parts = 0; + std::lock_guard lock(data_parts_mutex); + data_parts_indexes.clear(); + for (const String & file_name : part_file_names) { MergeTreePartInfo part_info; @@ -496,7 +499,8 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later part->state = DataPartState::Committed; - data_parts.insert(part); + if (!data_parts_indexes.insert(part).second) + throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART); } if (suspicious_broken_parts > settings.max_suspicious_broken_parts && !skip_sanity_checks) @@ -512,13 +516,21 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) /// were merged), but that for some reason are still not deleted from the filesystem. /// Deletion of files will be performed later in the clearOldParts() method. - if (data_parts.size() >= 2) + if (data_parts_indexes.size() >= 2) { - auto committed_parts = getDataPartsRange({DataPartState::Committed}); - auto prev_jt = committed_parts.begin(); + /// Now all parts are committed, so data_parts_by_state_and_name == committed_parts_range + auto prev_jt = data_parts_by_state_and_name.begin(); auto curr_jt = std::next(prev_jt); - while (curr_jt != committed_parts.end()) + auto deactivate_part = [&] (DataPartIteratorByStateAndName it) + { + (*it)->remove_time = (*it)->modification_time; + modifyPartState(it, DataPartState::Outdated); + }; + + (*prev_jt)->assertState({DataPartState::Committed}); + + while (curr_jt != data_parts_by_state_and_name.end() && (*curr_jt)->state == DataPartState::Committed) { /// Don't consider data parts belonging to different partitions. if ((*curr_jt)->info.partition_id != (*prev_jt)->info.partition_id) @@ -530,16 +542,15 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) if ((*curr_jt)->contains(**prev_jt)) { - (*prev_jt)->remove_time = (*prev_jt)->modification_time; - (*prev_jt)->state = DataPartState::Outdated; /// prev_jt becomes invalid here + deactivate_part(prev_jt); prev_jt = curr_jt; ++curr_jt; } else if ((*prev_jt)->contains(**curr_jt)) { - (*curr_jt)->remove_time = (*curr_jt)->modification_time; - (*curr_jt)->state = DataPartState::Outdated; /// curr_jt becomes invalid here - ++curr_jt; + auto next = std::next(curr_jt); + deactivate_part(curr_jt); + curr_jt = next; } else { @@ -551,7 +562,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) calculateColumnSizesImpl(); - LOG_DEBUG(log, "Loaded data parts (" << data_parts.size() << " items)"); + LOG_DEBUG(log, "Loaded data parts (" << data_parts_indexes.size() << " items)"); } @@ -619,21 +630,30 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts() return res; time_t now = time(nullptr); + std::vector parts_to_delete; { std::lock_guard lock_parts(data_parts_mutex); - for (auto it = data_parts.begin(); it != data_parts.end(); ++it) + auto outdated_parts_range = getDataPartsStateRange(DataPartState::Outdated); + for (auto it = outdated_parts_range.begin(); it != outdated_parts_range.end(); ++it) { - if ((*it)->state == DataPartState::Outdated && - it->unique() && /// Grab only parts that is not using by anyone (SELECTs for example) - (*it)->remove_time < now && - now - (*it)->remove_time > settings.old_parts_lifetime.totalSeconds()) + const DataPartPtr & part = *it; + + if (part.unique() && /// Grab only parts that is not using by anyone (SELECTs for example) + part->remove_time < now && + now - part->remove_time > settings.old_parts_lifetime.totalSeconds()) { - (*it)->state = DataPartState::Deleting; - res.push_back(*it); + parts_to_delete.emplace_back(it); } } + + res.reserve(parts_to_delete.size()); + for (const auto & it_to_delete : parts_to_delete) + { + res.emplace_back(*it_to_delete); + modifyPartState(it_to_delete, DataPartState::Deleting); + } } if (!res.empty()) @@ -650,7 +670,7 @@ void MergeTreeData::rollbackDeletingParts(const MergeTreeData::DataPartsVector & { /// We should modify it under data_parts_mutex part->assertState({DataPartState::Deleting}); - part->state = DataPartState::Outdated; + modifyPartState(part, DataPartState::Outdated); } } @@ -661,26 +681,27 @@ void MergeTreeData::removePartsFinally(const MergeTreeData::DataPartsVector & pa /// TODO: use data_parts iterators instead of pointers for (auto & part : parts) { - if (part->state != DataPartState::Deleting) - throw Exception("An attempt to delete part " + part->getNameWithState() + " with unexpected state", ErrorCodes::LOGICAL_ERROR); - - auto it = data_parts.find(part); - if (it == data_parts.end()) + auto it = data_parts_by_name.find(part->info); + if (it == data_parts_by_name.end()) throw Exception("Deleting data part " + part->name + " is not exist", ErrorCodes::LOGICAL_ERROR); - data_parts.erase(it); + (*it)->assertState({DataPartState::Deleting}); + + data_parts_indexes.erase(it); } } -void MergeTreeData::clearOldParts() +void MergeTreeData::clearOldPartsFromFilesystem() { auto parts_to_remove = grabOldParts(); for (const DataPartPtr & part : parts_to_remove) { - LOG_DEBUG(log, "Removing part " << part->name); + LOG_DEBUG(log, "Removing part from filesystem " << part->name); part->remove(); } + + removePartsFinally(parts_to_remove); } void MergeTreeData::setPath(const String & new_full_path, bool move_data) @@ -710,7 +731,7 @@ void MergeTreeData::dropAllData() LOG_TRACE(log, "dropAllData: removing data from memory."); - data_parts.clear(); + data_parts_indexes.clear(); column_sizes.clear(); context.dropCaches(); @@ -1319,9 +1340,13 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace( part->assertState({DataPartState::Temporary}); - DataPartsVector replaced; + MergeTreePartInfo part_info = part->info; + String part_name; + + DataPartsVector replaced_parts; + std::vector replaced_iterators; { - std::lock_guard lock(data_parts_mutex); + std::unique_lock lock(data_parts_mutex); if (DataPartPtr existing_part_in_partition = getAnyPartInPartition(part->info.partition_id, lock)) { @@ -1336,141 +1361,163 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace( * Otherwise there is race condition - merge of blocks could happen in interval that doesn't yet contain new part. */ if (increment) - part->info.min_block = part->info.max_block = increment->get(); + part_info.min_block = part_info.max_block = increment->get(); - String new_name; if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) - new_name = part->info.getPartNameV0(part->getMinDate(), part->getMaxDate()); + part_name = part_info.getPartNameV0(part->getMinDate(), part->getMaxDate()); else - new_name = part->info.getPartName(); + part_name = part_info.getPartName(); - LOG_TRACE(log, "Renaming temporary part " << part->relative_path << " to " << new_name << "."); + LOG_TRACE(log, "Renaming temporary part " << part->relative_path << " to " << part_name << "."); - auto it_duplicate = data_parts.find(part); - if (it_duplicate != data_parts.end()) + auto it_duplicate = data_parts_by_name.find(part_info); + if (it_duplicate != data_parts_by_name.end()) { String message = "Part " + (*it_duplicate)->getNameWithState() + " already exists"; + if ((*it_duplicate)->checkState({DataPartState::Outdated, DataPartState::Deleting})) - message += ", but it will be deleted soon"; + { + throw Exception(message + ", but it will be deleted soon", ErrorCodes::PART_IS_TEMPORARILY_LOCKED); + } throw Exception(message, ErrorCodes::DUPLICATE_DATA_PART); } - /// Rename the part only in memory. Will rename it on disk only if all check is passed. - /// It allows us maintain invariant: if non-temporary parts in filesystem then they are in data_parts - part->name = new_name; + /// Check that part is not covered and doesn't cover other in-progress parts, it makes sense only for Replicated* engines + if (out_transaction) + { + auto check_coverage = [&part_info, &part_name] (const DataPartPtr & part) + { + if (part_info.contains(part->info)) + { + throw Exception("Cannot add part " + part_name + " covering pre-committed part " + part->name, ErrorCodes::PART_IS_TEMPORARILY_LOCKED); + } + else + { + if (part->info.contains(part_info)) + throw Exception("Cannot add part " + part_name + " covered by pre-committed part " + part->name + ". It is a bug", ErrorCodes::LOGICAL_ERROR); + } + }; + + auto it_middle = data_parts_by_state_and_name.lower_bound(DataPartStateAndInfo(DataPartState::PreCommitted, part_info)); + + auto precommitted_parts_range = getDataPartsStateRange(DataPartState::PreCommitted); + + for (auto it = it_middle; it != precommitted_parts_range.begin();) + { + --it; + check_coverage(*it); + } + + for (auto it = it_middle; it != precommitted_parts_range.end();) + { + check_coverage(*it); + ++it; + } + } /// Is the part covered by some other part? - bool obsolete = false; + DataPartPtr covering_part; - auto check_replacing_part_state = [&] (const DataPartPtr & cur_part) - { - cur_part->assertState({DataPartState::PreCommitted, DataPartState::Committed}); - if (cur_part->state == DataPartState::PreCommitted) - throw Exception("Could not add part " + new_name + " while replacing part " + cur_part->name + " is in pre-committed state", ErrorCodes::LOGICAL_ERROR); - }; + auto it_middle = data_parts_by_state_and_name.lower_bound(DataPartStateAndInfo(DataPartState::Committed, part_info)); - /// Don't consider parts going to be deleted - auto active_parts = getDataPartsRange({DataPartState::Committed, DataPartState::PreCommitted}); /// Parts contained in the part are consecutive in data_parts, intersecting the insertion place for the part itself. - auto it_middle = active_parts.convert(data_parts.lower_bound(part)); + auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed); /// Go to the left. - for (auto it = it_middle; it != active_parts.begin();) + for (auto it = it_middle; it != committed_parts_range.begin();) { --it; - if (!part->contains(**it)) + if (!part_info.contains((*it)->info)) { - if ((*it)->contains(*part)) - obsolete = true; - ++it; + if ((*it)->info.contains(part_info)) + covering_part = *it; break; } - check_replacing_part_state(*it); - replaced.push_back(*it); -// replaced.push_back(*it); -// (*it)->remove_time = time(nullptr); -// (*it)->state = replaced_parts_state; -// removePartContributionToColumnSizes(*it); -// data_parts.erase(it++); /// Yes, ++, not --. + replaced_iterators.push_back(it); } /// Parts must be in ascending order. - std::reverse(replaced.begin(), replaced.end()); + std::reverse(replaced_iterators.begin(), replaced_iterators.end()); /// Go to the right. - for (auto it = it_middle; it != active_parts.end();) + for (auto it = it_middle; it != committed_parts_range.end();) { - if ((*it)->name == part->name) - throw Exception("Unexpected duplicate part " + part->getNameWithState() + ". It is a bug.", ErrorCodes::LOGICAL_ERROR); + if ((*it)->name == part_name) + throw Exception("Unexpected duplicate part " + (*it)->getNameWithState() + ". It is a bug.", ErrorCodes::LOGICAL_ERROR); - if (!part->contains(**it)) + if (!part_info.contains((*it)->info)) { - if ((*it)->contains(*part)) - obsolete = true; + if ((*it)->info.contains(part_info)) + covering_part = *it; break; } - check_replacing_part_state(*it); - replaced.push_back(*it); + replaced_iterators.push_back(it); ++it; -// replaced.push_back(*it); -// (*it)->remove_time = time(nullptr); -// (*it)->state = replaced_parts_state; -// removePartContributionToColumnSizes(*it); -// data_parts.erase(it++); } - if (obsolete) + if (covering_part) { - LOG_WARNING(log, "Obsolete part " << part->name << " added"); + LOG_WARNING(log, "Tried to add obsolete part " << part_name << " covered by " << covering_part->getNameWithState()); + + /// It is a temporary part, we want to delete it from filesystem immediately + /// Other fields remain the same part->remove_time = time(nullptr); - /// I case of fail, we want to delete part from filesystem immediately (to avoid any conflicts) part->is_temp = true; + + /// Nothing to commit or rollback + if (out_transaction) + { + out_transaction->data = this; + out_transaction->parts_to_add_on_rollback = {}; + out_transaction->parts_to_remove_on_rollback = {}; + } + + /// We replaced nothing + return {}; + } + + /// All checks are passed. Now we can rename the part on disk. + /// So, we maintain invariant: if a non-temporary part in filesystem then it is in data_parts + /// + /// Ordinary MergeTree engines (they don't use out_transaction) commit parts immediately, + /// whereas ReplicatedMergeTree uses intermediate PreCommitted state + part->name = part_name; + part->info = part_info; + part->is_temp = false; + part->state = (out_transaction) ? DataPartState::PreCommitted : DataPartState::Committed; + part->renameTo(part_name); + + data_parts_indexes.insert(part); + + replaced_parts.reserve(replaced_iterators.size()); + for (auto it_replacing_part : replaced_iterators) + replaced_parts.emplace_back(*it_replacing_part); + + if (!out_transaction) + { + addPartContributionToColumnSizes(part); + + auto current_time = time(nullptr); + for (auto it_replacing_part : replaced_iterators) + { + (*it_replacing_part)->remove_time = current_time; + modifyPartState(it_replacing_part, DataPartState::Outdated); + removePartContributionToColumnSizes(*it_replacing_part); + } } else { - /// Now we can rename part on filesystem - part->is_temp = false; - part->renameTo(new_name); - - if (!out_transaction) - { - /// Ordinary MergeTree engines (they don't use out_transaction) commit parts immediately - part->state = DataPartState::Committed; - addPartContributionToColumnSizes(part); - } - else - { - /// Whereas ReplicatedMergeTree uses intermediate PreCommitted state - part->state = DataPartState::PreCommitted; - } - - data_parts.insert(part); - - auto current_time = time(nullptr); - for (auto & replacing_part : replaced) - { - if (!out_transaction) - { - replacing_part->remove_time = current_time; - replacing_part->state = DataPartState::Outdated; - removePartContributionToColumnSizes(replacing_part); - } - } + out_transaction->data = this; + out_transaction->parts_to_add_on_rollback = replaced_parts; + out_transaction->parts_to_remove_on_rollback = {part}; } } - if (out_transaction) - { - out_transaction->data = this; - out_transaction->parts_to_add_on_rollback = replaced; - out_transaction->parts_to_remove_on_rollback = {part}; - } - - return replaced; + return replaced_parts; } void MergeTreeData::removePartsFromWorkingSet(const DataPartsVector & remove, bool clear_without_timeout) @@ -1479,7 +1526,7 @@ void MergeTreeData::removePartsFromWorkingSet(const DataPartsVector & remove, bo for (auto & part : remove) { - if (!data_parts.count(part)) + if (!data_parts_by_name.count(part->info)) throw Exception("Part " + part->getNameWithState() + " not found in data_parts", ErrorCodes::LOGICAL_ERROR); part->assertState({DataPartState::PreCommitted, DataPartState::Committed, DataPartState::Outdated}); @@ -1490,7 +1537,8 @@ void MergeTreeData::removePartsFromWorkingSet(const DataPartsVector & remove, bo { if (part->state == DataPartState::Committed) removePartContributionToColumnSizes(part); - part->state = DataPartState::Outdated; + + modifyPartState(part, DataPartState::Outdated); part->remove_time = remove_time; } } @@ -1502,65 +1550,93 @@ void MergeTreeData::renameAndDetachPart(const DataPartPtr & part_to_detach, cons LOG_INFO(log, "Renaming " << part_to_detach->relative_path << " to " << prefix << part_to_detach->name << " and detaching it."); std::lock_guard lock(data_parts_mutex); - //std::lock_guard lock_all(all_data_parts_mutex); - auto it_part = data_parts.find(part_to_detach); - if (it_part == data_parts.end()) + auto it_part = data_parts_by_name.find(part_to_detach->info); + if (it_part == data_parts_by_name.end()) throw Exception("No such data part " + part_to_detach->getNameWithState(), ErrorCodes::NO_SUCH_DATA_PART); /// What if part_to_detach is reference to *it_part? Make a new owner just in case. - auto part = *it_part; + DataPartPtr part = *it_part; - removePartContributionToColumnSizes(part); - part->state = DataPartState::Deleting; + if (part->state == DataPartState::Committed) + removePartContributionToColumnSizes(part); + modifyPartState(it_part, DataPartState::Deleting); if (move_to_detached || !prefix.empty()) part->renameAddPrefix(move_to_detached, prefix); + data_parts_indexes.erase(it_part); + + if (restore_covered && part->info.level == 0) + { + LOG_WARNING(log, "Will not recover parts covered by zero-level part " << part->name); + return; + } if (restore_covered) { - auto suitable_parts = getDataPartsRange({DataPartState::PreCommitted, DataPartState::Committed, DataPartState::Outdated}); - auto it = suitable_parts.convert(data_parts.lower_bound(part)); - Strings restored; bool error = false; + String error_parts; Int64 pos = part->info.min_block; - if (it != suitable_parts.begin()) + auto is_appropriate_state = [] (DataPartState state) { - --it; - if (part->contains(**it)) + return state == DataPartState::Committed || state == DataPartState::Outdated; + }; + + auto update_error = [&] (DataPartIteratorByAndName it) + { + error = true; + error_parts += (*it)->getNameWithState() + " "; + }; + + auto it_middle = data_parts_by_name.lower_bound(part->info); + + /// Restore the leftmost part covered by the part + if (it_middle != data_parts_by_name.begin()) + { + auto it = std::prev(it_middle); + + if (part->contains(**it) && is_appropriate_state((*it)->state)) { + /// Maybe, we must consider part level somehow if ((*it)->info.min_block != part->info.min_block) - error = true; + update_error(it); if ((*it)->state != DataPartState::Committed) { addPartContributionToColumnSizes(*it); - (*it)->state = DataPartState::Committed; + modifyPartState(it, DataPartState::Committed); // iterator is not invalidated here } pos = (*it)->info.max_block + 1; restored.push_back((*it)->name); } else - error = true; - ++it; + update_error(it); } else error = true; - for (; it != suitable_parts.end() && part->contains(**it); ++it) + /// Restore "right" parts + for (auto it = it_middle; it != data_parts_by_name.end() && part->contains(**it); ++it) { if ((*it)->info.min_block < pos) continue; + + if (!is_appropriate_state((*it)->state)) + { + update_error(it); + continue; + } + if ((*it)->info.min_block > pos) - error = true; + update_error(it); if ((*it)->state != DataPartState::Committed) { addPartContributionToColumnSizes(*it); - (*it)->state = DataPartState::Committed; + modifyPartState(it, DataPartState::Committed); } pos = (*it)->info.max_block + 1; @@ -1576,18 +1652,24 @@ void MergeTreeData::renameAndDetachPart(const DataPartPtr & part_to_detach, cons } if (error) - LOG_ERROR(log, "The set of parts restored in place of " << part->name << " looks incomplete. There might or might not be a data loss."); + { + LOG_ERROR(log, "The set of parts restored in place of " << part->name << " looks incomplete." + << " There might or might not be a data loss." + << (error_parts.empty() ? "" : " Suspicious parts: " + error_parts)); + } } } size_t MergeTreeData::getTotalActiveSizeInBytes() const { - std::lock_guard lock(data_parts_mutex); - size_t res = 0; - for (auto & part : getDataPartsRange({DataPartState::Committed})) - res += part->size_in_bytes; + { + std::lock_guard lock(data_parts_mutex); + + for (auto & part : getDataPartsStateRange(DataPartState::Committed)) + res += part->size_in_bytes; + } return res; } @@ -1601,7 +1683,7 @@ size_t MergeTreeData::getMaxPartsCountForPartition() const size_t cur_count = 0; const String * cur_partition_id = nullptr; - for (const auto & part : getDataPartsRange({DataPartState::Committed})) + for (const auto & part : getDataPartsStateRange(DataPartState::Committed)) { if (cur_partition_id && part->info.partition_id == *cur_partition_id) { @@ -1656,11 +1738,12 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String & std::lock_guard lock(data_parts_mutex); - /// The part can be covered only by the previous or the next one in data_parts. - auto committed_parts = getDataPartsRange({DataPartState::Committed}); - auto it = committed_parts.convert(data_parts.lower_bound(part_info)); + auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed); - if (it != committed_parts.end()) + /// The part can be covered only by the previous or the next one in data_parts. + auto it = data_parts_by_state_and_name.lower_bound(DataPartStateAndInfo(DataPartState::Committed, part_info)); + + if (it != committed_parts_range.end()) { if ((*it)->name == part_name) return *it; @@ -1668,7 +1751,7 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String & return *it; } - if (it != committed_parts.begin()) + if (it != committed_parts_range.begin()) { --it; if ((*it)->info.contains(part_info)) @@ -1685,10 +1768,15 @@ MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const String & part_na std::lock_guard lock(data_parts_mutex); - auto filtered_parts = getDataPartsRange(valid_states); - auto it = filtered_parts.convert(data_parts.find(part_info)); - if (it != filtered_parts.end() && (*it)->name == part_name) - return *it; + auto it = data_parts_by_name.find(part_info); + if (it == data_parts_by_name.end()) + return nullptr; + + for (auto state : valid_states) + { + if ((*it)->state == state) + return *it; + } return nullptr; } @@ -1742,7 +1830,8 @@ void MergeTreeData::calculateColumnSizesImpl() column_sizes.clear(); /// Take into account only committed parts - for (const auto & part : getDataPartsRange({DataPartState::Committed})) + auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed); + for (const auto & part : committed_parts_range) addPartContributionToColumnSizes(part); } @@ -1953,7 +2042,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context String partition_id = partition.getID(*this); { - std::lock_guard data_parts_lock(data_parts_mutex); + std::unique_lock data_parts_lock(data_parts_mutex); DataPartPtr existing_part_in_partition = getAnyPartInPartition(partition_id, data_parts_lock); if (existing_part_in_partition && existing_part_in_partition->partition.value != partition.value) { @@ -1969,28 +2058,48 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context return partition_id; } -MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector(const DataPartStates & affordable_states) const +MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector(const DataPartStates & affordable_states, DataPartStateVector * out_states) const { DataPartsVector res; + DataPartsVector buf; { std::lock_guard lock(data_parts_mutex); - std::copy_if(data_parts.begin(), data_parts.end(), std::back_inserter(res), DataPart::getStatesFilter(affordable_states)); + + for (auto state : affordable_states) + { + buf = std::move(res); + res.clear(); + + auto range = getDataPartsStateRange(state); + std::merge(range.begin(), range.end(), buf.begin(), buf.end(), std::back_inserter(res), LessDataPart()); + } + + if (out_states != nullptr) + { + out_states->resize(res.size()); + for (size_t i = 0; i < res.size(); ++i) + (*out_states)[i] = res[i]->state; + } } + return res; } -MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector(const MergeTreeData::DataPartStates & affordable_states, - MergeTreeData::DataPartStateVector & out_states_snapshot) const +MergeTreeData::DataPartsVector MergeTreeData::getAllDataPartsVector(MergeTreeData::DataPartStateVector * out_states) const { DataPartsVector res; { std::lock_guard lock(data_parts_mutex); - std::copy_if(data_parts.begin(), data_parts.end(), std::back_inserter(res), DataPart::getStatesFilter(affordable_states)); + res.assign(data_parts_by_name.begin(), data_parts_by_name.end()); - out_states_snapshot.resize(res.size()); - for (size_t i = 0; i < res.size(); ++i) - out_states_snapshot[i] = res[i]->state; + if (out_states != nullptr) + { + out_states->resize(res.size()); + for (size_t i = 0; i < res.size(); ++i) + (*out_states)[i] = res[i]->state; + } } + return res; } @@ -1999,7 +2108,11 @@ MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affo DataParts res; { std::lock_guard lock(data_parts_mutex); - std::copy_if(data_parts.begin(), data_parts.end(), std::inserter(res, res.end()), DataPart::getStatesFilter(affordable_states)); + for (auto state : affordable_states) + { + auto range = getDataPartsStateRange(state); + res.insert(range.begin(), range.end()); + } } return res; } @@ -2014,28 +2127,23 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector() const return getDataPartsVector({DataPartState::Committed}); } -MergeTreeData::DataParts MergeTreeData::getAllDataParts() const -{ - return getDataParts({DataPartState::PreCommitted, DataPartState::Committed, DataPartState::Outdated}); -} - MergeTreeData::DataPartPtr MergeTreeData::getAnyPartInPartition( - const String & partition_id, std::lock_guard & data_parts_lock) + const String & partition_id, std::unique_lock & data_parts_lock) { auto min_block = std::numeric_limits::min(); MergeTreePartInfo dummy_part_info(partition_id, min_block, min_block, 0); - auto committed_parts = getDataPartsRange({DataPartState::Committed}); - auto it = committed_parts.convert(data_parts.lower_bound(dummy_part_info)); + auto it = data_parts_by_state_and_name.lower_bound(DataPartStateAndInfo(DataPartState::Committed, dummy_part_info)); - if (it != committed_parts.end() && (*it)->info.partition_id == partition_id) + if (it != data_parts_by_state_and_name.end() && (*it)->state == DataPartState::Committed && (*it)->info.partition_id == partition_id) return *it; - return {}; + + return nullptr; } void MergeTreeData::Transaction::rollback() { - if (data && (!parts_to_remove_on_rollback.empty() || !parts_to_add_on_rollback.empty())) + if (!isEmpty()) { std::stringstream ss; if (!parts_to_remove_on_rollback.empty()) @@ -2057,14 +2165,19 @@ void MergeTreeData::Transaction::rollback() /// PreCommitted -> Outdated replaceParts(DataPartState::Outdated, DataPartState::Committed, true); - clear(); } + + clear(); } void MergeTreeData::Transaction::commit() { - /// PreCommitted -> Committed, Committed -> Outdated - replaceParts(DataPartState::Committed, DataPartState::Outdated, false); + if (!isEmpty()) + { + /// PreCommitted -> Committed, Committed -> Outdated + replaceParts(DataPartState::Committed, DataPartState::Outdated, false); + } + clear(); } @@ -2088,9 +2201,9 @@ void MergeTreeData::Transaction::replaceParts(MergeTreeData::DataPartState move_ /// If it is rollback then do nothing, else make it Outdated and remove their size contribution if (move_committed_to != DataPartState::Committed) { - for (auto & part : committed_parts) + for (const DataPartPtr & part : committed_parts) { - part->state = move_committed_to; + data->modifyPartState(part, move_committed_to); part->remove_time = remove_time; data->removePartContributionToColumnSizes(part); } @@ -2099,7 +2212,7 @@ void MergeTreeData::Transaction::replaceParts(MergeTreeData::DataPartState move_ /// If it is rollback just change state to Outdated, else change state to Committed and add their size contribution for (auto & part : precommitted_parts) { - part->state = move_precommitted_to; + data->modifyPartState(part, move_precommitted_to); if (move_precommitted_to == DataPartState::Committed) data->addPartContributionToColumnSizes(part); else diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 2235a73dbf1..fe793c5da9c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -15,7 +15,10 @@ #include #include -#include +#include +#include +#include +#include namespace DB { @@ -104,7 +107,16 @@ public: using DataPartStates = std::initializer_list; using DataPartStateVector = std::vector; - struct DataPartPtrLess + /// Auxiliary structure for index comparison. Keep in mind lifetime of MergeTreePartInfo. + struct DataPartStateAndInfo + { + DataPartState state; + const MergeTreePartInfo & info; + + DataPartStateAndInfo(DataPartState state, const MergeTreePartInfo & info) : state(state), info(info) {} + }; + + struct LessDataPart { using is_transparent = void; @@ -113,11 +125,32 @@ public: bool operator()(const DataPartPtr & lhs, const DataPartPtr & rhs) const { return lhs->info < rhs->info; } }; - using DataParts = std::set; + struct LessStateDataPart + { + using is_transparent = void; + + bool operator() (const DataPartStateAndInfo & lhs, const DataPartStateAndInfo & rhs) const + { + return std::forward_as_tuple(static_cast(lhs.state), lhs.info) + < std::forward_as_tuple(static_cast(rhs.state), rhs.info); + } + + bool operator() (DataPartStateAndInfo info, const DataPartState & state) const + { + return static_cast(info.state) < static_cast(state); + } + + bool operator() (const DataPartState & state, DataPartStateAndInfo info) const + { + return static_cast(state) < static_cast(info.state); + } + }; + + using DataParts = std::set; using DataPartsVector = std::vector; /// For resharding. - using MutableDataParts = std::set; + using MutableDataParts = std::set; using PerShardDataParts = std::unordered_map; /// Some operations on the set of parts return a Transaction object. @@ -131,6 +164,11 @@ public: void rollback(); + bool isEmpty() const + { + return parts_to_add_on_rollback.empty() && parts_to_remove_on_rollback.empty(); + } + ~Transaction() { try @@ -310,22 +348,17 @@ public: /// Returns a copy of the list so that the caller shouldn't worry about locks. DataParts getDataParts(const DataPartStates & affordable_states) const; - DataPartsVector getDataPartsVector(const DataPartStates & affordable_states) const; - DataPartsVector getDataPartsVector(const DataPartStates & affordable_states, DataPartStateVector & out_states_snapshot) const; + /// Returns sorted list of the parts with specified states + /// out_states will contain snapshot of each part state + DataPartsVector getDataPartsVector(const DataPartStates & affordable_states, DataPartStateVector * out_states = nullptr) const; - /// Returns a virtual container iteration only through parts with specified states - decltype(auto) getDataPartsRange(const DataPartStates & affordable_states) const - { - return createRangeFiltered(DataPart::getStatesFilter(affordable_states), data_parts); - } + /// Returns absolutely all parts (and snapshot of their states) + DataPartsVector getAllDataPartsVector(DataPartStateVector * out_states = nullptr) const; /// Returns Committed parts DataParts getDataParts() const; DataPartsVector getDataPartsVector() const; - /// Returns all parts except Temporary and Deleting ones - DataParts getAllDataParts() const; - /// Returns an comitted part with the given name or a part containing it. If there is no such part, returns nullptr. DataPartPtr getActiveContainingPart(const String & part_name); @@ -375,8 +408,8 @@ public: /// Removes parts from data_parts, they should be in Deleting state void removePartsFinally(const DataPartsVector & parts); - /// Delete irrelevant parts. - void clearOldParts(); + /// Delete irrelevant parts from memory and disk. + void clearOldPartsFromFilesystem(); /// Deleate all directories which names begin with "tmp" /// Set non-negative parameter value to override MergeTreeSettings temporary_directories_lifetime @@ -538,15 +571,81 @@ private: String log_name; Logger * log; - /// Current set of data parts. - DataParts data_parts; - mutable std::mutex data_parts_mutex; - /// The set of all data parts including already merged but not yet deleted. Usually it is small (tens of elements). - /// The part is referenced from here, from the list of current parts and from each thread reading from it. - /// This means that if reference count is 1 - the part is not used right now and can be deleted. -// DataParts all_data_parts; -// mutable std::mutex all_data_parts_mutex; + /// Work with data parts + + struct TagByName{}; + struct TagByStateAndName{}; + + static const MergeTreePartInfo & dataPartPtrToInfo(const DataPartPtr & part) + { + return part->info; + } + + static DataPartStateAndInfo dataPartPtrToStateAndInfo(const DataPartPtr & part) + { + return {part->state, part->info}; + }; + + using DataPartsIndexes = boost::multi_index_container, + boost::multi_index::global_fun + >, + /// Index by (State, Name), is used to obtain ordered slices of parts with the same state + boost::multi_index::ordered_unique< + boost::multi_index::tag, + boost::multi_index::global_fun, + LessStateDataPart + > + > + >; + + /// Current set of data parts. + mutable std::mutex data_parts_mutex; + DataPartsIndexes data_parts_indexes; + DataPartsIndexes::index::type & data_parts_by_name; + DataPartsIndexes::index::type & data_parts_by_state_and_name; + + using DataPartIteratorByAndName = DataPartsIndexes::index::type::iterator; + using DataPartIteratorByStateAndName = DataPartsIndexes::index::type::iterator; + + boost::iterator_range getDataPartsStateRange(DataPartState state) const + { + auto begin = data_parts_by_state_and_name.lower_bound(state, LessStateDataPart()); + auto end = data_parts_by_state_and_name.upper_bound(state, LessStateDataPart()); + return {begin, end}; + } + + static decltype(auto) getStateModifier(DataPartState state) + { + return [state] (const DataPartPtr & part) { part->state = state; }; + } + + void modifyPartState(DataPartIteratorByStateAndName it, DataPartState state) + { + if (!data_parts_by_state_and_name.modify(it, getStateModifier(state))) + throw Exception("Can't modify " + (*it)->getNameWithState(), ErrorCodes::LOGICAL_ERROR); + } + + void modifyPartState(DataPartIteratorByAndName it, DataPartState state) + { + if (!data_parts_by_state_and_name.modify(data_parts_indexes.project(it), getStateModifier(state))) + throw Exception("Can't modify " + (*it)->getNameWithState(), ErrorCodes::LOGICAL_ERROR); + } + + void modifyPartState(const DataPartPtr & part, DataPartState state) + { + auto it = data_parts_by_name.find(part->info); + if (it == data_parts_by_name.end() || (*it).get() != part.get()) + throw Exception("Part " + part->name + " is not exists", ErrorCodes::LOGICAL_ERROR); + + if (!data_parts_by_state_and_name.modify(data_parts_indexes.project(it), getStateModifier(state))) + throw Exception("Can't modify " + (*it)->getNameWithState(), ErrorCodes::LOGICAL_ERROR); + } + /// Used to serialize calls to grabOldParts. std::mutex grab_old_parts_mutex; @@ -582,7 +681,7 @@ private: void removePartContributionToColumnSizes(const DataPartPtr & part); /// If there is no part in the partition with ID `partition_id`, returns empty ptr. Should be called under the lock. - DataPartPtr getAnyPartInPartition(const String & partition_id, std::lock_guard & data_parts_lock); + DataPartPtr getAnyPartInPartition(const String & partition_id, std::unique_lock & data_parts_lock); }; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp index 280e1cc30a6..cc8094b8ee5 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp @@ -187,7 +187,7 @@ bool MergeTreeDataMerger::selectPartsToMerge( if (prev_part && part->info.partition_id == (*prev_part)->info.partition_id && part->info.min_block < (*prev_part)->info.max_block) { - LOG_ERROR(log, "Part " << part->name << " intersects previous part " << (*prev_part)->name); + LOG_ERROR(log, "Part " << part->getNameWithState() << " intersects previous part " << (*prev_part)->getNameWithState()); } prev_part = ∂ diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp index a893a8d26d3..d8e35552065 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp @@ -38,6 +38,7 @@ namespace ErrorCodes extern const int FORMAT_VERSION_TOO_OLD; extern const int UNKNOWN_FORMAT; extern const int UNEXPECTED_FILE_IN_DATA_PART; + extern const int NOT_FOUND_EXPECTED_DATA_PART; } @@ -935,4 +936,16 @@ String MergeTreeDataPart::stateString() const return stateToString(state); } +void MergeTreeDataPart::assertState(const std::initializer_list & affordable_states) const +{ + if (!checkState(affordable_states)) + { + String states_str; + for (auto state : affordable_states) + states_str += stateToString(state) + " "; + + throw Exception("Unexpected state of part " + getNameWithState() + ". Expected: " + states_str, ErrorCodes::NOT_FOUND_EXPECTED_DATA_PART); + } +} + } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h index 1863fcbc0f2..b767eb6414b 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h @@ -190,17 +190,7 @@ struct MergeTreeDataPart } /// Throws an exception if state of the part is not in affordable_states - void assertState(const std::initializer_list & affordable_states) const - { - if (!checkState(affordable_states)) - { - String states_str; - for (auto state : affordable_states) - states_str += stateToString(state) + " "; - - throw Exception("Unexpected state of part " + getNameWithState() + ". Expected: " + states_str); - } - } + void assertState(const std::initializer_list & affordable_states) const; /// In comparison with lambdas, it is move assignable and could has several overloaded operator() struct StatesFilter @@ -327,4 +317,7 @@ private: void checkConsistency(bool require_part_metadata); }; + +using MergeTreeDataPartState = MergeTreeDataPart::State; + } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 035d01de38d..c11ee29b10a 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -68,7 +68,7 @@ StorageMergeTree::StorageMergeTree( } else { - data.clearOldParts(); + data.clearOldPartsFromFilesystem(); } /// Temporary directories contain incomplete results of merges (after forced restart) @@ -188,7 +188,7 @@ void StorageMergeTree::alter( if (primary_key_is_modified && supportsSampling()) throw Exception("MODIFY PRIMARY KEY only supported for tables without sampling key", ErrorCodes::BAD_ARGUMENTS); - MergeTreeData::DataParts parts = data.getAllDataParts(); + auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}); for (const MergeTreeData::DataPartPtr & part : parts) { if (auto transaction = data.alterDataPart(part, columns_for_parts, new_primary_key_ast, false)) @@ -291,7 +291,7 @@ bool StorageMergeTree::merge( /// Clear old parts. It does not matter to do it more frequently than each second. if (auto lock = time_after_previous_cleanup.lockTestAndRestartAfter(1)) { - data.clearOldParts(); + data.clearOldPartsFromFilesystem(); data.clearOldTemporaryDirectories(); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 8e745721434..55f72ca520b 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -109,6 +109,7 @@ namespace ErrorCodes extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS; extern const int TOO_MUCH_FETCHES; extern const int BAD_DATA_PART_NAME; + extern const int PART_IS_TEMPORARILY_LOCKED; } @@ -800,7 +801,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) /// Parts in ZK. NameSet expected_parts(expected_parts_vec.begin(), expected_parts_vec.end()); - MergeTreeData::DataParts parts = data.getAllDataParts(); + auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}); /// Local parts that are not in ZK. MergeTreeData::DataParts unexpected_parts; @@ -1179,7 +1180,21 @@ bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry) if (!do_fetch) { merger.renameMergedTemporaryPart(part, parts, &transaction); - getZooKeeper()->multi(ops); /// After long merge, get fresh ZK handle, because previous session may be expired. + + /// Do not commit if the part is obsolete + if (!transaction.isEmpty()) + { + getZooKeeper()->multi(ops); /// After long merge, get fresh ZK handle, because previous session may be expired. + transaction.commit(); + } + + /** Removing old chunks from ZK and from the disk is delayed - see ReplicatedMergeTreeCleanupThread, clearOldParts. + */ + + /** With `ZCONNECTIONLOSS` or `ZOPERATIONTIMEOUT`, we can inadvertently roll back local changes to the parts. + * This is not a problem, because in this case the merge will remain in the queue, and we will try again. + */ + merge_selecting_event.set(); if (auto part_log = context.getPartLog(database_name, table_name)) { @@ -1212,15 +1227,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry) } } - /** Removing old chunks from ZK and from the disk is delayed - see ReplicatedMergeTreeCleanupThread, clearOldParts. - */ - - /** With `ZCONNECTIONLOSS` or `ZOPERATIONTIMEOUT`, we can inadvertently roll back local changes to the parts. - * This is not a problem, because in this case the merge will remain in the queue, and we will try again. - */ - transaction.commit(); - merge_selecting_event.set(); - ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges); } } @@ -1443,8 +1449,9 @@ void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTr /// It's important that no old parts remain (after the merge), because otherwise, /// after adding a new replica, this new replica downloads them, but does not delete them. /// And, if you do not, the parts will come to life after the server is restarted. - /// Therefore, we use getAllDataParts. - auto parts = data.getAllDataParts(); + /// Therefore, we use all data parts. + auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}); + for (const auto & part : parts) { if (!entry_part_info.contains(part->info)) @@ -1616,6 +1623,11 @@ bool StorageReplicatedMergeTree::queueTask() /// Interrupted merge or downloading a part is not an error. LOG_INFO(log, e.message()); } + else if (e.code() == ErrorCodes::PART_IS_TEMPORARILY_LOCKED) + { + /// Part cannot be added temporarily + LOG_INFO(log, e.displayText()); + } else tryLogCurrentException(__PRETTY_FUNCTION__); @@ -2205,6 +2217,13 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin MergeTreeData::Transaction transaction; auto removed_parts = data.renameTempPartAndReplace(part, nullptr, &transaction); + /// Do not commit if the part is obsolete + if (!transaction.isEmpty()) + { + getZooKeeper()->multi(ops); + transaction.commit(); + } + if (auto part_log = context.getPartLog(database_name, table_name)) { PartLogElement elem; @@ -2236,10 +2255,6 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin } } - - getZooKeeper()->multi(ops); - transaction.commit(); - /** If a quorum is tracked for this part, you must update it. * If you do not have time, in case of losing the session, when you restart the server - see the `ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart` method. */ diff --git a/dbms/src/Storages/System/StorageSystemParts.cpp b/dbms/src/Storages/System/StorageSystemParts.cpp index aba6db5fbbc..02cad078f02 100644 --- a/dbms/src/Storages/System/StorageSystemParts.cpp +++ b/dbms/src/Storages/System/StorageSystemParts.cpp @@ -39,7 +39,7 @@ StorageSystemParts::StorageSystemParts(const std::string & name_) {"database", std::make_shared()}, {"table", std::make_shared()}, - {"engine", std::make_shared()}, + {"engine", std::make_shared()} } { } @@ -53,9 +53,12 @@ BlockInputStreams StorageSystemParts::read( const size_t max_block_size, const unsigned num_streams) { - check(column_names); + //check(column_names); processed_stage = QueryProcessingStage::FetchColumns; + auto it_state_column = std::find(column_names.begin(), column_names.end(), "_state"); + bool has_state_column = it_state_column != column_names.end(); + /// Will apply WHERE to subset of columns and then add more columns. /// This is kind of complicated, but we use WHERE to do less work. @@ -142,6 +145,8 @@ BlockInputStreams StorageSystemParts::read( /// Finally, create the result. Block block = getSampleBlock(); + if (has_state_column) + block.insert(ColumnWithTypeAndName(std::make_shared(), "_state")); for (size_t i = 0; i < filtered_database_column->size();) { @@ -198,10 +203,18 @@ BlockInputStreams StorageSystemParts::read( using State = MergeTreeDataPart::State; MergeTreeData::DataPartStateVector all_parts_state; MergeTreeData::DataPartsVector all_parts; + if (need[0]) - all_parts = data->getDataPartsVector({State::Committed, State::Outdated}, all_parts_state); + { + /// If has_state_column is requested, return all states + if (!has_state_column) + all_parts = data->getDataPartsVector({State::Committed, State::Outdated}, &all_parts_state); + else + all_parts = data->getAllDataPartsVector(&all_parts_state); + } else - all_parts = data->getDataPartsVector({State::Committed}, all_parts_state); + all_parts = data->getDataPartsVector({State::Committed}, &all_parts_state); + /// Finally, we'll go through the list of parts. for (size_t part_number = 0; part_number < all_parts.size(); ++part_number) @@ -248,11 +261,30 @@ BlockInputStreams StorageSystemParts::read( block.getByPosition(i++).column->insert(database); block.getByPosition(i++).column->insert(table); block.getByPosition(i++).column->insert(engine); + + if (has_state_column) + block.getByPosition(i++).column->insert(part->stateString()); } } return BlockInputStreams(1, std::make_shared(block)); } +NameAndTypePair StorageSystemParts::getColumn(const String & column_name) const +{ + if (column_name == "_state") + return NameAndTypePair("_state", std::make_shared()); + + return ITableDeclaration::getColumn(column_name); +} + +bool StorageSystemParts::hasColumn(const String & column_name) const +{ + if (column_name == "_state") + return true; + + return ITableDeclaration::hasColumn(column_name); +} + } diff --git a/dbms/src/Storages/System/StorageSystemParts.h b/dbms/src/Storages/System/StorageSystemParts.h index 17c6a7f4e5c..09b14d72e56 100644 --- a/dbms/src/Storages/System/StorageSystemParts.h +++ b/dbms/src/Storages/System/StorageSystemParts.h @@ -21,6 +21,10 @@ public: const NamesAndTypesList & getColumnsListImpl() const override { return columns; } + NameAndTypePair getColumn(const String & column_name) const override; + + bool hasColumn(const String & column_name) const override; + BlockInputStreams read( const Names & column_names, const SelectQueryInfo & query_info, diff --git a/dbms/src/Storages/tests/gtest_range_filtered.cpp b/dbms/src/Storages/tests/gtest_range_filtered.cpp deleted file mode 100644 index 1a3b82f1a68..00000000000 --- a/dbms/src/Storages/tests/gtest_range_filtered.cpp +++ /dev/null @@ -1,44 +0,0 @@ -#include -#include -#include -#include - - -TEST(RangeFiltered, simple) -{ - std::vector v; - - for (int i = 0; i < 10; ++i) - v.push_back(i); - - auto v30 = createRangeFiltered([] (int i) { return i % 3 == 0; }, v); - auto v31 = createRangeFiltered([] (int i) { return i % 3 != 0; }, v); - - for (const int & i : v30) - ASSERT_EQ(i % 3, 0); - - for (const int & i : v31) - ASSERT_NE(i % 3, 0); - - { - auto it = v30.begin(); - ASSERT_EQ(*it, 0); - - auto it2 = std::next(it); - ASSERT_EQ(*it2, 3); - - it = std::next(it2); - ASSERT_EQ(*it, 6); - } - - { - auto it = std::next(v30.begin()); - ASSERT_EQ(*it, 3); - - *it = 2; /// it becomes invalid - ASSERT_EQ(*(++it), 6); /// but iteration is sucessfull - - *v30.begin() = 1; - ASSERT_EQ(*v30.begin(), 6); - } -} diff --git a/dbms/tests/integration/test_random_inserts/test.py b/dbms/tests/integration/test_random_inserts/test.py index d9325c91191..bfa5c451f44 100644 --- a/dbms/tests/integration/test_random_inserts/test.py +++ b/dbms/tests/integration/test_random_inserts/test.py @@ -26,6 +26,7 @@ def started_cluster(): pass cluster.shutdown() + def test_random_inserts(started_cluster): # Duration of the test, reduce it if don't want to wait DURATION_SECONDS = 10# * 60 @@ -55,7 +56,9 @@ def test_random_inserts(started_cluster): inserter.get_answer() answer="{}\t{}\t{}\t{}\n".format(num_timestamps, num_timestamps, min_timestamp, max_timestamp) + for node in nodes: - assert TSV(node.query("SELECT count(), uniqExact(i), min(i), max(i) FROM simple")) == TSV(answer), node.name + " : " + node.query("SELECT groupArray(_part), i, count() AS c FROM simple GROUP BY i ORDER BY c DESC LIMIT 1") + res = node.query("SELECT count(), uniqExact(i), min(i), max(i) FROM simple") + assert TSV(res) == TSV(answer), node.name + " : " + node.query("SELECT groupArray(_part), i, count() AS c FROM simple GROUP BY i ORDER BY c DESC LIMIT 1") node1.query("""DROP TABLE simple ON CLUSTER test_cluster""") diff --git a/dbms/tests/integration/test_random_inserts/test.sh b/dbms/tests/integration/test_random_inserts/test.sh index d743ffe4e91..006ee673fe9 100755 --- a/dbms/tests/integration/test_random_inserts/test.sh +++ b/dbms/tests/integration/test_random_inserts/test.sh @@ -4,6 +4,7 @@ [[ -n "$1" ]] && host="$1" || host="127.0.0.1" [[ -n "$2" ]] && min_timestamp="$2" || min_timestamp=$(( $(date +%s) - 10 )) [[ -n "$3" ]] && max_timestamp="$3" || max_timestamp=$(( $(date +%s) + 10 )) +[[ -n "$4" ]] && iters_per_timestamp="$4" || iters_per_timestamp=1 timestamps=`seq $min_timestamp $max_timestamp` @@ -40,6 +41,6 @@ for i in $timestamps; do cur_timestamp=$(date +%s) done - #echo $i >> $host".txt" reliable_insert "$i" -done \ No newline at end of file +done +sleep 1 diff --git a/libs/libcommon/include/common/RangeFiltered.h b/libs/libcommon/include/common/RangeFiltered.h deleted file mode 100644 index cdb8f902409..00000000000 --- a/libs/libcommon/include/common/RangeFiltered.h +++ /dev/null @@ -1,127 +0,0 @@ -#pragma once -#include - - -/// Similar to boost::filtered_range but a little bit easier and allows to convert ordinary iterators to filtered -template -struct RangeFiltered -{ - /// Template parameter C may be const. Then const_iterator is used. - using RawIterator = decltype(std::declval().begin()); - class Iterator; - - /// Will iterate over elements for which filter(*it) == true - template /// Another template for universal references to work. - RangeFiltered(F_ && filter, C_ && container) - : filter(std::move(filter)), container(container) {} - - Iterator begin() const - { - return Iterator{*this, std::begin(container)}; - } - - Iterator end() const - { - return Iterator{*this, std::end(container)}; - } - - /// Convert ordinary iterator to filtered one - /// Real position will be in range [ordinary_iterator; end()], so it is suitable to use with lower[upper]_bound() - inline Iterator convert(RawIterator ordinary_iterator) const - { - return Iterator{*this, ordinary_iterator}; - } - - - /// It is similar to boost::filtered_iterator, but has additional features: - /// it doesn't store end() iterator - /// it doesn't store predicate, so it allows to implement operator=() - /// it guarantees that operator++() works properly in case of filter(*it) == false - class Iterator - { - public: - using Range = RangeFiltered; - - typedef Iterator self_type; - typedef typename std::iterator_traits::value_type value_type; - typedef typename std::iterator_traits::reference reference; - typedef const value_type & const_reference; - typedef typename std::iterator_traits::pointer pointer; - typedef const value_type * const_pointer; - typedef typename std::iterator_traits::difference_type difference_type; - typedef std::bidirectional_iterator_tag iterator_category; - - Iterator(const Range & range_, RawIterator iter_) - : range(&range_), iter(iter_) - { - for (; iter != std::end(range->container) && !range->filter(*iter); ++iter); - } - - Iterator(const Iterator & rhs) = default; - Iterator(Iterator && rhs) noexcept = default; - - Iterator operator++() - { - ++iter; - for (; iter != std::end(range->container) && !range->filter(*iter); ++iter); - return *this; - } - - Iterator operator--() - { - --iter; - for (; !range->filter(*iter); --iter); /// Don't check std::begin() bound - return *this; - } - - pointer operator->() - { - return iter.operator->(); - } - - const_pointer operator->() const - { - return iter.operator->(); - } - - reference operator*() - { - return *iter; - } - - const_reference operator*() const - { - return *iter; - } - - bool operator==(const self_type & rhs) const - { - return iter == rhs.iter; - } - - bool operator!=(const self_type & rhs) const - { - return iter != rhs.iter; - } - - self_type & operator=(const self_type & rhs) = default; - self_type & operator=(self_type && rhs) noexcept = default; - - ~Iterator() = default; - - private: - const Range * range = nullptr; - RawIterator iter; - }; - -protected: - F filter; - C & container; -}; - - -template -inline RangeFiltered, std::remove_reference_t> createRangeFiltered(F && filter, C && container) -{ - return {std::forward(filter), std::forward(container)}; -}; From 2afbd8bceebeb4422009e4f48b3e6581a2737b3c Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Wed, 29 Nov 2017 14:54:37 +0300 Subject: [PATCH 005/509] Small enhancements. [#CLICKHOUSE-3452] --- .../MergeTree/MergeTreeDataMerger.cpp | 2 +- .../Storages/System/StorageSystemParts.cpp | 19 +++++++++++++++---- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp index cc8094b8ee5..280e1cc30a6 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp @@ -187,7 +187,7 @@ bool MergeTreeDataMerger::selectPartsToMerge( if (prev_part && part->info.partition_id == (*prev_part)->info.partition_id && part->info.min_block < (*prev_part)->info.max_block) { - LOG_ERROR(log, "Part " << part->getNameWithState() << " intersects previous part " << (*prev_part)->getNameWithState()); + LOG_ERROR(log, "Part " << part->name << " intersects previous part " << (*prev_part)->name); } prev_part = ∂ diff --git a/dbms/src/Storages/System/StorageSystemParts.cpp b/dbms/src/Storages/System/StorageSystemParts.cpp index 02cad078f02..91c03f58b4b 100644 --- a/dbms/src/Storages/System/StorageSystemParts.cpp +++ b/dbms/src/Storages/System/StorageSystemParts.cpp @@ -53,11 +53,22 @@ BlockInputStreams StorageSystemParts::read( const size_t max_block_size, const unsigned num_streams) { - //check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; + bool has_state_column = false; + Names real_column_names; - auto it_state_column = std::find(column_names.begin(), column_names.end(), "_state"); - bool has_state_column = it_state_column != column_names.end(); + for (const String & column_name : column_names) + { + if (column_name == "_state") + has_state_column = true; + else + real_column_names.emplace_back(column_name); + } + + /// Do not check if only _state column is requested + if (!(has_state_column && real_column_names.empty())) + check(real_column_names); + + processed_stage = QueryProcessingStage::FetchColumns; /// Will apply WHERE to subset of columns and then add more columns. /// This is kind of complicated, but we use WHERE to do less work. From 616697f14f404e119aff317680ed57d144aed741 Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 21 Nov 2017 19:18:18 +0300 Subject: [PATCH 006/509] Zlib-ng: enable zlib compat mode (this fixes log compression) (#CLICKHOUSE-3447) --- cmake/find_zlib.cmake | 1 + 1 file changed, 1 insertion(+) diff --git a/cmake/find_zlib.cmake b/cmake/find_zlib.cmake index bbfc75e5e24..93e62497c25 100644 --- a/cmake/find_zlib.cmake +++ b/cmake/find_zlib.cmake @@ -6,6 +6,7 @@ endif () if (NOT ZLIB_FOUND) set (USE_INTERNAL_ZLIB_LIBRARY 1) + set (ZLIB_COMPAT 1) # for zlib-ng, also enables WITH_GZFILEOP set (ZLIB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libzlib-ng" "${ClickHouse_BINARY_DIR}/contrib/libzlib-ng") # generated zconf.h set (ZLIB_INCLUDE_DIRS ${ZLIB_INCLUDE_DIR}) # for poco set (ZLIB_FOUND 1) # for poco From 8d3bc22fed080e20474df0e65c75fd0cad6deaa0 Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 21 Nov 2017 20:46:28 +0300 Subject: [PATCH 007/509] Zlib: as submodule, fix compile options; config: add logger options: logger.flush logger.rotateOnOpen Conflicts: .gitmodules dbms/src/Common/BackgroundSchedulePool.h --- .gitmodules | 3 +++ contrib/CMakeLists.txt | 6 +++++- contrib/zlib-ng | 1 + libs/libdaemon/src/BaseDaemon.cpp | 24 ++++++++++++++---------- 4 files changed, 23 insertions(+), 11 deletions(-) create mode 160000 contrib/zlib-ng diff --git a/.gitmodules b/.gitmodules index 87f07998f35..fc08ca60275 100644 --- a/.gitmodules +++ b/.gitmodules @@ -13,3 +13,6 @@ [submodule "contrib/librdkafka"] path = contrib/librdkafka url = https://github.com/edenhill/librdkafka.git +[submodule "contrib/zlib-ng"] + path = contrib/zlib-ng + url = https://github.com/Dead2/zlib-ng.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index f21bee5d979..137eea988bc 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -45,7 +45,11 @@ if (USE_INTERNAL_UNWIND_LIBRARY) endif () if (USE_INTERNAL_ZLIB_LIBRARY) - add_subdirectory (libzlib-ng) + add_subdirectory (zlib-ng) + # todo: make pull to Dead2/zlib-ng and remove: + # We should use same defines when including zlib.h as used when zlib compiled + target_compile_definitions (zlib PUBLIC ZLIB_COMPAT WITH_GZFILEOP) + target_compile_definitions (zlibstatic PUBLIC ZLIB_COMPAT WITH_GZFILEOP) endif () if (USE_INTERNAL_CCTZ_LIBRARY) diff --git a/contrib/zlib-ng b/contrib/zlib-ng new file mode 160000 index 00000000000..e07a52dbaa3 --- /dev/null +++ b/contrib/zlib-ng @@ -0,0 +1 @@ +Subproject commit e07a52dbaa35d003f5659b221b29d220c091667b diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index bf6a0a778de..4b8f1e8d766 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -604,11 +604,13 @@ void BaseDaemon::buildLoggers() pf->setProperty("times", "local"); Poco::AutoPtr log = new FormattingChannel(pf); log_file = new FileChannel; - log_file->setProperty("path", Poco::Path(config().getString("logger.log")).absolute().toString()); - log_file->setProperty("rotation", config().getRawString("logger.size", "100M")); - log_file->setProperty("archive", "number"); - log_file->setProperty("compress", config().getRawString("logger.compress", "true")); - log_file->setProperty("purgeCount", config().getRawString("logger.count", "1")); + log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(config().getString("logger.log")).absolute().toString()); + log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config().getRawString("logger.size", "100M")); + log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); + log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config().getRawString("logger.compress", "true")); + log_file->setProperty(Poco::FileChannel::PROP_PURGECOUNT, config().getRawString("logger.count", "1")); + log_file->setProperty(Poco::FileChannel::PROP_FLUSH, config().getRawString("logger.flush", "true")); + log_file->setProperty(Poco::FileChannel::PROP_ROTATEONOPEN, config().getRawString("logger.rotateOnOpen", "false")); log->setChannel(log_file); split->addChannel(log); log_file->open(); @@ -622,11 +624,13 @@ void BaseDaemon::buildLoggers() pf->setProperty("times", "local"); Poco::AutoPtr errorlog = new FormattingChannel(pf); error_log_file = new FileChannel; - error_log_file->setProperty("path", Poco::Path(config().getString("logger.errorlog")).absolute().toString()); - error_log_file->setProperty("rotation", config().getRawString("logger.size", "100M")); - error_log_file->setProperty("archive", "number"); - error_log_file->setProperty("compress", config().getRawString("logger.compress", "true")); - error_log_file->setProperty("purgeCount", config().getRawString("logger.count", "1")); + error_log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(config().getString("logger.errorlog")).absolute().toString()); + error_log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config().getRawString("logger.size", "100M")); + error_log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); + error_log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config().getRawString("logger.compress", "true")); + error_log_file->setProperty(Poco::FileChannel::PROP_PURGECOUNT, config().getRawString("logger.count", "1")); + error_log_file->setProperty(Poco::FileChannel::PROP_FLUSH, config().getRawString("logger.flush", "true")); + error_log_file->setProperty(Poco::FileChannel::PROP_ROTATEONOPEN, config().getRawString("logger.rotateOnOpen", "false")); errorlog->setChannel(error_log_file); level->setChannel(errorlog); split->addChannel(level); From 00d8ff3ced8d4f723ae1196b36f95b7dd16ecada Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 22 Nov 2017 00:21:22 +0300 Subject: [PATCH 008/509] Remove old contrib/libzlib-ng --- contrib/libzlib-ng/.gitignore | 52 - contrib/libzlib-ng/.travis.yml | 12 - contrib/libzlib-ng/CMakeLists.txt | 563 ---- contrib/libzlib-ng/ChangeLog.zlib | 1478 ---------- contrib/libzlib-ng/FAQ.zlib | 374 --- contrib/libzlib-ng/INDEX | 55 - contrib/libzlib-ng/LICENSE.md | 19 - contrib/libzlib-ng/Makefile.in | 329 --- contrib/libzlib-ng/README | 63 - contrib/libzlib-ng/README.clickhouse | 12 - contrib/libzlib-ng/README.md | 65 - contrib/libzlib-ng/README.zlib | 121 - contrib/libzlib-ng/adler32.c | 177 -- contrib/libzlib-ng/arch/.gitignore | 2 - contrib/libzlib-ng/arch/arm/Makefile.in | 20 - contrib/libzlib-ng/arch/generic/Makefile.in | 20 - contrib/libzlib-ng/arch/x86/INDEX | 3 - contrib/libzlib-ng/arch/x86/Makefile.in | 53 - contrib/libzlib-ng/arch/x86/crc_folding.c | 465 ---- contrib/libzlib-ng/arch/x86/deflate_quick.c | 2371 ----------------- contrib/libzlib-ng/arch/x86/fill_window_sse.c | 165 -- .../libzlib-ng/arch/x86/insert_string_sse.c | 50 - contrib/libzlib-ng/arch/x86/x86.c | 53 - contrib/libzlib-ng/arch/x86/x86.h | 23 - contrib/libzlib-ng/compress.c | 74 - contrib/libzlib-ng/configure | 923 ------- contrib/libzlib-ng/crc32.c | 458 ---- contrib/libzlib-ng/crc32.h | 444 --- contrib/libzlib-ng/deflate.c | 1407 ---------- contrib/libzlib-ng/deflate.h | 459 ---- contrib/libzlib-ng/deflate_fast.c | 114 - contrib/libzlib-ng/deflate_medium.c | 322 --- contrib/libzlib-ng/deflate_p.h | 96 - contrib/libzlib-ng/deflate_slow.c | 160 -- contrib/libzlib-ng/doc/algorithm.txt | 209 -- contrib/libzlib-ng/doc/rfc1950.txt | 619 ----- contrib/libzlib-ng/doc/rfc1951.txt | 955 ------- contrib/libzlib-ng/doc/rfc1952.txt | 675 ----- contrib/libzlib-ng/doc/txtvsbin.txt | 107 - contrib/libzlib-ng/gzclose.c | 23 - contrib/libzlib-ng/gzguts.h | 158 -- contrib/libzlib-ng/gzlib.c | 518 ---- contrib/libzlib-ng/gzread.c | 538 ---- contrib/libzlib-ng/gzwrite.c | 457 ---- contrib/libzlib-ng/infback.c | 612 ----- contrib/libzlib-ng/inffast.c | 328 --- contrib/libzlib-ng/inffast.h | 15 - contrib/libzlib-ng/inffixed.h | 94 - contrib/libzlib-ng/inflate.c | 1467 ---------- contrib/libzlib-ng/inflate.h | 127 - contrib/libzlib-ng/inftrees.c | 298 --- contrib/libzlib-ng/inftrees.h | 66 - contrib/libzlib-ng/match.c | 471 ---- contrib/libzlib-ng/match.h | 6 - contrib/libzlib-ng/test/.gitignore | 2 - contrib/libzlib-ng/test/CVE-2002-0059/test.gz | Bin 4610 -> 0 bytes contrib/libzlib-ng/test/CVE-2003-0107.c | 20 - contrib/libzlib-ng/test/CVE-2004-0797/test.gz | Bin 52 -> 0 bytes contrib/libzlib-ng/test/CVE-2005-1849/test.gz | Bin 52 -> 0 bytes contrib/libzlib-ng/test/CVE-2005-2096/test.gz | Bin 52 -> 0 bytes contrib/libzlib-ng/test/INDEX | 10 - contrib/libzlib-ng/test/Makefile.in | 77 - contrib/libzlib-ng/test/example.c | 544 ---- contrib/libzlib-ng/test/infcover.c | 668 ----- contrib/libzlib-ng/test/minigzip.c | 530 ---- contrib/libzlib-ng/test/testCVEinputs.sh | 22 - contrib/libzlib-ng/treebuild.xml | 116 - contrib/libzlib-ng/trees.c | 1119 -------- contrib/libzlib-ng/trees.h | 132 - contrib/libzlib-ng/uncompr.c | 75 - contrib/libzlib-ng/win32/DLL_FAQ.txt | 397 --- contrib/libzlib-ng/win32/Makefile.msc | 154 -- contrib/libzlib-ng/win32/README-WIN32.txt | 103 - contrib/libzlib-ng/win32/VisualC.txt | 3 - contrib/libzlib-ng/win32/zlib.def | 55 - contrib/libzlib-ng/win32/zlib1.rc | 40 - contrib/libzlib-ng/win32/zlibcompat.def | 86 - contrib/libzlib-ng/zconf.h.in | 176 -- contrib/libzlib-ng/zlib.3 | 169 -- contrib/libzlib-ng/zlib.3.pdf | Bin 8734 -> 0 bytes contrib/libzlib-ng/zlib.h | 1728 ------------ contrib/libzlib-ng/zlib.map | 83 - contrib/libzlib-ng/zlib.pc.cmakein | 13 - contrib/libzlib-ng/zlib.pc.in | 13 - contrib/libzlib-ng/zutil.c | 124 - contrib/libzlib-ng/zutil.h | 185 -- 86 files changed, 25389 deletions(-) delete mode 100644 contrib/libzlib-ng/.gitignore delete mode 100644 contrib/libzlib-ng/.travis.yml delete mode 100644 contrib/libzlib-ng/CMakeLists.txt delete mode 100644 contrib/libzlib-ng/ChangeLog.zlib delete mode 100644 contrib/libzlib-ng/FAQ.zlib delete mode 100644 contrib/libzlib-ng/INDEX delete mode 100644 contrib/libzlib-ng/LICENSE.md delete mode 100644 contrib/libzlib-ng/Makefile.in delete mode 100644 contrib/libzlib-ng/README delete mode 100644 contrib/libzlib-ng/README.clickhouse delete mode 100644 contrib/libzlib-ng/README.md delete mode 100644 contrib/libzlib-ng/README.zlib delete mode 100644 contrib/libzlib-ng/adler32.c delete mode 100644 contrib/libzlib-ng/arch/.gitignore delete mode 100644 contrib/libzlib-ng/arch/arm/Makefile.in delete mode 100644 contrib/libzlib-ng/arch/generic/Makefile.in delete mode 100644 contrib/libzlib-ng/arch/x86/INDEX delete mode 100644 contrib/libzlib-ng/arch/x86/Makefile.in delete mode 100644 contrib/libzlib-ng/arch/x86/crc_folding.c delete mode 100644 contrib/libzlib-ng/arch/x86/deflate_quick.c delete mode 100644 contrib/libzlib-ng/arch/x86/fill_window_sse.c delete mode 100644 contrib/libzlib-ng/arch/x86/insert_string_sse.c delete mode 100644 contrib/libzlib-ng/arch/x86/x86.c delete mode 100644 contrib/libzlib-ng/arch/x86/x86.h delete mode 100644 contrib/libzlib-ng/compress.c delete mode 100755 contrib/libzlib-ng/configure delete mode 100644 contrib/libzlib-ng/crc32.c delete mode 100644 contrib/libzlib-ng/crc32.h delete mode 100644 contrib/libzlib-ng/deflate.c delete mode 100644 contrib/libzlib-ng/deflate.h delete mode 100644 contrib/libzlib-ng/deflate_fast.c delete mode 100644 contrib/libzlib-ng/deflate_medium.c delete mode 100644 contrib/libzlib-ng/deflate_p.h delete mode 100644 contrib/libzlib-ng/deflate_slow.c delete mode 100644 contrib/libzlib-ng/doc/algorithm.txt delete mode 100644 contrib/libzlib-ng/doc/rfc1950.txt delete mode 100644 contrib/libzlib-ng/doc/rfc1951.txt delete mode 100644 contrib/libzlib-ng/doc/rfc1952.txt delete mode 100644 contrib/libzlib-ng/doc/txtvsbin.txt delete mode 100644 contrib/libzlib-ng/gzclose.c delete mode 100644 contrib/libzlib-ng/gzguts.h delete mode 100644 contrib/libzlib-ng/gzlib.c delete mode 100644 contrib/libzlib-ng/gzread.c delete mode 100644 contrib/libzlib-ng/gzwrite.c delete mode 100644 contrib/libzlib-ng/infback.c delete mode 100644 contrib/libzlib-ng/inffast.c delete mode 100644 contrib/libzlib-ng/inffast.h delete mode 100644 contrib/libzlib-ng/inffixed.h delete mode 100644 contrib/libzlib-ng/inflate.c delete mode 100644 contrib/libzlib-ng/inflate.h delete mode 100644 contrib/libzlib-ng/inftrees.c delete mode 100644 contrib/libzlib-ng/inftrees.h delete mode 100644 contrib/libzlib-ng/match.c delete mode 100644 contrib/libzlib-ng/match.h delete mode 100644 contrib/libzlib-ng/test/.gitignore delete mode 100644 contrib/libzlib-ng/test/CVE-2002-0059/test.gz delete mode 100644 contrib/libzlib-ng/test/CVE-2003-0107.c delete mode 100644 contrib/libzlib-ng/test/CVE-2004-0797/test.gz delete mode 100644 contrib/libzlib-ng/test/CVE-2005-1849/test.gz delete mode 100644 contrib/libzlib-ng/test/CVE-2005-2096/test.gz delete mode 100644 contrib/libzlib-ng/test/INDEX delete mode 100644 contrib/libzlib-ng/test/Makefile.in delete mode 100644 contrib/libzlib-ng/test/example.c delete mode 100644 contrib/libzlib-ng/test/infcover.c delete mode 100644 contrib/libzlib-ng/test/minigzip.c delete mode 100755 contrib/libzlib-ng/test/testCVEinputs.sh delete mode 100644 contrib/libzlib-ng/treebuild.xml delete mode 100644 contrib/libzlib-ng/trees.c delete mode 100644 contrib/libzlib-ng/trees.h delete mode 100644 contrib/libzlib-ng/uncompr.c delete mode 100644 contrib/libzlib-ng/win32/DLL_FAQ.txt delete mode 100644 contrib/libzlib-ng/win32/Makefile.msc delete mode 100644 contrib/libzlib-ng/win32/README-WIN32.txt delete mode 100644 contrib/libzlib-ng/win32/VisualC.txt delete mode 100644 contrib/libzlib-ng/win32/zlib.def delete mode 100644 contrib/libzlib-ng/win32/zlib1.rc delete mode 100644 contrib/libzlib-ng/win32/zlibcompat.def delete mode 100644 contrib/libzlib-ng/zconf.h.in delete mode 100644 contrib/libzlib-ng/zlib.3 delete mode 100644 contrib/libzlib-ng/zlib.3.pdf delete mode 100644 contrib/libzlib-ng/zlib.h delete mode 100644 contrib/libzlib-ng/zlib.map delete mode 100644 contrib/libzlib-ng/zlib.pc.cmakein delete mode 100644 contrib/libzlib-ng/zlib.pc.in delete mode 100644 contrib/libzlib-ng/zutil.c delete mode 100644 contrib/libzlib-ng/zutil.h diff --git a/contrib/libzlib-ng/.gitignore b/contrib/libzlib-ng/.gitignore deleted file mode 100644 index 0beb44b722a..00000000000 --- a/contrib/libzlib-ng/.gitignore +++ /dev/null @@ -1,52 +0,0 @@ -*.diff -*.patch -*.orig -*.rej - -*~ -*.a -*.lo -*.o -*.dylib - -*.gcda -*.gcno -*.gcov - -/example -/example64 -/examplesh -/libz.so* -/minigzip -/minigzip64 -/minigzipsh -/zlib.pc -/CVE-2003-0107 - -.DS_Store -*.obj -*.exe -*.pdb -*.exp -*.lib -*.dll -*.res -foo.gz -*.manifest - -CMakeCache.txt -CMakeFiles -Testing -*.cmake -*.stackdump -zconf.h -zconf.h.cmakein -zconf.h.included -ztest* - -configure.log -a.out - -/arch/arm/Makefile -/arch/generic/Makefile -/arch/x86/Makefile diff --git a/contrib/libzlib-ng/.travis.yml b/contrib/libzlib-ng/.travis.yml deleted file mode 100644 index 6080169cf68..00000000000 --- a/contrib/libzlib-ng/.travis.yml +++ /dev/null @@ -1,12 +0,0 @@ -language: c -compiler: - - gcc - - clang -env: - - BUILDDIR=. TOOL="./configure --zlib-compat" - - BUILDDIR=../build TOOL="../zlib-ng/configure --zlib-compat" - - BUILDDIR=. TOOL="./configure --zlib-compat --without-optimizations --without-new-strategies" - - BUILDDIR=. TOOL="cmake ." - - BUILDDIR=../build TOOL="cmake ../zlib-ng" -script: mkdir -p $BUILDDIR && cd $BUILDDIR && - $TOOL && make && make test diff --git a/contrib/libzlib-ng/CMakeLists.txt b/contrib/libzlib-ng/CMakeLists.txt deleted file mode 100644 index 3d5be9eabf2..00000000000 --- a/contrib/libzlib-ng/CMakeLists.txt +++ /dev/null @@ -1,563 +0,0 @@ -cmake_minimum_required(VERSION 2.8.4) -set(CMAKE_ALLOW_LOOSE_LOOP_CONSTRUCTS ON) - -project(zlib C) - -set(VERSION "1.2.8") - -set(INSTALL_BIN_DIR "${CMAKE_INSTALL_PREFIX}/bin" CACHE PATH "Installation directory for executables") -set(INSTALL_LIB_DIR "${CMAKE_INSTALL_PREFIX}/lib" CACHE PATH "Installation directory for libraries") -set(INSTALL_INC_DIR "${CMAKE_INSTALL_PREFIX}/include" CACHE PATH "Installation directory for headers") -set(INSTALL_MAN_DIR "${CMAKE_INSTALL_PREFIX}/share/man" CACHE PATH "Installation directory for manual pages") -set(INSTALL_PKGCONFIG_DIR "${CMAKE_INSTALL_PREFIX}/share/pkgconfig" CACHE PATH "Installation directory for pkgconfig (.pc) files") - -include(CheckTypeSize) -include(CheckSymbolExists) -include(CheckFunctionExists) -include(CheckIncludeFile) -include(CheckCSourceCompiles) -include(CheckCSourceRuns) -include(CheckLibraryExists) -include(FeatureSummary) - -# make sure we use an appropriate BUILD_TYPE by default, "Release" to be exact -# this should select the maximum generic optimisation on the current platform (i.e. -O3 for gcc/clang) -if(NOT CMAKE_BUILD_TYPE) - set(CMAKE_BUILD_TYPE "Release" CACHE STRING - "Choose the type of build, standard options are: Debug Release RelWithDebInfo MinSizeRel." - FORCE) - add_feature_info(CMAKE_BUILD_TYPE 1 "Build type: ${CMAKE_BUILD_TYPE} (default)") -else() - add_feature_info(CMAKE_BUILD_TYPE 1 "Build type: ${CMAKE_BUILD_TYPE} (selected)") -endif() - -enable_testing() - -check_include_file(sys/types.h HAVE_SYS_TYPES_H) -check_include_file(stdint.h HAVE_STDINT_H) -check_include_file(stddef.h HAVE_STDDEF_H) - -# -# Options parsing -# -set(ARCH ${CMAKE_HOST_SYSTEM_PROCESSOR}) -message(STATUS "Architecture: ${ARCH}") - -option (ZLIB_COMPAT "Compile with zlib compatible API" OFF) -if (ZLIB_COMPAT) - add_definitions(-DZLIB_COMPAT) - set (WITH_GZFILEOP ON) -endif (ZLIB_COMPAT) - -option (WITH_GZFILEOP "Compile with support for gzFile related functions" OFF) -if (WITH_GZFILEOP) - add_definitions(-DWITH_GZFILEOP) -endif (WITH_GZFILEOP) - -option(WITH_OPTIM "Build with optimisation" ON) -option(WITH_NEW_QUICK_STRATEGY "Use new quick strategy for compression level 1" OFF) # this option produces corrupt gzip stream so turn it off for now. -option(WITH_NEW_MEDIUM_STRATEGY "Use new medium strategy for compression levels 4-6" ON) -option(WITH_NATIVE_INSTRUCTIONS - "Instruct the compiler to use the full instruction set on this host (gcc/clang -march=native)" OFF) - -if(${CMAKE_C_COMPILER} MATCHES "icc" OR ${CMAKE_C_COMPILER} MATCHES "icpc" OR ${CMAKE_C_COMPILER} MATCHES "icl") - if(WITH_NATIVE_INSTRUCTIONS) - message(STATUS "Ignoring WITH_NATIVE_INSTRUCTIONS; not supported on this configuration") - endif() - if(CMAKE_HOST_UNIX) - if(NOT SSE2FLAG) - set(SSE2FLAG "-msse2") - endif() - if(NOT SSE4FLAG) - set(SSE4FLAG "-msse4.2") - endif() - else() - if(NOT SSE2FLAG) - set(SSE2FLAG "/arch:SSE2") - endif() - if(NOT SSE4FLAG) - set(SSE4FLAG "/arch:SSE4.2") - endif() - endif() -elseif(MSVC) - # TODO. ICC can be used through MSVC. I'm not sure if we'd ever see that combination - # (who'd use cmake from an IDE...) but checking for ICC before checking for MSVC should - # avoid mistakes. - # /Oi ? - if(NOT ${ARCH} MATCHES "AMD64") - set(SSE2FLAG "/arch:SSE2") - endif() - if(WITH_NATIVE_INSTRUCTIONS) - message(STATUS "Ignoring WITH_NATIVE_INSTRUCTIONS; not supported on this configuration") - endif() -else() - execute_process(COMMAND ${CMAKE_C_COMPILER} --version OUTPUT_VARIABLE COMPILER_VERSION) - if("${COMPILER_VERSION}" MATCHES "gcc" OR "${COMPILER_VERSION}" MATCHES "clang") - set(__GNUC__ ON) - endif() - if(WITH_NATIVE_INSTRUCTIONS) - if(__GNUC__) - set(NATIVEFLAG "-march=native") - else() - message(STATUS "Ignoring WITH_NATIVE_INSTRUCTIONS; not implemented yet on this configuration") - endif() - endif() - if(NOT NATIVEFLAG) - if(NOT SSE2FLAG) - if(__GNUC__) - set(SSE2FLAG "-msse2") - endif() - endif() - if(NOT SSE4FLAG) - if(__GNUC__) - set(SSE4FLAG "-msse4") - endif() - endif() - if(NOT PCLMULFLAG) - if(__GNUC__) - set(PCLMULFLAG "-mpclmul") - endif() - endif() - else(NOT NATIVEFLAG) - set(SSE2FLAG ${NATIVEFLAG}) - set(SSE4FLAG ${NATIVEFLAG}) - set(PCLMULFLAG ${NATIVEFLAG}) - endif(NOT NATIVEFLAG) -endif() - -add_feature_info(ZLIB_COMPAT ZLIB_COMPAT "Provide a zlib-compatible API") -add_feature_info(WITH_GZFILEOP WITH_GZFILEOP "Compile with support for gzFile-related functions") -add_feature_info(WITH_OPTIM WITH_OPTIM "Build with optimisation") -add_feature_info(WITH_NEW_QUICK_STRATEGY WITH_NEW_QUICK_STRATEGY "Use new quick strategy for compression level 1") -add_feature_info(WITH_NEW_MEDIUM_STRATEGY WITH_NEW_MEDIUM_STRATEGY "Use new medium strategy for compression levels 4-6") - -# -# Check to see if we have large file support -# -set(CMAKE_REQUIRED_DEFINITIONS -D_LARGEFILE64_SOURCE=1) -# We add these other definitions here because CheckTypeSize.cmake -# in CMake 2.4.x does not automatically do so and we want -# compatibility with CMake 2.4.x. -if(HAVE_SYS_TYPES_H) - list(APPEND CMAKE_REQUIRED_DEFINITIONS -DHAVE_SYS_TYPES_H) -endif() -if(HAVE_STDINT_H) - list(APPEND CMAKE_REQUIRED_DEFINITIONS -DHAVE_STDINT_H) -endif() -if(HAVE_STDDEF_H) - list(APPEND CMAKE_REQUIRED_DEFINITIONS -DHAVE_STDDEF_H) -endif() -check_type_size(off64_t OFF64_T) -if(HAVE_OFF64_T) - add_definitions(-D_LARGEFILE64_SOURCE=1) -else() - check_type_size(_off64_t _OFF64_T) - if (HAVE__OFF64_T) - add_definitions(-D_LARGEFILE64_SOURCE=1) - endif() -endif() -set(CMAKE_REQUIRED_DEFINITIONS) # clear variable - -# -# Check for fseeko and other optional functions -# -check_function_exists(fseeko HAVE_FSEEKO) -if(NOT HAVE_FSEEKO) - add_definitions(-DNO_FSEEKO) -endif() -check_function_exists(strerror HAVE_STRERROR) -if(NOT HAVE_STRERROR) - add_definitions(-DNO_STRERROR) -endif() - -# -# Check for unistd.h and stdarg.h -# -check_include_file(unistd.h Z_HAVE_UNISTD_H) -check_include_file(stdarg.h Z_HAVE_STDARG_H) - -# -# Check if we can hide zlib internal symbols that are linked between separate source files using hidden -# -check_c_source_compiles( - "#define ZLIB_INTERNAL __attribute__((visibility (\"hidden\"))) - int ZLIB_INTERNAL foo; - int main() - { - return 0; - }" - HAVE_ATTRIBUTE_VISIBILITY_HIDDEN FAIL_REGEX "not supported") -if(HAVE_ATTRIBUTE_VISIBILITY_HIDDEN) - add_definitions(-DHAVE_HIDDEN) -endif() - -# -# Check if we can hide zlib internal symbols that are linked between separate source files using internal -# -check_c_source_compiles( - "#define ZLIB_INTERNAL __attribute__((visibility (\"internal\"))) - int ZLIB_INTERNAL foo; - int main() - { - return 0; - }" - HAVE_ATTRIBUTE_VISIBILITY_INTERNAL FAIL_REGEX "not supported") -if(HAVE_ATTRIBUTE_VISIBILITY_INTERNAL) - add_definitions(-DHAVE_INTERNAL) -endif() - -# -# check for __builtin_ctzl() support in the compiler -# -check_c_source_compiles( - "int main(void) - { - unsigned int zero = 0; - long test = __builtin_ctzl(zero); - (void)test; - return 0; - }" - HAVE_BUILTIN_CTZL -) -if(HAVE_BUILTIN_CTZL) - add_definitions(-DHAVE_BUILTIN_CTZL) -endif() - -# Macro to check if source compiles when cross-compiling -# or runs when compiling natively -macro(check_c_source_compile_or_run source flag) - if(CMAKE_CROSSCOMPILING) - check_c_source_compiles("${source}" ${flag}) - else() - check_c_source_runs("${source}" ${flag}) - endif() -endmacro(check_c_source_compile_or_run) -if(MSVC) - set(CMAKE_DEBUG_POSTFIX "d") - add_definitions(-D_CRT_SECURE_NO_DEPRECATE) - add_definitions(-D_CRT_NONSTDC_NO_DEPRECATE) - include_directories(${CMAKE_CURRENT_SOURCE_DIR}) -else() - # - # not MSVC, so we need to check if we have the MS-style SSE etc. intrinsics - # - if(WITH_NATIVE_INSTRUCTIONS) - set(CMAKE_REQUIRED_FLAGS "${NATIVEFLAG}") - else() - set(CMAKE_REQUIRED_FLAGS "${SSE2FLAG}") - endif() - check_c_source_compile_or_run( - "#include - int main(void) - { - __m128i zero = _mm_setzero_si128(); - (void)zero; - return 0; - }" - HAVE_SSE2_INTRIN - ) - if(WITH_NATIVE_INSTRUCTIONS) - set(CMAKE_REQUIRED_FLAGS "${NATIVEFLAG}") - else() - # use the generic SSE4 enabler option to check for the SSE4.2 instruction we require: - set(CMAKE_REQUIRED_FLAGS "${SSE4FLAG}") - endif() - check_c_source_compile_or_run( - "int main(void) - { - unsigned val = 0, h = 0; - __asm__ __volatile__ ( \"crc32 %1,%0\" : \"+r\" (h) : \"r\" (val) ); - return (int) h; - }" - HAVE_SSE42_INTRIN - ) - if(WITH_NATIVE_INSTRUCTIONS) - set(CMAKE_REQUIRED_FLAGS "${NATIVEFLAG}") - else() - # the PCLMUL instruction we use also requires an SSE4.1 instruction check for both - set(CMAKE_REQUIRED_FLAGS "${SSE4FLAG} ${PCLMULFLAG}") - endif() - check_c_source_compile_or_run( - "#include - #include - #include - int main(void) - { - __m128i a = _mm_setzero_si128(); - __m128i b = _mm_setzero_si128(); - __m128i c = _mm_clmulepi64_si128(a, b, 0x10); - int d = _mm_extract_epi32(c, 2); - return d; - }" - HAVE_PCLMULQDQ_INTRIN - ) -endif() - -# -# Enable deflate_medium at level 4-6 -# -if(WITH_NEW_MEDIUM_STRATEGY) - add_definitions(-DMEDIUM_STRATEGY) -endif() - -# -# macro to add either the given intrinsics option to the global compiler options, -# or ${NATIVEFLAG} (-march=native) if that is appropriate and possible. -# An alternative version of this macro would take a file argument, and set ${flag} -# only for that file as opposed to ${NATIVEFLAG} globally, to limit side-effect of -# using ${flag} globally. -# -macro(add_intrinsics_option flag) - if(WITH_NATIVE_INSTRUCTIONS AND NATIVEFLAG) - if (NOT "${CMAKE_C_FLAGS} " MATCHES ".*${NATIVEFLAG} .*") - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${NATIVEFLAG}") - endif() - else() - if (NOT "${CMAKE_C_FLAGS} " MATCHES ".*${flag} .*") - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${flag}") - endif() - endif() -endmacro(add_intrinsics_option) - -set(ZLIB_ARCH_SRCS) -set(ARCHDIR "arch/generic") -if("${ARCH}" MATCHES "x86_64" OR "${ARCH}" MATCHES "AMD64") - set(ARCHDIR "arch/x86") - add_definitions(-DX86_64 -DX86_NOCHECK_SSE2 -DUNALIGNED_OK -DUNROLL_LESS -DX86_CPUID) - add_feature_info(SSE2 1 "Use the SSE2 instruction set, using \"${SSE2FLAG}\"") -elseif("${ARCH}" MATCHES "arm" OR "${ARCH}" MATCHES "aarch64") - set(ARCHDIR "arch/arm") - add_definitions(-DUNALIGNED_OK -DUNROLL_LESS) -else() - set(ARCHDIR "arch/x86") - add_definitions(-DX86 -DUNALIGNED_OK -DUNROLL_LESS -DX86_CPUID) - add_feature_info(SSE2 1 "Support the SSE2 instruction set, using \"${SSE2FLAG}\"") -endif() -if(WITH_OPTIM) - if(NOT CMAKE_SYSTEM_PROCESSOR MATCHES "arm" AND NOT CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64") - set(ZLIB_ARCH_SRCS ${ZLIB_ARCH_SRCS} ${ARCHDIR}/x86.c) - endif() - if(HAVE_SSE42_INTRIN) - add_definitions(-DX86_SSE4_2_CRC_HASH) - set(ZLIB_ARCH_SRCS ${ZLIB_ARCH_SRCS} ${ARCHDIR}/insert_string_sse.c) - add_feature_info(SSE4_CRC 1 "Support CRC hash generation using the SSE4.2 instruction set, using \"${SSE4FLAG}\"") - add_intrinsics_option("${SSE4FLAG}") - if(WITH_NEW_QUICK_STRATEGY) - add_definitions(-DX86_QUICK_STRATEGY) - set(ZLIB_ARCH_SRCS ${ZLIB_ARCH_SRCS} ${ARCHDIR}/deflate_quick.c) - add_feature_info(SSE4DEFLATE 1 "Support SSE4.2-accelerated quick decompression") - endif() - endif() - if(HAVE_SSE2_INTRIN) - add_definitions(-DX86_SSE2_FILL_WINDOW) - set(ZLIB_ARCH_SRCS ${ZLIB_ARCH_SRCS} ${ARCHDIR}/fill_window_sse.c) - if(NOT ${ARCH} MATCHES "x86_64") - add_intrinsics_option("${SSE2FLAG}") - endif() - endif() - if(HAVE_PCLMULQDQ_INTRIN) - if (CMAKE_SYSTEM_PROCESSOR MATCHES "^x86") - add_definitions(-DX86_PCLMULQDQ_CRC) - endif() - set(ZLIB_ARCH_SRCS ${ZLIB_ARCH_SRCS} ${ARCHDIR}/crc_folding.c) - add_feature_info(PCLMUL_CRC 1 "Support CRC hash generation using PCLMULQDQ, using \"${SSE4FLAG} ${PCLMULFLAG}\"") - add_intrinsics_option("${PCLMULFLAG}") - if(NOT HAVE_SSE42_INTRIN) - add_intrinsics_option("${SSE4FLAG}") - endif() - endif() -endif() -message(STATUS "Architecture-specific source files: ${ZLIB_ARCH_SRCS}") - -#============================================================================ -# zconf.h -#============================================================================ - -macro(generate_cmakein input output) - execute_process(COMMAND sed "/#define ZCONF_H/ a\\\n#cmakedefine Z_HAVE_UNISTD_H\\\n#cmakedefine Z_HAVE_STDARG_H\n" - INPUT_FILE ${input} - OUTPUT_FILE ${output} -) - -endmacro(generate_cmakein) - -generate_cmakein( ${CMAKE_CURRENT_SOURCE_DIR}/zconf.h.in ${CMAKE_CURRENT_BINARY_DIR}/zconf.h.cmakein ) - -if(NOT CMAKE_CURRENT_SOURCE_DIR STREQUAL CMAKE_CURRENT_BINARY_DIR) - # If we're doing an out of source build and the user has a zconf.h - # in their source tree... - if(EXISTS ${CMAKE_CURRENT_SOURCE_DIR}/zconf.h) - message(STATUS "Renaming") - message(STATUS " ${CMAKE_CURRENT_SOURCE_DIR}/zconf.h") - message(STATUS "to 'zconf.h.included' because this file is included with zlib") - message(STATUS "but CMake generates it automatically in the build directory.") - file(RENAME ${CMAKE_CURRENT_SOURCE_DIR}/zconf.h ${CMAKE_CURRENT_SOURCE_DIR}/zconf.h.included) - endif() - - # If we're doing an out of source build and the user has a zconf.h.cmakein - # in their source tree... - if(EXISTS ${CMAKE_CURRENT_SOURCE_DIR}/zconf.h.cmakein) - message(STATUS "Renaming") - message(STATUS " ${CMAKE_CURRENT_SOURCE_DIR}/zconf.h.cmakein") - message(STATUS "to 'zconf.h.cmakeincluded' because this file is included with zlib") - message(STATUS "but CMake generates it automatically in the build directory.") - file(RENAME ${CMAKE_CURRENT_SOURCE_DIR}/zconf.h.cmakein ${CMAKE_CURRENT_SOURCE_DIR}/zconf.h.cmakeincluded) - endif() -endif() - -set(ZLIB_PC ${CMAKE_CURRENT_BINARY_DIR}/zlib.pc) -configure_file( ${CMAKE_CURRENT_SOURCE_DIR}/zlib.pc.cmakein - ${ZLIB_PC} @ONLY) -configure_file( ${CMAKE_CURRENT_BINARY_DIR}/zconf.h.cmakein - ${CMAKE_CURRENT_BINARY_DIR}/zconf.h @ONLY) -include_directories(${CMAKE_CURRENT_BINARY_DIR} ${CMAKE_CURRENT_SOURCE_DIR}) - - -#============================================================================ -# zlib -#============================================================================ - -set(ZLIB_PUBLIC_HDRS - ${CMAKE_CURRENT_BINARY_DIR}/zconf.h - zlib.h -) -set(ZLIB_PRIVATE_HDRS - crc32.h - deflate.h - gzguts.h - inffast.h - inffixed.h - inflate.h - inftrees.h - trees.h - zutil.h -) -set(ZLIB_SRCS - adler32.c - compress.c - crc32.c - deflate.c - deflate_fast.c - deflate_medium.c - deflate_slow.c - inflate.c - infback.c - inftrees.c - inffast.c - match.c - trees.c - uncompr.c - zutil.c -) -if (WITH_GZFILEOP) - set(ZLIB_GZFILE_SRCS - gzclose.c - gzlib.c - gzread.c - gzwrite.c - ) -else (WITH_GZFILEOP) - set(ZLIB_GZFILE_SRCS - ) -endif (WITH_GZFILEOP) - - -if(NOT MINGW AND NOT MSYS) - set(ZLIB_DLL_SRCS - win32/zlib1.rc # If present will override custom build rule below. - ) -endif() - -# parse the full version number from zlib.h and include in ZLIB_FULL_VERSION -file(READ ${CMAKE_CURRENT_SOURCE_DIR}/zlib.h _zlib_h_contents) -string(REGEX REPLACE ".*#define[ \t]+ZLIB_VERSION[ \t]+\"([-0-9A-Za-z.]+)\".*" - "\\1" ZLIB_FULL_VERSION ${_zlib_h_contents}) - -if(MINGW OR MSYS) - # This gets us DLL resource information when compiling on MinGW. - if(NOT CMAKE_RC_COMPILER) - set(CMAKE_RC_COMPILER windres.exe) - endif() - - add_custom_command(OUTPUT ${CMAKE_CURRENT_BINARY_DIR}/zlib1rc.obj - COMMAND ${CMAKE_RC_COMPILER} - -D GCC_WINDRES - -I ${CMAKE_CURRENT_SOURCE_DIR} - -I ${CMAKE_CURRENT_BINARY_DIR} - -o ${CMAKE_CURRENT_BINARY_DIR}/zlib1rc.obj - -i ${CMAKE_CURRENT_SOURCE_DIR}/win32/zlib1.rc) - set(ZLIB_DLL_SRCS ${CMAKE_CURRENT_BINARY_DIR}/zlib1rc.obj) -endif(MINGW OR MSYS) - -add_library(zlib SHARED ${ZLIB_SRCS} ${ZLIB_GZFILE_SRCS} ${ZLIB_ARCH_SRCS} ${ZLIB_ASMS} ${ZLIB_DLL_SRCS} ${ZLIB_PUBLIC_HDRS} ${ZLIB_PRIVATE_HDRS}) -add_library(zlibstatic STATIC ${ZLIB_SRCS} ${ZLIB_GZFILE_SRCS} ${ZLIB_ARCH_SRCS} ${ZLIB_ASMS} ${ZLIB_PUBLIC_HDRS} ${ZLIB_PRIVATE_HDRS}) - -set_target_properties(zlib PROPERTIES DEFINE_SYMBOL ZLIB_DLL) -set_target_properties(zlib PROPERTIES SOVERSION 1) - -if(NOT CYGWIN) - # This property causes shared libraries on Linux to have the full version - # encoded into their final filename. We disable this on Cygwin because - # it causes cygz-${ZLIB_FULL_VERSION}.dll to be created when cygz.dll - # seems to be the default. - # - # This has no effect with MSVC, on that platform the version info for - # the DLL comes from the resource file win32/zlib1.rc - set_target_properties(zlib PROPERTIES VERSION ${ZLIB_FULL_VERSION}) -endif() - -if(UNIX) - # On unix-like platforms the library is almost always called libz - set_target_properties(zlib zlibstatic PROPERTIES OUTPUT_NAME z) - if(NOT APPLE) - set_target_properties(zlib PROPERTIES LINK_FLAGS "-Wl,--version-script,\"${CMAKE_CURRENT_SOURCE_DIR}/zlib.map\"") - endif() -elseif(MSYS) - # Suppress version number from shared library name - set(CMAKE_SHARED_LIBRARY_NAME_WITH_VERSION 0) -elseif(BUILD_SHARED_LIBS AND WIN32) - # Creates zlib1.dll when building shared library version - set_target_properties(zlib PROPERTIES SUFFIX "1.dll") -endif() - -if(NOT SKIP_INSTALL_LIBRARIES AND NOT SKIP_INSTALL_ALL ) - install(TARGETS zlib zlibstatic - RUNTIME DESTINATION "${INSTALL_BIN_DIR}" - ARCHIVE DESTINATION "${INSTALL_LIB_DIR}" - LIBRARY DESTINATION "${INSTALL_LIB_DIR}" ) -endif() -if(NOT SKIP_INSTALL_HEADERS AND NOT SKIP_INSTALL_ALL ) - install(FILES ${ZLIB_PUBLIC_HDRS} DESTINATION "${INSTALL_INC_DIR}") -endif() -if(NOT SKIP_INSTALL_FILES AND NOT SKIP_INSTALL_ALL ) - install(FILES zlib.3 DESTINATION "${INSTALL_MAN_DIR}/man3") -endif() -if(NOT SKIP_INSTALL_FILES AND NOT SKIP_INSTALL_ALL ) - install(FILES ${ZLIB_PC} DESTINATION "${INSTALL_PKGCONFIG_DIR}") -endif() - -#============================================================================ -# Example binaries -#============================================================================ - -if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang") - set (CMAKE_EXE_LINKER_FLAGS "") -endif () - -add_executable(example test/example.c) -target_link_libraries(example zlib) -add_test(example example) - -add_executable(minigzip test/minigzip.c) -target_link_libraries(minigzip zlib) - -if(HAVE_OFF64_T) - add_executable(example64 test/example.c) - target_link_libraries(example64 zlib) - set_target_properties(example64 PROPERTIES COMPILE_FLAGS "-D_FILE_OFFSET_BITS=64") - add_test(example64 example64) - - add_executable(minigzip64 test/minigzip.c) - target_link_libraries(minigzip64 zlib) - set_target_properties(minigzip64 PROPERTIES COMPILE_FLAGS "-D_FILE_OFFSET_BITS=64") -endif() diff --git a/contrib/libzlib-ng/ChangeLog.zlib b/contrib/libzlib-ng/ChangeLog.zlib deleted file mode 100644 index 909d11f9b47..00000000000 --- a/contrib/libzlib-ng/ChangeLog.zlib +++ /dev/null @@ -1,1478 +0,0 @@ -## -# THIS IS AN UNMAINTAINED COPY OF THE ORIGINAL FILE DISTRIBUTED WITH ZLIB 1.2.8 -## - - - - - ChangeLog file for zlib - -Changes in 1.2.8 (28 Apr 2013) -- Update contrib/minizip/iowin32.c for Windows RT [Vollant] -- Do not force Z_CONST for C++ -- Clean up contrib/vstudio [Roß] -- Correct spelling error in zlib.h -- Fix mixed line endings in contrib/vstudio - -Changes in 1.2.7.3 (13 Apr 2013) -- Fix version numbers and DLL names in contrib/vstudio/*/zlib.rc - -Changes in 1.2.7.2 (13 Apr 2013) -- Change check for a four-byte type back to hexadecimal -- Fix typo in win32/Makefile.msc -- Add casts in gzwrite.c for pointer differences - -Changes in 1.2.7.1 (24 Mar 2013) -- Replace use of unsafe string functions with snprintf if available -- Avoid including stddef.h on Windows for Z_SOLO compile [Niessink] -- Fix gzgetc undefine when Z_PREFIX set [Turk] -- Eliminate use of mktemp in Makefile (not always available) -- Fix bug in 'F' mode for gzopen() -- Add inflateGetDictionary() function -- Correct comment in deflate.h -- Use _snprintf for snprintf in Microsoft C -- On Darwin, only use /usr/bin/libtool if libtool is not Apple -- Delete "--version" file if created by "ar --version" [Richard G.] -- Fix configure check for veracity of compiler error return codes -- Fix CMake compilation of static lib for MSVC2010 x64 -- Remove unused variable in infback9.c -- Fix argument checks in gzlog_compress() and gzlog_write() -- Clean up the usage of z_const and respect const usage within zlib -- Clean up examples/gzlog.[ch] comparisons of different types -- Avoid shift equal to bits in type (caused endless loop) -- Fix uninitialized value bug in gzputc() introduced by const patches -- Fix memory allocation error in examples/zran.c [Nor] -- Fix bug where gzopen(), gzclose() would write an empty file -- Fix bug in gzclose() when gzwrite() runs out of memory -- Check for input buffer malloc failure in examples/gzappend.c -- Add note to contrib/blast to use binary mode in stdio -- Fix comparisons of differently signed integers in contrib/blast -- Check for invalid code length codes in contrib/puff -- Fix serious but very rare decompression bug in inftrees.c -- Update inflateBack() comments, since inflate() can be faster -- Use underscored I/O function names for WINAPI_FAMILY -- Add _tr_flush_bits to the external symbols prefixed by --zprefix -- Add contrib/vstudio/vc10 pre-build step for static only -- Quote --version-script argument in CMakeLists.txt -- Don't specify --version-script on Apple platforms in CMakeLists.txt -- Fix casting error in contrib/testzlib/testzlib.c -- Fix types in contrib/minizip to match result of get_crc_table() -- Simplify contrib/vstudio/vc10 with 'd' suffix -- Add TOP support to win32/Makefile.msc -- Suport i686 and amd64 assembler builds in CMakeLists.txt -- Fix typos in the use of _LARGEFILE64_SOURCE in zconf.h -- Add vc11 and vc12 build files to contrib/vstudio -- Add gzvprintf() as an undocumented function in zlib -- Fix configure for Sun shell -- Remove runtime check in configure for four-byte integer type -- Add casts and consts to ease user conversion to C++ -- Add man pages for minizip and miniunzip -- In Makefile uninstall, don't rm if preceding cd fails -- Do not return Z_BUF_ERROR if deflateParam() has nothing to write - -Changes in 1.2.7 (2 May 2012) -- Replace use of memmove() with a simple copy for portability -- Test for existence of strerror -- Restore gzgetc_ for backward compatibility with 1.2.6 -- Fix build with non-GNU make on Solaris -- Require gcc 4.0 or later on Mac OS X to use the hidden attribute -- Include unistd.h for Watcom C -- Use __WATCOMC__ instead of __WATCOM__ -- Do not use the visibility attribute if NO_VIZ defined -- Improve the detection of no hidden visibility attribute -- Avoid using __int64 for gcc or solo compilation -- Cast to char * in gzprintf to avoid warnings [Zinser] -- Fix make_vms.com for VAX [Zinser] -- Don't use library or built-in byte swaps -- Simplify test and use of gcc hidden attribute -- Fix bug in gzclose_w() when gzwrite() fails to allocate memory -- Add "x" (O_EXCL) and "e" (O_CLOEXEC) modes support to gzopen() -- Fix bug in test/minigzip.c for configure --solo -- Fix contrib/vstudio project link errors [Mohanathas] -- Add ability to choose the builder in make_vms.com [Schweda] -- Add DESTDIR support to mingw32 win32/Makefile.gcc -- Fix comments in win32/Makefile.gcc for proper usage -- Allow overriding the default install locations for cmake -- Generate and install the pkg-config file with cmake -- Build both a static and a shared version of zlib with cmake -- Include version symbols for cmake builds -- If using cmake with MSVC, add the source directory to the includes -- Remove unneeded EXTRA_CFLAGS from win32/Makefile.gcc [Truta] -- Move obsolete emx makefile to old [Truta] -- Allow the use of -Wundef when compiling or using zlib -- Avoid the use of the -u option with mktemp -- Improve inflate() documentation on the use of Z_FINISH -- Recognize clang as gcc -- Add gzopen_w() in Windows for wide character path names -- Rename zconf.h in CMakeLists.txt to move it out of the way -- Add source directory in CMakeLists.txt for building examples -- Look in build directory for zlib.pc in CMakeLists.txt -- Remove gzflags from zlibvc.def in vc9 and vc10 -- Fix contrib/minizip compilation in the MinGW environment -- Update ./configure for Solaris, support --64 [Mooney] -- Remove -R. from Solaris shared build (possible security issue) -- Avoid race condition for parallel make (-j) running example -- Fix type mismatch between get_crc_table() and crc_table -- Fix parsing of version with "-" in CMakeLists.txt [Snider, Ziegler] -- Fix the path to zlib.map in CMakeLists.txt -- Force the native libtool in Mac OS X to avoid GNU libtool [Beebe] -- Add instructions to win32/Makefile.gcc for shared install [Torri] - -Changes in 1.2.6.1 (12 Feb 2012) -- Avoid the use of the Objective-C reserved name "id" -- Include io.h in gzguts.h for Microsoft compilers -- Fix problem with ./configure --prefix and gzgetc macro -- Include gz_header definition when compiling zlib solo -- Put gzflags() functionality back in zutil.c -- Avoid library header include in crc32.c for Z_SOLO -- Use name in GCC_CLASSIC as C compiler for coverage testing, if set -- Minor cleanup in contrib/minizip/zip.c [Vollant] -- Update make_vms.com [Zinser] -- Remove unnecessary gzgetc_ function -- Use optimized byte swap operations for Microsoft and GNU [Snyder] -- Fix minor typo in zlib.h comments [Rzesniowiecki] - -Changes in 1.2.6 (29 Jan 2012) -- Update the Pascal interface in contrib/pascal -- Fix function numbers for gzgetc_ in zlibvc.def files -- Fix configure.ac for contrib/minizip [Schiffer] -- Fix large-entry detection in minizip on 64-bit systems [Schiffer] -- Have ./configure use the compiler return code for error indication -- Fix CMakeLists.txt for cross compilation [McClure] -- Fix contrib/minizip/zip.c for 64-bit architectures [Dalsnes] -- Fix compilation of contrib/minizip on FreeBSD [Marquez] -- Correct suggested usages in win32/Makefile.msc [Shachar, Horvath] -- Include io.h for Turbo C / Borland C on all platforms [Truta] -- Make version explicit in contrib/minizip/configure.ac [Bosmans] -- Avoid warning for no encryption in contrib/minizip/zip.c [Vollant] -- Minor cleanup up contrib/minizip/unzip.c [Vollant] -- Fix bug when compiling minizip with C++ [Vollant] -- Protect for long name and extra fields in contrib/minizip [Vollant] -- Avoid some warnings in contrib/minizip [Vollant] -- Add -I../.. -L../.. to CFLAGS for minizip and miniunzip -- Add missing libs to minizip linker command -- Add support for VPATH builds in contrib/minizip -- Add an --enable-demos option to contrib/minizip/configure -- Add the generation of configure.log by ./configure -- Exit when required parameters not provided to win32/Makefile.gcc -- Have gzputc return the character written instead of the argument -- Use the -m option on ldconfig for BSD systems [Tobias] -- Correct in zlib.map when deflateResetKeep was added - -Changes in 1.2.5.3 (15 Jan 2012) -- Restore gzgetc function for binary compatibility -- Do not use _lseeki64 under Borland C++ [Truta] -- Update win32/Makefile.msc to build test/*.c [Truta] -- Remove old/visualc6 given CMakefile and other alternatives -- Update AS400 build files and documentation [Monnerat] -- Update win32/Makefile.gcc to build test/*.c [Truta] -- Permit stronger flushes after Z_BLOCK flushes -- Avoid extraneous empty blocks when doing empty flushes -- Permit Z_NULL arguments to deflatePending -- Allow deflatePrime() to insert bits in the middle of a stream -- Remove second empty static block for Z_PARTIAL_FLUSH -- Write out all of the available bits when using Z_BLOCK -- Insert the first two strings in the hash table after a flush - -Changes in 1.2.5.2 (17 Dec 2011) -- fix ld error: unable to find version dependency 'ZLIB_1.2.5' -- use relative symlinks for shared libs -- Avoid searching past window for Z_RLE strategy -- Assure that high-water mark initialization is always applied in deflate -- Add assertions to fill_window() in deflate.c to match comments -- Update python link in README -- Correct spelling error in gzread.c -- Fix bug in gzgets() for a concatenated empty gzip stream -- Correct error in comment for gz_make() -- Change gzread() and related to ignore junk after gzip streams -- Allow gzread() and related to continue after gzclearerr() -- Allow gzrewind() and gzseek() after a premature end-of-file -- Simplify gzseek() now that raw after gzip is ignored -- Change gzgetc() to a macro for speed (~40% speedup in testing) -- Fix gzclose() to return the actual error last encountered -- Always add large file support for windows -- Include zconf.h for windows large file support -- Include zconf.h.cmakein for windows large file support -- Update zconf.h.cmakein on make distclean -- Merge vestigial vsnprintf determination from zutil.h to gzguts.h -- Clarify how gzopen() appends in zlib.h comments -- Correct documentation of gzdirect() since junk at end now ignored -- Add a transparent write mode to gzopen() when 'T' is in the mode -- Update python link in zlib man page -- Get inffixed.h and MAKEFIXED result to match -- Add a ./config --solo option to make zlib subset with no libary use -- Add undocumented inflateResetKeep() function for CAB file decoding -- Add --cover option to ./configure for gcc coverage testing -- Add #define ZLIB_CONST option to use const in the z_stream interface -- Add comment to gzdopen() in zlib.h to use dup() when using fileno() -- Note behavior of uncompress() to provide as much data as it can -- Add files in contrib/minizip to aid in building libminizip -- Split off AR options in Makefile.in and configure -- Change ON macro to Z_ARG to avoid application conflicts -- Facilitate compilation with Borland C++ for pragmas and vsnprintf -- Include io.h for Turbo C / Borland C++ -- Move example.c and minigzip.c to test/ -- Simplify incomplete code table filling in inflate_table() -- Remove code from inflate.c and infback.c that is impossible to execute -- Test the inflate code with full coverage -- Allow deflateSetDictionary, inflateSetDictionary at any time (in raw) -- Add deflateResetKeep and fix inflateResetKeep to retain dictionary -- Fix gzwrite.c to accommodate reduced memory zlib compilation -- Have inflate() with Z_FINISH avoid the allocation of a window -- Do not set strm->adler when doing raw inflate -- Fix gzeof() to behave just like feof() when read is not past end of file -- Fix bug in gzread.c when end-of-file is reached -- Avoid use of Z_BUF_ERROR in gz* functions except for premature EOF -- Document gzread() capability to read concurrently written files -- Remove hard-coding of resource compiler in CMakeLists.txt [Blammo] - -Changes in 1.2.5.1 (10 Sep 2011) -- Update FAQ entry on shared builds (#13) -- Avoid symbolic argument to chmod in Makefile.in -- Fix bug and add consts in contrib/puff [Oberhumer] -- Update contrib/puff/zeros.raw test file to have all block types -- Add full coverage test for puff in contrib/puff/Makefile -- Fix static-only-build install in Makefile.in -- Fix bug in unzGetCurrentFileInfo() in contrib/minizip [Kuno] -- Add libz.a dependency to shared in Makefile.in for parallel builds -- Spell out "number" (instead of "nb") in zlib.h for total_in, total_out -- Replace $(...) with `...` in configure for non-bash sh [Bowler] -- Add darwin* to Darwin* and solaris* to SunOS\ 5* in configure [Groffen] -- Add solaris* to Linux* in configure to allow gcc use [Groffen] -- Add *bsd* to Linux* case in configure [Bar-Lev] -- Add inffast.obj to dependencies in win32/Makefile.msc -- Correct spelling error in deflate.h [Kohler] -- Change libzdll.a again to libz.dll.a (!) in win32/Makefile.gcc -- Add test to configure for GNU C looking for gcc in output of $cc -v -- Add zlib.pc generation to win32/Makefile.gcc [Weigelt] -- Fix bug in zlib.h for _FILE_OFFSET_BITS set and _LARGEFILE64_SOURCE not -- Add comment in zlib.h that adler32_combine with len2 < 0 makes no sense -- Make NO_DIVIDE option in adler32.c much faster (thanks to John Reiser) -- Make stronger test in zconf.h to include unistd.h for LFS -- Apply Darwin patches for 64-bit file offsets to contrib/minizip [Slack] -- Fix zlib.h LFS support when Z_PREFIX used -- Add updated as400 support (removed from old) [Monnerat] -- Avoid deflate sensitivity to volatile input data -- Avoid division in adler32_combine for NO_DIVIDE -- Clarify the use of Z_FINISH with deflateBound() amount of space -- Set binary for output file in puff.c -- Use u4 type for crc_table to avoid conversion warnings -- Apply casts in zlib.h to avoid conversion warnings -- Add OF to prototypes for adler32_combine_ and crc32_combine_ [Miller] -- Improve inflateSync() documentation to note indeterminancy -- Add deflatePending() function to return the amount of pending output -- Correct the spelling of "specification" in FAQ [Randers-Pehrson] -- Add a check in configure for stdarg.h, use for gzprintf() -- Check that pointers fit in ints when gzprint() compiled old style -- Add dummy name before $(SHAREDLIBV) in Makefile [Bar-Lev, Bowler] -- Delete line in configure that adds -L. libz.a to LDFLAGS [Weigelt] -- Add debug records in assmebler code [Londer] -- Update RFC references to use http://tools.ietf.org/html/... [Li] -- Add --archs option, use of libtool to configure for Mac OS X [Borstel] - -Changes in 1.2.5 (19 Apr 2010) -- Disable visibility attribute in win32/Makefile.gcc [Bar-Lev] -- Default to libdir as sharedlibdir in configure [Nieder] -- Update copyright dates on modified source files -- Update trees.c to be able to generate modified trees.h -- Exit configure for MinGW, suggesting win32/Makefile.gcc -- Check for NULL path in gz_open [Homurlu] - -Changes in 1.2.4.5 (18 Apr 2010) -- Set sharedlibdir in configure [Torok] -- Set LDFLAGS in Makefile.in [Bar-Lev] -- Avoid mkdir objs race condition in Makefile.in [Bowler] -- Add ZLIB_INTERNAL in front of internal inter-module functions and arrays -- Define ZLIB_INTERNAL to hide internal functions and arrays for GNU C -- Don't use hidden attribute when it is a warning generator (e.g. Solaris) - -Changes in 1.2.4.4 (18 Apr 2010) -- Fix CROSS_PREFIX executable testing, CHOST extract, mingw* [Torok] -- Undefine _LARGEFILE64_SOURCE in zconf.h if it is zero, but not if empty -- Try to use bash or ksh regardless of functionality of /bin/sh -- Fix configure incompatibility with NetBSD sh -- Remove attempt to run under bash or ksh since have better NetBSD fix -- Fix win32/Makefile.gcc for MinGW [Bar-Lev] -- Add diagnostic messages when using CROSS_PREFIX in configure -- Added --sharedlibdir option to configure [Weigelt] -- Use hidden visibility attribute when available [Frysinger] - -Changes in 1.2.4.3 (10 Apr 2010) -- Only use CROSS_PREFIX in configure for ar and ranlib if they exist -- Use CROSS_PREFIX for nm [Bar-Lev] -- Assume _LARGEFILE64_SOURCE defined is equivalent to true -- Avoid use of undefined symbols in #if with && and || -- Make *64 prototypes in gzguts.h consistent with functions -- Add -shared load option for MinGW in configure [Bowler] -- Move z_off64_t to public interface, use instead of off64_t -- Remove ! from shell test in configure (not portable to Solaris) -- Change +0 macro tests to -0 for possibly increased portability - -Changes in 1.2.4.2 (9 Apr 2010) -- Add consistent carriage returns to readme.txt's in masmx86 and masmx64 -- Really provide prototypes for *64 functions when building without LFS -- Only define unlink() in minigzip.c if unistd.h not included -- Update README to point to contrib/vstudio project files -- Move projects/vc6 to old/ and remove projects/ -- Include stdlib.h in minigzip.c for setmode() definition under WinCE -- Clean up assembler builds in win32/Makefile.msc [Rowe] -- Include sys/types.h for Microsoft for off_t definition -- Fix memory leak on error in gz_open() -- Symbolize nm as $NM in configure [Weigelt] -- Use TEST_LDSHARED instead of LDSHARED to link test programs [Weigelt] -- Add +0 to _FILE_OFFSET_BITS and _LFS64_LARGEFILE in case not defined -- Fix bug in gzeof() to take into account unused input data -- Avoid initialization of structures with variables in puff.c -- Updated win32/README-WIN32.txt [Rowe] - -Changes in 1.2.4.1 (28 Mar 2010) -- Remove the use of [a-z] constructs for sed in configure [gentoo 310225] -- Remove $(SHAREDLIB) from LIBS in Makefile.in [Creech] -- Restore "for debugging" comment on sprintf() in gzlib.c -- Remove fdopen for MVS from gzguts.h -- Put new README-WIN32.txt in win32 [Rowe] -- Add check for shell to configure and invoke another shell if needed -- Fix big fat stinking bug in gzseek() on uncompressed files -- Remove vestigial F_OPEN64 define in zutil.h -- Set and check the value of _LARGEFILE_SOURCE and _LARGEFILE64_SOURCE -- Avoid errors on non-LFS systems when applications define LFS macros -- Set EXE to ".exe" in configure for MINGW [Kahle] -- Match crc32() in crc32.c exactly to the prototype in zlib.h [Sherrill] -- Add prefix for cross-compilation in win32/makefile.gcc [Bar-Lev] -- Add DLL install in win32/makefile.gcc [Bar-Lev] -- Allow Linux* or linux* from uname in configure [Bar-Lev] -- Allow ldconfig to be redefined in configure and Makefile.in [Bar-Lev] -- Add cross-compilation prefixes to configure [Bar-Lev] -- Match type exactly in gz_load() invocation in gzread.c -- Match type exactly of zcalloc() in zutil.c to zlib.h alloc_func -- Provide prototypes for *64 functions when building zlib without LFS -- Don't use -lc when linking shared library on MinGW -- Remove errno.h check in configure and vestigial errno code in zutil.h - -Changes in 1.2.4 (14 Mar 2010) -- Fix VER3 extraction in configure for no fourth subversion -- Update zlib.3, add docs to Makefile.in to make .pdf out of it -- Add zlib.3.pdf to distribution -- Don't set error code in gzerror() if passed pointer is NULL -- Apply destination directory fixes to CMakeLists.txt [Lowman] -- Move #cmakedefine's to a new zconf.in.cmakein -- Restore zconf.h for builds that don't use configure or cmake -- Add distclean to dummy Makefile for convenience -- Update and improve INDEX, README, and FAQ -- Update CMakeLists.txt for the return of zconf.h [Lowman] -- Update contrib/vstudio/vc9 and vc10 [Vollant] -- Change libz.dll.a back to libzdll.a in win32/Makefile.gcc -- Apply license and readme changes to contrib/asm686 [Raiter] -- Check file name lengths and add -c option in minigzip.c [Li] -- Update contrib/amd64 and contrib/masmx86/ [Vollant] -- Avoid use of "eof" parameter in trees.c to not shadow library variable -- Update make_vms.com for removal of zlibdefs.h [Zinser] -- Update assembler code and vstudio projects in contrib [Vollant] -- Remove outdated assembler code contrib/masm686 and contrib/asm586 -- Remove old vc7 and vc8 from contrib/vstudio -- Update win32/Makefile.msc, add ZLIB_VER_SUBREVISION [Rowe] -- Fix memory leaks in gzclose_r() and gzclose_w(), file leak in gz_open() -- Add contrib/gcc_gvmat64 for longest_match and inflate_fast [Vollant] -- Remove *64 functions from win32/zlib.def (they're not 64-bit yet) -- Fix bug in void-returning vsprintf() case in gzwrite.c -- Fix name change from inflate.h in contrib/inflate86/inffas86.c -- Check if temporary file exists before removing in make_vms.com [Zinser] -- Fix make install and uninstall for --static option -- Fix usage of _MSC_VER in gzguts.h and zutil.h [Truta] -- Update readme.txt in contrib/masmx64 and masmx86 to assemble - -Changes in 1.2.3.9 (21 Feb 2010) -- Expunge gzio.c -- Move as400 build information to old -- Fix updates in contrib/minizip and contrib/vstudio -- Add const to vsnprintf test in configure to avoid warnings [Weigelt] -- Delete zconf.h (made by configure) [Weigelt] -- Change zconf.in.h to zconf.h.in per convention [Weigelt] -- Check for NULL buf in gzgets() -- Return empty string for gzgets() with len == 1 (like fgets()) -- Fix description of gzgets() in zlib.h for end-of-file, NULL return -- Update minizip to 1.1 [Vollant] -- Avoid MSVC loss of data warnings in gzread.c, gzwrite.c -- Note in zlib.h that gzerror() should be used to distinguish from EOF -- Remove use of snprintf() from gzlib.c -- Fix bug in gzseek() -- Update contrib/vstudio, adding vc9 and vc10 [Kuno, Vollant] -- Fix zconf.h generation in CMakeLists.txt [Lowman] -- Improve comments in zconf.h where modified by configure - -Changes in 1.2.3.8 (13 Feb 2010) -- Clean up text files (tabs, trailing whitespace, etc.) [Oberhumer] -- Use z_off64_t in gz_zero() and gz_skip() to match state->skip -- Avoid comparison problem when sizeof(int) == sizeof(z_off64_t) -- Revert to Makefile.in from 1.2.3.6 (live with the clutter) -- Fix missing error return in gzflush(), add zlib.h note -- Add *64 functions to zlib.map [Levin] -- Fix signed/unsigned comparison in gz_comp() -- Use SFLAGS when testing shared linking in configure -- Add --64 option to ./configure to use -m64 with gcc -- Fix ./configure --help to correctly name options -- Have make fail if a test fails [Levin] -- Avoid buffer overrun in contrib/masmx64/gvmat64.asm [Simpson] -- Remove assembler object files from contrib - -Changes in 1.2.3.7 (24 Jan 2010) -- Always gzopen() with O_LARGEFILE if available -- Fix gzdirect() to work immediately after gzopen() or gzdopen() -- Make gzdirect() more precise when the state changes while reading -- Improve zlib.h documentation in many places -- Catch memory allocation failure in gz_open() -- Complete close operation if seek forward in gzclose_w() fails -- Return Z_ERRNO from gzclose_r() if close() fails -- Return Z_STREAM_ERROR instead of EOF for gzclose() being passed NULL -- Return zero for gzwrite() errors to match zlib.h description -- Return -1 on gzputs() error to match zlib.h description -- Add zconf.in.h to allow recovery from configure modification [Weigelt] -- Fix static library permissions in Makefile.in [Weigelt] -- Avoid warnings in configure tests that hide functionality [Weigelt] -- Add *BSD and DragonFly to Linux case in configure [gentoo 123571] -- Change libzdll.a to libz.dll.a in win32/Makefile.gcc [gentoo 288212] -- Avoid access of uninitialized data for first inflateReset2 call [Gomes] -- Keep object files in subdirectories to reduce the clutter somewhat -- Remove default Makefile and zlibdefs.h, add dummy Makefile -- Add new external functions to Z_PREFIX, remove duplicates, z_z_ -> z_ -- Remove zlibdefs.h completely -- modify zconf.h instead - -Changes in 1.2.3.6 (17 Jan 2010) -- Avoid void * arithmetic in gzread.c and gzwrite.c -- Make compilers happier with const char * for gz_error message -- Avoid unused parameter warning in inflate.c -- Avoid signed-unsigned comparison warning in inflate.c -- Indent #pragma's for traditional C -- Fix usage of strwinerror() in glib.c, change to gz_strwinerror() -- Correct email address in configure for system options -- Update make_vms.com and add make_vms.com to contrib/minizip [Zinser] -- Update zlib.map [Brown] -- Fix Makefile.in for Solaris 10 make of example64 and minizip64 [Torok] -- Apply various fixes to CMakeLists.txt [Lowman] -- Add checks on len in gzread() and gzwrite() -- Add error message for no more room for gzungetc() -- Remove zlib version check in gzwrite() -- Defer compression of gzprintf() result until need to -- Use snprintf() in gzdopen() if available -- Remove USE_MMAP configuration determination (only used by minigzip) -- Remove examples/pigz.c (available separately) -- Update examples/gun.c to 1.6 - -Changes in 1.2.3.5 (8 Jan 2010) -- Add space after #if in zutil.h for some compilers -- Fix relatively harmless bug in deflate_fast() [Exarevsky] -- Fix same problem in deflate_slow() -- Add $(SHAREDLIBV) to LIBS in Makefile.in [Brown] -- Add deflate_rle() for faster Z_RLE strategy run-length encoding -- Add deflate_huff() for faster Z_HUFFMAN_ONLY encoding -- Change name of "write" variable in inffast.c to avoid library collisions -- Fix premature EOF from gzread() in gzio.c [Brown] -- Use zlib header window size if windowBits is 0 in inflateInit2() -- Remove compressBound() call in deflate.c to avoid linking compress.o -- Replace use of errno in gz* with functions, support WinCE [Alves] -- Provide alternative to perror() in minigzip.c for WinCE [Alves] -- Don't use _vsnprintf on later versions of MSVC [Lowman] -- Add CMake build script and input file [Lowman] -- Update contrib/minizip to 1.1 [Svensson, Vollant] -- Moved nintendods directory from contrib to . -- Replace gzio.c with a new set of routines with the same functionality -- Add gzbuffer(), gzoffset(), gzclose_r(), gzclose_w() as part of above -- Update contrib/minizip to 1.1b -- Change gzeof() to return 0 on error instead of -1 to agree with zlib.h - -Changes in 1.2.3.4 (21 Dec 2009) -- Use old school .SUFFIXES in Makefile.in for FreeBSD compatibility -- Update comments in configure and Makefile.in for default --shared -- Fix test -z's in configure [Marquess] -- Build examplesh and minigzipsh when not testing -- Change NULL's to Z_NULL's in deflate.c and in comments in zlib.h -- Import LDFLAGS from the environment in configure -- Fix configure to populate SFLAGS with discovered CFLAGS options -- Adapt make_vms.com to the new Makefile.in [Zinser] -- Add zlib2ansi script for C++ compilation [Marquess] -- Add _FILE_OFFSET_BITS=64 test to make test (when applicable) -- Add AMD64 assembler code for longest match to contrib [Teterin] -- Include options from $SFLAGS when doing $LDSHARED -- Simplify 64-bit file support by introducing z_off64_t type -- Make shared object files in objs directory to work around old Sun cc -- Use only three-part version number for Darwin shared compiles -- Add rc option to ar in Makefile.in for when ./configure not run -- Add -WI,-rpath,. to LDFLAGS for OSF 1 V4* -- Set LD_LIBRARYN32_PATH for SGI IRIX shared compile -- Protect against _FILE_OFFSET_BITS being defined when compiling zlib -- Rename Makefile.in targets allstatic to static and allshared to shared -- Fix static and shared Makefile.in targets to be independent -- Correct error return bug in gz_open() by setting state [Brown] -- Put spaces before ;;'s in configure for better sh compatibility -- Add pigz.c (parallel implementation of gzip) to examples/ -- Correct constant in crc32.c to UL [Leventhal] -- Reject negative lengths in crc32_combine() -- Add inflateReset2() function to work like inflateEnd()/inflateInit2() -- Include sys/types.h for _LARGEFILE64_SOURCE [Brown] -- Correct typo in doc/algorithm.txt [Janik] -- Fix bug in adler32_combine() [Zhu] -- Catch missing-end-of-block-code error in all inflates and in puff - Assures that random input to inflate eventually results in an error -- Added enough.c (calculation of ENOUGH for inftrees.h) to examples/ -- Update ENOUGH and its usage to reflect discovered bounds -- Fix gzerror() error report on empty input file [Brown] -- Add ush casts in trees.c to avoid pedantic runtime errors -- Fix typo in zlib.h uncompress() description [Reiss] -- Correct inflate() comments with regard to automatic header detection -- Remove deprecation comment on Z_PARTIAL_FLUSH (it stays) -- Put new version of gzlog (2.0) in examples with interruption recovery -- Add puff compile option to permit invalid distance-too-far streams -- Add puff TEST command options, ability to read piped input -- Prototype the *64 functions in zlib.h when _FILE_OFFSET_BITS == 64, but - _LARGEFILE64_SOURCE not defined -- Fix Z_FULL_FLUSH to truly erase the past by resetting s->strstart -- Fix deflateSetDictionary() to use all 32K for output consistency -- Remove extraneous #define MIN_LOOKAHEAD in deflate.c (in deflate.h) -- Clear bytes after deflate lookahead to avoid use of uninitialized data -- Change a limit in inftrees.c to be more transparent to Coverity Prevent -- Update win32/zlib.def with exported symbols from zlib.h -- Correct spelling errors in zlib.h [Willem, Sobrado] -- Allow Z_BLOCK for deflate() to force a new block -- Allow negative bits in inflatePrime() to delete existing bit buffer -- Add Z_TREES flush option to inflate() to return at end of trees -- Add inflateMark() to return current state information for random access -- Add Makefile for NintendoDS to contrib [Costa] -- Add -w in configure compile tests to avoid spurious warnings [Beucler] -- Fix typos in zlib.h comments for deflateSetDictionary() -- Fix EOF detection in transparent gzread() [Maier] - -Changes in 1.2.3.3 (2 October 2006) -- Make --shared the default for configure, add a --static option -- Add compile option to permit invalid distance-too-far streams -- Add inflateUndermine() function which is required to enable above -- Remove use of "this" variable name for C++ compatibility [Marquess] -- Add testing of shared library in make test, if shared library built -- Use ftello() and fseeko() if available instead of ftell() and fseek() -- Provide two versions of all functions that use the z_off_t type for - binary compatibility -- a normal version and a 64-bit offset version, - per the Large File Support Extension when _LARGEFILE64_SOURCE is - defined; use the 64-bit versions by default when _FILE_OFFSET_BITS - is defined to be 64 -- Add a --uname= option to configure to perhaps help with cross-compiling - -Changes in 1.2.3.2 (3 September 2006) -- Turn off silly Borland warnings [Hay] -- Use off64_t and define _LARGEFILE64_SOURCE when present -- Fix missing dependency on inffixed.h in Makefile.in -- Rig configure --shared to build both shared and static [Teredesai, Truta] -- Remove zconf.in.h and instead create a new zlibdefs.h file -- Fix contrib/minizip/unzip.c non-encrypted after encrypted [Vollant] -- Add treebuild.xml (see http://treebuild.metux.de/) [Weigelt] - -Changes in 1.2.3.1 (16 August 2006) -- Add watcom directory with OpenWatcom make files [Daniel] -- Remove #undef of FAR in zconf.in.h for MVS [Fedtke] -- Update make_vms.com [Zinser] -- Use -fPIC for shared build in configure [Teredesai, Nicholson] -- Use only major version number for libz.so on IRIX and OSF1 [Reinholdtsen] -- Use fdopen() (not _fdopen()) for Interix in zutil.h [Bäck] -- Add some FAQ entries about the contrib directory -- Update the MVS question in the FAQ -- Avoid extraneous reads after EOF in gzio.c [Brown] -- Correct spelling of "successfully" in gzio.c [Randers-Pehrson] -- Add comments to zlib.h about gzerror() usage [Brown] -- Set extra flags in gzip header in gzopen() like deflate() does -- Make configure options more compatible with double-dash conventions - [Weigelt] -- Clean up compilation under Solaris SunStudio cc [Rowe, Reinholdtsen] -- Fix uninstall target in Makefile.in [Truta] -- Add pkgconfig support [Weigelt] -- Use $(DESTDIR) macro in Makefile.in [Reinholdtsen, Weigelt] -- Replace set_data_type() with a more accurate detect_data_type() in - trees.c, according to the txtvsbin.txt document [Truta] -- Swap the order of #include and #include "zlib.h" in - gzio.c, example.c and minigzip.c [Truta] -- Shut up annoying VS2005 warnings about standard C deprecation [Rowe, - Truta] (where?) -- Fix target "clean" from win32/Makefile.bor [Truta] -- Create .pdb and .manifest files in win32/makefile.msc [Ziegler, Rowe] -- Update zlib www home address in win32/DLL_FAQ.txt [Truta] -- Update contrib/masmx86/inffas32.asm for VS2005 [Vollant, Van Wassenhove] -- Enable browse info in the "Debug" and "ASM Debug" configurations in - the Visual C++ 6 project, and set (non-ASM) "Debug" as default [Truta] -- Add pkgconfig support [Weigelt] -- Add ZLIB_VER_MAJOR, ZLIB_VER_MINOR and ZLIB_VER_REVISION in zlib.h, - for use in win32/zlib1.rc [Polushin, Rowe, Truta] -- Add a document that explains the new text detection scheme to - doc/txtvsbin.txt [Truta] -- Add rfc1950.txt, rfc1951.txt and rfc1952.txt to doc/ [Truta] -- Move algorithm.txt into doc/ [Truta] -- Synchronize FAQ with website -- Fix compressBound(), was low for some pathological cases [Fearnley] -- Take into account wrapper variations in deflateBound() -- Set examples/zpipe.c input and output to binary mode for Windows -- Update examples/zlib_how.html with new zpipe.c (also web site) -- Fix some warnings in examples/gzlog.c and examples/zran.c (it seems - that gcc became pickier in 4.0) -- Add zlib.map for Linux: "All symbols from zlib-1.1.4 remain - un-versioned, the patch adds versioning only for symbols introduced in - zlib-1.2.0 or later. It also declares as local those symbols which are - not designed to be exported." [Levin] -- Update Z_PREFIX list in zconf.in.h, add --zprefix option to configure -- Do not initialize global static by default in trees.c, add a response - NO_INIT_GLOBAL_POINTERS to initialize them if needed [Marquess] -- Don't use strerror() in gzio.c under WinCE [Yakimov] -- Don't use errno.h in zutil.h under WinCE [Yakimov] -- Move arguments for AR to its usage to allow replacing ar [Marot] -- Add HAVE_VISIBILITY_PRAGMA in zconf.in.h for Mozilla [Randers-Pehrson] -- Improve inflateInit() and inflateInit2() documentation -- Fix structure size comment in inflate.h -- Change configure help option from --h* to --help [Santos] - -Changes in 1.2.3 (18 July 2005) -- Apply security vulnerability fixes to contrib/infback9 as well -- Clean up some text files (carriage returns, trailing space) -- Update testzlib, vstudio, masmx64, and masmx86 in contrib [Vollant] - -Changes in 1.2.2.4 (11 July 2005) -- Add inflatePrime() function for starting inflation at bit boundary -- Avoid some Visual C warnings in deflate.c -- Avoid more silly Visual C warnings in inflate.c and inftrees.c for 64-bit - compile -- Fix some spelling errors in comments [Betts] -- Correct inflateInit2() error return documentation in zlib.h -- Add zran.c example of compressed data random access to examples - directory, shows use of inflatePrime() -- Fix cast for assignments to strm->state in inflate.c and infback.c -- Fix zlibCompileFlags() in zutil.c to use 1L for long shifts [Oberhumer] -- Move declarations of gf2 functions to right place in crc32.c [Oberhumer] -- Add cast in trees.c t avoid a warning [Oberhumer] -- Avoid some warnings in fitblk.c, gun.c, gzjoin.c in examples [Oberhumer] -- Update make_vms.com [Zinser] -- Initialize state->write in inflateReset() since copied in inflate_fast() -- Be more strict on incomplete code sets in inflate_table() and increase - ENOUGH and MAXD -- this repairs a possible security vulnerability for - invalid inflate input. Thanks to Tavis Ormandy and Markus Oberhumer for - discovering the vulnerability and providing test cases. -- Add ia64 support to configure for HP-UX [Smith] -- Add error return to gzread() for format or i/o error [Levin] -- Use malloc.h for OS/2 [Necasek] - -Changes in 1.2.2.3 (27 May 2005) -- Replace 1U constants in inflate.c and inftrees.c for 64-bit compile -- Typecast fread() return values in gzio.c [Vollant] -- Remove trailing space in minigzip.c outmode (VC++ can't deal with it) -- Fix crc check bug in gzread() after gzungetc() [Heiner] -- Add the deflateTune() function to adjust internal compression parameters -- Add a fast gzip decompressor, gun.c, to examples (use of inflateBack) -- Remove an incorrect assertion in examples/zpipe.c -- Add C++ wrapper in infback9.h [Donais] -- Fix bug in inflateCopy() when decoding fixed codes -- Note in zlib.h how much deflateSetDictionary() actually uses -- Remove USE_DICT_HEAD in deflate.c (would mess up inflate if used) -- Add _WIN32_WCE to define WIN32 in zconf.in.h [Spencer] -- Don't include stderr.h or errno.h for _WIN32_WCE in zutil.h [Spencer] -- Add gzdirect() function to indicate transparent reads -- Update contrib/minizip [Vollant] -- Fix compilation of deflate.c when both ASMV and FASTEST [Oberhumer] -- Add casts in crc32.c to avoid warnings [Oberhumer] -- Add contrib/masmx64 [Vollant] -- Update contrib/asm586, asm686, masmx86, testzlib, vstudio [Vollant] - -Changes in 1.2.2.2 (30 December 2004) -- Replace structure assignments in deflate.c and inflate.c with zmemcpy to - avoid implicit memcpy calls (portability for no-library compilation) -- Increase sprintf() buffer size in gzdopen() to allow for large numbers -- Add INFLATE_STRICT to check distances against zlib header -- Improve WinCE errno handling and comments [Chang] -- Remove comment about no gzip header processing in FAQ -- Add Z_FIXED strategy option to deflateInit2() to force fixed trees -- Add updated make_vms.com [Coghlan], update README -- Create a new "examples" directory, move gzappend.c there, add zpipe.c, - fitblk.c, gzlog.[ch], gzjoin.c, and zlib_how.html. -- Add FAQ entry and comments in deflate.c on uninitialized memory access -- Add Solaris 9 make options in configure [Gilbert] -- Allow strerror() usage in gzio.c for STDC -- Fix DecompressBuf in contrib/delphi/ZLib.pas [ManChesTer] -- Update contrib/masmx86/inffas32.asm and gvmat32.asm [Vollant] -- Use z_off_t for adler32_combine() and crc32_combine() lengths -- Make adler32() much faster for small len -- Use OS_CODE in deflate() default gzip header - -Changes in 1.2.2.1 (31 October 2004) -- Allow inflateSetDictionary() call for raw inflate -- Fix inflate header crc check bug for file names and comments -- Add deflateSetHeader() and gz_header structure for custom gzip headers -- Add inflateGetheader() to retrieve gzip headers -- Add crc32_combine() and adler32_combine() functions -- Add alloc_func, free_func, in_func, out_func to Z_PREFIX list -- Use zstreamp consistently in zlib.h (inflate_back functions) -- Remove GUNZIP condition from definition of inflate_mode in inflate.h - and in contrib/inflate86/inffast.S [Truta, Anderson] -- Add support for AMD64 in contrib/inflate86/inffas86.c [Anderson] -- Update projects/README.projects and projects/visualc6 [Truta] -- Update win32/DLL_FAQ.txt [Truta] -- Avoid warning under NO_GZCOMPRESS in gzio.c; fix typo [Truta] -- Deprecate Z_ASCII; use Z_TEXT instead [Truta] -- Use a new algorithm for setting strm->data_type in trees.c [Truta] -- Do not define an exit() prototype in zutil.c unless DEBUG defined -- Remove prototype of exit() from zutil.c, example.c, minigzip.c [Truta] -- Add comment in zlib.h for Z_NO_FLUSH parameter to deflate() -- Fix Darwin build version identification [Peterson] - -Changes in 1.2.2 (3 October 2004) -- Update zlib.h comments on gzip in-memory processing -- Set adler to 1 in inflateReset() to support Java test suite [Walles] -- Add contrib/dotzlib [Ravn] -- Update win32/DLL_FAQ.txt [Truta] -- Update contrib/minizip [Vollant] -- Move contrib/visual-basic.txt to old/ [Truta] -- Fix assembler builds in projects/visualc6/ [Truta] - -Changes in 1.2.1.2 (9 September 2004) -- Update INDEX file -- Fix trees.c to update strm->data_type (no one ever noticed!) -- Fix bug in error case in inflate.c, infback.c, and infback9.c [Brown] -- Add "volatile" to crc table flag declaration (for DYNAMIC_CRC_TABLE) -- Add limited multitasking protection to DYNAMIC_CRC_TABLE -- Add NO_vsnprintf for VMS in zutil.h [Mozilla] -- Don't declare strerror() under VMS [Mozilla] -- Add comment to DYNAMIC_CRC_TABLE to use get_crc_table() to initialize -- Update contrib/ada [Anisimkov] -- Update contrib/minizip [Vollant] -- Fix configure to not hardcode directories for Darwin [Peterson] -- Fix gzio.c to not return error on empty files [Brown] -- Fix indentation; update version in contrib/delphi/ZLib.pas and - contrib/pascal/zlibpas.pas [Truta] -- Update mkasm.bat in contrib/masmx86 [Truta] -- Update contrib/untgz [Truta] -- Add projects/README.projects [Truta] -- Add project for MS Visual C++ 6.0 in projects/visualc6 [Cadieux, Truta] -- Update win32/DLL_FAQ.txt [Truta] -- Update list of Z_PREFIX symbols in zconf.h [Randers-Pehrson, Truta] -- Remove an unnecessary assignment to curr in inftrees.c [Truta] -- Add OS/2 to exe builds in configure [Poltorak] -- Remove err dummy parameter in zlib.h [Kientzle] - -Changes in 1.2.1.1 (9 January 2004) -- Update email address in README -- Several FAQ updates -- Fix a big fat bug in inftrees.c that prevented decoding valid - dynamic blocks with only literals and no distance codes -- - Thanks to "Hot Emu" for the bug report and sample file -- Add a note to puff.c on no distance codes case. - -Changes in 1.2.1 (17 November 2003) -- Remove a tab in contrib/gzappend/gzappend.c -- Update some interfaces in contrib for new zlib functions -- Update zlib version number in some contrib entries -- Add Windows CE definition for ptrdiff_t in zutil.h [Mai, Truta] -- Support shared libraries on Hurd and KFreeBSD [Brown] -- Fix error in NO_DIVIDE option of adler32.c - -Changes in 1.2.0.8 (4 November 2003) -- Update version in contrib/delphi/ZLib.pas and contrib/pascal/zlibpas.pas -- Add experimental NO_DIVIDE #define in adler32.c - - Possibly faster on some processors (let me know if it is) -- Correct Z_BLOCK to not return on first inflate call if no wrap -- Fix strm->data_type on inflate() return to correctly indicate EOB -- Add deflatePrime() function for appending in the middle of a byte -- Add contrib/gzappend for an example of appending to a stream -- Update win32/DLL_FAQ.txt [Truta] -- Delete Turbo C comment in README [Truta] -- Improve some indentation in zconf.h [Truta] -- Fix infinite loop on bad input in configure script [Church] -- Fix gzeof() for concatenated gzip files [Johnson] -- Add example to contrib/visual-basic.txt [Michael B.] -- Add -p to mkdir's in Makefile.in [vda] -- Fix configure to properly detect presence or lack of printf functions -- Add AS400 support [Monnerat] -- Add a little Cygwin support [Wilson] - -Changes in 1.2.0.7 (21 September 2003) -- Correct some debug formats in contrib/infback9 -- Cast a type in a debug statement in trees.c -- Change search and replace delimiter in configure from % to # [Beebe] -- Update contrib/untgz to 0.2 with various fixes [Truta] -- Add build support for Amiga [Nikl] -- Remove some directories in old that have been updated to 1.2 -- Add dylib building for Mac OS X in configure and Makefile.in -- Remove old distribution stuff from Makefile -- Update README to point to DLL_FAQ.txt, and add comment on Mac OS X -- Update links in README - -Changes in 1.2.0.6 (13 September 2003) -- Minor FAQ updates -- Update contrib/minizip to 1.00 [Vollant] -- Remove test of gz functions in example.c when GZ_COMPRESS defined [Truta] -- Update POSTINC comment for 68060 [Nikl] -- Add contrib/infback9 with deflate64 decoding (unsupported) -- For MVS define NO_vsnprintf and undefine FAR [van Burik] -- Add pragma for fdopen on MVS [van Burik] - -Changes in 1.2.0.5 (8 September 2003) -- Add OF to inflateBackEnd() declaration in zlib.h -- Remember start when using gzdopen in the middle of a file -- Use internal off_t counters in gz* functions to properly handle seeks -- Perform more rigorous check for distance-too-far in inffast.c -- Add Z_BLOCK flush option to return from inflate at block boundary -- Set strm->data_type on return from inflate - - Indicate bits unused, if at block boundary, and if in last block -- Replace size_t with ptrdiff_t in crc32.c, and check for correct size -- Add condition so old NO_DEFLATE define still works for compatibility -- FAQ update regarding the Windows DLL [Truta] -- INDEX update: add qnx entry, remove aix entry [Truta] -- Install zlib.3 into mandir [Wilson] -- Move contrib/zlib_dll_FAQ.txt to win32/DLL_FAQ.txt; update [Truta] -- Adapt the zlib interface to the new DLL convention guidelines [Truta] -- Introduce ZLIB_WINAPI macro to allow the export of functions using - the WINAPI calling convention, for Visual Basic [Vollant, Truta] -- Update msdos and win32 scripts and makefiles [Truta] -- Export symbols by name, not by ordinal, in win32/zlib.def [Truta] -- Add contrib/ada [Anisimkov] -- Move asm files from contrib/vstudio/vc70_32 to contrib/asm386 [Truta] -- Rename contrib/asm386 to contrib/masmx86 [Truta, Vollant] -- Add contrib/masm686 [Truta] -- Fix offsets in contrib/inflate86 and contrib/masmx86/inffas32.asm - [Truta, Vollant] -- Update contrib/delphi; rename to contrib/pascal; add example [Truta] -- Remove contrib/delphi2; add a new contrib/delphi [Truta] -- Avoid inclusion of the nonstandard in contrib/iostream, - and fix some method prototypes [Truta] -- Fix the ZCR_SEED2 constant to avoid warnings in contrib/minizip - [Truta] -- Avoid the use of backslash (\) in contrib/minizip [Vollant] -- Fix file time handling in contrib/untgz; update makefiles [Truta] -- Update contrib/vstudio/vc70_32 to comply with the new DLL guidelines - [Vollant] -- Remove contrib/vstudio/vc15_16 [Vollant] -- Rename contrib/vstudio/vc70_32 to contrib/vstudio/vc7 [Truta] -- Update README.contrib [Truta] -- Invert the assignment order of match_head and s->prev[...] in - INSERT_STRING [Truta] -- Compare TOO_FAR with 32767 instead of 32768, to avoid 16-bit warnings - [Truta] -- Compare function pointers with 0, not with NULL or Z_NULL [Truta] -- Fix prototype of syncsearch in inflate.c [Truta] -- Introduce ASMINF macro to be enabled when using an ASM implementation - of inflate_fast [Truta] -- Change NO_DEFLATE to NO_GZCOMPRESS [Truta] -- Modify test_gzio in example.c to take a single file name as a - parameter [Truta] -- Exit the example.c program if gzopen fails [Truta] -- Add type casts around strlen in example.c [Truta] -- Remove casting to sizeof in minigzip.c; give a proper type - to the variable compared with SUFFIX_LEN [Truta] -- Update definitions of STDC and STDC99 in zconf.h [Truta] -- Synchronize zconf.h with the new Windows DLL interface [Truta] -- Use SYS16BIT instead of __32BIT__ to distinguish between - 16- and 32-bit platforms [Truta] -- Use far memory allocators in small 16-bit memory models for - Turbo C [Truta] -- Add info about the use of ASMV, ASMINF and ZLIB_WINAPI in - zlibCompileFlags [Truta] -- Cygwin has vsnprintf [Wilson] -- In Windows16, OS_CODE is 0, as in MSDOS [Truta] -- In Cygwin, OS_CODE is 3 (Unix), not 11 (Windows32) [Wilson] - -Changes in 1.2.0.4 (10 August 2003) -- Minor FAQ updates -- Be more strict when checking inflateInit2's windowBits parameter -- Change NO_GUNZIP compile option to NO_GZIP to cover deflate as well -- Add gzip wrapper option to deflateInit2 using windowBits -- Add updated QNX rule in configure and qnx directory [Bonnefoy] -- Make inflate distance-too-far checks more rigorous -- Clean up FAR usage in inflate -- Add casting to sizeof() in gzio.c and minigzip.c - -Changes in 1.2.0.3 (19 July 2003) -- Fix silly error in gzungetc() implementation [Vollant] -- Update contrib/minizip and contrib/vstudio [Vollant] -- Fix printf format in example.c -- Correct cdecl support in zconf.in.h [Anisimkov] -- Minor FAQ updates - -Changes in 1.2.0.2 (13 July 2003) -- Add ZLIB_VERNUM in zlib.h for numerical preprocessor comparisons -- Attempt to avoid warnings in crc32.c for pointer-int conversion -- Add AIX to configure, remove aix directory [Bakker] -- Add some casts to minigzip.c -- Improve checking after insecure sprintf() or vsprintf() calls -- Remove #elif's from crc32.c -- Change leave label to inf_leave in inflate.c and infback.c to avoid - library conflicts -- Remove inflate gzip decoding by default--only enable gzip decoding by - special request for stricter backward compatibility -- Add zlibCompileFlags() function to return compilation information -- More typecasting in deflate.c to avoid warnings -- Remove leading underscore from _Capital #defines [Truta] -- Fix configure to link shared library when testing -- Add some Windows CE target adjustments [Mai] -- Remove #define ZLIB_DLL in zconf.h [Vollant] -- Add zlib.3 [Rodgers] -- Update RFC URL in deflate.c and algorithm.txt [Mai] -- Add zlib_dll_FAQ.txt to contrib [Truta] -- Add UL to some constants [Truta] -- Update minizip and vstudio [Vollant] -- Remove vestigial NEED_DUMMY_RETURN from zconf.in.h -- Expand use of NO_DUMMY_DECL to avoid all dummy structures -- Added iostream3 to contrib [Schwardt] -- Replace rewind() with fseek() for WinCE [Truta] -- Improve setting of zlib format compression level flags - - Report 0 for huffman and rle strategies and for level == 0 or 1 - - Report 2 only for level == 6 -- Only deal with 64K limit when necessary at compile time [Truta] -- Allow TOO_FAR check to be turned off at compile time [Truta] -- Add gzclearerr() function [Souza] -- Add gzungetc() function - -Changes in 1.2.0.1 (17 March 2003) -- Add Z_RLE strategy for run-length encoding [Truta] - - When Z_RLE requested, restrict matches to distance one - - Update zlib.h, minigzip.c, gzopen(), gzdopen() for Z_RLE -- Correct FASTEST compilation to allow level == 0 -- Clean up what gets compiled for FASTEST -- Incorporate changes to zconf.in.h [Vollant] - - Refine detection of Turbo C need for dummy returns - - Refine ZLIB_DLL compilation - - Include additional header file on VMS for off_t typedef -- Try to use _vsnprintf where it supplants vsprintf [Vollant] -- Add some casts in inffast.c -- Enchance comments in zlib.h on what happens if gzprintf() tries to - write more than 4095 bytes before compression -- Remove unused state from inflateBackEnd() -- Remove exit(0) from minigzip.c, example.c -- Get rid of all those darn tabs -- Add "check" target to Makefile.in that does the same thing as "test" -- Add "mostlyclean" and "maintainer-clean" targets to Makefile.in -- Update contrib/inflate86 [Anderson] -- Update contrib/testzlib, contrib/vstudio, contrib/minizip [Vollant] -- Add msdos and win32 directories with makefiles [Truta] -- More additions and improvements to the FAQ - -Changes in 1.2.0 (9 March 2003) -- New and improved inflate code - - About 20% faster - - Does not allocate 32K window unless and until needed - - Automatically detects and decompresses gzip streams - - Raw inflate no longer needs an extra dummy byte at end - - Added inflateBack functions using a callback interface--even faster - than inflate, useful for file utilities (gzip, zip) - - Added inflateCopy() function to record state for random access on - externally generated deflate streams (e.g. in gzip files) - - More readable code (I hope) -- New and improved crc32() - - About 50% faster, thanks to suggestions from Rodney Brown -- Add deflateBound() and compressBound() functions -- Fix memory leak in deflateInit2() -- Permit setting dictionary for raw deflate (for parallel deflate) -- Fix const declaration for gzwrite() -- Check for some malloc() failures in gzio.c -- Fix bug in gzopen() on single-byte file 0x1f -- Fix bug in gzread() on concatenated file with 0x1f at end of buffer - and next buffer doesn't start with 0x8b -- Fix uncompress() to return Z_DATA_ERROR on truncated input -- Free memory at end of example.c -- Remove MAX #define in trees.c (conflicted with some libraries) -- Fix static const's in deflate.c, gzio.c, and zutil.[ch] -- Declare malloc() and free() in gzio.c if STDC not defined -- Use malloc() instead of calloc() in zutil.c if int big enough -- Define STDC for AIX -- Add aix/ with approach for compiling shared library on AIX -- Add HP-UX support for shared libraries in configure -- Add OpenUNIX support for shared libraries in configure -- Use $cc instead of gcc to build shared library -- Make prefix directory if needed when installing -- Correct Macintosh avoidance of typedef Byte in zconf.h -- Correct Turbo C memory allocation when under Linux -- Use libz.a instead of -lz in Makefile (assure use of compiled library) -- Update configure to check for snprintf or vsnprintf functions and their - return value, warn during make if using an insecure function -- Fix configure problem with compile-time knowledge of HAVE_UNISTD_H that - is lost when library is used--resolution is to build new zconf.h -- Documentation improvements (in zlib.h): - - Document raw deflate and inflate - - Update RFCs URL - - Point out that zlib and gzip formats are different - - Note that Z_BUF_ERROR is not fatal - - Document string limit for gzprintf() and possible buffer overflow - - Note requirement on avail_out when flushing - - Note permitted values of flush parameter of inflate() -- Add some FAQs (and even answers) to the FAQ -- Add contrib/inflate86/ for x86 faster inflate -- Add contrib/blast/ for PKWare Data Compression Library decompression -- Add contrib/puff/ simple inflate for deflate format description - -Changes in 1.1.4 (11 March 2002) -- ZFREE was repeated on same allocation on some error conditions. - This creates a security problem described in - http://www.zlib.org/advisory-2002-03-11.txt -- Returned incorrect error (Z_MEM_ERROR) on some invalid data -- Avoid accesses before window for invalid distances with inflate window - less than 32K. -- force windowBits > 8 to avoid a bug in the encoder for a window size - of 256 bytes. (A complete fix will be available in 1.1.5). - -Changes in 1.1.3 (9 July 1998) -- fix "an inflate input buffer bug that shows up on rare but persistent - occasions" (Mark) -- fix gzread and gztell for concatenated .gz files (Didier Le Botlan) -- fix gzseek(..., SEEK_SET) in write mode -- fix crc check after a gzeek (Frank Faubert) -- fix miniunzip when the last entry in a zip file is itself a zip file - (J Lillge) -- add contrib/asm586 and contrib/asm686 (Brian Raiter) - See http://www.muppetlabs.com/~breadbox/software/assembly.html -- add support for Delphi 3 in contrib/delphi (Bob Dellaca) -- add support for C++Builder 3 and Delphi 3 in contrib/delphi2 (Davide Moretti) -- do not exit prematurely in untgz if 0 at start of block (Magnus Holmgren) -- use macro EXTERN instead of extern to support DLL for BeOS (Sander Stoks) -- added a FAQ file - -- Support gzdopen on Mac with Metrowerks (Jason Linhart) -- Do not redefine Byte on Mac (Brad Pettit & Jason Linhart) -- define SEEK_END too if SEEK_SET is not defined (Albert Chin-A-Young) -- avoid some warnings with Borland C (Tom Tanner) -- fix a problem in contrib/minizip/zip.c for 16-bit MSDOS (Gilles Vollant) -- emulate utime() for WIN32 in contrib/untgz (Gilles Vollant) -- allow several arguments to configure (Tim Mooney, Frodo Looijaard) -- use libdir and includedir in Makefile.in (Tim Mooney) -- support shared libraries on OSF1 V4 (Tim Mooney) -- remove so_locations in "make clean" (Tim Mooney) -- fix maketree.c compilation error (Glenn, Mark) -- Python interface to zlib now in Python 1.5 (Jeremy Hylton) -- new Makefile.riscos (Rich Walker) -- initialize static descriptors in trees.c for embedded targets (Nick Smith) -- use "foo-gz" in example.c for RISCOS and VMS (Nick Smith) -- add the OS/2 files in Makefile.in too (Andrew Zabolotny) -- fix fdopen and halloc macros for Microsoft C 6.0 (Tom Lane) -- fix maketree.c to allow clean compilation of inffixed.h (Mark) -- fix parameter check in deflateCopy (Gunther Nikl) -- cleanup trees.c, use compressed_len only in debug mode (Christian Spieler) -- Many portability patches by Christian Spieler: - . zutil.c, zutil.h: added "const" for zmem* - . Make_vms.com: fixed some typos - . Make_vms.com: msdos/Makefile.*: removed zutil.h from some dependency lists - . msdos/Makefile.msc: remove "default rtl link library" info from obj files - . msdos/Makefile.*: use model-dependent name for the built zlib library - . msdos/Makefile.emx, nt/Makefile.emx, nt/Makefile.gcc: - new makefiles, for emx (DOS/OS2), emx&rsxnt and mingw32 (Windows 9x / NT) -- use define instead of typedef for Bytef also for MSC small/medium (Tom Lane) -- replace __far with _far for better portability (Christian Spieler, Tom Lane) -- fix test for errno.h in configure (Tim Newsham) - -Changes in 1.1.2 (19 March 98) -- added contrib/minzip, mini zip and unzip based on zlib (Gilles Vollant) - See http://www.winimage.com/zLibDll/unzip.html -- preinitialize the inflate tables for fixed codes, to make the code - completely thread safe (Mark) -- some simplifications and slight speed-up to the inflate code (Mark) -- fix gzeof on non-compressed files (Allan Schrum) -- add -std1 option in configure for OSF1 to fix gzprintf (Martin Mokrejs) -- use default value of 4K for Z_BUFSIZE for 16-bit MSDOS (Tim Wegner + Glenn) -- added os2/Makefile.def and os2/zlib.def (Andrew Zabolotny) -- add shared lib support for UNIX_SV4.2MP (MATSUURA Takanori) -- do not wrap extern "C" around system includes (Tom Lane) -- mention zlib binding for TCL in README (Andreas Kupries) -- added amiga/Makefile.pup for Amiga powerUP SAS/C PPC (Andreas Kleinert) -- allow "make install prefix=..." even after configure (Glenn Randers-Pehrson) -- allow "configure --prefix $HOME" (Tim Mooney) -- remove warnings in example.c and gzio.c (Glenn Randers-Pehrson) -- move Makefile.sas to amiga/Makefile.sas - -Changes in 1.1.1 (27 Feb 98) -- fix macros _tr_tally_* in deflate.h for debug mode (Glenn Randers-Pehrson) -- remove block truncation heuristic which had very marginal effect for zlib - (smaller lit_bufsize than in gzip 1.2.4) and degraded a little the - compression ratio on some files. This also allows inlining _tr_tally for - matches in deflate_slow. -- added msdos/Makefile.w32 for WIN32 Microsoft Visual C++ (Bob Frazier) - -Changes in 1.1.0 (24 Feb 98) -- do not return STREAM_END prematurely in inflate (John Bowler) -- revert to the zlib 1.0.8 inflate to avoid the gcc 2.8.0 bug (Jeremy Buhler) -- compile with -DFASTEST to get compression code optimized for speed only -- in minigzip, try mmap'ing the input file first (Miguel Albrecht) -- increase size of I/O buffers in minigzip.c and gzio.c (not a big gain - on Sun but significant on HP) - -- add a pointer to experimental unzip library in README (Gilles Vollant) -- initialize variable gcc in configure (Chris Herborth) - -Changes in 1.0.9 (17 Feb 1998) -- added gzputs and gzgets functions -- do not clear eof flag in gzseek (Mark Diekhans) -- fix gzseek for files in transparent mode (Mark Diekhans) -- do not assume that vsprintf returns the number of bytes written (Jens Krinke) -- replace EXPORT with ZEXPORT to avoid conflict with other programs -- added compress2 in zconf.h, zlib.def, zlib.dnt -- new asm code from Gilles Vollant in contrib/asm386 -- simplify the inflate code (Mark): - . Replace ZALLOC's in huft_build() with single ZALLOC in inflate_blocks_new() - . ZALLOC the length list in inflate_trees_fixed() instead of using stack - . ZALLOC the value area for huft_build() instead of using stack - . Simplify Z_FINISH check in inflate() - -- Avoid gcc 2.8.0 comparison bug a little differently than zlib 1.0.8 -- in inftrees.c, avoid cc -O bug on HP (Farshid Elahi) -- in zconf.h move the ZLIB_DLL stuff earlier to avoid problems with - the declaration of FAR (Gilles VOllant) -- install libz.so* with mode 755 (executable) instead of 644 (Marc Lehmann) -- read_buf buf parameter of type Bytef* instead of charf* -- zmemcpy parameters are of type Bytef*, not charf* (Joseph Strout) -- do not redeclare unlink in minigzip.c for WIN32 (John Bowler) -- fix check for presence of directories in "make install" (Ian Willis) - -Changes in 1.0.8 (27 Jan 1998) -- fixed offsets in contrib/asm386/gvmat32.asm (Gilles Vollant) -- fix gzgetc and gzputc for big endian systems (Markus Oberhumer) -- added compress2() to allow setting the compression level -- include sys/types.h to get off_t on some systems (Marc Lehmann & QingLong) -- use constant arrays for the static trees in trees.c instead of computing - them at run time (thanks to Ken Raeburn for this suggestion). To create - trees.h, compile with GEN_TREES_H and run "make test". -- check return code of example in "make test" and display result -- pass minigzip command line options to file_compress -- simplifying code of inflateSync to avoid gcc 2.8 bug - -- support CC="gcc -Wall" in configure -s (QingLong) -- avoid a flush caused by ftell in gzopen for write mode (Ken Raeburn) -- fix test for shared library support to avoid compiler warnings -- zlib.lib -> zlib.dll in msdos/zlib.rc (Gilles Vollant) -- check for TARGET_OS_MAC in addition to MACOS (Brad Pettit) -- do not use fdopen for Metrowerks on Mac (Brad Pettit)) -- add checks for gzputc and gzputc in example.c -- avoid warnings in gzio.c and deflate.c (Andreas Kleinert) -- use const for the CRC table (Ken Raeburn) -- fixed "make uninstall" for shared libraries -- use Tracev instead of Trace in infblock.c -- in example.c use correct compressed length for test_sync -- suppress +vnocompatwarnings in configure for HPUX (not always supported) - -Changes in 1.0.7 (20 Jan 1998) -- fix gzseek which was broken in write mode -- return error for gzseek to negative absolute position -- fix configure for Linux (Chun-Chung Chen) -- increase stack space for MSC (Tim Wegner) -- get_crc_table and inflateSyncPoint are EXPORTed (Gilles Vollant) -- define EXPORTVA for gzprintf (Gilles Vollant) -- added man page zlib.3 (Rick Rodgers) -- for contrib/untgz, fix makedir() and improve Makefile - -- check gzseek in write mode in example.c -- allocate extra buffer for seeks only if gzseek is actually called -- avoid signed/unsigned comparisons (Tim Wegner, Gilles Vollant) -- add inflateSyncPoint in zconf.h -- fix list of exported functions in nt/zlib.dnt and mdsos/zlib.def - -Changes in 1.0.6 (19 Jan 1998) -- add functions gzprintf, gzputc, gzgetc, gztell, gzeof, gzseek, gzrewind and - gzsetparams (thanks to Roland Giersig and Kevin Ruland for some of this code) -- Fix a deflate bug occurring only with compression level 0 (thanks to - Andy Buckler for finding this one). -- In minigzip, pass transparently also the first byte for .Z files. -- return Z_BUF_ERROR instead of Z_OK if output buffer full in uncompress() -- check Z_FINISH in inflate (thanks to Marc Schluper) -- Implement deflateCopy (thanks to Adam Costello) -- make static libraries by default in configure, add --shared option. -- move MSDOS or Windows specific files to directory msdos -- suppress the notion of partial flush to simplify the interface - (but the symbol Z_PARTIAL_FLUSH is kept for compatibility with 1.0.4) -- suppress history buffer provided by application to simplify the interface - (this feature was not implemented anyway in 1.0.4) -- next_in and avail_in must be initialized before calling inflateInit or - inflateInit2 -- add EXPORT in all exported functions (for Windows DLL) -- added Makefile.nt (thanks to Stephen Williams) -- added the unsupported "contrib" directory: - contrib/asm386/ by Gilles Vollant - 386 asm code replacing longest_match(). - contrib/iostream/ by Kevin Ruland - A C++ I/O streams interface to the zlib gz* functions - contrib/iostream2/ by Tyge Løvset - Another C++ I/O streams interface - contrib/untgz/ by "Pedro A. Aranda Guti\irrez" - A very simple tar.gz file extractor using zlib - contrib/visual-basic.txt by Carlos Rios - How to use compress(), uncompress() and the gz* functions from VB. -- pass params -f (filtered data), -h (huffman only), -1 to -9 (compression - level) in minigzip (thanks to Tom Lane) - -- use const for rommable constants in deflate -- added test for gzseek and gztell in example.c -- add undocumented function inflateSyncPoint() (hack for Paul Mackerras) -- add undocumented function zError to convert error code to string - (for Tim Smithers) -- Allow compilation of gzio with -DNO_DEFLATE to avoid the compression code. -- Use default memcpy for Symantec MSDOS compiler. -- Add EXPORT keyword for check_func (needed for Windows DLL) -- add current directory to LD_LIBRARY_PATH for "make test" -- create also a link for libz.so.1 -- added support for FUJITSU UXP/DS (thanks to Toshiaki Nomura) -- use $(SHAREDLIB) instead of libz.so in Makefile.in (for HPUX) -- added -soname for Linux in configure (Chun-Chung Chen, -- assign numbers to the exported functions in zlib.def (for Windows DLL) -- add advice in zlib.h for best usage of deflateSetDictionary -- work around compiler bug on Atari (cast Z_NULL in call of s->checkfn) -- allow compilation with ANSI keywords only enabled for TurboC in large model -- avoid "versionString"[0] (Borland bug) -- add NEED_DUMMY_RETURN for Borland -- use variable z_verbose for tracing in debug mode (L. Peter Deutsch). -- allow compilation with CC -- defined STDC for OS/2 (David Charlap) -- limit external names to 8 chars for MVS (Thomas Lund) -- in minigzip.c, use static buffers only for 16-bit systems -- fix suffix check for "minigzip -d foo.gz" -- do not return an error for the 2nd of two consecutive gzflush() (Felix Lee) -- use _fdopen instead of fdopen for MSC >= 6.0 (Thomas Fanslau) -- added makelcc.bat for lcc-win32 (Tom St Denis) -- in Makefile.dj2, use copy and del instead of install and rm (Frank Donahoe) -- Avoid expanded $Id$. Use "rcs -kb" or "cvs admin -kb" to avoid Id expansion. -- check for unistd.h in configure (for off_t) -- remove useless check parameter in inflate_blocks_free -- avoid useless assignment of s->check to itself in inflate_blocks_new -- do not flush twice in gzclose (thanks to Ken Raeburn) -- rename FOPEN as F_OPEN to avoid clash with /usr/include/sys/file.h -- use NO_ERRNO_H instead of enumeration of operating systems with errno.h -- work around buggy fclose on pipes for HP/UX -- support zlib DLL with BORLAND C++ 5.0 (thanks to Glenn Randers-Pehrson) -- fix configure if CC is already equal to gcc - -Changes in 1.0.5 (3 Jan 98) -- Fix inflate to terminate gracefully when fed corrupted or invalid data -- Use const for rommable constants in inflate -- Eliminate memory leaks on error conditions in inflate -- Removed some vestigial code in inflate -- Update web address in README - -Changes in 1.0.4 (24 Jul 96) -- In very rare conditions, deflate(s, Z_FINISH) could fail to produce an EOF - bit, so the decompressor could decompress all the correct data but went - on to attempt decompressing extra garbage data. This affected minigzip too. -- zlibVersion and gzerror return const char* (needed for DLL) -- port to RISCOS (no fdopen, no multiple dots, no unlink, no fileno) -- use z_error only for DEBUG (avoid problem with DLLs) - -Changes in 1.0.3 (2 Jul 96) -- use z_streamp instead of z_stream *, which is now a far pointer in MSDOS - small and medium models; this makes the library incompatible with previous - versions for these models. (No effect in large model or on other systems.) -- return OK instead of BUF_ERROR if previous deflate call returned with - avail_out as zero but there is nothing to do -- added memcmp for non STDC compilers -- define NO_DUMMY_DECL for more Mac compilers (.h files merged incorrectly) -- define __32BIT__ if __386__ or i386 is defined (pb. with Watcom and SCO) -- better check for 16-bit mode MSC (avoids problem with Symantec) - -Changes in 1.0.2 (23 May 96) -- added Windows DLL support -- added a function zlibVersion (for the DLL support) -- fixed declarations using Bytef in infutil.c (pb with MSDOS medium model) -- Bytef is define's instead of typedef'd only for Borland C -- avoid reading uninitialized memory in example.c -- mention in README that the zlib format is now RFC1950 -- updated Makefile.dj2 -- added algorithm.doc - -Changes in 1.0.1 (20 May 96) [1.0 skipped to avoid confusion] -- fix array overlay in deflate.c which sometimes caused bad compressed data -- fix inflate bug with empty stored block -- fix MSDOS medium model which was broken in 0.99 -- fix deflateParams() which could generated bad compressed data. -- Bytef is define'd instead of typedef'ed (work around Borland bug) -- added an INDEX file -- new makefiles for DJGPP (Makefile.dj2), 32-bit Borland (Makefile.b32), - Watcom (Makefile.wat), Amiga SAS/C (Makefile.sas) -- speed up adler32 for modern machines without auto-increment -- added -ansi for IRIX in configure -- static_init_done in trees.c is an int -- define unlink as delete for VMS -- fix configure for QNX -- add configure branch for SCO and HPUX -- avoid many warnings (unused variables, dead assignments, etc...) -- no fdopen for BeOS -- fix the Watcom fix for 32 bit mode (define FAR as empty) -- removed redefinition of Byte for MKWERKS -- work around an MWKERKS bug (incorrect merge of all .h files) - -Changes in 0.99 (27 Jan 96) -- allow preset dictionary shared between compressor and decompressor -- allow compression level 0 (no compression) -- add deflateParams in zlib.h: allow dynamic change of compression level - and compression strategy. -- test large buffers and deflateParams in example.c -- add optional "configure" to build zlib as a shared library -- suppress Makefile.qnx, use configure instead -- fixed deflate for 64-bit systems (detected on Cray) -- fixed inflate_blocks for 64-bit systems (detected on Alpha) -- declare Z_DEFLATED in zlib.h (possible parameter for deflateInit2) -- always return Z_BUF_ERROR when deflate() has nothing to do -- deflateInit and inflateInit are now macros to allow version checking -- prefix all global functions and types with z_ with -DZ_PREFIX -- make falloc completely reentrant (inftrees.c) -- fixed very unlikely race condition in ct_static_init -- free in reverse order of allocation to help memory manager -- use zlib-1.0/* instead of zlib/* inside the tar.gz -- make zlib warning-free with "gcc -O3 -Wall -Wwrite-strings -Wpointer-arith - -Wconversion -Wstrict-prototypes -Wmissing-prototypes" -- allow gzread on concatenated .gz files -- deflateEnd now returns Z_DATA_ERROR if it was premature -- deflate is finally (?) fully deterministic (no matches beyond end of input) -- Document Z_SYNC_FLUSH -- add uninstall in Makefile -- Check for __cpluplus in zlib.h -- Better test in ct_align for partial flush -- avoid harmless warnings for Borland C++ -- initialize hash_head in deflate.c -- avoid warning on fdopen (gzio.c) for HP cc -Aa -- include stdlib.h for STDC compilers -- include errno.h for Cray -- ignore error if ranlib doesn't exist -- call ranlib twice for NeXTSTEP -- use exec_prefix instead of prefix for libz.a -- renamed ct_* as _tr_* to avoid conflict with applications -- clear z->msg in inflateInit2 before any error return -- initialize opaque in example.c, gzio.c, deflate.c and inflate.c -- fixed typo in zconf.h (_GNUC__ => __GNUC__) -- check for WIN32 in zconf.h and zutil.c (avoid farmalloc in 32-bit mode) -- fix typo in Make_vms.com (f$trnlnm -> f$getsyi) -- in fcalloc, normalize pointer if size > 65520 bytes -- don't use special fcalloc for 32 bit Borland C++ -- use STDC instead of __GO32__ to avoid redeclaring exit, calloc, etc... -- use Z_BINARY instead of BINARY -- document that gzclose after gzdopen will close the file -- allow "a" as mode in gzopen. -- fix error checking in gzread -- allow skipping .gz extra-field on pipes -- added reference to Perl interface in README -- put the crc table in FAR data (I dislike more and more the medium model :) -- added get_crc_table -- added a dimension to all arrays (Borland C can't count). -- workaround Borland C bug in declaration of inflate_codes_new & inflate_fast -- guard against multiple inclusion of *.h (for precompiled header on Mac) -- Watcom C pretends to be Microsoft C small model even in 32 bit mode. -- don't use unsized arrays to avoid silly warnings by Visual C++: - warning C4746: 'inflate_mask' : unsized array treated as '__far' - (what's wrong with far data in far model?). -- define enum out of inflate_blocks_state to allow compilation with C++ - -Changes in 0.95 (16 Aug 95) -- fix MSDOS small and medium model (now easier to adapt to any compiler) -- inlined send_bits -- fix the final (:-) bug for deflate with flush (output was correct but - not completely flushed in rare occasions). -- default window size is same for compression and decompression - (it's now sufficient to set MAX_WBITS in zconf.h). -- voidp -> voidpf and voidnp -> voidp (for consistency with other - typedefs and because voidnp was not near in large model). - -Changes in 0.94 (13 Aug 95) -- support MSDOS medium model -- fix deflate with flush (could sometimes generate bad output) -- fix deflateReset (zlib header was incorrectly suppressed) -- added support for VMS -- allow a compression level in gzopen() -- gzflush now calls fflush -- For deflate with flush, flush even if no more input is provided. -- rename libgz.a as libz.a -- avoid complex expression in infcodes.c triggering Turbo C bug -- work around a problem with gcc on Alpha (in INSERT_STRING) -- don't use inline functions (problem with some gcc versions) -- allow renaming of Byte, uInt, etc... with #define. -- avoid warning about (unused) pointer before start of array in deflate.c -- avoid various warnings in gzio.c, example.c, infblock.c, adler32.c, zutil.c -- avoid reserved word 'new' in trees.c - -Changes in 0.93 (25 June 95) -- temporarily disable inline functions -- make deflate deterministic -- give enough lookahead for PARTIAL_FLUSH -- Set binary mode for stdin/stdout in minigzip.c for OS/2 -- don't even use signed char in inflate (not portable enough) -- fix inflate memory leak for segmented architectures - -Changes in 0.92 (3 May 95) -- don't assume that char is signed (problem on SGI) -- Clear bit buffer when starting a stored block -- no memcpy on Pyramid -- suppressed inftest.c -- optimized fill_window, put longest_match inline for gcc -- optimized inflate on stored blocks. -- untabify all sources to simplify patches - -Changes in 0.91 (2 May 95) -- Default MEM_LEVEL is 8 (not 9 for Unix) as documented in zlib.h -- Document the memory requirements in zconf.h -- added "make install" -- fix sync search logic in inflateSync -- deflate(Z_FULL_FLUSH) now works even if output buffer too short -- after inflateSync, don't scare people with just "lo world" -- added support for DJGPP - -Changes in 0.9 (1 May 95) -- don't assume that zalloc clears the allocated memory (the TurboC bug - was Mark's bug after all :) -- let again gzread copy uncompressed data unchanged (was working in 0.71) -- deflate(Z_FULL_FLUSH), inflateReset and inflateSync are now fully implemented -- added a test of inflateSync in example.c -- moved MAX_WBITS to zconf.h because users might want to change that. -- document explicitly that zalloc(64K) on MSDOS must return a normalized - pointer (zero offset) -- added Makefiles for Microsoft C, Turbo C, Borland C++ -- faster crc32() - -Changes in 0.8 (29 April 95) -- added fast inflate (inffast.c) -- deflate(Z_FINISH) now returns Z_STREAM_END when done. Warning: this - is incompatible with previous versions of zlib which returned Z_OK. -- work around a TurboC compiler bug (bad code for b << 0, see infutil.h) - (actually that was not a compiler bug, see 0.81 above) -- gzread no longer reads one extra byte in certain cases -- In gzio destroy(), don't reference a freed structure -- avoid many warnings for MSDOS -- avoid the ERROR symbol which is used by MS Windows - -Changes in 0.71 (14 April 95) -- Fixed more MSDOS compilation problems :( There is still a bug with - TurboC large model. - -Changes in 0.7 (14 April 95) -- Added full inflate support. -- Simplified the crc32() interface. The pre- and post-conditioning - (one's complement) is now done inside crc32(). WARNING: this is - incompatible with previous versions; see zlib.h for the new usage. - -Changes in 0.61 (12 April 95) -- workaround for a bug in TurboC. example and minigzip now work on MSDOS. - -Changes in 0.6 (11 April 95) -- added minigzip.c -- added gzdopen to reopen a file descriptor as gzFile -- added transparent reading of non-gziped files in gzread. -- fixed bug in gzread (don't read crc as data) -- fixed bug in destroy (gzio.c) (don't return Z_STREAM_END for gzclose). -- don't allocate big arrays in the stack (for MSDOS) -- fix some MSDOS compilation problems - -Changes in 0.5: -- do real compression in deflate.c. Z_PARTIAL_FLUSH is supported but - not yet Z_FULL_FLUSH. -- support decompression but only in a single step (forced Z_FINISH) -- added opaque object for zalloc and zfree. -- added deflateReset and inflateReset -- added a variable zlib_version for consistency checking. -- renamed the 'filter' parameter of deflateInit2 as 'strategy'. - Added Z_FILTERED and Z_HUFFMAN_ONLY constants. - -Changes in 0.4: -- avoid "zip" everywhere, use zlib instead of ziplib. -- suppress Z_BLOCK_FLUSH, interpret Z_PARTIAL_FLUSH as block flush - if compression method == 8. -- added adler32 and crc32 -- renamed deflateOptions as deflateInit2, call one or the other but not both -- added the method parameter for deflateInit2. -- added inflateInit2 -- simplied considerably deflateInit and inflateInit by not supporting - user-provided history buffer. This is supported only in deflateInit2 - and inflateInit2. - -Changes in 0.3: -- prefix all macro names with Z_ -- use Z_FINISH instead of deflateEnd to finish compression. -- added Z_HUFFMAN_ONLY -- added gzerror() diff --git a/contrib/libzlib-ng/FAQ.zlib b/contrib/libzlib-ng/FAQ.zlib deleted file mode 100644 index 8da63b40767..00000000000 --- a/contrib/libzlib-ng/FAQ.zlib +++ /dev/null @@ -1,374 +0,0 @@ -## -# THIS IS AN UNMAINTAINED COPY OF THE ORIGINAL FILE DISTRIBUTED WITH ZLIB 1.2.8 -## - - - - - Frequently Asked Questions about zlib - - -If your question is not there, please check the zlib home page -http://zlib.net/ which may have more recent information. -The lastest zlib FAQ is at http://zlib.net/zlib_faq.html - - - 1. Is zlib Y2K-compliant? - - Yes. zlib doesn't handle dates. - - 2. Where can I get a Windows DLL version? - - The zlib sources can be compiled without change to produce a DLL. See the - file win32/DLL_FAQ.txt in the zlib distribution. Pointers to the - precompiled DLL are found in the zlib web site at http://zlib.net/ . - - 3. Where can I get a Visual Basic interface to zlib? - - See - * http://marknelson.us/1997/01/01/zlib-engine/ - * win32/DLL_FAQ.txt in the zlib distribution - - 4. compress() returns Z_BUF_ERROR. - - Make sure that before the call of compress(), the length of the compressed - buffer is equal to the available size of the compressed buffer and not - zero. For Visual Basic, check that this parameter is passed by reference - ("as any"), not by value ("as long"). - - 5. deflate() or inflate() returns Z_BUF_ERROR. - - Before making the call, make sure that avail_in and avail_out are not zero. - When setting the parameter flush equal to Z_FINISH, also make sure that - avail_out is big enough to allow processing all pending input. Note that a - Z_BUF_ERROR is not fatal--another call to deflate() or inflate() can be - made with more input or output space. A Z_BUF_ERROR may in fact be - unavoidable depending on how the functions are used, since it is not - possible to tell whether or not there is more output pending when - strm.avail_out returns with zero. See http://zlib.net/zlib_how.html for a - heavily annotated example. - - 6. Where's the zlib documentation (man pages, etc.)? - - It's in zlib.h . Examples of zlib usage are in the files test/example.c - and test/minigzip.c, with more in examples/ . - - 7. Why don't you use GNU autoconf or libtool or ...? - - Because we would like to keep zlib as a very small and simple package. - zlib is rather portable and doesn't need much configuration. - - 8. I found a bug in zlib. - - Most of the time, such problems are due to an incorrect usage of zlib. - Please try to reproduce the problem with a small program and send the - corresponding source to us at zlib@gzip.org . Do not send multi-megabyte - data files without prior agreement. - - 9. Why do I get "undefined reference to gzputc"? - - If "make test" produces something like - - example.o(.text+0x154): undefined reference to `gzputc' - - check that you don't have old files libz.* in /usr/lib, /usr/local/lib or - /usr/X11R6/lib. Remove any old versions, then do "make install". - -10. I need a Delphi interface to zlib. - - See the contrib/delphi directory in the zlib distribution. - -11. Can zlib handle .zip archives? - - Not by itself, no. See the directory contrib/minizip in the zlib - distribution. - -12. Can zlib handle .Z files? - - No, sorry. You have to spawn an uncompress or gunzip subprocess, or adapt - the code of uncompress on your own. - -13. How can I make a Unix shared library? - - By default a shared (and a static) library is built for Unix. So: - - make distclean - ./configure - make - -14. How do I install a shared zlib library on Unix? - - After the above, then: - - make install - - However, many flavors of Unix come with a shared zlib already installed. - Before going to the trouble of compiling a shared version of zlib and - trying to install it, you may want to check if it's already there! If you - can #include , it's there. The -lz option will probably link to - it. You can check the version at the top of zlib.h or with the - ZLIB_VERSION symbol defined in zlib.h . - -15. I have a question about OttoPDF. - - We are not the authors of OttoPDF. The real author is on the OttoPDF web - site: Joel Hainley, jhainley@myndkryme.com. - -16. Can zlib decode Flate data in an Adobe PDF file? - - Yes. See http://www.pdflib.com/ . To modify PDF forms, see - http://sourceforge.net/projects/acroformtool/ . - -17. Why am I getting this "register_frame_info not found" error on Solaris? - - After installing zlib 1.1.4 on Solaris 2.6, running applications using zlib - generates an error such as: - - ld.so.1: rpm: fatal: relocation error: file /usr/local/lib/libz.so: - symbol __register_frame_info: referenced symbol not found - - The symbol __register_frame_info is not part of zlib, it is generated by - the C compiler (cc or gcc). You must recompile applications using zlib - which have this problem. This problem is specific to Solaris. See - http://www.sunfreeware.com for Solaris versions of zlib and applications - using zlib. - -18. Why does gzip give an error on a file I make with compress/deflate? - - The compress and deflate functions produce data in the zlib format, which - is different and incompatible with the gzip format. The gz* functions in - zlib on the other hand use the gzip format. Both the zlib and gzip formats - use the same compressed data format internally, but have different headers - and trailers around the compressed data. - -19. Ok, so why are there two different formats? - - The gzip format was designed to retain the directory information about a - single file, such as the name and last modification date. The zlib format - on the other hand was designed for in-memory and communication channel - applications, and has a much more compact header and trailer and uses a - faster integrity check than gzip. - -20. Well that's nice, but how do I make a gzip file in memory? - - You can request that deflate write the gzip format instead of the zlib - format using deflateInit2(). You can also request that inflate decode the - gzip format using inflateInit2(). Read zlib.h for more details. - -21. Is zlib thread-safe? - - Yes. However any library routines that zlib uses and any application- - provided memory allocation routines must also be thread-safe. zlib's gz* - functions use stdio library routines, and most of zlib's functions use the - library memory allocation routines by default. zlib's *Init* functions - allow for the application to provide custom memory allocation routines. - - Of course, you should only operate on any given zlib or gzip stream from a - single thread at a time. - -22. Can I use zlib in my commercial application? - - Yes. Please read the license in zlib.h. - -23. Is zlib under the GNU license? - - No. Please read the license in zlib.h. - -24. The license says that altered source versions must be "plainly marked". So - what exactly do I need to do to meet that requirement? - - You need to change the ZLIB_VERSION and ZLIB_VERNUM #defines in zlib.h. In - particular, the final version number needs to be changed to "f", and an - identification string should be appended to ZLIB_VERSION. Version numbers - x.x.x.f are reserved for modifications to zlib by others than the zlib - maintainers. For example, if the version of the base zlib you are altering - is "1.2.3.4", then in zlib.h you should change ZLIB_VERNUM to 0x123f, and - ZLIB_VERSION to something like "1.2.3.f-zachary-mods-v3". You can also - update the version strings in deflate.c and inftrees.c. - - For altered source distributions, you should also note the origin and - nature of the changes in zlib.h, as well as in ChangeLog and README, along - with the dates of the alterations. The origin should include at least your - name (or your company's name), and an email address to contact for help or - issues with the library. - - Note that distributing a compiled zlib library along with zlib.h and - zconf.h is also a source distribution, and so you should change - ZLIB_VERSION and ZLIB_VERNUM and note the origin and nature of the changes - in zlib.h as you would for a full source distribution. - -25. Will zlib work on a big-endian or little-endian architecture, and can I - exchange compressed data between them? - - Yes and yes. - -26. Will zlib work on a 64-bit machine? - - Yes. It has been tested on 64-bit machines, and has no dependence on any - data types being limited to 32-bits in length. If you have any - difficulties, please provide a complete problem report to zlib@gzip.org - -27. Will zlib decompress data from the PKWare Data Compression Library? - - No. The PKWare DCL uses a completely different compressed data format than - does PKZIP and zlib. However, you can look in zlib's contrib/blast - directory for a possible solution to your problem. - -28. Can I access data randomly in a compressed stream? - - No, not without some preparation. If when compressing you periodically use - Z_FULL_FLUSH, carefully write all the pending data at those points, and - keep an index of those locations, then you can start decompression at those - points. You have to be careful to not use Z_FULL_FLUSH too often, since it - can significantly degrade compression. Alternatively, you can scan a - deflate stream once to generate an index, and then use that index for - random access. See examples/zran.c . - -29. Does zlib work on MVS, OS/390, CICS, etc.? - - It has in the past, but we have not heard of any recent evidence. There - were working ports of zlib 1.1.4 to MVS, but those links no longer work. - If you know of recent, successful applications of zlib on these operating - systems, please let us know. Thanks. - -30. Is there some simpler, easier to read version of inflate I can look at to - understand the deflate format? - - First off, you should read RFC 1951. Second, yes. Look in zlib's - contrib/puff directory. - -31. Does zlib infringe on any patents? - - As far as we know, no. In fact, that was originally the whole point behind - zlib. Look here for some more information: - - http://www.gzip.org/#faq11 - -32. Can zlib work with greater than 4 GB of data? - - Yes. inflate() and deflate() will process any amount of data correctly. - Each call of inflate() or deflate() is limited to input and output chunks - of the maximum value that can be stored in the compiler's "unsigned int" - type, but there is no limit to the number of chunks. Note however that the - strm.total_in and strm_total_out counters may be limited to 4 GB. These - counters are provided as a convenience and are not used internally by - inflate() or deflate(). The application can easily set up its own counters - updated after each call of inflate() or deflate() to count beyond 4 GB. - compress() and uncompress() may be limited to 4 GB, since they operate in a - single call. gzseek() and gztell() may be limited to 4 GB depending on how - zlib is compiled. See the zlibCompileFlags() function in zlib.h. - - The word "may" appears several times above since there is a 4 GB limit only - if the compiler's "long" type is 32 bits. If the compiler's "long" type is - 64 bits, then the limit is 16 exabytes. - -33. Does zlib have any security vulnerabilities? - - The only one that we are aware of is potentially in gzprintf(). If zlib is - compiled to use sprintf() or vsprintf(), then there is no protection - against a buffer overflow of an 8K string space (or other value as set by - gzbuffer()), other than the caller of gzprintf() assuring that the output - will not exceed 8K. On the other hand, if zlib is compiled to use - snprintf() or vsnprintf(), which should normally be the case, then there is - no vulnerability. The ./configure script will display warnings if an - insecure variation of sprintf() will be used by gzprintf(). Also the - zlibCompileFlags() function will return information on what variant of - sprintf() is used by gzprintf(). - - If you don't have snprintf() or vsnprintf() and would like one, you can - find a portable implementation here: - - http://www.ijs.si/software/snprintf/ - - Note that you should be using the most recent version of zlib. Versions - 1.1.3 and before were subject to a double-free vulnerability, and versions - 1.2.1 and 1.2.2 were subject to an access exception when decompressing - invalid compressed data. - -34. Is there a Java version of zlib? - - Probably what you want is to use zlib in Java. zlib is already included - as part of the Java SDK in the java.util.zip package. If you really want - a version of zlib written in the Java language, look on the zlib home - page for links: http://zlib.net/ . - -35. I get this or that compiler or source-code scanner warning when I crank it - up to maximally-pedantic. Can't you guys write proper code? - - Many years ago, we gave up attempting to avoid warnings on every compiler - in the universe. It just got to be a waste of time, and some compilers - were downright silly as well as contradicted each other. So now, we simply - make sure that the code always works. - -36. Valgrind (or some similar memory access checker) says that deflate is - performing a conditional jump that depends on an uninitialized value. - Isn't that a bug? - - No. That is intentional for performance reasons, and the output of deflate - is not affected. This only started showing up recently since zlib 1.2.x - uses malloc() by default for allocations, whereas earlier versions used - calloc(), which zeros out the allocated memory. Even though the code was - correct, versions 1.2.4 and later was changed to not stimulate these - checkers. - -37. Will zlib read the (insert any ancient or arcane format here) compressed - data format? - - Probably not. Look in the comp.compression FAQ for pointers to various - formats and associated software. - -38. How can I encrypt/decrypt zip files with zlib? - - zlib doesn't support encryption. The original PKZIP encryption is very - weak and can be broken with freely available programs. To get strong - encryption, use GnuPG, http://www.gnupg.org/ , which already includes zlib - compression. For PKZIP compatible "encryption", look at - http://www.info-zip.org/ - -39. What's the difference between the "gzip" and "deflate" HTTP 1.1 encodings? - - "gzip" is the gzip format, and "deflate" is the zlib format. They should - probably have called the second one "zlib" instead to avoid confusion with - the raw deflate compressed data format. While the HTTP 1.1 RFC 2616 - correctly points to the zlib specification in RFC 1950 for the "deflate" - transfer encoding, there have been reports of servers and browsers that - incorrectly produce or expect raw deflate data per the deflate - specification in RFC 1951, most notably Microsoft. So even though the - "deflate" transfer encoding using the zlib format would be the more - efficient approach (and in fact exactly what the zlib format was designed - for), using the "gzip" transfer encoding is probably more reliable due to - an unfortunate choice of name on the part of the HTTP 1.1 authors. - - Bottom line: use the gzip format for HTTP 1.1 encoding. - -40. Does zlib support the new "Deflate64" format introduced by PKWare? - - No. PKWare has apparently decided to keep that format proprietary, since - they have not documented it as they have previous compression formats. In - any case, the compression improvements are so modest compared to other more - modern approaches, that it's not worth the effort to implement. - -41. I'm having a problem with the zip functions in zlib, can you help? - - There are no zip functions in zlib. You are probably using minizip by - Giles Vollant, which is found in the contrib directory of zlib. It is not - part of zlib. In fact none of the stuff in contrib is part of zlib. The - files in there are not supported by the zlib authors. You need to contact - the authors of the respective contribution for help. - -42. The match.asm code in contrib is under the GNU General Public License. - Since it's part of zlib, doesn't that mean that all of zlib falls under the - GNU GPL? - - No. The files in contrib are not part of zlib. They were contributed by - other authors and are provided as a convenience to the user within the zlib - distribution. Each item in contrib has its own license. - -43. Is zlib subject to export controls? What is its ECCN? - - zlib is not subject to export controls, and so is classified as EAR99. - -44. Can you please sign these lengthy legal documents and fax them back to us - so that we can use your software in our product? - - No. Go away. Shoo. diff --git a/contrib/libzlib-ng/INDEX b/contrib/libzlib-ng/INDEX deleted file mode 100644 index acb9aba64c3..00000000000 --- a/contrib/libzlib-ng/INDEX +++ /dev/null @@ -1,55 +0,0 @@ -CMakeLists.txt cmake build file -ChangeLog.zlib history of changes up to the fork from zlib 1.2.8 -FAQ.zlib Frequently Asked Questions about zlib, as distributed in zlib 1.2.8 -INDEX this file -Makefile dummy Makefile that tells you to ./configure -Makefile.in template for Unix Makefile -README guess what -README.zlib Copy of the original README file distributed in zlib 1.2.8 -configure configure script for Unix -test/example.c zlib usages examples for build testing -test/minigzip.c minimal gzip-like functionality for build testing -test/infcover.c inf*.c code coverage for build coverage testing -treebuild.xml XML description of source file dependencies -zconf.h.cmakein zconf.h template for cmake -zconf.h.in zconf.h template for configure -zlib.3 Man page for zlib -zlib.3.pdf Man page in PDF format -zlib.map Linux symbol information -zlib.pc.in Template for pkg-config descriptor -zlib.pc.cmakein zlib.pc template for cmake -zlib2ansi perl script to convert source files for C++ compilation - -arch/ architecture-specific code -doc/ documentation for formats and algorithms -win32/ makefiles for Windows - - zlib public header files (required for library use): -zconf.h -zlib.h - - private source files used to build the zlib library: -adler32.c -compress.c -crc32.c -crc32.h -deflate.c -deflate.h -gzclose.c -gzguts.h -gzlib.c -gzread.c -gzwrite.c -infback.c -inffast.c -inffast.h -inffixed.h -inflate.c -inflate.h -inftrees.c -inftrees.h -trees.c -trees.h -uncompr.c -zutil.c -zutil.h diff --git a/contrib/libzlib-ng/LICENSE.md b/contrib/libzlib-ng/LICENSE.md deleted file mode 100644 index adb48d47296..00000000000 --- a/contrib/libzlib-ng/LICENSE.md +++ /dev/null @@ -1,19 +0,0 @@ -(C) 1995-2013 Jean-loup Gailly and Mark Adler - -This software is provided 'as-is', without any express or implied -warranty. In no event will the authors be held liable for any damages -arising from the use of this software. - -Permission is granted to anyone to use this software for any purpose, -including commercial applications, and to alter it and redistribute it -freely, subject to the following restrictions: - -1. The origin of this software must not be misrepresented; you must not - claim that you wrote the original software. If you use this software - in a product, an acknowledgment in the product documentation would be - appreciated but is not required. - -2. Altered source versions must be plainly marked as such, and must not be - misrepresented as being the original software. - -3. This notice may not be removed or altered from any source distribution. diff --git a/contrib/libzlib-ng/Makefile.in b/contrib/libzlib-ng/Makefile.in deleted file mode 100644 index c69175e2d9a..00000000000 --- a/contrib/libzlib-ng/Makefile.in +++ /dev/null @@ -1,329 +0,0 @@ -# Makefile for zlib -# Copyright (C) 1995-2013 Jean-loup Gailly, Mark Adler -# For conditions of distribution and use, see copyright notice in zlib.h - -# To compile and test, type: -# ./configure; make test -# Normally configure builds both a static and a shared library. -# If you want to build just a static library, use: ./configure --static - -# To install /usr/local/lib/libz.* and /usr/local/include/zlib.h, type: -# make install -# To install in $HOME instead of /usr/local, use: -# make install prefix=$HOME - -CC=cc - -CFLAGS=-O -#CFLAGS=-O -DMAX_WBITS=14 -DMAX_MEM_LEVEL=7 -#CFLAGS=-g -DDEBUG -#CFLAGS=-O3 -Wall -Wwrite-strings -Wpointer-arith -Wconversion \ -# -Wstrict-prototypes -Wmissing-prototypes - -SFLAGS=-O -LDFLAGS= -TEST_LDFLAGS=-L. libz.a -LDSHARED=$(CC) - -STATICLIB=libz.a -SHAREDLIB=libz.so -SHAREDLIBV=libz.so.1.2.8 -SHAREDLIBM=libz.so.1 -IMPORTLIB= -SHAREDTARGET=libz.so.1.2.8 - -LIBS=$(STATICLIB) $(SHAREDTARGET) - -AR=ar -ARFLAGS=rc -DEFFILE= -RC= -RCFLAGS= -RCOBJS= -STRIP= -RANLIB=ranlib -LDCONFIG=ldconfig -LDSHAREDLIBC=-lc -TAR=tar -SHELL=/bin/sh -EXE= - -SRCDIR=. -INCLUDES=-I$(SRCDIR) - -ARCHDIR=arch/generic -ARCH_STATIC_OBJS= -ARCH_SHARED_OBJS= - -prefix = /usr/local -exec_prefix = ${prefix} -bindir = ${exec_prefix}/bin -libdir = ${exec_prefix}/lib -sharedlibdir = ${libdir} -includedir = ${prefix}/include -mandir = ${prefix}/share/man -man3dir = ${mandir}/man3 -pkgconfigdir = ${libdir}/pkgconfig - -OBJZ = adler32.o compress.o crc32.o deflate.o deflate_fast.o deflate_medium.o deflate_slow.o match.o infback.o inffast.o inflate.o inftrees.o trees.o uncompr.o zutil.o $(ARCH_STATIC_OBJS) -OBJG = gzclose.o gzlib.o gzread.o gzwrite.o -OBJC = $(OBJZ) $(OBJG) - -PIC_OBJZ = adler32.lo compress.lo crc32.lo deflate.lo deflate_fast.lo deflate_medium.lo deflate_slow.lo match.lo infback.lo inffast.lo inflate.lo inftrees.lo trees.lo uncompr.lo zutil.lo $(ARCH_SHARED_OBJS) -PIC_OBJG = gzclose.lo gzlib.lo gzread.lo gzwrite.lo -PIC_OBJC = $(PIC_OBJZ) $(PIC_OBJG) - -OBJS = $(OBJC) - -PIC_OBJS = $(PIC_OBJC) - -all: static shared - -static: example$(EXE) minigzip$(EXE) - -shared: examplesh$(EXE) minigzipsh$(EXE) - -all64: example64$(EXE) minigzip64$(EXE) - -check: test - -$(ARCHDIR)/%.o: $(SRCDIR)/$(ARCHDIR)/%.c - $(MAKE) -C $(ARCHDIR) $(notdir $@) - -$(ARCHDIR)/%.lo: $(SRCDIR)/$(ARCHDIR)/%.c - $(MAKE) -C $(ARCHDIR) $(notdir $@) - -%.o: $(ARCHDIR)/%.o - -cp $< $@ - -%.lo: $(ARCHDIR)/%.lo - -cp $< $@ - -test: all - $(MAKE) -C test - -infcover.o: $(SRCDIR)/test/infcover.c $(SRCDIR)/zlib.h zconf.h - $(CC) $(CFLAGS) $(INCLUDES) -c -o $@ $(SRCDIR)/test/infcover.c - -infcover$(EXE): infcover.o $(STATICLIB) - $(CC) $(CFLAGS) -o $@ infcover.o $(STATICLIB) -ifneq ($(STRIP),) - $(STRIP) $@ -endif - -cover: infcover$(EXE) - rm -f *.gcda - ./infcover - gcov inf*.c - -$(STATICLIB): $(OBJS) - $(AR) $(ARFLAGS) $@ $(OBJS) - -@ ($(RANLIB) $@ || true) >/dev/null 2>&1 - -example.o: $(SRCDIR)/test/example.c $(SRCDIR)/zlib.h zconf.h - $(CC) $(CFLAGS) $(INCLUDES) -c -o $@ $(SRCDIR)/test/example.c - -minigzip.o: $(SRCDIR)/test/minigzip.c $(SRCDIR)/zlib.h zconf.h - $(CC) $(CFLAGS) $(INCLUDES) -c -o $@ $(SRCDIR)/test/minigzip.c - -example64.o: $(SRCDIR)/test/example.c $(SRCDIR)/zlib.h zconf.h - $(CC) $(CFLAGS) $(INCLUDES) -D_FILE_OFFSET_BITS=64 -c -o $@ $(SRCDIR)/test/example.c - -minigzip64.o: $(SRCDIR)/test/minigzip.c $(SRCDIR)/zlib.h zconf.h - $(CC) $(CFLAGS) $(INCLUDES) -D_FILE_OFFSET_BITS=64 -c -o $@ $(SRCDIR)/test/minigzip.c - -zlibrc.o: win32/zlib1.rc - $(RC) $(RCFLAGS) -o $@ win32/zlib1.rc - -.SUFFIXES: .lo - -%.o: $(SRCDIR)/%.c - $(CC) $(INCLUDES) $(CFLAGS) -c -o $@ $< - -%.lo: $(SRCDIR)/%.c - $(CC) $(INCLUDES) $(SFLAGS) -DPIC -c -o $@ $< - -$(SHAREDTARGET): $(PIC_OBJS) $(DEFFILE) $(RCOBJS) -ifneq ($(SHAREDTARGET),) - $(LDSHARED) $(SFLAGS) -o $@ $(DEFFILE) $(PIC_OBJS) $(RCOBJS) $(LDSHAREDLIBC) $(LDFLAGS) -ifneq ($(STRIP),) - $(STRIP) $@ -endif -ifneq ($(SHAREDLIB),$(SHAREDTARGET)) - rm -f $(SHAREDLIB) $(SHAREDLIBM) - ln -s $@ $(SHAREDLIB) - ln -s $@ $(SHAREDLIBM) -endif -endif - -example$(EXE): example.o $(STATICLIB) - $(CC) $(CFLAGS) -o $@ example.o $(TEST_LDFLAGS) -ifneq ($(STRIP),) - $(STRIP) $@ -endif - -minigzip$(EXE): minigzip.o $(STATICLIB) - $(CC) $(CFLAGS) -o $@ minigzip.o $(TEST_LDFLAGS) -ifneq ($(STRIP),) - $(STRIP) $@ -endif - -examplesh$(EXE): example.o $(SHAREDTARGET) - $(CC) $(CFLAGS) -o $@ example.o -L. $(SHAREDTARGET) -ifneq ($(STRIP),) - $(STRIP) $@ -endif - -minigzipsh$(EXE): minigzip.o $(SHAREDTARGET) - $(CC) $(CFLAGS) -o $@ minigzip.o -L. $(SHAREDTARGET) -ifneq ($(STRIP),) - $(STRIP) $@ -endif - -example64$(EXE): example64.o $(STATICLIB) - $(CC) $(CFLAGS) -o $@ example64.o $(TEST_LDFLAGS) -ifneq ($(STRIP),) - $(STRIP) $@ -endif - -minigzip64$(EXE): minigzip64.o $(STATICLIB) - $(CC) $(CFLAGS) -o $@ minigzip64.o $(TEST_LDFLAGS) -ifneq ($(STRIP),) - $(STRIP) $@ -endif - -install-shared: $(SHAREDTARGET) -ifneq ($(SHAREDTARGET),) - -@if [ ! -d $(DESTDIR)$(sharedlibdir) ]; then mkdir -p $(DESTDIR)$(sharedlibdir); fi - cp $(SHAREDTARGET) $(DESTDIR)$(sharedlibdir) - chmod 644 $(DESTDIR)$(sharedlibdir)/$(SHAREDTARGET) -ifneq ($(SHAREDLIB),$(SHAREDTARGET)) - rm -f $(DESTDIR)$(sharedlibdir)/$(SHAREDLIB) $(DESTDIR)$(sharedlibdir)/$(SHAREDLIBM) - ln -s $(SHAREDLIBV) $(DESTDIR)$(sharedlibdir)/$(SHAREDLIB) - ln -s $(SHAREDLIBV) $(DESTDIR)$(sharedlibdir)/$(SHAREDLIBM) - ($(LDCONFIG) || true) >/dev/null 2>&1 -# ldconfig is for Linux -endif -ifneq ($(IMPORTLIB),) - cp $(IMPORTLIB) $(DESTDIR)$(sharedlibdir) - chmod 644 $(DESTDIR)$(sharedlibdir)/$(IMPORTLIB) -endif -endif - -install-static: $(STATICLIB) - -@if [ ! -d $(DESTDIR)$(libdir) ]; then mkdir -p $(DESTDIR)$(libdir); fi - cp $(STATICLIB) $(DESTDIR)$(libdir) - chmod 644 $(DESTDIR)$(libdir)/$(STATICLIB) - -@($(RANLIB) $(DESTDIR)$(libdir)/$(STATICLIB) || true) >/dev/null 2>&1 -# The ranlib in install-static is needed on NeXTSTEP which checks file times - -install-libs: install-shared install-static - -@if [ ! -d $(DESTDIR)$(man3dir) ]; then mkdir -p $(DESTDIR)$(man3dir); fi - -@if [ ! -d $(DESTDIR)$(pkgconfigdir) ]; then mkdir -p $(DESTDIR)$(pkgconfigdir); fi - cp $(SRCDIR)/zlib.3 $(DESTDIR)$(man3dir) - chmod 644 $(DESTDIR)$(man3dir)/zlib.3 - cp zlib.pc $(DESTDIR)$(pkgconfigdir) - chmod 644 $(DESTDIR)$(pkgconfigdir)/zlib.pc - -install: install-libs - -@if [ ! -d $(DESTDIR)$(includedir) ]; then mkdir -p $(DESTDIR)$(includedir); fi - cp $(SRCDIR)/zlib.h zconf.h $(DESTDIR)$(includedir) - chmod 644 $(DESTDIR)$(includedir)/zlib.h $(DESTDIR)$(includedir)/zconf.h - -uninstall-static: - cd $(DESTDIR)$(libdir) && rm -f $(STATICLIB) - -uninstall-shared: -ifneq ($(SHAREDLIB),) - cd $(DESTDIR)$(sharedlibdir) && rm -f $(SHAREDLIBV) $(SHAREDLIB) $(SHAREDLIBM) -endif -ifneq ($(IMPORTLIB),) - cd $(DESTDIR)$(sharedlibdir) && rm -f $(IMPORTLIB) -endif - -uninstall: uninstall-static uninstall-shared - cd $(DESTDIR)$(includedir) && rm -f zlib.h zconf.h - cd $(DESTDIR)$(man3dir) && rm -f zlib.3 - cd $(DESTDIR)$(pkgconfigdir) && rm -f zlib.pc - -docs: zlib.3.pdf - -zlib.3.pdf: $(SRCDIR)/zlib.3 - groff -mandoc -f H -T ps $(SRCDIR)/zlib.3 | ps2pdf - zlib.3.pdf - -mostlyclean: clean -clean: - @if [ -f $(ARCHDIR)/Makefile ]; then $(MAKE) -C $(ARCHDIR) clean; fi - @if [ -f test/Makefile ]; then $(MAKE) -C test clean; fi - rm -f *.o *.lo *~ \ - example$(EXE) minigzip$(EXE) examplesh$(EXE) minigzipsh$(EXE) \ - example64$(EXE) minigzip64$(EXE) \ - infcover \ - $(STATICLIB) $(IMPORTLIB) $(SHAREDLIB) $(SHAREDLIBV) $(SHAREDLIBM) \ - foo.gz so_locations \ - _match.s maketree - rm -rf objs - rm -f *.gcda *.gcno *.gcov - rm -f a.out - -maintainer-clean: distclean -distclean: clean - @if [ -f $(ARCHDIR)/Makefile ]; then $(MAKE) -C $(ARCHDIR) distclean; fi - @if [ -f test/Makefile ]; then $(MAKE) -C test distclean; fi - rm -f zlib.pc configure.log zconf.h zconf.h.cmakein - -@rm -f .DS_Store -# Reset Makefile if building inside source tree - @if [ -f Makefile.in ]; then \ - printf 'all:\n\t-@echo "Please use ./configure first. Thank you."\n' > Makefile ; \ - printf '\ndistclean:\n\tmake -f Makefile.in distclean\n' >> Makefile ; \ - touch -r $(SRCDIR)/Makefile.in Makefile ; fi -# Reset zconf.h and zconf.h.cmakein if building inside source tree - @if [ -f zconf.h.in ]; then \ - cp -p $(SRCDIR)/zconf.h.in zconf.h ; \ - TEMPFILE=zconfh_$$ ; \ - echo "/#define ZCONF_H/ a\\\\\n#cmakedefine Z_HAVE_UNISTD_H\n" >> $$TEMPFILE &&\ - sed -f $$TEMPFILE $(SRCDIR)/zconf.h.in > zconf.h.cmakein &&\ - touch -r $(SRCDIR)/zconf.h.in zconf.h.cmakein &&\ - rm $$TEMPFILE ; fi -# Cleanup these files if building outside source tree - @if [ ! -f zlib.3 ]; then rm -f zlib.3.pdf Makefile; fi -# Remove arch and test directory if building outside source tree - @if [ ! -f $(ARCHDIR)/Makefile.in ]; then rm -rf arch; fi - @if [ ! -f test/Makefile.in ]; then rm -rf test; fi - -tags: - etags $(SRCDIR)/*.[ch] - -depend: - makedepend -- $(CFLAGS) -- $(SRCDIR)/*.[ch] - makedepend -a -o.lo -- $(SFLAGS) -- $(SRCDIR)/*.[ch] - @sed "s=^$(SRCDIR)/\([a-zA-Z0-9_]*\.\(lo\|o\):\)=\1=g" < Makefile > Makefile.tmp - @mv -f Makefile.tmp Makefile - -# DO NOT DELETE THIS LINE -- make depend depends on it. - -adler32.o zutil.o: $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h -gzclose.o gzlib.o gzread.o gzwrite.o: $(SRCDIR)/zlib.h zconf.h $(SRCDIR)/gzguts.h -compress.o example.o minigzip.o uncompr.o: $(SRCDIR)/zlib.h zconf.h -crc32.o: $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h $(SRCDIR)/crc32.h -deflate.o: $(SRCDIR)/deflate.h $(SRCDIR)/deflate_p.h $(SRCDIR)/match.h $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h -deflate_fast.o: $(SRCDIR)/deflate.h $(SRCDIR)/deflate_p.h $(SRCDIR)/match.h -deflate_medium.o: $(SRCDIR)/deflate.h $(SRCDIR)/deflate_p.h $(SRCDIR)/match.h -deflate_slow.o: $(SRCDIR)/deflate.h $(SRCDIR)/deflate_p.h $(SRCDIR)/match.h -infback.o inflate.o: $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h $(SRCDIR)/inftrees.h $(SRCDIR)/inflate.h $(SRCDIR)/inffast.h $(SRCDIR)/inffixed.h -inffast.o: $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h $(SRCDIR)/inftrees.h $(SRCDIR)/inflate.h $(SRCDIR)/inffast.h -inftrees.o: $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h $(SRCDIR)/inftrees.h -trees.o: $(SRCDIR)/deflate.h $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h $(SRCDIR)/trees.h - -adler32.lo zutil.lo: $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h -gzclose.lo gzlib.lo gzread.lo gzwrite.lo: $(SRCDIR)/zlib.h zconf.h $(SRCDIR)/gzguts.h -compress.lo example.lo minigzip.lo uncompr.lo: $(SRCDIR)/zlib.h zconf.h -crc32.lo: $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h $(SRCDIR)/crc32.h -deflate.lo: $(SRCDIR)/deflate.h $(SRCDIR)/deflate_p.h $(SRCDIR)/match.h $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h -deflate_fast.lo: $(SRCDIR)/deflate.h $(SRCDIR)/deflate_p.h $(SRCDIR)/match.h -deflate_medium.lo: $(SRCDIR)/deflate.h $(SRCDIR)/deflate_p.h $(SRCDIR)/match.h -deflate_slow.lo: $(SRCDIR)/deflate.h $(SRCDIR)/deflate_p.h $(SRCDIR)/match.h -infback.lo inflate.lo: $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h $(SRCDIR)/inftrees.h $(SRCDIR)/inflate.h $(SRCDIR)/inffast.h $(SRCDIR)/inffixed.h -inffast.lo: $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h $(SRCDIR)/inftrees.h $(SRCDIR)/inflate.h $(SRCDIR)/inffast.h -inftrees.lo: $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h $(SRCDIR)/inftrees.h -trees.lo: $(SRCDIR)/deflate.h $(SRCDIR)/zutil.h $(SRCDIR)/zlib.h zconf.h $(SRCDIR)/trees.h diff --git a/contrib/libzlib-ng/README b/contrib/libzlib-ng/README deleted file mode 100644 index db98f1112fb..00000000000 --- a/contrib/libzlib-ng/README +++ /dev/null @@ -1,63 +0,0 @@ -zlib-ng - zlib for the next generation systems - -Maintained by Hans Kristian Rosbach - aka Dead2 (zlib-ng àt circlestorm dót org) - - -Fork Motivation and History ---------------------------- - -The motivation for this fork was due to seeing several 3rd party -contributions containing new optimizations not getting implemented -into the official zlib repository. - -Mark Adler has been maintaining zlib for a very long time, and he has -done a great job and hopefully he will continue for a long time yet. -The idea of zlib-ng is not to replace zlib, but to co-exist as a -drop-in replacement with a lower threshold for code change. - -zlib has a long history and is incredibly portable, even supporting -lots of systems that predate the Internet. This is great, but it does -complicate further development and maintainability. -The zlib code has to make numerous workarounds for old compilers that -do not understand ANSI-C or to accommodate systems with limitations -such as operating in a 16-bit environment. - -Many of these workarounds are only maintenance burdens, some of them -are pretty huge code-wise. For example, the [v]s[n]printf workaround -code has a whopping 8 different implementations just to cater to -various old compilers. With this many workarounds cluttered throughout -the code, new programmers with an idea/interest for zlib will need -to take some time to figure out why all of these seemingly strange -things are used, and how to work within those confines. - -So I decided to make a fork, merge all the Intel optimizations, merge -the Cloudflare optimizations that did not conflict, plus a couple -of other smaller patches. Then I started cleaning out workarounds, -various dead code, all contrib and example code as there is little -point in having those in this fork for different reasons. - -Lastly I have been cleaning up the handling of different arches, -so that it will be easier to implement arch-specific code without -cluttering up the main code too much. - -Now, there is still a lot to do and I am sure there are better ways -of doing several of the changes I have done. And I would be delighted -to receive patches, preferably as pull requests on github. -Just remember that any code you submit must be your own and it must -be zlib licensed. - -Please read LICENSE.md, it is very simple and very liberal. - - -Acknowledgments ----------------- - -Big thanks to Raske Sider AS / raskesider.no for sponsoring my -maintainership of zlib-ng. - -The deflate format used by zlib was defined by Phil Katz. -The deflate and zlib specifications were written by L. Peter Deutsch. - -zlib was originally created by Jean-loup Gailly (compression) -and Mark Adler (decompression). diff --git a/contrib/libzlib-ng/README.clickhouse b/contrib/libzlib-ng/README.clickhouse deleted file mode 100644 index 82f318a88ee..00000000000 --- a/contrib/libzlib-ng/README.clickhouse +++ /dev/null @@ -1,12 +0,0 @@ -Sources imported from https://github.com/Dead2/zlib-ng/tree/343c4c549107d31f6eeabfb4b31bec4502a2ea0e -CMakeLists.txt taken from https://github.com/mtl1979/zlib-ng/tree/ad8868ab0e78a87fb0485d4bc67b8cfe96e00891 - -Zlib with CloudFlare patches provides slightly better performance for decompression and compression -with level > 1. Unfortunately, we can't use that version due to GPL-licensed code. If you still -want to use it, replace contents of this directory with contents of https://github.com/cloudflare/zlib -and add the following lines at the beginning of CMakeLists.txt: - -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-variable -DHAS_PCLMUL") -set (ZLIB_ASMS - contrib/amd64/crc32-pclmul_asm.S -) diff --git a/contrib/libzlib-ng/README.md b/contrib/libzlib-ng/README.md deleted file mode 100644 index 35ea0545568..00000000000 --- a/contrib/libzlib-ng/README.md +++ /dev/null @@ -1,65 +0,0 @@ -Travis CI: [![build status](https://api.travis-ci.org/Dead2/zlib-ng.svg)](https://travis-ci.org/Dead2/zlib-ng/) - -zlib-ng - zlib for the next generation systems - -Maintained by Hans Kristian Rosbach - aka Dead2 (zlib-ng àt circlestorm dót org) - - -Fork Motivation and History ---------------------------- - -The motivation for this fork was due to seeing several 3rd party -contributions containing new optimizations not getting implemented -into the official zlib repository. - -Mark Adler has been maintaining zlib for a very long time, and he has -done a great job and hopefully he will continue for a long time yet. -The idea of zlib-ng is not to replace zlib, but to co-exist as a -drop-in replacement with a lower threshold for code change. - -zlib has a long history and is incredibly portable, even supporting -lots of systems that predate the Internet. This is great, but it does -complicate further development and maintainability. -The zlib code has to make numerous workarounds for old compilers that -do not understand ANSI-C or to accommodate systems with limitations -such as operating in a 16-bit environment. - -Many of these workarounds are only maintenance burdens, some of them -are pretty huge code-wise. For example, the [v]s[n]printf workaround -code has a whopping 8 different implementations just to cater to -various old compilers. With this many workarounds cluttered throughout -the code, new programmers with an idea/interest for zlib will need -to take some time to figure out why all of these seemingly strange -things are used, and how to work within those confines. - -So I decided to make a fork, merge all the Intel optimizations, merge -the Cloudflare optimizations that did not conflict, plus a couple -of other smaller patches. Then I started cleaning out workarounds, -various dead code, all contrib and example code as there is little -point in having those in this fork for different reasons. - -Lastly I have been cleaning up the handling of different arches, -so that it will be easier to implement arch-specific code without -cluttering up the main code too much. - -Now, there is still a lot to do and I am sure there are better ways -of doing several of the changes I have done. And I would be delighted -to receive patches, preferably as pull requests on github. -Just remember that any code you submit must be your own and it must -be zlib licensed. - -Please read LICENSE.md, it is very simple and very liberal. - - -Acknowledgments ----------------- - -Big thanks to Raske Sider AS / raskesider.no for sponsoring my -maintainership of zlib-ng. - -The deflate format used by zlib was defined by Phil Katz. -The deflate and zlib specifications were written by L. Peter Deutsch. - -zlib was originally created by Jean-loup Gailly (compression) -and Mark Adler (decompression). diff --git a/contrib/libzlib-ng/README.zlib b/contrib/libzlib-ng/README.zlib deleted file mode 100644 index 200579260ac..00000000000 --- a/contrib/libzlib-ng/README.zlib +++ /dev/null @@ -1,121 +0,0 @@ -## -# THIS IS AN UNMAINTAINED COPY OF THE ORIGINAL FILE DISTRIBUTED WITH ZLIB 1.2.8 -## - - - -ZLIB DATA COMPRESSION LIBRARY - -zlib 1.2.8 is a general purpose data compression library. All the code is -thread safe. The data format used by the zlib library is described by RFCs -(Request for Comments) 1950 to 1952 in the files -http://tools.ietf.org/html/rfc1950 (zlib format), rfc1951 (deflate format) and -rfc1952 (gzip format). - -All functions of the compression library are documented in the file zlib.h -(volunteer to write man pages welcome, contact zlib@gzip.org). A usage example -of the library is given in the file test/example.c which also tests that -the library is working correctly. Another example is given in the file -test/minigzip.c. The compression library itself is composed of all source -files in the root directory. - -To compile all files and run the test program, follow the instructions given at -the top of Makefile.in. In short "./configure; make test", and if that goes -well, "make install" should work for most flavors of Unix. For Windows, use -one of the special makefiles in win32/ or contrib/vstudio/ . For VMS, use -make_vms.com. - -Questions about zlib should be sent to , or to Gilles Vollant - for the Windows DLL version. The zlib home page is -http://zlib.net/ . Before reporting a problem, please check this site to -verify that you have the latest version of zlib; otherwise get the latest -version and check whether the problem still exists or not. - -PLEASE read the zlib FAQ http://zlib.net/zlib_faq.html before asking for help. - -Mark Nelson wrote an article about zlib for the Jan. 1997 -issue of Dr. Dobb's Journal; a copy of the article is available at -http://marknelson.us/1997/01/01/zlib-engine/ . - -The changes made in version 1.2.8 are documented in the file ChangeLog. - -Unsupported third party contributions are provided in directory contrib/ . - -zlib is available in Java using the java.util.zip package, documented at -http://java.sun.com/developer/technicalArticles/Programming/compression/ . - -A Perl interface to zlib written by Paul Marquess is available -at CPAN (Comprehensive Perl Archive Network) sites, including -http://search.cpan.org/~pmqs/IO-Compress-Zlib/ . - -A Python interface to zlib written by A.M. Kuchling is -available in Python 1.5 and later versions, see -http://docs.python.org/library/zlib.html . - -zlib is built into tcl: http://wiki.tcl.tk/4610 . - -An experimental package to read and write files in .zip format, written on top -of zlib by Gilles Vollant , is available in the -contrib/minizip directory of zlib. - - -Notes for some targets: - -- For Windows DLL versions, please see win32/DLL_FAQ.txt - -- For 64-bit Irix, deflate.c must be compiled without any optimization. With - -O, one libpng test fails. The test works in 32 bit mode (with the -n32 - compiler flag). The compiler bug has been reported to SGI. - -- zlib doesn't work with gcc 2.6.3 on a DEC 3000/300LX under OSF/1 2.1 it works - when compiled with cc. - -- On Digital Unix 4.0D (formely OSF/1) on AlphaServer, the cc option -std1 is - necessary to get gzprintf working correctly. This is done by configure. - -- zlib doesn't work on HP-UX 9.05 with some versions of /bin/cc. It works with - other compilers. Use "make test" to check your compiler. - -- gzdopen is not supported on RISCOS or BEOS. - -- For PalmOs, see http://palmzlib.sourceforge.net/ - - -Acknowledgments: - - The deflate format used by zlib was defined by Phil Katz. The deflate and - zlib specifications were written by L. Peter Deutsch. Thanks to all the - people who reported problems and suggested various improvements in zlib; they - are too numerous to cite here. - -Copyright notice: - - (C) 1995-2013 Jean-loup Gailly and Mark Adler - - This software is provided 'as-is', without any express or implied - warranty. In no event will the authors be held liable for any damages - arising from the use of this software. - - Permission is granted to anyone to use this software for any purpose, - including commercial applications, and to alter it and redistribute it - freely, subject to the following restrictions: - - 1. The origin of this software must not be misrepresented; you must not - claim that you wrote the original software. If you use this software - in a product, an acknowledgment in the product documentation would be - appreciated but is not required. - 2. Altered source versions must be plainly marked as such, and must not be - misrepresented as being the original software. - 3. This notice may not be removed or altered from any source distribution. - - Jean-loup Gailly Mark Adler - jloup@gzip.org madler@alumni.caltech.edu - -If you use the zlib library in a product, we would appreciate *not* receiving -lengthy legal documents to sign. The sources are provided for free but without -warranty of any kind. The library has been entirely written by Jean-loup -Gailly and Mark Adler; it does not include third-party code. - -If you redistribute modified sources, we would appreciate that you include in -the file ChangeLog history information documenting your changes. Please read -the FAQ for more information on the distribution of modified source versions. diff --git a/contrib/libzlib-ng/adler32.c b/contrib/libzlib-ng/adler32.c deleted file mode 100644 index 495101dd5d0..00000000000 --- a/contrib/libzlib-ng/adler32.c +++ /dev/null @@ -1,177 +0,0 @@ -/* adler32.c -- compute the Adler-32 checksum of a data stream - * Copyright (C) 1995-2011 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* @(#) $Id$ */ - -#include "zutil.h" - -static uint32_t adler32_combine_(uint32_t adler1, uint32_t adler2, z_off64_t len2); - -#define BASE 65521U /* largest prime smaller than 65536 */ -#define NMAX 5552 -/* NMAX is the largest n such that 255n(n+1)/2 + (n+1)(BASE-1) <= 2^32-1 */ - -#define DO1(buf, i) {adler += (buf)[i]; sum2 += adler;} -#define DO2(buf, i) DO1(buf, i); DO1(buf, i+1); -#define DO4(buf, i) DO2(buf, i); DO2(buf, i+2); -#define DO8(buf, i) DO4(buf, i); DO4(buf, i+4); -#define DO16(buf) DO8(buf, 0); DO8(buf, 8); - -/* use NO_DIVIDE if your processor does not do division in hardware -- - try it both ways to see which is faster */ -#ifdef NO_DIVIDE -/* note that this assumes BASE is 65521, where 65536 % 65521 == 15 - (thank you to John Reiser for pointing this out) */ -# define CHOP(a) \ - do { \ - uint32_t tmp = a >> 16; \ - a &= 0xffff; \ - a += (tmp << 4) - tmp; \ - } while (0) -# define MOD28(a) \ - do { \ - CHOP(a); \ - if (a >= BASE) a -= BASE; \ - } while (0) -# define MOD(a) \ - do { \ - CHOP(a); \ - MOD28(a); \ - } while (0) -# define MOD63(a) \ - do { /* this assumes a is not negative */ \ - z_off64_t tmp = a >> 32; \ - a &= 0xffffffffL; \ - a += (tmp << 8) - (tmp << 5) + tmp; \ - tmp = a >> 16; \ - a &= 0xffffL; \ - a += (tmp << 4) - tmp; \ - tmp = a >> 16; \ - a &= 0xffffL; \ - a += (tmp << 4) - tmp; \ - if (a >= BASE) a -= BASE; \ - } while (0) -#else -# define MOD(a) a %= BASE -# define MOD28(a) a %= BASE -# define MOD63(a) a %= BASE -#endif - -/* ========================================================================= */ -uint32_t ZEXPORT adler32(uint32_t adler, const unsigned char *buf, uint32_t len) { - uint32_t sum2; - unsigned n; - - /* split Adler-32 into component sums */ - sum2 = (adler >> 16) & 0xffff; - adler &= 0xffff; - - /* in case user likes doing a byte at a time, keep it fast */ - if (len == 1) { - adler += buf[0]; - if (adler >= BASE) - adler -= BASE; - sum2 += adler; - if (sum2 >= BASE) - sum2 -= BASE; - return adler | (sum2 << 16); - } - - /* initial Adler-32 value (deferred check for len == 1 speed) */ - if (buf == Z_NULL) - return 1L; - - /* in case short lengths are provided, keep it somewhat fast */ - if (len < 16) { - while (len--) { - adler += *buf++; - sum2 += adler; - } - if (adler >= BASE) - adler -= BASE; - MOD28(sum2); /* only added so many BASE's */ - return adler | (sum2 << 16); - } - - /* do length NMAX blocks -- requires just one modulo operation */ - while (len >= NMAX) { - len -= NMAX; -#ifndef UNROLL_LESS - n = NMAX / 16; /* NMAX is divisible by 16 */ -#else - n = NMAX / 8; /* NMAX is divisible by 8 */ -#endif - do { -#ifndef UNROLL_LESS - DO16(buf); /* 16 sums unrolled */ - buf += 16; -#else - DO8(buf, 0); /* 8 sums unrolled */ - buf += 8; -#endif - } while (--n); - MOD(adler); - MOD(sum2); - } - - /* do remaining bytes (less than NMAX, still just one modulo) */ - if (len) { /* avoid modulos if none remaining */ -#ifndef UNROLL_LESS - while (len >= 16) { - len -= 16; - DO16(buf); - buf += 16; -#else - while (len >= 8) { - len -= 8; - DO8(buf, 0); - buf += 8; -#endif - } - while (len--) { - adler += *buf++; - sum2 += adler; - } - MOD(adler); - MOD(sum2); - } - - /* return recombined sums */ - return adler | (sum2 << 16); -} - -/* ========================================================================= */ -static uint32_t adler32_combine_(uint32_t adler1, uint32_t adler2, z_off64_t len2) { - uint32_t sum1; - uint32_t sum2; - unsigned rem; - - /* for negative len, return invalid adler32 as a clue for debugging */ - if (len2 < 0) - return 0xffffffff; - - /* the derivation of this formula is left as an exercise for the reader */ - MOD63(len2); /* assumes len2 >= 0 */ - rem = (unsigned)len2; - sum1 = adler1 & 0xffff; - sum2 = rem * sum1; - MOD(sum2); - sum1 += (adler2 & 0xffff) + BASE - 1; - sum2 += ((adler1 >> 16) & 0xffff) + ((adler2 >> 16) & 0xffff) + BASE - rem; - if (sum1 >= BASE) sum1 -= BASE; - if (sum1 >= BASE) sum1 -= BASE; - if (sum2 >= ((unsigned long)BASE << 1)) sum2 -= ((unsigned long)BASE << 1); - if (sum2 >= BASE) sum2 -= BASE; - return sum1 | (sum2 << 16); -} - -/* ========================================================================= */ -uint32_t ZEXPORT adler32_combine(uint32_t adler1, uint32_t adler2, z_off_t len2) { - return adler32_combine_(adler1, adler2, len2); -} - -uint32_t ZEXPORT adler32_combine64(uint32_t adler1, uint32_t adler2, z_off64_t len2) { - return adler32_combine_(adler1, adler2, len2); -} diff --git a/contrib/libzlib-ng/arch/.gitignore b/contrib/libzlib-ng/arch/.gitignore deleted file mode 100644 index 2c3af0a08cb..00000000000 --- a/contrib/libzlib-ng/arch/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -# ignore Makefiles; they're all automatically generated -Makefile diff --git a/contrib/libzlib-ng/arch/arm/Makefile.in b/contrib/libzlib-ng/arch/arm/Makefile.in deleted file mode 100644 index 759a1213ba8..00000000000 --- a/contrib/libzlib-ng/arch/arm/Makefile.in +++ /dev/null @@ -1,20 +0,0 @@ -# Makefile for zlib -# Copyright (C) 1995-2013 Jean-loup Gailly, Mark Adler -# For conditions of distribution and use, see copyright notice in zlib.h - -CC= -CFLAGS= -SFLAGS= -INCLUDES= - -SRCDIR= -SRCTOP= - -all: - - -mostlyclean: clean -clean: - rm -f *.o *.lo *~ \ - rm -rf objs - rm -f *.gcda *.gcno *.gcov diff --git a/contrib/libzlib-ng/arch/generic/Makefile.in b/contrib/libzlib-ng/arch/generic/Makefile.in deleted file mode 100644 index 759a1213ba8..00000000000 --- a/contrib/libzlib-ng/arch/generic/Makefile.in +++ /dev/null @@ -1,20 +0,0 @@ -# Makefile for zlib -# Copyright (C) 1995-2013 Jean-loup Gailly, Mark Adler -# For conditions of distribution and use, see copyright notice in zlib.h - -CC= -CFLAGS= -SFLAGS= -INCLUDES= - -SRCDIR= -SRCTOP= - -all: - - -mostlyclean: clean -clean: - rm -f *.o *.lo *~ \ - rm -rf objs - rm -f *.gcda *.gcno *.gcov diff --git a/contrib/libzlib-ng/arch/x86/INDEX b/contrib/libzlib-ng/arch/x86/INDEX deleted file mode 100644 index 9ee3802a2cd..00000000000 --- a/contrib/libzlib-ng/arch/x86/INDEX +++ /dev/null @@ -1,3 +0,0 @@ -fill_window_sse.c SSE2 optimized fill_window -deflate_quick.c SSE4 optimized deflate strategy for use as level 1 -crc_folding.c SSE4 + PCLMULQDQ optimized CRC folding implementation diff --git a/contrib/libzlib-ng/arch/x86/Makefile.in b/contrib/libzlib-ng/arch/x86/Makefile.in deleted file mode 100644 index 3604ba85226..00000000000 --- a/contrib/libzlib-ng/arch/x86/Makefile.in +++ /dev/null @@ -1,53 +0,0 @@ -# Makefile for zlib -# Copyright (C) 1995-2013 Jean-loup Gailly, Mark Adler -# For conditions of distribution and use, see copyright notice in zlib.h - -CC= -CFLAGS= -SFLAGS= -INCLUDES= - -SRCDIR= -SRCTOP= - -all: x86.o x86.lo fill_window_sse.o fill_window_sse.lo deflate_quick.o deflate_quick.lo insert_string_sse.o insert_string_sse.lo crc_folding.o crc_folding.lo - -x86.o: - $(CC) $(CFLAGS) $(INCLUDES) -c -o $@ $(SRCDIR)/x86.c - -x86.lo: - $(CC) $(SFLAGS) $(INCLUDES) -c -o $@ $(SRCDIR)/x86.c - -fill_window_sse.o: - $(CC) $(CFLAGS) -msse2 $(INCLUDES) -c -o $@ $(SRCDIR)/fill_window_sse.c - -fill_window_sse.lo: - $(CC) $(SFLAGS) -msse2 -DPIC $(INCLUDES) -c -o $@ $(SRCDIR)/fill_window_sse.c - -deflate_quick.o: - $(CC) $(CFLAGS) -msse4 $(INCLUDES) -c -o $@ $(SRCDIR)/deflate_quick.c - -deflate_quick.lo: - $(CC) $(SFLAGS) -msse4 -DPIC $(INCLUDES) -c -o $@ $(SRCDIR)/deflate_quick.c - -insert_string_sse.o: - $(CC) $(CFLAGS) -msse4 $(INCLUDES) -c -o $@ $(SRCDIR)/insert_string_sse.c - -insert_string_sse.lo: - $(CC) $(SFLAGS) -msse4 -DPIC $(INCLUDES) -c -o $@ $(SRCDIR)/insert_string_sse.c - -crc_folding.o: - $(CC) $(CFLAGS) -mpclmul -msse4 $(INCLUDES) -c -o $@ $(SRCDIR)/crc_folding.c - -crc_folding.lo: - $(CC) $(SFLAGS) -mpclmul -msse4 -DPIC $(INCLUDES) -c -o $@ $(SRCDIR)/crc_folding.c - - -mostlyclean: clean -clean: - rm -f *.o *.lo *~ \ - rm -rf objs - rm -f *.gcda *.gcno *.gcov - -distclean: - rm -f Makefile diff --git a/contrib/libzlib-ng/arch/x86/crc_folding.c b/contrib/libzlib-ng/arch/x86/crc_folding.c deleted file mode 100644 index b1fb6ec3a35..00000000000 --- a/contrib/libzlib-ng/arch/x86/crc_folding.c +++ /dev/null @@ -1,465 +0,0 @@ -/* - * Compute the CRC32 using a parallelized folding approach with the PCLMULQDQ - * instruction. - * - * A white paper describing this algorithm can be found at: - * http://www.intel.com/content/dam/www/public/us/en/documents/white-papers/fast-crc-computation-generic-polynomials-pclmulqdq-paper.pdf - * - * Copyright (C) 2013 Intel Corporation. All rights reserved. - * Authors: - * Wajdi Feghali - * Jim Guilford - * Vinodh Gopal - * Erdinc Ozturk - * Jim Kukunas - * - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -#ifdef X86_PCLMULQDQ_CRC - -#include -#include -#include - -#include "deflate.h" - - -#define CRC_LOAD(s) \ - do { \ - __m128i xmm_crc0 = _mm_loadu_si128((__m128i *)s->crc0 + 0);\ - __m128i xmm_crc1 = _mm_loadu_si128((__m128i *)s->crc0 + 1);\ - __m128i xmm_crc2 = _mm_loadu_si128((__m128i *)s->crc0 + 2);\ - __m128i xmm_crc3 = _mm_loadu_si128((__m128i *)s->crc0 + 3);\ - __m128i xmm_crc_part = _mm_loadu_si128((__m128i *)s->crc0 + 4); - -#define CRC_SAVE(s) \ - _mm_storeu_si128((__m128i *)s->crc0 + 0, xmm_crc0);\ - _mm_storeu_si128((__m128i *)s->crc0 + 1, xmm_crc1);\ - _mm_storeu_si128((__m128i *)s->crc0 + 2, xmm_crc2);\ - _mm_storeu_si128((__m128i *)s->crc0 + 3, xmm_crc3);\ - _mm_storeu_si128((__m128i *)s->crc0 + 4, xmm_crc_part);\ - } while (0); - -ZLIB_INTERNAL void crc_fold_init(deflate_state *const s) { - CRC_LOAD(s) - - xmm_crc0 = _mm_cvtsi32_si128(0x9db42487); - xmm_crc1 = _mm_setzero_si128(); - xmm_crc2 = _mm_setzero_si128(); - xmm_crc3 = _mm_setzero_si128(); - - CRC_SAVE(s) - - s->strm->adler = 0; -} - -local void fold_1(deflate_state *const s, __m128i *xmm_crc0, __m128i *xmm_crc1, __m128i *xmm_crc2, __m128i *xmm_crc3) { - const __m128i xmm_fold4 = _mm_set_epi32( - 0x00000001, 0x54442bd4, - 0x00000001, 0xc6e41596); - - __m128i x_tmp3; - __m128 ps_crc0, ps_crc3, ps_res; - - x_tmp3 = *xmm_crc3; - - *xmm_crc3 = *xmm_crc0; - *xmm_crc0 = _mm_clmulepi64_si128(*xmm_crc0, xmm_fold4, 0x01); - *xmm_crc3 = _mm_clmulepi64_si128(*xmm_crc3, xmm_fold4, 0x10); - ps_crc0 = _mm_castsi128_ps(*xmm_crc0); - ps_crc3 = _mm_castsi128_ps(*xmm_crc3); - ps_res = _mm_xor_ps(ps_crc0, ps_crc3); - - *xmm_crc0 = *xmm_crc1; - *xmm_crc1 = *xmm_crc2; - *xmm_crc2 = x_tmp3; - *xmm_crc3 = _mm_castps_si128(ps_res); -} - -local void fold_2(deflate_state *const s, __m128i *xmm_crc0, __m128i *xmm_crc1, __m128i *xmm_crc2, __m128i *xmm_crc3) { - const __m128i xmm_fold4 = _mm_set_epi32( - 0x00000001, 0x54442bd4, - 0x00000001, 0xc6e41596); - - __m128i x_tmp3, x_tmp2; - __m128 ps_crc0, ps_crc1, ps_crc2, ps_crc3, ps_res31, ps_res20; - - x_tmp3 = *xmm_crc3; - x_tmp2 = *xmm_crc2; - - *xmm_crc3 = *xmm_crc1; - *xmm_crc1 = _mm_clmulepi64_si128(*xmm_crc1, xmm_fold4, 0x01); - *xmm_crc3 = _mm_clmulepi64_si128(*xmm_crc3, xmm_fold4, 0x10); - ps_crc3 = _mm_castsi128_ps(*xmm_crc3); - ps_crc1 = _mm_castsi128_ps(*xmm_crc1); - ps_res31 = _mm_xor_ps(ps_crc3, ps_crc1); - - *xmm_crc2 = *xmm_crc0; - *xmm_crc0 = _mm_clmulepi64_si128(*xmm_crc0, xmm_fold4, 0x01); - *xmm_crc2 = _mm_clmulepi64_si128(*xmm_crc2, xmm_fold4, 0x10); - ps_crc0 = _mm_castsi128_ps(*xmm_crc0); - ps_crc2 = _mm_castsi128_ps(*xmm_crc2); - ps_res20 = _mm_xor_ps(ps_crc0, ps_crc2); - - *xmm_crc0 = x_tmp2; - *xmm_crc1 = x_tmp3; - *xmm_crc2 = _mm_castps_si128(ps_res20); - *xmm_crc3 = _mm_castps_si128(ps_res31); -} - -local void fold_3(deflate_state *const s, __m128i *xmm_crc0, __m128i *xmm_crc1, __m128i *xmm_crc2, __m128i *xmm_crc3) { - const __m128i xmm_fold4 = _mm_set_epi32( - 0x00000001, 0x54442bd4, - 0x00000001, 0xc6e41596); - - __m128i x_tmp3; - __m128 ps_crc0, ps_crc1, ps_crc2, ps_crc3, ps_res32, ps_res21, ps_res10; - - x_tmp3 = *xmm_crc3; - - *xmm_crc3 = *xmm_crc2; - *xmm_crc2 = _mm_clmulepi64_si128(*xmm_crc2, xmm_fold4, 0x01); - *xmm_crc3 = _mm_clmulepi64_si128(*xmm_crc3, xmm_fold4, 0x10); - ps_crc2 = _mm_castsi128_ps(*xmm_crc2); - ps_crc3 = _mm_castsi128_ps(*xmm_crc3); - ps_res32 = _mm_xor_ps(ps_crc2, ps_crc3); - - *xmm_crc2 = *xmm_crc1; - *xmm_crc1 = _mm_clmulepi64_si128(*xmm_crc1, xmm_fold4, 0x01); - *xmm_crc2 = _mm_clmulepi64_si128(*xmm_crc2, xmm_fold4, 0x10); - ps_crc1 = _mm_castsi128_ps(*xmm_crc1); - ps_crc2 = _mm_castsi128_ps(*xmm_crc2); - ps_res21 = _mm_xor_ps(ps_crc1, ps_crc2); - - *xmm_crc1 = *xmm_crc0; - *xmm_crc0 = _mm_clmulepi64_si128(*xmm_crc0, xmm_fold4, 0x01); - *xmm_crc1 = _mm_clmulepi64_si128(*xmm_crc1, xmm_fold4, 0x10); - ps_crc0 = _mm_castsi128_ps(*xmm_crc0); - ps_crc1 = _mm_castsi128_ps(*xmm_crc1); - ps_res10 = _mm_xor_ps(ps_crc0, ps_crc1); - - *xmm_crc0 = x_tmp3; - *xmm_crc1 = _mm_castps_si128(ps_res10); - *xmm_crc2 = _mm_castps_si128(ps_res21); - *xmm_crc3 = _mm_castps_si128(ps_res32); -} - -local void fold_4(deflate_state *const s, __m128i *xmm_crc0, __m128i *xmm_crc1, __m128i *xmm_crc2, __m128i *xmm_crc3) { - const __m128i xmm_fold4 = _mm_set_epi32( - 0x00000001, 0x54442bd4, - 0x00000001, 0xc6e41596); - - __m128i x_tmp0, x_tmp1, x_tmp2, x_tmp3; - __m128 ps_crc0, ps_crc1, ps_crc2, ps_crc3; - __m128 ps_t0, ps_t1, ps_t2, ps_t3; - __m128 ps_res0, ps_res1, ps_res2, ps_res3; - - x_tmp0 = *xmm_crc0; - x_tmp1 = *xmm_crc1; - x_tmp2 = *xmm_crc2; - x_tmp3 = *xmm_crc3; - - *xmm_crc0 = _mm_clmulepi64_si128(*xmm_crc0, xmm_fold4, 0x01); - x_tmp0 = _mm_clmulepi64_si128(x_tmp0, xmm_fold4, 0x10); - ps_crc0 = _mm_castsi128_ps(*xmm_crc0); - ps_t0 = _mm_castsi128_ps(x_tmp0); - ps_res0 = _mm_xor_ps(ps_crc0, ps_t0); - - *xmm_crc1 = _mm_clmulepi64_si128(*xmm_crc1, xmm_fold4, 0x01); - x_tmp1 = _mm_clmulepi64_si128(x_tmp1, xmm_fold4, 0x10); - ps_crc1 = _mm_castsi128_ps(*xmm_crc1); - ps_t1 = _mm_castsi128_ps(x_tmp1); - ps_res1 = _mm_xor_ps(ps_crc1, ps_t1); - - *xmm_crc2 = _mm_clmulepi64_si128(*xmm_crc2, xmm_fold4, 0x01); - x_tmp2 = _mm_clmulepi64_si128(x_tmp2, xmm_fold4, 0x10); - ps_crc2 = _mm_castsi128_ps(*xmm_crc2); - ps_t2 = _mm_castsi128_ps(x_tmp2); - ps_res2 = _mm_xor_ps(ps_crc2, ps_t2); - - *xmm_crc3 = _mm_clmulepi64_si128(*xmm_crc3, xmm_fold4, 0x01); - x_tmp3 = _mm_clmulepi64_si128(x_tmp3, xmm_fold4, 0x10); - ps_crc3 = _mm_castsi128_ps(*xmm_crc3); - ps_t3 = _mm_castsi128_ps(x_tmp3); - ps_res3 = _mm_xor_ps(ps_crc3, ps_t3); - - *xmm_crc0 = _mm_castps_si128(ps_res0); - *xmm_crc1 = _mm_castps_si128(ps_res1); - *xmm_crc2 = _mm_castps_si128(ps_res2); - *xmm_crc3 = _mm_castps_si128(ps_res3); -} - -local const unsigned ALIGNED_(32) pshufb_shf_table[60] = { - 0x84838281, 0x88878685, 0x8c8b8a89, 0x008f8e8d, /* shl 15 (16 - 1)/shr1 */ - 0x85848382, 0x89888786, 0x8d8c8b8a, 0x01008f8e, /* shl 14 (16 - 3)/shr2 */ - 0x86858483, 0x8a898887, 0x8e8d8c8b, 0x0201008f, /* shl 13 (16 - 4)/shr3 */ - 0x87868584, 0x8b8a8988, 0x8f8e8d8c, 0x03020100, /* shl 12 (16 - 4)/shr4 */ - 0x88878685, 0x8c8b8a89, 0x008f8e8d, 0x04030201, /* shl 11 (16 - 5)/shr5 */ - 0x89888786, 0x8d8c8b8a, 0x01008f8e, 0x05040302, /* shl 10 (16 - 6)/shr6 */ - 0x8a898887, 0x8e8d8c8b, 0x0201008f, 0x06050403, /* shl 9 (16 - 7)/shr7 */ - 0x8b8a8988, 0x8f8e8d8c, 0x03020100, 0x07060504, /* shl 8 (16 - 8)/shr8 */ - 0x8c8b8a89, 0x008f8e8d, 0x04030201, 0x08070605, /* shl 7 (16 - 9)/shr9 */ - 0x8d8c8b8a, 0x01008f8e, 0x05040302, 0x09080706, /* shl 6 (16 -10)/shr10*/ - 0x8e8d8c8b, 0x0201008f, 0x06050403, 0x0a090807, /* shl 5 (16 -11)/shr11*/ - 0x8f8e8d8c, 0x03020100, 0x07060504, 0x0b0a0908, /* shl 4 (16 -12)/shr12*/ - 0x008f8e8d, 0x04030201, 0x08070605, 0x0c0b0a09, /* shl 3 (16 -13)/shr13*/ - 0x01008f8e, 0x05040302, 0x09080706, 0x0d0c0b0a, /* shl 2 (16 -14)/shr14*/ - 0x0201008f, 0x06050403, 0x0a090807, 0x0e0d0c0b /* shl 1 (16 -15)/shr15*/ -}; - -local void partial_fold(deflate_state *const s, const size_t len, __m128i *xmm_crc0, __m128i *xmm_crc1, - __m128i *xmm_crc2, __m128i *xmm_crc3, __m128i *xmm_crc_part) { - - const __m128i xmm_fold4 = _mm_set_epi32( - 0x00000001, 0x54442bd4, - 0x00000001, 0xc6e41596); - const __m128i xmm_mask3 = _mm_set1_epi32(0x80808080); - - __m128i xmm_shl, xmm_shr, xmm_tmp1, xmm_tmp2, xmm_tmp3; - __m128i xmm_a0_0, xmm_a0_1; - __m128 ps_crc3, psa0_0, psa0_1, ps_res; - - xmm_shl = _mm_load_si128((__m128i *)pshufb_shf_table + (len - 1)); - xmm_shr = xmm_shl; - xmm_shr = _mm_xor_si128(xmm_shr, xmm_mask3); - - xmm_a0_0 = _mm_shuffle_epi8(*xmm_crc0, xmm_shl); - - *xmm_crc0 = _mm_shuffle_epi8(*xmm_crc0, xmm_shr); - xmm_tmp1 = _mm_shuffle_epi8(*xmm_crc1, xmm_shl); - *xmm_crc0 = _mm_or_si128(*xmm_crc0, xmm_tmp1); - - *xmm_crc1 = _mm_shuffle_epi8(*xmm_crc1, xmm_shr); - xmm_tmp2 = _mm_shuffle_epi8(*xmm_crc2, xmm_shl); - *xmm_crc1 = _mm_or_si128(*xmm_crc1, xmm_tmp2); - - *xmm_crc2 = _mm_shuffle_epi8(*xmm_crc2, xmm_shr); - xmm_tmp3 = _mm_shuffle_epi8(*xmm_crc3, xmm_shl); - *xmm_crc2 = _mm_or_si128(*xmm_crc2, xmm_tmp3); - - *xmm_crc3 = _mm_shuffle_epi8(*xmm_crc3, xmm_shr); - *xmm_crc_part = _mm_shuffle_epi8(*xmm_crc_part, xmm_shl); - *xmm_crc3 = _mm_or_si128(*xmm_crc3, *xmm_crc_part); - - xmm_a0_1 = _mm_clmulepi64_si128(xmm_a0_0, xmm_fold4, 0x10); - xmm_a0_0 = _mm_clmulepi64_si128(xmm_a0_0, xmm_fold4, 0x01); - - ps_crc3 = _mm_castsi128_ps(*xmm_crc3); - psa0_0 = _mm_castsi128_ps(xmm_a0_0); - psa0_1 = _mm_castsi128_ps(xmm_a0_1); - - ps_res = _mm_xor_ps(ps_crc3, psa0_0); - ps_res = _mm_xor_ps(ps_res, psa0_1); - - *xmm_crc3 = _mm_castps_si128(ps_res); -} - -ZLIB_INTERNAL void crc_fold_copy(deflate_state *const s, unsigned char *dst, const unsigned char *src, long len) { - unsigned long algn_diff; - __m128i xmm_t0, xmm_t1, xmm_t2, xmm_t3; - - CRC_LOAD(s) - - if (len < 16) { - if (len == 0) - return; - xmm_crc_part = _mm_loadu_si128((__m128i *)src); - goto partial; - } - - algn_diff = (0 - (uintptr_t)src) & 0xF; - if (algn_diff) { - xmm_crc_part = _mm_loadu_si128((__m128i *)src); - _mm_storeu_si128((__m128i *)dst, xmm_crc_part); - - dst += algn_diff; - src += algn_diff; - len -= algn_diff; - - partial_fold(s, algn_diff, &xmm_crc0, &xmm_crc1, &xmm_crc2, &xmm_crc3, - &xmm_crc_part); - } - - while ((len -= 64) >= 0) { - xmm_t0 = _mm_load_si128((__m128i *)src); - xmm_t1 = _mm_load_si128((__m128i *)src + 1); - xmm_t2 = _mm_load_si128((__m128i *)src + 2); - xmm_t3 = _mm_load_si128((__m128i *)src + 3); - - fold_4(s, &xmm_crc0, &xmm_crc1, &xmm_crc2, &xmm_crc3); - - _mm_storeu_si128((__m128i *)dst, xmm_t0); - _mm_storeu_si128((__m128i *)dst + 1, xmm_t1); - _mm_storeu_si128((__m128i *)dst + 2, xmm_t2); - _mm_storeu_si128((__m128i *)dst + 3, xmm_t3); - - xmm_crc0 = _mm_xor_si128(xmm_crc0, xmm_t0); - xmm_crc1 = _mm_xor_si128(xmm_crc1, xmm_t1); - xmm_crc2 = _mm_xor_si128(xmm_crc2, xmm_t2); - xmm_crc3 = _mm_xor_si128(xmm_crc3, xmm_t3); - - src += 64; - dst += 64; - } - - /* - * len = num bytes left - 64 - */ - if (len + 16 >= 0) { - len += 16; - - xmm_t0 = _mm_load_si128((__m128i *)src); - xmm_t1 = _mm_load_si128((__m128i *)src + 1); - xmm_t2 = _mm_load_si128((__m128i *)src + 2); - - fold_3(s, &xmm_crc0, &xmm_crc1, &xmm_crc2, &xmm_crc3); - - _mm_storeu_si128((__m128i *)dst, xmm_t0); - _mm_storeu_si128((__m128i *)dst + 1, xmm_t1); - _mm_storeu_si128((__m128i *)dst + 2, xmm_t2); - - xmm_crc1 = _mm_xor_si128(xmm_crc1, xmm_t0); - xmm_crc2 = _mm_xor_si128(xmm_crc2, xmm_t1); - xmm_crc3 = _mm_xor_si128(xmm_crc3, xmm_t2); - - if (len == 0) - goto done; - - dst += 48; - xmm_crc_part = _mm_load_si128((__m128i *)src + 3); - } else if (len + 32 >= 0) { - len += 32; - - xmm_t0 = _mm_load_si128((__m128i *)src); - xmm_t1 = _mm_load_si128((__m128i *)src + 1); - - fold_2(s, &xmm_crc0, &xmm_crc1, &xmm_crc2, &xmm_crc3); - - _mm_storeu_si128((__m128i *)dst, xmm_t0); - _mm_storeu_si128((__m128i *)dst + 1, xmm_t1); - - xmm_crc2 = _mm_xor_si128(xmm_crc2, xmm_t0); - xmm_crc3 = _mm_xor_si128(xmm_crc3, xmm_t1); - - if (len == 0) - goto done; - - dst += 32; - xmm_crc_part = _mm_load_si128((__m128i *)src + 2); - } else if (len + 48 >= 0) { - len += 48; - - xmm_t0 = _mm_load_si128((__m128i *)src); - - fold_1(s, &xmm_crc0, &xmm_crc1, &xmm_crc2, &xmm_crc3); - - _mm_storeu_si128((__m128i *)dst, xmm_t0); - - xmm_crc3 = _mm_xor_si128(xmm_crc3, xmm_t0); - - if (len == 0) - goto done; - - dst += 16; - xmm_crc_part = _mm_load_si128((__m128i *)src + 1); - } else { - len += 64; - if (len == 0) - goto done; - xmm_crc_part = _mm_load_si128((__m128i *)src); - } - -partial: - _mm_storeu_si128((__m128i *)dst, xmm_crc_part); - partial_fold(s, len, &xmm_crc0, &xmm_crc1, &xmm_crc2, &xmm_crc3, - &xmm_crc_part); -done: - CRC_SAVE(s) -} - -local const unsigned ALIGNED_(16) crc_k[] = { - 0xccaa009e, 0x00000000, /* rk1 */ - 0x751997d0, 0x00000001, /* rk2 */ - 0xccaa009e, 0x00000000, /* rk5 */ - 0x63cd6124, 0x00000001, /* rk6 */ - 0xf7011640, 0x00000001, /* rk7 */ - 0xdb710640, 0x00000001 /* rk8 */ -}; - -local const unsigned ALIGNED_(16) crc_mask[4] = { - 0xFFFFFFFF, 0xFFFFFFFF, 0x00000000, 0x00000000 -}; - -local const unsigned ALIGNED_(16) crc_mask2[4] = { - 0x00000000, 0xFFFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF -}; - -uint32_t ZLIB_INTERNAL crc_fold_512to32(deflate_state *const s) { - const __m128i xmm_mask = _mm_load_si128((__m128i *)crc_mask); - const __m128i xmm_mask2 = _mm_load_si128((__m128i *)crc_mask2); - - uint32_t crc; - __m128i x_tmp0, x_tmp1, x_tmp2, crc_fold; - - CRC_LOAD(s) - - /* - * k1 - */ - crc_fold = _mm_load_si128((__m128i *)crc_k); - - x_tmp0 = _mm_clmulepi64_si128(xmm_crc0, crc_fold, 0x10); - xmm_crc0 = _mm_clmulepi64_si128(xmm_crc0, crc_fold, 0x01); - xmm_crc1 = _mm_xor_si128(xmm_crc1, x_tmp0); - xmm_crc1 = _mm_xor_si128(xmm_crc1, xmm_crc0); - - x_tmp1 = _mm_clmulepi64_si128(xmm_crc1, crc_fold, 0x10); - xmm_crc1 = _mm_clmulepi64_si128(xmm_crc1, crc_fold, 0x01); - xmm_crc2 = _mm_xor_si128(xmm_crc2, x_tmp1); - xmm_crc2 = _mm_xor_si128(xmm_crc2, xmm_crc1); - - x_tmp2 = _mm_clmulepi64_si128(xmm_crc2, crc_fold, 0x10); - xmm_crc2 = _mm_clmulepi64_si128(xmm_crc2, crc_fold, 0x01); - xmm_crc3 = _mm_xor_si128(xmm_crc3, x_tmp2); - xmm_crc3 = _mm_xor_si128(xmm_crc3, xmm_crc2); - - /* - * k5 - */ - crc_fold = _mm_load_si128((__m128i *)crc_k + 1); - - xmm_crc0 = xmm_crc3; - xmm_crc3 = _mm_clmulepi64_si128(xmm_crc3, crc_fold, 0); - xmm_crc0 = _mm_srli_si128(xmm_crc0, 8); - xmm_crc3 = _mm_xor_si128(xmm_crc3, xmm_crc0); - - xmm_crc0 = xmm_crc3; - xmm_crc3 = _mm_slli_si128(xmm_crc3, 4); - xmm_crc3 = _mm_clmulepi64_si128(xmm_crc3, crc_fold, 0x10); - xmm_crc3 = _mm_xor_si128(xmm_crc3, xmm_crc0); - xmm_crc3 = _mm_and_si128(xmm_crc3, xmm_mask2); - - /* - * k7 - */ - xmm_crc1 = xmm_crc3; - xmm_crc2 = xmm_crc3; - crc_fold = _mm_load_si128((__m128i *)crc_k + 2); - - xmm_crc3 = _mm_clmulepi64_si128(xmm_crc3, crc_fold, 0); - xmm_crc3 = _mm_xor_si128(xmm_crc3, xmm_crc2); - xmm_crc3 = _mm_and_si128(xmm_crc3, xmm_mask); - - xmm_crc2 = xmm_crc3; - xmm_crc3 = _mm_clmulepi64_si128(xmm_crc3, crc_fold, 0x10); - xmm_crc3 = _mm_xor_si128(xmm_crc3, xmm_crc2); - xmm_crc3 = _mm_xor_si128(xmm_crc3, xmm_crc1); - - crc = _mm_extract_epi32(xmm_crc3, 2); - return ~crc; - CRC_SAVE(s) -} - -#endif - diff --git a/contrib/libzlib-ng/arch/x86/deflate_quick.c b/contrib/libzlib-ng/arch/x86/deflate_quick.c deleted file mode 100644 index b5190d59898..00000000000 --- a/contrib/libzlib-ng/arch/x86/deflate_quick.c +++ /dev/null @@ -1,2371 +0,0 @@ -/* - * The deflate_quick deflate strategy, designed to be used when cycles are - * at a premium. - * - * Copyright (C) 2013 Intel Corporation. All rights reserved. - * Authors: - * Wajdi Feghali - * Jim Guilford - * Vinodh Gopal - * Erdinc Ozturk - * Jim Kukunas - * - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -#include -#ifdef _MSC_VER -# include -#endif -#include "deflate.h" - -extern void fill_window_sse(deflate_state *s); -extern void flush_pending(z_stream *strm); - -local inline long compare258(const unsigned char *const src0, const unsigned char *const src1) { -#ifdef _MSC_VER - long cnt; - - cnt = 0; - do { -#define mode _SIDD_UBYTE_OPS | _SIDD_CMP_EQUAL_EACH | _SIDD_NEGATIVE_POLARITY - - int ret; - __m128i xmm_src0, xmm_src1; - - xmm_src0 = _mm_loadu_si128((__m128i *)(src0 + cnt)); - xmm_src1 = _mm_loadu_si128((__m128i *)(src1 + cnt)); - ret = _mm_cmpestri(xmm_src0, 16, xmm_src1, 16, mode); - if (_mm_cmpestrc(xmm_src0, 16, xmm_src1, 16, mode)) { - cnt += ret; - break; - } - cnt += 16; - - xmm_src0 = _mm_loadu_si128((__m128i *)(src0 + cnt)); - xmm_src1 = _mm_loadu_si128((__m128i *)(src1 + cnt)); - ret = _mm_cmpestri(xmm_src0, 16, xmm_src1, 16, mode); - if (_mm_cmpestrc(xmm_src0, 16, xmm_src1, 16, mode)) { - cnt += ret; - break; - } - cnt += 16; - } while (cnt < 256); - - if (*(unsigned short *)(src0 + cnt) == *(unsigned short *)(src1 + cnt)) { - cnt += 2; - } else if (*(src0 + cnt) == *(src1 + cnt)) { - cnt++; - } - return cnt; -#else - uintptr_t ax, dx, cx; - __m128i xmm_src0; - - ax = 16; - dx = 16; - /* set cx to something, otherwise gcc thinks it's used - uninitalised */ - cx = 0; - - __asm__ __volatile__ ( - "1:" - "movdqu -16(%[src0], %[ax]), %[xmm_src0]\n\t" - "pcmpestri $0x18, -16(%[src1], %[ax]), %[xmm_src0]\n\t" - "jc 2f\n\t" - "add $16, %[ax]\n\t" - - "movdqu -16(%[src0], %[ax]), %[xmm_src0]\n\t" - "pcmpestri $0x18, -16(%[src1], %[ax]), %[xmm_src0]\n\t" - "jc 2f\n\t" - "add $16, %[ax]\n\t" - - "cmp $256 + 16, %[ax]\n\t" - "jb 1b\n\t" - -#ifdef X86 - "movzwl -16(%[src0], %[ax]), %[dx]\n\t" -#else - "movzwq -16(%[src0], %[ax]), %[dx]\n\t" -#endif - "xorw -16(%[src1], %[ax]), %%dx\n\t" - "jnz 3f\n\t" - - "add $2, %[ax]\n\t" - "jmp 4f\n\t" - "3:\n\t" - "rep; bsf %[dx], %[cx]\n\t" - "shr $3, %[cx]\n\t" - "2:" - "add %[cx], %[ax]\n\t" - "4:" - : [ax] "+a" (ax), - [cx] "+c" (cx), - [dx] "+d" (dx), - [xmm_src0] "=x" (xmm_src0) - : [src0] "r" (src0), - [src1] "r" (src1) - : "cc" - ); - return ax - 16; -#endif -} - -local const unsigned quick_len_codes[MAX_MATCH-MIN_MATCH+1]; -local const unsigned quick_dist_codes[8192]; - -local inline void quick_send_bits(deflate_state *const s, const int value, const int length) { - unsigned code, out, w, b; - - out = s->bi_buf; - w = s->bi_valid; - - code = value << s->bi_valid; - out |= code; - w += length; - - if (s->pending + 4 >= s->pending_buf_size) - flush_pending(s->strm); - - *(unsigned *)(s->pending_buf + s->pending) = out; - - b = w >> 3; - s->pending += b; - s->bi_buf = out >> (b << 3); - s->bi_valid = w - (b << 3); -} - -local inline void static_emit_ptr(deflate_state *const s, const int lc, const unsigned dist) { - unsigned code, len; - - code = quick_len_codes[lc] >> 8; - len = quick_len_codes[lc] & 0xFF; - quick_send_bits(s, code, len); - - code = quick_dist_codes[dist-1] >> 8; - len = quick_dist_codes[dist-1] & 0xFF; - quick_send_bits(s, code, len); -} - -const ct_data static_ltree[L_CODES+2]; - -local inline void static_emit_lit(deflate_state *const s, const int lit) { - quick_send_bits(s, static_ltree[lit].Code, static_ltree[lit].Len); - Tracecv(isgraph(lit), (stderr, " '%c' ", lit)); -} - -local void static_emit_tree(deflate_state *const s, const int flush) { - unsigned last; - - last = flush == Z_FINISH ? 1 : 0; - send_bits(s, (STATIC_TREES << 1)+ last, 3); -} - - -local void static_emit_end_block(deflate_state *const s, int last) { - send_code(s, END_BLOCK, static_ltree); - - if (last) - bi_windup(s); - - s->block_start = s->strstart; - flush_pending(s->strm); -} - -local inline Pos quick_insert_string(deflate_state *const s, const Pos str) { - Pos ret; - unsigned h = 0; - -#ifdef _MSC_VER - h = _mm_crc32_u32(h, *(unsigned *)(s->window + str)); -#else - __asm__ __volatile__ ( - "crc32l (%[window], %[str], 1), %0\n\t" - : "+r" (h) - : [window] "r" (s->window), - [str] "r" ((uintptr_t)str) - ); -#endif - - ret = s->head[h & s->hash_mask]; - s->head[h & s->hash_mask] = str; - return ret; -} - -ZLIB_INTERNAL block_state deflate_quick(deflate_state *s, int flush) { - IPos hash_head; - unsigned dist, match_len; - - static_emit_tree(s, flush); - - do { - if (s->lookahead < MIN_LOOKAHEAD) { - fill_window_sse(s); - if (s->lookahead < MIN_LOOKAHEAD && flush == Z_NO_FLUSH) { - static_emit_end_block(s, 0); - return need_more; - } - if (s->lookahead == 0) - break; - } - - if (s->lookahead >= MIN_MATCH) { - hash_head = quick_insert_string(s, s->strstart); - dist = s->strstart - hash_head; - - if ((dist-1) < (s->w_size - 1)) { - match_len = compare258(s->window + s->strstart, s->window + s->strstart - dist); - - if (match_len >= MIN_MATCH) { - if (match_len > s->lookahead) - match_len = s->lookahead; - - if (match_len > MAX_MATCH) - match_len = MAX_MATCH; - - static_emit_ptr(s, match_len - MIN_MATCH, s->strstart - hash_head); - s->lookahead -= match_len; - s->strstart += match_len; - continue; - } - } - } - - static_emit_lit(s, s->window[s->strstart]); - s->strstart++; - s->lookahead--; - } while (s->strm->avail_out != 0); - - if (s->strm->avail_out == 0 && flush != Z_FINISH) - return need_more; - - s->insert = s->strstart < MIN_MATCH - 1 ? s->strstart : MIN_MATCH-1; - if (flush == Z_FINISH) { - static_emit_end_block(s, 1); - if (s->strm->avail_out == 0) - return s->strm->avail_in == 0 ? finish_started : need_more; - else - return finish_done; - } - - static_emit_end_block(s, 0); - return block_done; -} - -local const unsigned quick_len_codes[MAX_MATCH-MIN_MATCH+1] = { - 0x00004007, 0x00002007, 0x00006007, 0x00001007, - 0x00005007, 0x00003007, 0x00007007, 0x00000807, - 0x00004808, 0x0000c808, 0x00002808, 0x0000a808, - 0x00006808, 0x0000e808, 0x00001808, 0x00009808, - 0x00005809, 0x0000d809, 0x00015809, 0x0001d809, - 0x00003809, 0x0000b809, 0x00013809, 0x0001b809, - 0x00007809, 0x0000f809, 0x00017809, 0x0001f809, - 0x00000409, 0x00008409, 0x00010409, 0x00018409, - 0x0000440a, 0x0000c40a, 0x0001440a, 0x0001c40a, - 0x0002440a, 0x0002c40a, 0x0003440a, 0x0003c40a, - 0x0000240a, 0x0000a40a, 0x0001240a, 0x0001a40a, - 0x0002240a, 0x0002a40a, 0x0003240a, 0x0003a40a, - 0x0000640a, 0x0000e40a, 0x0001640a, 0x0001e40a, - 0x0002640a, 0x0002e40a, 0x0003640a, 0x0003e40a, - 0x0000140a, 0x0000940a, 0x0001140a, 0x0001940a, - 0x0002140a, 0x0002940a, 0x0003140a, 0x0003940a, - 0x0000540b, 0x0000d40b, 0x0001540b, 0x0001d40b, - 0x0002540b, 0x0002d40b, 0x0003540b, 0x0003d40b, - 0x0004540b, 0x0004d40b, 0x0005540b, 0x0005d40b, - 0x0006540b, 0x0006d40b, 0x0007540b, 0x0007d40b, - 0x0000340b, 0x0000b40b, 0x0001340b, 0x0001b40b, - 0x0002340b, 0x0002b40b, 0x0003340b, 0x0003b40b, - 0x0004340b, 0x0004b40b, 0x0005340b, 0x0005b40b, - 0x0006340b, 0x0006b40b, 0x0007340b, 0x0007b40b, - 0x0000740b, 0x0000f40b, 0x0001740b, 0x0001f40b, - 0x0002740b, 0x0002f40b, 0x0003740b, 0x0003f40b, - 0x0004740b, 0x0004f40b, 0x0005740b, 0x0005f40b, - 0x0006740b, 0x0006f40b, 0x0007740b, 0x0007f40b, - 0x0000030c, 0x0001030c, 0x0002030c, 0x0003030c, - 0x0004030c, 0x0005030c, 0x0006030c, 0x0007030c, - 0x0008030c, 0x0009030c, 0x000a030c, 0x000b030c, - 0x000c030c, 0x000d030c, 0x000e030c, 0x000f030c, - 0x0000830d, 0x0001830d, 0x0002830d, 0x0003830d, - 0x0004830d, 0x0005830d, 0x0006830d, 0x0007830d, - 0x0008830d, 0x0009830d, 0x000a830d, 0x000b830d, - 0x000c830d, 0x000d830d, 0x000e830d, 0x000f830d, - 0x0010830d, 0x0011830d, 0x0012830d, 0x0013830d, - 0x0014830d, 0x0015830d, 0x0016830d, 0x0017830d, - 0x0018830d, 0x0019830d, 0x001a830d, 0x001b830d, - 0x001c830d, 0x001d830d, 0x001e830d, 0x001f830d, - 0x0000430d, 0x0001430d, 0x0002430d, 0x0003430d, - 0x0004430d, 0x0005430d, 0x0006430d, 0x0007430d, - 0x0008430d, 0x0009430d, 0x000a430d, 0x000b430d, - 0x000c430d, 0x000d430d, 0x000e430d, 0x000f430d, - 0x0010430d, 0x0011430d, 0x0012430d, 0x0013430d, - 0x0014430d, 0x0015430d, 0x0016430d, 0x0017430d, - 0x0018430d, 0x0019430d, 0x001a430d, 0x001b430d, - 0x001c430d, 0x001d430d, 0x001e430d, 0x001f430d, - 0x0000c30d, 0x0001c30d, 0x0002c30d, 0x0003c30d, - 0x0004c30d, 0x0005c30d, 0x0006c30d, 0x0007c30d, - 0x0008c30d, 0x0009c30d, 0x000ac30d, 0x000bc30d, - 0x000cc30d, 0x000dc30d, 0x000ec30d, 0x000fc30d, - 0x0010c30d, 0x0011c30d, 0x0012c30d, 0x0013c30d, - 0x0014c30d, 0x0015c30d, 0x0016c30d, 0x0017c30d, - 0x0018c30d, 0x0019c30d, 0x001ac30d, 0x001bc30d, - 0x001cc30d, 0x001dc30d, 0x001ec30d, 0x001fc30d, - 0x0000230d, 0x0001230d, 0x0002230d, 0x0003230d, - 0x0004230d, 0x0005230d, 0x0006230d, 0x0007230d, - 0x0008230d, 0x0009230d, 0x000a230d, 0x000b230d, - 0x000c230d, 0x000d230d, 0x000e230d, 0x000f230d, - 0x0010230d, 0x0011230d, 0x0012230d, 0x0013230d, - 0x0014230d, 0x0015230d, 0x0016230d, 0x0017230d, - 0x0018230d, 0x0019230d, 0x001a230d, 0x001b230d, - 0x001c230d, 0x001d230d, 0x001e230d, 0x0000a308, -}; - -local const unsigned quick_dist_codes[8192] = { - 0x00000005, 0x00001005, 0x00000805, 0x00001805, - 0x00000406, 0x00002406, 0x00001406, 0x00003406, - 0x00000c07, 0x00002c07, 0x00004c07, 0x00006c07, - 0x00001c07, 0x00003c07, 0x00005c07, 0x00007c07, - 0x00000208, 0x00002208, 0x00004208, 0x00006208, - 0x00008208, 0x0000a208, 0x0000c208, 0x0000e208, - 0x00001208, 0x00003208, 0x00005208, 0x00007208, - 0x00009208, 0x0000b208, 0x0000d208, 0x0000f208, - 0x00000a09, 0x00002a09, 0x00004a09, 0x00006a09, - 0x00008a09, 0x0000aa09, 0x0000ca09, 0x0000ea09, - 0x00010a09, 0x00012a09, 0x00014a09, 0x00016a09, - 0x00018a09, 0x0001aa09, 0x0001ca09, 0x0001ea09, - 0x00001a09, 0x00003a09, 0x00005a09, 0x00007a09, - 0x00009a09, 0x0000ba09, 0x0000da09, 0x0000fa09, - 0x00011a09, 0x00013a09, 0x00015a09, 0x00017a09, - 0x00019a09, 0x0001ba09, 0x0001da09, 0x0001fa09, - 0x0000060a, 0x0000260a, 0x0000460a, 0x0000660a, - 0x0000860a, 0x0000a60a, 0x0000c60a, 0x0000e60a, - 0x0001060a, 0x0001260a, 0x0001460a, 0x0001660a, - 0x0001860a, 0x0001a60a, 0x0001c60a, 0x0001e60a, - 0x0002060a, 0x0002260a, 0x0002460a, 0x0002660a, - 0x0002860a, 0x0002a60a, 0x0002c60a, 0x0002e60a, - 0x0003060a, 0x0003260a, 0x0003460a, 0x0003660a, - 0x0003860a, 0x0003a60a, 0x0003c60a, 0x0003e60a, - 0x0000160a, 0x0000360a, 0x0000560a, 0x0000760a, - 0x0000960a, 0x0000b60a, 0x0000d60a, 0x0000f60a, - 0x0001160a, 0x0001360a, 0x0001560a, 0x0001760a, - 0x0001960a, 0x0001b60a, 0x0001d60a, 0x0001f60a, - 0x0002160a, 0x0002360a, 0x0002560a, 0x0002760a, - 0x0002960a, 0x0002b60a, 0x0002d60a, 0x0002f60a, - 0x0003160a, 0x0003360a, 0x0003560a, 0x0003760a, - 0x0003960a, 0x0003b60a, 0x0003d60a, 0x0003f60a, - 0x00000e0b, 0x00002e0b, 0x00004e0b, 0x00006e0b, - 0x00008e0b, 0x0000ae0b, 0x0000ce0b, 0x0000ee0b, - 0x00010e0b, 0x00012e0b, 0x00014e0b, 0x00016e0b, - 0x00018e0b, 0x0001ae0b, 0x0001ce0b, 0x0001ee0b, - 0x00020e0b, 0x00022e0b, 0x00024e0b, 0x00026e0b, - 0x00028e0b, 0x0002ae0b, 0x0002ce0b, 0x0002ee0b, - 0x00030e0b, 0x00032e0b, 0x00034e0b, 0x00036e0b, - 0x00038e0b, 0x0003ae0b, 0x0003ce0b, 0x0003ee0b, - 0x00040e0b, 0x00042e0b, 0x00044e0b, 0x00046e0b, - 0x00048e0b, 0x0004ae0b, 0x0004ce0b, 0x0004ee0b, - 0x00050e0b, 0x00052e0b, 0x00054e0b, 0x00056e0b, - 0x00058e0b, 0x0005ae0b, 0x0005ce0b, 0x0005ee0b, - 0x00060e0b, 0x00062e0b, 0x00064e0b, 0x00066e0b, - 0x00068e0b, 0x0006ae0b, 0x0006ce0b, 0x0006ee0b, - 0x00070e0b, 0x00072e0b, 0x00074e0b, 0x00076e0b, - 0x00078e0b, 0x0007ae0b, 0x0007ce0b, 0x0007ee0b, - 0x00001e0b, 0x00003e0b, 0x00005e0b, 0x00007e0b, - 0x00009e0b, 0x0000be0b, 0x0000de0b, 0x0000fe0b, - 0x00011e0b, 0x00013e0b, 0x00015e0b, 0x00017e0b, - 0x00019e0b, 0x0001be0b, 0x0001de0b, 0x0001fe0b, - 0x00021e0b, 0x00023e0b, 0x00025e0b, 0x00027e0b, - 0x00029e0b, 0x0002be0b, 0x0002de0b, 0x0002fe0b, - 0x00031e0b, 0x00033e0b, 0x00035e0b, 0x00037e0b, - 0x00039e0b, 0x0003be0b, 0x0003de0b, 0x0003fe0b, - 0x00041e0b, 0x00043e0b, 0x00045e0b, 0x00047e0b, - 0x00049e0b, 0x0004be0b, 0x0004de0b, 0x0004fe0b, - 0x00051e0b, 0x00053e0b, 0x00055e0b, 0x00057e0b, - 0x00059e0b, 0x0005be0b, 0x0005de0b, 0x0005fe0b, - 0x00061e0b, 0x00063e0b, 0x00065e0b, 0x00067e0b, - 0x00069e0b, 0x0006be0b, 0x0006de0b, 0x0006fe0b, - 0x00071e0b, 0x00073e0b, 0x00075e0b, 0x00077e0b, - 0x00079e0b, 0x0007be0b, 0x0007de0b, 0x0007fe0b, - 0x0000010c, 0x0000210c, 0x0000410c, 0x0000610c, - 0x0000810c, 0x0000a10c, 0x0000c10c, 0x0000e10c, - 0x0001010c, 0x0001210c, 0x0001410c, 0x0001610c, - 0x0001810c, 0x0001a10c, 0x0001c10c, 0x0001e10c, - 0x0002010c, 0x0002210c, 0x0002410c, 0x0002610c, - 0x0002810c, 0x0002a10c, 0x0002c10c, 0x0002e10c, - 0x0003010c, 0x0003210c, 0x0003410c, 0x0003610c, - 0x0003810c, 0x0003a10c, 0x0003c10c, 0x0003e10c, - 0x0004010c, 0x0004210c, 0x0004410c, 0x0004610c, - 0x0004810c, 0x0004a10c, 0x0004c10c, 0x0004e10c, - 0x0005010c, 0x0005210c, 0x0005410c, 0x0005610c, - 0x0005810c, 0x0005a10c, 0x0005c10c, 0x0005e10c, - 0x0006010c, 0x0006210c, 0x0006410c, 0x0006610c, - 0x0006810c, 0x0006a10c, 0x0006c10c, 0x0006e10c, - 0x0007010c, 0x0007210c, 0x0007410c, 0x0007610c, - 0x0007810c, 0x0007a10c, 0x0007c10c, 0x0007e10c, - 0x0008010c, 0x0008210c, 0x0008410c, 0x0008610c, - 0x0008810c, 0x0008a10c, 0x0008c10c, 0x0008e10c, - 0x0009010c, 0x0009210c, 0x0009410c, 0x0009610c, - 0x0009810c, 0x0009a10c, 0x0009c10c, 0x0009e10c, - 0x000a010c, 0x000a210c, 0x000a410c, 0x000a610c, - 0x000a810c, 0x000aa10c, 0x000ac10c, 0x000ae10c, - 0x000b010c, 0x000b210c, 0x000b410c, 0x000b610c, - 0x000b810c, 0x000ba10c, 0x000bc10c, 0x000be10c, - 0x000c010c, 0x000c210c, 0x000c410c, 0x000c610c, - 0x000c810c, 0x000ca10c, 0x000cc10c, 0x000ce10c, - 0x000d010c, 0x000d210c, 0x000d410c, 0x000d610c, - 0x000d810c, 0x000da10c, 0x000dc10c, 0x000de10c, - 0x000e010c, 0x000e210c, 0x000e410c, 0x000e610c, - 0x000e810c, 0x000ea10c, 0x000ec10c, 0x000ee10c, - 0x000f010c, 0x000f210c, 0x000f410c, 0x000f610c, - 0x000f810c, 0x000fa10c, 0x000fc10c, 0x000fe10c, - 0x0000110c, 0x0000310c, 0x0000510c, 0x0000710c, - 0x0000910c, 0x0000b10c, 0x0000d10c, 0x0000f10c, - 0x0001110c, 0x0001310c, 0x0001510c, 0x0001710c, - 0x0001910c, 0x0001b10c, 0x0001d10c, 0x0001f10c, - 0x0002110c, 0x0002310c, 0x0002510c, 0x0002710c, - 0x0002910c, 0x0002b10c, 0x0002d10c, 0x0002f10c, - 0x0003110c, 0x0003310c, 0x0003510c, 0x0003710c, - 0x0003910c, 0x0003b10c, 0x0003d10c, 0x0003f10c, - 0x0004110c, 0x0004310c, 0x0004510c, 0x0004710c, - 0x0004910c, 0x0004b10c, 0x0004d10c, 0x0004f10c, - 0x0005110c, 0x0005310c, 0x0005510c, 0x0005710c, - 0x0005910c, 0x0005b10c, 0x0005d10c, 0x0005f10c, - 0x0006110c, 0x0006310c, 0x0006510c, 0x0006710c, - 0x0006910c, 0x0006b10c, 0x0006d10c, 0x0006f10c, - 0x0007110c, 0x0007310c, 0x0007510c, 0x0007710c, - 0x0007910c, 0x0007b10c, 0x0007d10c, 0x0007f10c, - 0x0008110c, 0x0008310c, 0x0008510c, 0x0008710c, - 0x0008910c, 0x0008b10c, 0x0008d10c, 0x0008f10c, - 0x0009110c, 0x0009310c, 0x0009510c, 0x0009710c, - 0x0009910c, 0x0009b10c, 0x0009d10c, 0x0009f10c, - 0x000a110c, 0x000a310c, 0x000a510c, 0x000a710c, - 0x000a910c, 0x000ab10c, 0x000ad10c, 0x000af10c, - 0x000b110c, 0x000b310c, 0x000b510c, 0x000b710c, - 0x000b910c, 0x000bb10c, 0x000bd10c, 0x000bf10c, - 0x000c110c, 0x000c310c, 0x000c510c, 0x000c710c, - 0x000c910c, 0x000cb10c, 0x000cd10c, 0x000cf10c, - 0x000d110c, 0x000d310c, 0x000d510c, 0x000d710c, - 0x000d910c, 0x000db10c, 0x000dd10c, 0x000df10c, - 0x000e110c, 0x000e310c, 0x000e510c, 0x000e710c, - 0x000e910c, 0x000eb10c, 0x000ed10c, 0x000ef10c, - 0x000f110c, 0x000f310c, 0x000f510c, 0x000f710c, - 0x000f910c, 0x000fb10c, 0x000fd10c, 0x000ff10c, - 0x0000090d, 0x0000290d, 0x0000490d, 0x0000690d, - 0x0000890d, 0x0000a90d, 0x0000c90d, 0x0000e90d, - 0x0001090d, 0x0001290d, 0x0001490d, 0x0001690d, - 0x0001890d, 0x0001a90d, 0x0001c90d, 0x0001e90d, - 0x0002090d, 0x0002290d, 0x0002490d, 0x0002690d, - 0x0002890d, 0x0002a90d, 0x0002c90d, 0x0002e90d, - 0x0003090d, 0x0003290d, 0x0003490d, 0x0003690d, - 0x0003890d, 0x0003a90d, 0x0003c90d, 0x0003e90d, - 0x0004090d, 0x0004290d, 0x0004490d, 0x0004690d, - 0x0004890d, 0x0004a90d, 0x0004c90d, 0x0004e90d, - 0x0005090d, 0x0005290d, 0x0005490d, 0x0005690d, - 0x0005890d, 0x0005a90d, 0x0005c90d, 0x0005e90d, - 0x0006090d, 0x0006290d, 0x0006490d, 0x0006690d, - 0x0006890d, 0x0006a90d, 0x0006c90d, 0x0006e90d, - 0x0007090d, 0x0007290d, 0x0007490d, 0x0007690d, - 0x0007890d, 0x0007a90d, 0x0007c90d, 0x0007e90d, - 0x0008090d, 0x0008290d, 0x0008490d, 0x0008690d, - 0x0008890d, 0x0008a90d, 0x0008c90d, 0x0008e90d, - 0x0009090d, 0x0009290d, 0x0009490d, 0x0009690d, - 0x0009890d, 0x0009a90d, 0x0009c90d, 0x0009e90d, - 0x000a090d, 0x000a290d, 0x000a490d, 0x000a690d, - 0x000a890d, 0x000aa90d, 0x000ac90d, 0x000ae90d, - 0x000b090d, 0x000b290d, 0x000b490d, 0x000b690d, - 0x000b890d, 0x000ba90d, 0x000bc90d, 0x000be90d, - 0x000c090d, 0x000c290d, 0x000c490d, 0x000c690d, - 0x000c890d, 0x000ca90d, 0x000cc90d, 0x000ce90d, - 0x000d090d, 0x000d290d, 0x000d490d, 0x000d690d, - 0x000d890d, 0x000da90d, 0x000dc90d, 0x000de90d, - 0x000e090d, 0x000e290d, 0x000e490d, 0x000e690d, - 0x000e890d, 0x000ea90d, 0x000ec90d, 0x000ee90d, - 0x000f090d, 0x000f290d, 0x000f490d, 0x000f690d, - 0x000f890d, 0x000fa90d, 0x000fc90d, 0x000fe90d, - 0x0010090d, 0x0010290d, 0x0010490d, 0x0010690d, - 0x0010890d, 0x0010a90d, 0x0010c90d, 0x0010e90d, - 0x0011090d, 0x0011290d, 0x0011490d, 0x0011690d, - 0x0011890d, 0x0011a90d, 0x0011c90d, 0x0011e90d, - 0x0012090d, 0x0012290d, 0x0012490d, 0x0012690d, - 0x0012890d, 0x0012a90d, 0x0012c90d, 0x0012e90d, - 0x0013090d, 0x0013290d, 0x0013490d, 0x0013690d, - 0x0013890d, 0x0013a90d, 0x0013c90d, 0x0013e90d, - 0x0014090d, 0x0014290d, 0x0014490d, 0x0014690d, - 0x0014890d, 0x0014a90d, 0x0014c90d, 0x0014e90d, - 0x0015090d, 0x0015290d, 0x0015490d, 0x0015690d, - 0x0015890d, 0x0015a90d, 0x0015c90d, 0x0015e90d, - 0x0016090d, 0x0016290d, 0x0016490d, 0x0016690d, - 0x0016890d, 0x0016a90d, 0x0016c90d, 0x0016e90d, - 0x0017090d, 0x0017290d, 0x0017490d, 0x0017690d, - 0x0017890d, 0x0017a90d, 0x0017c90d, 0x0017e90d, - 0x0018090d, 0x0018290d, 0x0018490d, 0x0018690d, - 0x0018890d, 0x0018a90d, 0x0018c90d, 0x0018e90d, - 0x0019090d, 0x0019290d, 0x0019490d, 0x0019690d, - 0x0019890d, 0x0019a90d, 0x0019c90d, 0x0019e90d, - 0x001a090d, 0x001a290d, 0x001a490d, 0x001a690d, - 0x001a890d, 0x001aa90d, 0x001ac90d, 0x001ae90d, - 0x001b090d, 0x001b290d, 0x001b490d, 0x001b690d, - 0x001b890d, 0x001ba90d, 0x001bc90d, 0x001be90d, - 0x001c090d, 0x001c290d, 0x001c490d, 0x001c690d, - 0x001c890d, 0x001ca90d, 0x001cc90d, 0x001ce90d, - 0x001d090d, 0x001d290d, 0x001d490d, 0x001d690d, - 0x001d890d, 0x001da90d, 0x001dc90d, 0x001de90d, - 0x001e090d, 0x001e290d, 0x001e490d, 0x001e690d, - 0x001e890d, 0x001ea90d, 0x001ec90d, 0x001ee90d, - 0x001f090d, 0x001f290d, 0x001f490d, 0x001f690d, - 0x001f890d, 0x001fa90d, 0x001fc90d, 0x001fe90d, - 0x0000190d, 0x0000390d, 0x0000590d, 0x0000790d, - 0x0000990d, 0x0000b90d, 0x0000d90d, 0x0000f90d, - 0x0001190d, 0x0001390d, 0x0001590d, 0x0001790d, - 0x0001990d, 0x0001b90d, 0x0001d90d, 0x0001f90d, - 0x0002190d, 0x0002390d, 0x0002590d, 0x0002790d, - 0x0002990d, 0x0002b90d, 0x0002d90d, 0x0002f90d, - 0x0003190d, 0x0003390d, 0x0003590d, 0x0003790d, - 0x0003990d, 0x0003b90d, 0x0003d90d, 0x0003f90d, - 0x0004190d, 0x0004390d, 0x0004590d, 0x0004790d, - 0x0004990d, 0x0004b90d, 0x0004d90d, 0x0004f90d, - 0x0005190d, 0x0005390d, 0x0005590d, 0x0005790d, - 0x0005990d, 0x0005b90d, 0x0005d90d, 0x0005f90d, - 0x0006190d, 0x0006390d, 0x0006590d, 0x0006790d, - 0x0006990d, 0x0006b90d, 0x0006d90d, 0x0006f90d, - 0x0007190d, 0x0007390d, 0x0007590d, 0x0007790d, - 0x0007990d, 0x0007b90d, 0x0007d90d, 0x0007f90d, - 0x0008190d, 0x0008390d, 0x0008590d, 0x0008790d, - 0x0008990d, 0x0008b90d, 0x0008d90d, 0x0008f90d, - 0x0009190d, 0x0009390d, 0x0009590d, 0x0009790d, - 0x0009990d, 0x0009b90d, 0x0009d90d, 0x0009f90d, - 0x000a190d, 0x000a390d, 0x000a590d, 0x000a790d, - 0x000a990d, 0x000ab90d, 0x000ad90d, 0x000af90d, - 0x000b190d, 0x000b390d, 0x000b590d, 0x000b790d, - 0x000b990d, 0x000bb90d, 0x000bd90d, 0x000bf90d, - 0x000c190d, 0x000c390d, 0x000c590d, 0x000c790d, - 0x000c990d, 0x000cb90d, 0x000cd90d, 0x000cf90d, - 0x000d190d, 0x000d390d, 0x000d590d, 0x000d790d, - 0x000d990d, 0x000db90d, 0x000dd90d, 0x000df90d, - 0x000e190d, 0x000e390d, 0x000e590d, 0x000e790d, - 0x000e990d, 0x000eb90d, 0x000ed90d, 0x000ef90d, - 0x000f190d, 0x000f390d, 0x000f590d, 0x000f790d, - 0x000f990d, 0x000fb90d, 0x000fd90d, 0x000ff90d, - 0x0010190d, 0x0010390d, 0x0010590d, 0x0010790d, - 0x0010990d, 0x0010b90d, 0x0010d90d, 0x0010f90d, - 0x0011190d, 0x0011390d, 0x0011590d, 0x0011790d, - 0x0011990d, 0x0011b90d, 0x0011d90d, 0x0011f90d, - 0x0012190d, 0x0012390d, 0x0012590d, 0x0012790d, - 0x0012990d, 0x0012b90d, 0x0012d90d, 0x0012f90d, - 0x0013190d, 0x0013390d, 0x0013590d, 0x0013790d, - 0x0013990d, 0x0013b90d, 0x0013d90d, 0x0013f90d, - 0x0014190d, 0x0014390d, 0x0014590d, 0x0014790d, - 0x0014990d, 0x0014b90d, 0x0014d90d, 0x0014f90d, - 0x0015190d, 0x0015390d, 0x0015590d, 0x0015790d, - 0x0015990d, 0x0015b90d, 0x0015d90d, 0x0015f90d, - 0x0016190d, 0x0016390d, 0x0016590d, 0x0016790d, - 0x0016990d, 0x0016b90d, 0x0016d90d, 0x0016f90d, - 0x0017190d, 0x0017390d, 0x0017590d, 0x0017790d, - 0x0017990d, 0x0017b90d, 0x0017d90d, 0x0017f90d, - 0x0018190d, 0x0018390d, 0x0018590d, 0x0018790d, - 0x0018990d, 0x0018b90d, 0x0018d90d, 0x0018f90d, - 0x0019190d, 0x0019390d, 0x0019590d, 0x0019790d, - 0x0019990d, 0x0019b90d, 0x0019d90d, 0x0019f90d, - 0x001a190d, 0x001a390d, 0x001a590d, 0x001a790d, - 0x001a990d, 0x001ab90d, 0x001ad90d, 0x001af90d, - 0x001b190d, 0x001b390d, 0x001b590d, 0x001b790d, - 0x001b990d, 0x001bb90d, 0x001bd90d, 0x001bf90d, - 0x001c190d, 0x001c390d, 0x001c590d, 0x001c790d, - 0x001c990d, 0x001cb90d, 0x001cd90d, 0x001cf90d, - 0x001d190d, 0x001d390d, 0x001d590d, 0x001d790d, - 0x001d990d, 0x001db90d, 0x001dd90d, 0x001df90d, - 0x001e190d, 0x001e390d, 0x001e590d, 0x001e790d, - 0x001e990d, 0x001eb90d, 0x001ed90d, 0x001ef90d, - 0x001f190d, 0x001f390d, 0x001f590d, 0x001f790d, - 0x001f990d, 0x001fb90d, 0x001fd90d, 0x001ff90d, - 0x0000050e, 0x0000250e, 0x0000450e, 0x0000650e, - 0x0000850e, 0x0000a50e, 0x0000c50e, 0x0000e50e, - 0x0001050e, 0x0001250e, 0x0001450e, 0x0001650e, - 0x0001850e, 0x0001a50e, 0x0001c50e, 0x0001e50e, - 0x0002050e, 0x0002250e, 0x0002450e, 0x0002650e, - 0x0002850e, 0x0002a50e, 0x0002c50e, 0x0002e50e, - 0x0003050e, 0x0003250e, 0x0003450e, 0x0003650e, - 0x0003850e, 0x0003a50e, 0x0003c50e, 0x0003e50e, - 0x0004050e, 0x0004250e, 0x0004450e, 0x0004650e, - 0x0004850e, 0x0004a50e, 0x0004c50e, 0x0004e50e, - 0x0005050e, 0x0005250e, 0x0005450e, 0x0005650e, - 0x0005850e, 0x0005a50e, 0x0005c50e, 0x0005e50e, - 0x0006050e, 0x0006250e, 0x0006450e, 0x0006650e, - 0x0006850e, 0x0006a50e, 0x0006c50e, 0x0006e50e, - 0x0007050e, 0x0007250e, 0x0007450e, 0x0007650e, - 0x0007850e, 0x0007a50e, 0x0007c50e, 0x0007e50e, - 0x0008050e, 0x0008250e, 0x0008450e, 0x0008650e, - 0x0008850e, 0x0008a50e, 0x0008c50e, 0x0008e50e, - 0x0009050e, 0x0009250e, 0x0009450e, 0x0009650e, - 0x0009850e, 0x0009a50e, 0x0009c50e, 0x0009e50e, - 0x000a050e, 0x000a250e, 0x000a450e, 0x000a650e, - 0x000a850e, 0x000aa50e, 0x000ac50e, 0x000ae50e, - 0x000b050e, 0x000b250e, 0x000b450e, 0x000b650e, - 0x000b850e, 0x000ba50e, 0x000bc50e, 0x000be50e, - 0x000c050e, 0x000c250e, 0x000c450e, 0x000c650e, - 0x000c850e, 0x000ca50e, 0x000cc50e, 0x000ce50e, - 0x000d050e, 0x000d250e, 0x000d450e, 0x000d650e, - 0x000d850e, 0x000da50e, 0x000dc50e, 0x000de50e, - 0x000e050e, 0x000e250e, 0x000e450e, 0x000e650e, - 0x000e850e, 0x000ea50e, 0x000ec50e, 0x000ee50e, - 0x000f050e, 0x000f250e, 0x000f450e, 0x000f650e, - 0x000f850e, 0x000fa50e, 0x000fc50e, 0x000fe50e, - 0x0010050e, 0x0010250e, 0x0010450e, 0x0010650e, - 0x0010850e, 0x0010a50e, 0x0010c50e, 0x0010e50e, - 0x0011050e, 0x0011250e, 0x0011450e, 0x0011650e, - 0x0011850e, 0x0011a50e, 0x0011c50e, 0x0011e50e, - 0x0012050e, 0x0012250e, 0x0012450e, 0x0012650e, - 0x0012850e, 0x0012a50e, 0x0012c50e, 0x0012e50e, - 0x0013050e, 0x0013250e, 0x0013450e, 0x0013650e, - 0x0013850e, 0x0013a50e, 0x0013c50e, 0x0013e50e, - 0x0014050e, 0x0014250e, 0x0014450e, 0x0014650e, - 0x0014850e, 0x0014a50e, 0x0014c50e, 0x0014e50e, - 0x0015050e, 0x0015250e, 0x0015450e, 0x0015650e, - 0x0015850e, 0x0015a50e, 0x0015c50e, 0x0015e50e, - 0x0016050e, 0x0016250e, 0x0016450e, 0x0016650e, - 0x0016850e, 0x0016a50e, 0x0016c50e, 0x0016e50e, - 0x0017050e, 0x0017250e, 0x0017450e, 0x0017650e, - 0x0017850e, 0x0017a50e, 0x0017c50e, 0x0017e50e, - 0x0018050e, 0x0018250e, 0x0018450e, 0x0018650e, - 0x0018850e, 0x0018a50e, 0x0018c50e, 0x0018e50e, - 0x0019050e, 0x0019250e, 0x0019450e, 0x0019650e, - 0x0019850e, 0x0019a50e, 0x0019c50e, 0x0019e50e, - 0x001a050e, 0x001a250e, 0x001a450e, 0x001a650e, - 0x001a850e, 0x001aa50e, 0x001ac50e, 0x001ae50e, - 0x001b050e, 0x001b250e, 0x001b450e, 0x001b650e, - 0x001b850e, 0x001ba50e, 0x001bc50e, 0x001be50e, - 0x001c050e, 0x001c250e, 0x001c450e, 0x001c650e, - 0x001c850e, 0x001ca50e, 0x001cc50e, 0x001ce50e, - 0x001d050e, 0x001d250e, 0x001d450e, 0x001d650e, - 0x001d850e, 0x001da50e, 0x001dc50e, 0x001de50e, - 0x001e050e, 0x001e250e, 0x001e450e, 0x001e650e, - 0x001e850e, 0x001ea50e, 0x001ec50e, 0x001ee50e, - 0x001f050e, 0x001f250e, 0x001f450e, 0x001f650e, - 0x001f850e, 0x001fa50e, 0x001fc50e, 0x001fe50e, - 0x0020050e, 0x0020250e, 0x0020450e, 0x0020650e, - 0x0020850e, 0x0020a50e, 0x0020c50e, 0x0020e50e, - 0x0021050e, 0x0021250e, 0x0021450e, 0x0021650e, - 0x0021850e, 0x0021a50e, 0x0021c50e, 0x0021e50e, - 0x0022050e, 0x0022250e, 0x0022450e, 0x0022650e, - 0x0022850e, 0x0022a50e, 0x0022c50e, 0x0022e50e, - 0x0023050e, 0x0023250e, 0x0023450e, 0x0023650e, - 0x0023850e, 0x0023a50e, 0x0023c50e, 0x0023e50e, - 0x0024050e, 0x0024250e, 0x0024450e, 0x0024650e, - 0x0024850e, 0x0024a50e, 0x0024c50e, 0x0024e50e, - 0x0025050e, 0x0025250e, 0x0025450e, 0x0025650e, - 0x0025850e, 0x0025a50e, 0x0025c50e, 0x0025e50e, - 0x0026050e, 0x0026250e, 0x0026450e, 0x0026650e, - 0x0026850e, 0x0026a50e, 0x0026c50e, 0x0026e50e, - 0x0027050e, 0x0027250e, 0x0027450e, 0x0027650e, - 0x0027850e, 0x0027a50e, 0x0027c50e, 0x0027e50e, - 0x0028050e, 0x0028250e, 0x0028450e, 0x0028650e, - 0x0028850e, 0x0028a50e, 0x0028c50e, 0x0028e50e, - 0x0029050e, 0x0029250e, 0x0029450e, 0x0029650e, - 0x0029850e, 0x0029a50e, 0x0029c50e, 0x0029e50e, - 0x002a050e, 0x002a250e, 0x002a450e, 0x002a650e, - 0x002a850e, 0x002aa50e, 0x002ac50e, 0x002ae50e, - 0x002b050e, 0x002b250e, 0x002b450e, 0x002b650e, - 0x002b850e, 0x002ba50e, 0x002bc50e, 0x002be50e, - 0x002c050e, 0x002c250e, 0x002c450e, 0x002c650e, - 0x002c850e, 0x002ca50e, 0x002cc50e, 0x002ce50e, - 0x002d050e, 0x002d250e, 0x002d450e, 0x002d650e, - 0x002d850e, 0x002da50e, 0x002dc50e, 0x002de50e, - 0x002e050e, 0x002e250e, 0x002e450e, 0x002e650e, - 0x002e850e, 0x002ea50e, 0x002ec50e, 0x002ee50e, - 0x002f050e, 0x002f250e, 0x002f450e, 0x002f650e, - 0x002f850e, 0x002fa50e, 0x002fc50e, 0x002fe50e, - 0x0030050e, 0x0030250e, 0x0030450e, 0x0030650e, - 0x0030850e, 0x0030a50e, 0x0030c50e, 0x0030e50e, - 0x0031050e, 0x0031250e, 0x0031450e, 0x0031650e, - 0x0031850e, 0x0031a50e, 0x0031c50e, 0x0031e50e, - 0x0032050e, 0x0032250e, 0x0032450e, 0x0032650e, - 0x0032850e, 0x0032a50e, 0x0032c50e, 0x0032e50e, - 0x0033050e, 0x0033250e, 0x0033450e, 0x0033650e, - 0x0033850e, 0x0033a50e, 0x0033c50e, 0x0033e50e, - 0x0034050e, 0x0034250e, 0x0034450e, 0x0034650e, - 0x0034850e, 0x0034a50e, 0x0034c50e, 0x0034e50e, - 0x0035050e, 0x0035250e, 0x0035450e, 0x0035650e, - 0x0035850e, 0x0035a50e, 0x0035c50e, 0x0035e50e, - 0x0036050e, 0x0036250e, 0x0036450e, 0x0036650e, - 0x0036850e, 0x0036a50e, 0x0036c50e, 0x0036e50e, - 0x0037050e, 0x0037250e, 0x0037450e, 0x0037650e, - 0x0037850e, 0x0037a50e, 0x0037c50e, 0x0037e50e, - 0x0038050e, 0x0038250e, 0x0038450e, 0x0038650e, - 0x0038850e, 0x0038a50e, 0x0038c50e, 0x0038e50e, - 0x0039050e, 0x0039250e, 0x0039450e, 0x0039650e, - 0x0039850e, 0x0039a50e, 0x0039c50e, 0x0039e50e, - 0x003a050e, 0x003a250e, 0x003a450e, 0x003a650e, - 0x003a850e, 0x003aa50e, 0x003ac50e, 0x003ae50e, - 0x003b050e, 0x003b250e, 0x003b450e, 0x003b650e, - 0x003b850e, 0x003ba50e, 0x003bc50e, 0x003be50e, - 0x003c050e, 0x003c250e, 0x003c450e, 0x003c650e, - 0x003c850e, 0x003ca50e, 0x003cc50e, 0x003ce50e, - 0x003d050e, 0x003d250e, 0x003d450e, 0x003d650e, - 0x003d850e, 0x003da50e, 0x003dc50e, 0x003de50e, - 0x003e050e, 0x003e250e, 0x003e450e, 0x003e650e, - 0x003e850e, 0x003ea50e, 0x003ec50e, 0x003ee50e, - 0x003f050e, 0x003f250e, 0x003f450e, 0x003f650e, - 0x003f850e, 0x003fa50e, 0x003fc50e, 0x003fe50e, - 0x0000150e, 0x0000350e, 0x0000550e, 0x0000750e, - 0x0000950e, 0x0000b50e, 0x0000d50e, 0x0000f50e, - 0x0001150e, 0x0001350e, 0x0001550e, 0x0001750e, - 0x0001950e, 0x0001b50e, 0x0001d50e, 0x0001f50e, - 0x0002150e, 0x0002350e, 0x0002550e, 0x0002750e, - 0x0002950e, 0x0002b50e, 0x0002d50e, 0x0002f50e, - 0x0003150e, 0x0003350e, 0x0003550e, 0x0003750e, - 0x0003950e, 0x0003b50e, 0x0003d50e, 0x0003f50e, - 0x0004150e, 0x0004350e, 0x0004550e, 0x0004750e, - 0x0004950e, 0x0004b50e, 0x0004d50e, 0x0004f50e, - 0x0005150e, 0x0005350e, 0x0005550e, 0x0005750e, - 0x0005950e, 0x0005b50e, 0x0005d50e, 0x0005f50e, - 0x0006150e, 0x0006350e, 0x0006550e, 0x0006750e, - 0x0006950e, 0x0006b50e, 0x0006d50e, 0x0006f50e, - 0x0007150e, 0x0007350e, 0x0007550e, 0x0007750e, - 0x0007950e, 0x0007b50e, 0x0007d50e, 0x0007f50e, - 0x0008150e, 0x0008350e, 0x0008550e, 0x0008750e, - 0x0008950e, 0x0008b50e, 0x0008d50e, 0x0008f50e, - 0x0009150e, 0x0009350e, 0x0009550e, 0x0009750e, - 0x0009950e, 0x0009b50e, 0x0009d50e, 0x0009f50e, - 0x000a150e, 0x000a350e, 0x000a550e, 0x000a750e, - 0x000a950e, 0x000ab50e, 0x000ad50e, 0x000af50e, - 0x000b150e, 0x000b350e, 0x000b550e, 0x000b750e, - 0x000b950e, 0x000bb50e, 0x000bd50e, 0x000bf50e, - 0x000c150e, 0x000c350e, 0x000c550e, 0x000c750e, - 0x000c950e, 0x000cb50e, 0x000cd50e, 0x000cf50e, - 0x000d150e, 0x000d350e, 0x000d550e, 0x000d750e, - 0x000d950e, 0x000db50e, 0x000dd50e, 0x000df50e, - 0x000e150e, 0x000e350e, 0x000e550e, 0x000e750e, - 0x000e950e, 0x000eb50e, 0x000ed50e, 0x000ef50e, - 0x000f150e, 0x000f350e, 0x000f550e, 0x000f750e, - 0x000f950e, 0x000fb50e, 0x000fd50e, 0x000ff50e, - 0x0010150e, 0x0010350e, 0x0010550e, 0x0010750e, - 0x0010950e, 0x0010b50e, 0x0010d50e, 0x0010f50e, - 0x0011150e, 0x0011350e, 0x0011550e, 0x0011750e, - 0x0011950e, 0x0011b50e, 0x0011d50e, 0x0011f50e, - 0x0012150e, 0x0012350e, 0x0012550e, 0x0012750e, - 0x0012950e, 0x0012b50e, 0x0012d50e, 0x0012f50e, - 0x0013150e, 0x0013350e, 0x0013550e, 0x0013750e, - 0x0013950e, 0x0013b50e, 0x0013d50e, 0x0013f50e, - 0x0014150e, 0x0014350e, 0x0014550e, 0x0014750e, - 0x0014950e, 0x0014b50e, 0x0014d50e, 0x0014f50e, - 0x0015150e, 0x0015350e, 0x0015550e, 0x0015750e, - 0x0015950e, 0x0015b50e, 0x0015d50e, 0x0015f50e, - 0x0016150e, 0x0016350e, 0x0016550e, 0x0016750e, - 0x0016950e, 0x0016b50e, 0x0016d50e, 0x0016f50e, - 0x0017150e, 0x0017350e, 0x0017550e, 0x0017750e, - 0x0017950e, 0x0017b50e, 0x0017d50e, 0x0017f50e, - 0x0018150e, 0x0018350e, 0x0018550e, 0x0018750e, - 0x0018950e, 0x0018b50e, 0x0018d50e, 0x0018f50e, - 0x0019150e, 0x0019350e, 0x0019550e, 0x0019750e, - 0x0019950e, 0x0019b50e, 0x0019d50e, 0x0019f50e, - 0x001a150e, 0x001a350e, 0x001a550e, 0x001a750e, - 0x001a950e, 0x001ab50e, 0x001ad50e, 0x001af50e, - 0x001b150e, 0x001b350e, 0x001b550e, 0x001b750e, - 0x001b950e, 0x001bb50e, 0x001bd50e, 0x001bf50e, - 0x001c150e, 0x001c350e, 0x001c550e, 0x001c750e, - 0x001c950e, 0x001cb50e, 0x001cd50e, 0x001cf50e, - 0x001d150e, 0x001d350e, 0x001d550e, 0x001d750e, - 0x001d950e, 0x001db50e, 0x001dd50e, 0x001df50e, - 0x001e150e, 0x001e350e, 0x001e550e, 0x001e750e, - 0x001e950e, 0x001eb50e, 0x001ed50e, 0x001ef50e, - 0x001f150e, 0x001f350e, 0x001f550e, 0x001f750e, - 0x001f950e, 0x001fb50e, 0x001fd50e, 0x001ff50e, - 0x0020150e, 0x0020350e, 0x0020550e, 0x0020750e, - 0x0020950e, 0x0020b50e, 0x0020d50e, 0x0020f50e, - 0x0021150e, 0x0021350e, 0x0021550e, 0x0021750e, - 0x0021950e, 0x0021b50e, 0x0021d50e, 0x0021f50e, - 0x0022150e, 0x0022350e, 0x0022550e, 0x0022750e, - 0x0022950e, 0x0022b50e, 0x0022d50e, 0x0022f50e, - 0x0023150e, 0x0023350e, 0x0023550e, 0x0023750e, - 0x0023950e, 0x0023b50e, 0x0023d50e, 0x0023f50e, - 0x0024150e, 0x0024350e, 0x0024550e, 0x0024750e, - 0x0024950e, 0x0024b50e, 0x0024d50e, 0x0024f50e, - 0x0025150e, 0x0025350e, 0x0025550e, 0x0025750e, - 0x0025950e, 0x0025b50e, 0x0025d50e, 0x0025f50e, - 0x0026150e, 0x0026350e, 0x0026550e, 0x0026750e, - 0x0026950e, 0x0026b50e, 0x0026d50e, 0x0026f50e, - 0x0027150e, 0x0027350e, 0x0027550e, 0x0027750e, - 0x0027950e, 0x0027b50e, 0x0027d50e, 0x0027f50e, - 0x0028150e, 0x0028350e, 0x0028550e, 0x0028750e, - 0x0028950e, 0x0028b50e, 0x0028d50e, 0x0028f50e, - 0x0029150e, 0x0029350e, 0x0029550e, 0x0029750e, - 0x0029950e, 0x0029b50e, 0x0029d50e, 0x0029f50e, - 0x002a150e, 0x002a350e, 0x002a550e, 0x002a750e, - 0x002a950e, 0x002ab50e, 0x002ad50e, 0x002af50e, - 0x002b150e, 0x002b350e, 0x002b550e, 0x002b750e, - 0x002b950e, 0x002bb50e, 0x002bd50e, 0x002bf50e, - 0x002c150e, 0x002c350e, 0x002c550e, 0x002c750e, - 0x002c950e, 0x002cb50e, 0x002cd50e, 0x002cf50e, - 0x002d150e, 0x002d350e, 0x002d550e, 0x002d750e, - 0x002d950e, 0x002db50e, 0x002dd50e, 0x002df50e, - 0x002e150e, 0x002e350e, 0x002e550e, 0x002e750e, - 0x002e950e, 0x002eb50e, 0x002ed50e, 0x002ef50e, - 0x002f150e, 0x002f350e, 0x002f550e, 0x002f750e, - 0x002f950e, 0x002fb50e, 0x002fd50e, 0x002ff50e, - 0x0030150e, 0x0030350e, 0x0030550e, 0x0030750e, - 0x0030950e, 0x0030b50e, 0x0030d50e, 0x0030f50e, - 0x0031150e, 0x0031350e, 0x0031550e, 0x0031750e, - 0x0031950e, 0x0031b50e, 0x0031d50e, 0x0031f50e, - 0x0032150e, 0x0032350e, 0x0032550e, 0x0032750e, - 0x0032950e, 0x0032b50e, 0x0032d50e, 0x0032f50e, - 0x0033150e, 0x0033350e, 0x0033550e, 0x0033750e, - 0x0033950e, 0x0033b50e, 0x0033d50e, 0x0033f50e, - 0x0034150e, 0x0034350e, 0x0034550e, 0x0034750e, - 0x0034950e, 0x0034b50e, 0x0034d50e, 0x0034f50e, - 0x0035150e, 0x0035350e, 0x0035550e, 0x0035750e, - 0x0035950e, 0x0035b50e, 0x0035d50e, 0x0035f50e, - 0x0036150e, 0x0036350e, 0x0036550e, 0x0036750e, - 0x0036950e, 0x0036b50e, 0x0036d50e, 0x0036f50e, - 0x0037150e, 0x0037350e, 0x0037550e, 0x0037750e, - 0x0037950e, 0x0037b50e, 0x0037d50e, 0x0037f50e, - 0x0038150e, 0x0038350e, 0x0038550e, 0x0038750e, - 0x0038950e, 0x0038b50e, 0x0038d50e, 0x0038f50e, - 0x0039150e, 0x0039350e, 0x0039550e, 0x0039750e, - 0x0039950e, 0x0039b50e, 0x0039d50e, 0x0039f50e, - 0x003a150e, 0x003a350e, 0x003a550e, 0x003a750e, - 0x003a950e, 0x003ab50e, 0x003ad50e, 0x003af50e, - 0x003b150e, 0x003b350e, 0x003b550e, 0x003b750e, - 0x003b950e, 0x003bb50e, 0x003bd50e, 0x003bf50e, - 0x003c150e, 0x003c350e, 0x003c550e, 0x003c750e, - 0x003c950e, 0x003cb50e, 0x003cd50e, 0x003cf50e, - 0x003d150e, 0x003d350e, 0x003d550e, 0x003d750e, - 0x003d950e, 0x003db50e, 0x003dd50e, 0x003df50e, - 0x003e150e, 0x003e350e, 0x003e550e, 0x003e750e, - 0x003e950e, 0x003eb50e, 0x003ed50e, 0x003ef50e, - 0x003f150e, 0x003f350e, 0x003f550e, 0x003f750e, - 0x003f950e, 0x003fb50e, 0x003fd50e, 0x003ff50e, - 0x00000d0f, 0x00002d0f, 0x00004d0f, 0x00006d0f, - 0x00008d0f, 0x0000ad0f, 0x0000cd0f, 0x0000ed0f, - 0x00010d0f, 0x00012d0f, 0x00014d0f, 0x00016d0f, - 0x00018d0f, 0x0001ad0f, 0x0001cd0f, 0x0001ed0f, - 0x00020d0f, 0x00022d0f, 0x00024d0f, 0x00026d0f, - 0x00028d0f, 0x0002ad0f, 0x0002cd0f, 0x0002ed0f, - 0x00030d0f, 0x00032d0f, 0x00034d0f, 0x00036d0f, - 0x00038d0f, 0x0003ad0f, 0x0003cd0f, 0x0003ed0f, - 0x00040d0f, 0x00042d0f, 0x00044d0f, 0x00046d0f, - 0x00048d0f, 0x0004ad0f, 0x0004cd0f, 0x0004ed0f, - 0x00050d0f, 0x00052d0f, 0x00054d0f, 0x00056d0f, - 0x00058d0f, 0x0005ad0f, 0x0005cd0f, 0x0005ed0f, - 0x00060d0f, 0x00062d0f, 0x00064d0f, 0x00066d0f, - 0x00068d0f, 0x0006ad0f, 0x0006cd0f, 0x0006ed0f, - 0x00070d0f, 0x00072d0f, 0x00074d0f, 0x00076d0f, - 0x00078d0f, 0x0007ad0f, 0x0007cd0f, 0x0007ed0f, - 0x00080d0f, 0x00082d0f, 0x00084d0f, 0x00086d0f, - 0x00088d0f, 0x0008ad0f, 0x0008cd0f, 0x0008ed0f, - 0x00090d0f, 0x00092d0f, 0x00094d0f, 0x00096d0f, - 0x00098d0f, 0x0009ad0f, 0x0009cd0f, 0x0009ed0f, - 0x000a0d0f, 0x000a2d0f, 0x000a4d0f, 0x000a6d0f, - 0x000a8d0f, 0x000aad0f, 0x000acd0f, 0x000aed0f, - 0x000b0d0f, 0x000b2d0f, 0x000b4d0f, 0x000b6d0f, - 0x000b8d0f, 0x000bad0f, 0x000bcd0f, 0x000bed0f, - 0x000c0d0f, 0x000c2d0f, 0x000c4d0f, 0x000c6d0f, - 0x000c8d0f, 0x000cad0f, 0x000ccd0f, 0x000ced0f, - 0x000d0d0f, 0x000d2d0f, 0x000d4d0f, 0x000d6d0f, - 0x000d8d0f, 0x000dad0f, 0x000dcd0f, 0x000ded0f, - 0x000e0d0f, 0x000e2d0f, 0x000e4d0f, 0x000e6d0f, - 0x000e8d0f, 0x000ead0f, 0x000ecd0f, 0x000eed0f, - 0x000f0d0f, 0x000f2d0f, 0x000f4d0f, 0x000f6d0f, - 0x000f8d0f, 0x000fad0f, 0x000fcd0f, 0x000fed0f, - 0x00100d0f, 0x00102d0f, 0x00104d0f, 0x00106d0f, - 0x00108d0f, 0x0010ad0f, 0x0010cd0f, 0x0010ed0f, - 0x00110d0f, 0x00112d0f, 0x00114d0f, 0x00116d0f, - 0x00118d0f, 0x0011ad0f, 0x0011cd0f, 0x0011ed0f, - 0x00120d0f, 0x00122d0f, 0x00124d0f, 0x00126d0f, - 0x00128d0f, 0x0012ad0f, 0x0012cd0f, 0x0012ed0f, - 0x00130d0f, 0x00132d0f, 0x00134d0f, 0x00136d0f, - 0x00138d0f, 0x0013ad0f, 0x0013cd0f, 0x0013ed0f, - 0x00140d0f, 0x00142d0f, 0x00144d0f, 0x00146d0f, - 0x00148d0f, 0x0014ad0f, 0x0014cd0f, 0x0014ed0f, - 0x00150d0f, 0x00152d0f, 0x00154d0f, 0x00156d0f, - 0x00158d0f, 0x0015ad0f, 0x0015cd0f, 0x0015ed0f, - 0x00160d0f, 0x00162d0f, 0x00164d0f, 0x00166d0f, - 0x00168d0f, 0x0016ad0f, 0x0016cd0f, 0x0016ed0f, - 0x00170d0f, 0x00172d0f, 0x00174d0f, 0x00176d0f, - 0x00178d0f, 0x0017ad0f, 0x0017cd0f, 0x0017ed0f, - 0x00180d0f, 0x00182d0f, 0x00184d0f, 0x00186d0f, - 0x00188d0f, 0x0018ad0f, 0x0018cd0f, 0x0018ed0f, - 0x00190d0f, 0x00192d0f, 0x00194d0f, 0x00196d0f, - 0x00198d0f, 0x0019ad0f, 0x0019cd0f, 0x0019ed0f, - 0x001a0d0f, 0x001a2d0f, 0x001a4d0f, 0x001a6d0f, - 0x001a8d0f, 0x001aad0f, 0x001acd0f, 0x001aed0f, - 0x001b0d0f, 0x001b2d0f, 0x001b4d0f, 0x001b6d0f, - 0x001b8d0f, 0x001bad0f, 0x001bcd0f, 0x001bed0f, - 0x001c0d0f, 0x001c2d0f, 0x001c4d0f, 0x001c6d0f, - 0x001c8d0f, 0x001cad0f, 0x001ccd0f, 0x001ced0f, - 0x001d0d0f, 0x001d2d0f, 0x001d4d0f, 0x001d6d0f, - 0x001d8d0f, 0x001dad0f, 0x001dcd0f, 0x001ded0f, - 0x001e0d0f, 0x001e2d0f, 0x001e4d0f, 0x001e6d0f, - 0x001e8d0f, 0x001ead0f, 0x001ecd0f, 0x001eed0f, - 0x001f0d0f, 0x001f2d0f, 0x001f4d0f, 0x001f6d0f, - 0x001f8d0f, 0x001fad0f, 0x001fcd0f, 0x001fed0f, - 0x00200d0f, 0x00202d0f, 0x00204d0f, 0x00206d0f, - 0x00208d0f, 0x0020ad0f, 0x0020cd0f, 0x0020ed0f, - 0x00210d0f, 0x00212d0f, 0x00214d0f, 0x00216d0f, - 0x00218d0f, 0x0021ad0f, 0x0021cd0f, 0x0021ed0f, - 0x00220d0f, 0x00222d0f, 0x00224d0f, 0x00226d0f, - 0x00228d0f, 0x0022ad0f, 0x0022cd0f, 0x0022ed0f, - 0x00230d0f, 0x00232d0f, 0x00234d0f, 0x00236d0f, - 0x00238d0f, 0x0023ad0f, 0x0023cd0f, 0x0023ed0f, - 0x00240d0f, 0x00242d0f, 0x00244d0f, 0x00246d0f, - 0x00248d0f, 0x0024ad0f, 0x0024cd0f, 0x0024ed0f, - 0x00250d0f, 0x00252d0f, 0x00254d0f, 0x00256d0f, - 0x00258d0f, 0x0025ad0f, 0x0025cd0f, 0x0025ed0f, - 0x00260d0f, 0x00262d0f, 0x00264d0f, 0x00266d0f, - 0x00268d0f, 0x0026ad0f, 0x0026cd0f, 0x0026ed0f, - 0x00270d0f, 0x00272d0f, 0x00274d0f, 0x00276d0f, - 0x00278d0f, 0x0027ad0f, 0x0027cd0f, 0x0027ed0f, - 0x00280d0f, 0x00282d0f, 0x00284d0f, 0x00286d0f, - 0x00288d0f, 0x0028ad0f, 0x0028cd0f, 0x0028ed0f, - 0x00290d0f, 0x00292d0f, 0x00294d0f, 0x00296d0f, - 0x00298d0f, 0x0029ad0f, 0x0029cd0f, 0x0029ed0f, - 0x002a0d0f, 0x002a2d0f, 0x002a4d0f, 0x002a6d0f, - 0x002a8d0f, 0x002aad0f, 0x002acd0f, 0x002aed0f, - 0x002b0d0f, 0x002b2d0f, 0x002b4d0f, 0x002b6d0f, - 0x002b8d0f, 0x002bad0f, 0x002bcd0f, 0x002bed0f, - 0x002c0d0f, 0x002c2d0f, 0x002c4d0f, 0x002c6d0f, - 0x002c8d0f, 0x002cad0f, 0x002ccd0f, 0x002ced0f, - 0x002d0d0f, 0x002d2d0f, 0x002d4d0f, 0x002d6d0f, - 0x002d8d0f, 0x002dad0f, 0x002dcd0f, 0x002ded0f, - 0x002e0d0f, 0x002e2d0f, 0x002e4d0f, 0x002e6d0f, - 0x002e8d0f, 0x002ead0f, 0x002ecd0f, 0x002eed0f, - 0x002f0d0f, 0x002f2d0f, 0x002f4d0f, 0x002f6d0f, - 0x002f8d0f, 0x002fad0f, 0x002fcd0f, 0x002fed0f, - 0x00300d0f, 0x00302d0f, 0x00304d0f, 0x00306d0f, - 0x00308d0f, 0x0030ad0f, 0x0030cd0f, 0x0030ed0f, - 0x00310d0f, 0x00312d0f, 0x00314d0f, 0x00316d0f, - 0x00318d0f, 0x0031ad0f, 0x0031cd0f, 0x0031ed0f, - 0x00320d0f, 0x00322d0f, 0x00324d0f, 0x00326d0f, - 0x00328d0f, 0x0032ad0f, 0x0032cd0f, 0x0032ed0f, - 0x00330d0f, 0x00332d0f, 0x00334d0f, 0x00336d0f, - 0x00338d0f, 0x0033ad0f, 0x0033cd0f, 0x0033ed0f, - 0x00340d0f, 0x00342d0f, 0x00344d0f, 0x00346d0f, - 0x00348d0f, 0x0034ad0f, 0x0034cd0f, 0x0034ed0f, - 0x00350d0f, 0x00352d0f, 0x00354d0f, 0x00356d0f, - 0x00358d0f, 0x0035ad0f, 0x0035cd0f, 0x0035ed0f, - 0x00360d0f, 0x00362d0f, 0x00364d0f, 0x00366d0f, - 0x00368d0f, 0x0036ad0f, 0x0036cd0f, 0x0036ed0f, - 0x00370d0f, 0x00372d0f, 0x00374d0f, 0x00376d0f, - 0x00378d0f, 0x0037ad0f, 0x0037cd0f, 0x0037ed0f, - 0x00380d0f, 0x00382d0f, 0x00384d0f, 0x00386d0f, - 0x00388d0f, 0x0038ad0f, 0x0038cd0f, 0x0038ed0f, - 0x00390d0f, 0x00392d0f, 0x00394d0f, 0x00396d0f, - 0x00398d0f, 0x0039ad0f, 0x0039cd0f, 0x0039ed0f, - 0x003a0d0f, 0x003a2d0f, 0x003a4d0f, 0x003a6d0f, - 0x003a8d0f, 0x003aad0f, 0x003acd0f, 0x003aed0f, - 0x003b0d0f, 0x003b2d0f, 0x003b4d0f, 0x003b6d0f, - 0x003b8d0f, 0x003bad0f, 0x003bcd0f, 0x003bed0f, - 0x003c0d0f, 0x003c2d0f, 0x003c4d0f, 0x003c6d0f, - 0x003c8d0f, 0x003cad0f, 0x003ccd0f, 0x003ced0f, - 0x003d0d0f, 0x003d2d0f, 0x003d4d0f, 0x003d6d0f, - 0x003d8d0f, 0x003dad0f, 0x003dcd0f, 0x003ded0f, - 0x003e0d0f, 0x003e2d0f, 0x003e4d0f, 0x003e6d0f, - 0x003e8d0f, 0x003ead0f, 0x003ecd0f, 0x003eed0f, - 0x003f0d0f, 0x003f2d0f, 0x003f4d0f, 0x003f6d0f, - 0x003f8d0f, 0x003fad0f, 0x003fcd0f, 0x003fed0f, - 0x00400d0f, 0x00402d0f, 0x00404d0f, 0x00406d0f, - 0x00408d0f, 0x0040ad0f, 0x0040cd0f, 0x0040ed0f, - 0x00410d0f, 0x00412d0f, 0x00414d0f, 0x00416d0f, - 0x00418d0f, 0x0041ad0f, 0x0041cd0f, 0x0041ed0f, - 0x00420d0f, 0x00422d0f, 0x00424d0f, 0x00426d0f, - 0x00428d0f, 0x0042ad0f, 0x0042cd0f, 0x0042ed0f, - 0x00430d0f, 0x00432d0f, 0x00434d0f, 0x00436d0f, - 0x00438d0f, 0x0043ad0f, 0x0043cd0f, 0x0043ed0f, - 0x00440d0f, 0x00442d0f, 0x00444d0f, 0x00446d0f, - 0x00448d0f, 0x0044ad0f, 0x0044cd0f, 0x0044ed0f, - 0x00450d0f, 0x00452d0f, 0x00454d0f, 0x00456d0f, - 0x00458d0f, 0x0045ad0f, 0x0045cd0f, 0x0045ed0f, - 0x00460d0f, 0x00462d0f, 0x00464d0f, 0x00466d0f, - 0x00468d0f, 0x0046ad0f, 0x0046cd0f, 0x0046ed0f, - 0x00470d0f, 0x00472d0f, 0x00474d0f, 0x00476d0f, - 0x00478d0f, 0x0047ad0f, 0x0047cd0f, 0x0047ed0f, - 0x00480d0f, 0x00482d0f, 0x00484d0f, 0x00486d0f, - 0x00488d0f, 0x0048ad0f, 0x0048cd0f, 0x0048ed0f, - 0x00490d0f, 0x00492d0f, 0x00494d0f, 0x00496d0f, - 0x00498d0f, 0x0049ad0f, 0x0049cd0f, 0x0049ed0f, - 0x004a0d0f, 0x004a2d0f, 0x004a4d0f, 0x004a6d0f, - 0x004a8d0f, 0x004aad0f, 0x004acd0f, 0x004aed0f, - 0x004b0d0f, 0x004b2d0f, 0x004b4d0f, 0x004b6d0f, - 0x004b8d0f, 0x004bad0f, 0x004bcd0f, 0x004bed0f, - 0x004c0d0f, 0x004c2d0f, 0x004c4d0f, 0x004c6d0f, - 0x004c8d0f, 0x004cad0f, 0x004ccd0f, 0x004ced0f, - 0x004d0d0f, 0x004d2d0f, 0x004d4d0f, 0x004d6d0f, - 0x004d8d0f, 0x004dad0f, 0x004dcd0f, 0x004ded0f, - 0x004e0d0f, 0x004e2d0f, 0x004e4d0f, 0x004e6d0f, - 0x004e8d0f, 0x004ead0f, 0x004ecd0f, 0x004eed0f, - 0x004f0d0f, 0x004f2d0f, 0x004f4d0f, 0x004f6d0f, - 0x004f8d0f, 0x004fad0f, 0x004fcd0f, 0x004fed0f, - 0x00500d0f, 0x00502d0f, 0x00504d0f, 0x00506d0f, - 0x00508d0f, 0x0050ad0f, 0x0050cd0f, 0x0050ed0f, - 0x00510d0f, 0x00512d0f, 0x00514d0f, 0x00516d0f, - 0x00518d0f, 0x0051ad0f, 0x0051cd0f, 0x0051ed0f, - 0x00520d0f, 0x00522d0f, 0x00524d0f, 0x00526d0f, - 0x00528d0f, 0x0052ad0f, 0x0052cd0f, 0x0052ed0f, - 0x00530d0f, 0x00532d0f, 0x00534d0f, 0x00536d0f, - 0x00538d0f, 0x0053ad0f, 0x0053cd0f, 0x0053ed0f, - 0x00540d0f, 0x00542d0f, 0x00544d0f, 0x00546d0f, - 0x00548d0f, 0x0054ad0f, 0x0054cd0f, 0x0054ed0f, - 0x00550d0f, 0x00552d0f, 0x00554d0f, 0x00556d0f, - 0x00558d0f, 0x0055ad0f, 0x0055cd0f, 0x0055ed0f, - 0x00560d0f, 0x00562d0f, 0x00564d0f, 0x00566d0f, - 0x00568d0f, 0x0056ad0f, 0x0056cd0f, 0x0056ed0f, - 0x00570d0f, 0x00572d0f, 0x00574d0f, 0x00576d0f, - 0x00578d0f, 0x0057ad0f, 0x0057cd0f, 0x0057ed0f, - 0x00580d0f, 0x00582d0f, 0x00584d0f, 0x00586d0f, - 0x00588d0f, 0x0058ad0f, 0x0058cd0f, 0x0058ed0f, - 0x00590d0f, 0x00592d0f, 0x00594d0f, 0x00596d0f, - 0x00598d0f, 0x0059ad0f, 0x0059cd0f, 0x0059ed0f, - 0x005a0d0f, 0x005a2d0f, 0x005a4d0f, 0x005a6d0f, - 0x005a8d0f, 0x005aad0f, 0x005acd0f, 0x005aed0f, - 0x005b0d0f, 0x005b2d0f, 0x005b4d0f, 0x005b6d0f, - 0x005b8d0f, 0x005bad0f, 0x005bcd0f, 0x005bed0f, - 0x005c0d0f, 0x005c2d0f, 0x005c4d0f, 0x005c6d0f, - 0x005c8d0f, 0x005cad0f, 0x005ccd0f, 0x005ced0f, - 0x005d0d0f, 0x005d2d0f, 0x005d4d0f, 0x005d6d0f, - 0x005d8d0f, 0x005dad0f, 0x005dcd0f, 0x005ded0f, - 0x005e0d0f, 0x005e2d0f, 0x005e4d0f, 0x005e6d0f, - 0x005e8d0f, 0x005ead0f, 0x005ecd0f, 0x005eed0f, - 0x005f0d0f, 0x005f2d0f, 0x005f4d0f, 0x005f6d0f, - 0x005f8d0f, 0x005fad0f, 0x005fcd0f, 0x005fed0f, - 0x00600d0f, 0x00602d0f, 0x00604d0f, 0x00606d0f, - 0x00608d0f, 0x0060ad0f, 0x0060cd0f, 0x0060ed0f, - 0x00610d0f, 0x00612d0f, 0x00614d0f, 0x00616d0f, - 0x00618d0f, 0x0061ad0f, 0x0061cd0f, 0x0061ed0f, - 0x00620d0f, 0x00622d0f, 0x00624d0f, 0x00626d0f, - 0x00628d0f, 0x0062ad0f, 0x0062cd0f, 0x0062ed0f, - 0x00630d0f, 0x00632d0f, 0x00634d0f, 0x00636d0f, - 0x00638d0f, 0x0063ad0f, 0x0063cd0f, 0x0063ed0f, - 0x00640d0f, 0x00642d0f, 0x00644d0f, 0x00646d0f, - 0x00648d0f, 0x0064ad0f, 0x0064cd0f, 0x0064ed0f, - 0x00650d0f, 0x00652d0f, 0x00654d0f, 0x00656d0f, - 0x00658d0f, 0x0065ad0f, 0x0065cd0f, 0x0065ed0f, - 0x00660d0f, 0x00662d0f, 0x00664d0f, 0x00666d0f, - 0x00668d0f, 0x0066ad0f, 0x0066cd0f, 0x0066ed0f, - 0x00670d0f, 0x00672d0f, 0x00674d0f, 0x00676d0f, - 0x00678d0f, 0x0067ad0f, 0x0067cd0f, 0x0067ed0f, - 0x00680d0f, 0x00682d0f, 0x00684d0f, 0x00686d0f, - 0x00688d0f, 0x0068ad0f, 0x0068cd0f, 0x0068ed0f, - 0x00690d0f, 0x00692d0f, 0x00694d0f, 0x00696d0f, - 0x00698d0f, 0x0069ad0f, 0x0069cd0f, 0x0069ed0f, - 0x006a0d0f, 0x006a2d0f, 0x006a4d0f, 0x006a6d0f, - 0x006a8d0f, 0x006aad0f, 0x006acd0f, 0x006aed0f, - 0x006b0d0f, 0x006b2d0f, 0x006b4d0f, 0x006b6d0f, - 0x006b8d0f, 0x006bad0f, 0x006bcd0f, 0x006bed0f, - 0x006c0d0f, 0x006c2d0f, 0x006c4d0f, 0x006c6d0f, - 0x006c8d0f, 0x006cad0f, 0x006ccd0f, 0x006ced0f, - 0x006d0d0f, 0x006d2d0f, 0x006d4d0f, 0x006d6d0f, - 0x006d8d0f, 0x006dad0f, 0x006dcd0f, 0x006ded0f, - 0x006e0d0f, 0x006e2d0f, 0x006e4d0f, 0x006e6d0f, - 0x006e8d0f, 0x006ead0f, 0x006ecd0f, 0x006eed0f, - 0x006f0d0f, 0x006f2d0f, 0x006f4d0f, 0x006f6d0f, - 0x006f8d0f, 0x006fad0f, 0x006fcd0f, 0x006fed0f, - 0x00700d0f, 0x00702d0f, 0x00704d0f, 0x00706d0f, - 0x00708d0f, 0x0070ad0f, 0x0070cd0f, 0x0070ed0f, - 0x00710d0f, 0x00712d0f, 0x00714d0f, 0x00716d0f, - 0x00718d0f, 0x0071ad0f, 0x0071cd0f, 0x0071ed0f, - 0x00720d0f, 0x00722d0f, 0x00724d0f, 0x00726d0f, - 0x00728d0f, 0x0072ad0f, 0x0072cd0f, 0x0072ed0f, - 0x00730d0f, 0x00732d0f, 0x00734d0f, 0x00736d0f, - 0x00738d0f, 0x0073ad0f, 0x0073cd0f, 0x0073ed0f, - 0x00740d0f, 0x00742d0f, 0x00744d0f, 0x00746d0f, - 0x00748d0f, 0x0074ad0f, 0x0074cd0f, 0x0074ed0f, - 0x00750d0f, 0x00752d0f, 0x00754d0f, 0x00756d0f, - 0x00758d0f, 0x0075ad0f, 0x0075cd0f, 0x0075ed0f, - 0x00760d0f, 0x00762d0f, 0x00764d0f, 0x00766d0f, - 0x00768d0f, 0x0076ad0f, 0x0076cd0f, 0x0076ed0f, - 0x00770d0f, 0x00772d0f, 0x00774d0f, 0x00776d0f, - 0x00778d0f, 0x0077ad0f, 0x0077cd0f, 0x0077ed0f, - 0x00780d0f, 0x00782d0f, 0x00784d0f, 0x00786d0f, - 0x00788d0f, 0x0078ad0f, 0x0078cd0f, 0x0078ed0f, - 0x00790d0f, 0x00792d0f, 0x00794d0f, 0x00796d0f, - 0x00798d0f, 0x0079ad0f, 0x0079cd0f, 0x0079ed0f, - 0x007a0d0f, 0x007a2d0f, 0x007a4d0f, 0x007a6d0f, - 0x007a8d0f, 0x007aad0f, 0x007acd0f, 0x007aed0f, - 0x007b0d0f, 0x007b2d0f, 0x007b4d0f, 0x007b6d0f, - 0x007b8d0f, 0x007bad0f, 0x007bcd0f, 0x007bed0f, - 0x007c0d0f, 0x007c2d0f, 0x007c4d0f, 0x007c6d0f, - 0x007c8d0f, 0x007cad0f, 0x007ccd0f, 0x007ced0f, - 0x007d0d0f, 0x007d2d0f, 0x007d4d0f, 0x007d6d0f, - 0x007d8d0f, 0x007dad0f, 0x007dcd0f, 0x007ded0f, - 0x007e0d0f, 0x007e2d0f, 0x007e4d0f, 0x007e6d0f, - 0x007e8d0f, 0x007ead0f, 0x007ecd0f, 0x007eed0f, - 0x007f0d0f, 0x007f2d0f, 0x007f4d0f, 0x007f6d0f, - 0x007f8d0f, 0x007fad0f, 0x007fcd0f, 0x007fed0f, - 0x00001d0f, 0x00003d0f, 0x00005d0f, 0x00007d0f, - 0x00009d0f, 0x0000bd0f, 0x0000dd0f, 0x0000fd0f, - 0x00011d0f, 0x00013d0f, 0x00015d0f, 0x00017d0f, - 0x00019d0f, 0x0001bd0f, 0x0001dd0f, 0x0001fd0f, - 0x00021d0f, 0x00023d0f, 0x00025d0f, 0x00027d0f, - 0x00029d0f, 0x0002bd0f, 0x0002dd0f, 0x0002fd0f, - 0x00031d0f, 0x00033d0f, 0x00035d0f, 0x00037d0f, - 0x00039d0f, 0x0003bd0f, 0x0003dd0f, 0x0003fd0f, - 0x00041d0f, 0x00043d0f, 0x00045d0f, 0x00047d0f, - 0x00049d0f, 0x0004bd0f, 0x0004dd0f, 0x0004fd0f, - 0x00051d0f, 0x00053d0f, 0x00055d0f, 0x00057d0f, - 0x00059d0f, 0x0005bd0f, 0x0005dd0f, 0x0005fd0f, - 0x00061d0f, 0x00063d0f, 0x00065d0f, 0x00067d0f, - 0x00069d0f, 0x0006bd0f, 0x0006dd0f, 0x0006fd0f, - 0x00071d0f, 0x00073d0f, 0x00075d0f, 0x00077d0f, - 0x00079d0f, 0x0007bd0f, 0x0007dd0f, 0x0007fd0f, - 0x00081d0f, 0x00083d0f, 0x00085d0f, 0x00087d0f, - 0x00089d0f, 0x0008bd0f, 0x0008dd0f, 0x0008fd0f, - 0x00091d0f, 0x00093d0f, 0x00095d0f, 0x00097d0f, - 0x00099d0f, 0x0009bd0f, 0x0009dd0f, 0x0009fd0f, - 0x000a1d0f, 0x000a3d0f, 0x000a5d0f, 0x000a7d0f, - 0x000a9d0f, 0x000abd0f, 0x000add0f, 0x000afd0f, - 0x000b1d0f, 0x000b3d0f, 0x000b5d0f, 0x000b7d0f, - 0x000b9d0f, 0x000bbd0f, 0x000bdd0f, 0x000bfd0f, - 0x000c1d0f, 0x000c3d0f, 0x000c5d0f, 0x000c7d0f, - 0x000c9d0f, 0x000cbd0f, 0x000cdd0f, 0x000cfd0f, - 0x000d1d0f, 0x000d3d0f, 0x000d5d0f, 0x000d7d0f, - 0x000d9d0f, 0x000dbd0f, 0x000ddd0f, 0x000dfd0f, - 0x000e1d0f, 0x000e3d0f, 0x000e5d0f, 0x000e7d0f, - 0x000e9d0f, 0x000ebd0f, 0x000edd0f, 0x000efd0f, - 0x000f1d0f, 0x000f3d0f, 0x000f5d0f, 0x000f7d0f, - 0x000f9d0f, 0x000fbd0f, 0x000fdd0f, 0x000ffd0f, - 0x00101d0f, 0x00103d0f, 0x00105d0f, 0x00107d0f, - 0x00109d0f, 0x0010bd0f, 0x0010dd0f, 0x0010fd0f, - 0x00111d0f, 0x00113d0f, 0x00115d0f, 0x00117d0f, - 0x00119d0f, 0x0011bd0f, 0x0011dd0f, 0x0011fd0f, - 0x00121d0f, 0x00123d0f, 0x00125d0f, 0x00127d0f, - 0x00129d0f, 0x0012bd0f, 0x0012dd0f, 0x0012fd0f, - 0x00131d0f, 0x00133d0f, 0x00135d0f, 0x00137d0f, - 0x00139d0f, 0x0013bd0f, 0x0013dd0f, 0x0013fd0f, - 0x00141d0f, 0x00143d0f, 0x00145d0f, 0x00147d0f, - 0x00149d0f, 0x0014bd0f, 0x0014dd0f, 0x0014fd0f, - 0x00151d0f, 0x00153d0f, 0x00155d0f, 0x00157d0f, - 0x00159d0f, 0x0015bd0f, 0x0015dd0f, 0x0015fd0f, - 0x00161d0f, 0x00163d0f, 0x00165d0f, 0x00167d0f, - 0x00169d0f, 0x0016bd0f, 0x0016dd0f, 0x0016fd0f, - 0x00171d0f, 0x00173d0f, 0x00175d0f, 0x00177d0f, - 0x00179d0f, 0x0017bd0f, 0x0017dd0f, 0x0017fd0f, - 0x00181d0f, 0x00183d0f, 0x00185d0f, 0x00187d0f, - 0x00189d0f, 0x0018bd0f, 0x0018dd0f, 0x0018fd0f, - 0x00191d0f, 0x00193d0f, 0x00195d0f, 0x00197d0f, - 0x00199d0f, 0x0019bd0f, 0x0019dd0f, 0x0019fd0f, - 0x001a1d0f, 0x001a3d0f, 0x001a5d0f, 0x001a7d0f, - 0x001a9d0f, 0x001abd0f, 0x001add0f, 0x001afd0f, - 0x001b1d0f, 0x001b3d0f, 0x001b5d0f, 0x001b7d0f, - 0x001b9d0f, 0x001bbd0f, 0x001bdd0f, 0x001bfd0f, - 0x001c1d0f, 0x001c3d0f, 0x001c5d0f, 0x001c7d0f, - 0x001c9d0f, 0x001cbd0f, 0x001cdd0f, 0x001cfd0f, - 0x001d1d0f, 0x001d3d0f, 0x001d5d0f, 0x001d7d0f, - 0x001d9d0f, 0x001dbd0f, 0x001ddd0f, 0x001dfd0f, - 0x001e1d0f, 0x001e3d0f, 0x001e5d0f, 0x001e7d0f, - 0x001e9d0f, 0x001ebd0f, 0x001edd0f, 0x001efd0f, - 0x001f1d0f, 0x001f3d0f, 0x001f5d0f, 0x001f7d0f, - 0x001f9d0f, 0x001fbd0f, 0x001fdd0f, 0x001ffd0f, - 0x00201d0f, 0x00203d0f, 0x00205d0f, 0x00207d0f, - 0x00209d0f, 0x0020bd0f, 0x0020dd0f, 0x0020fd0f, - 0x00211d0f, 0x00213d0f, 0x00215d0f, 0x00217d0f, - 0x00219d0f, 0x0021bd0f, 0x0021dd0f, 0x0021fd0f, - 0x00221d0f, 0x00223d0f, 0x00225d0f, 0x00227d0f, - 0x00229d0f, 0x0022bd0f, 0x0022dd0f, 0x0022fd0f, - 0x00231d0f, 0x00233d0f, 0x00235d0f, 0x00237d0f, - 0x00239d0f, 0x0023bd0f, 0x0023dd0f, 0x0023fd0f, - 0x00241d0f, 0x00243d0f, 0x00245d0f, 0x00247d0f, - 0x00249d0f, 0x0024bd0f, 0x0024dd0f, 0x0024fd0f, - 0x00251d0f, 0x00253d0f, 0x00255d0f, 0x00257d0f, - 0x00259d0f, 0x0025bd0f, 0x0025dd0f, 0x0025fd0f, - 0x00261d0f, 0x00263d0f, 0x00265d0f, 0x00267d0f, - 0x00269d0f, 0x0026bd0f, 0x0026dd0f, 0x0026fd0f, - 0x00271d0f, 0x00273d0f, 0x00275d0f, 0x00277d0f, - 0x00279d0f, 0x0027bd0f, 0x0027dd0f, 0x0027fd0f, - 0x00281d0f, 0x00283d0f, 0x00285d0f, 0x00287d0f, - 0x00289d0f, 0x0028bd0f, 0x0028dd0f, 0x0028fd0f, - 0x00291d0f, 0x00293d0f, 0x00295d0f, 0x00297d0f, - 0x00299d0f, 0x0029bd0f, 0x0029dd0f, 0x0029fd0f, - 0x002a1d0f, 0x002a3d0f, 0x002a5d0f, 0x002a7d0f, - 0x002a9d0f, 0x002abd0f, 0x002add0f, 0x002afd0f, - 0x002b1d0f, 0x002b3d0f, 0x002b5d0f, 0x002b7d0f, - 0x002b9d0f, 0x002bbd0f, 0x002bdd0f, 0x002bfd0f, - 0x002c1d0f, 0x002c3d0f, 0x002c5d0f, 0x002c7d0f, - 0x002c9d0f, 0x002cbd0f, 0x002cdd0f, 0x002cfd0f, - 0x002d1d0f, 0x002d3d0f, 0x002d5d0f, 0x002d7d0f, - 0x002d9d0f, 0x002dbd0f, 0x002ddd0f, 0x002dfd0f, - 0x002e1d0f, 0x002e3d0f, 0x002e5d0f, 0x002e7d0f, - 0x002e9d0f, 0x002ebd0f, 0x002edd0f, 0x002efd0f, - 0x002f1d0f, 0x002f3d0f, 0x002f5d0f, 0x002f7d0f, - 0x002f9d0f, 0x002fbd0f, 0x002fdd0f, 0x002ffd0f, - 0x00301d0f, 0x00303d0f, 0x00305d0f, 0x00307d0f, - 0x00309d0f, 0x0030bd0f, 0x0030dd0f, 0x0030fd0f, - 0x00311d0f, 0x00313d0f, 0x00315d0f, 0x00317d0f, - 0x00319d0f, 0x0031bd0f, 0x0031dd0f, 0x0031fd0f, - 0x00321d0f, 0x00323d0f, 0x00325d0f, 0x00327d0f, - 0x00329d0f, 0x0032bd0f, 0x0032dd0f, 0x0032fd0f, - 0x00331d0f, 0x00333d0f, 0x00335d0f, 0x00337d0f, - 0x00339d0f, 0x0033bd0f, 0x0033dd0f, 0x0033fd0f, - 0x00341d0f, 0x00343d0f, 0x00345d0f, 0x00347d0f, - 0x00349d0f, 0x0034bd0f, 0x0034dd0f, 0x0034fd0f, - 0x00351d0f, 0x00353d0f, 0x00355d0f, 0x00357d0f, - 0x00359d0f, 0x0035bd0f, 0x0035dd0f, 0x0035fd0f, - 0x00361d0f, 0x00363d0f, 0x00365d0f, 0x00367d0f, - 0x00369d0f, 0x0036bd0f, 0x0036dd0f, 0x0036fd0f, - 0x00371d0f, 0x00373d0f, 0x00375d0f, 0x00377d0f, - 0x00379d0f, 0x0037bd0f, 0x0037dd0f, 0x0037fd0f, - 0x00381d0f, 0x00383d0f, 0x00385d0f, 0x00387d0f, - 0x00389d0f, 0x0038bd0f, 0x0038dd0f, 0x0038fd0f, - 0x00391d0f, 0x00393d0f, 0x00395d0f, 0x00397d0f, - 0x00399d0f, 0x0039bd0f, 0x0039dd0f, 0x0039fd0f, - 0x003a1d0f, 0x003a3d0f, 0x003a5d0f, 0x003a7d0f, - 0x003a9d0f, 0x003abd0f, 0x003add0f, 0x003afd0f, - 0x003b1d0f, 0x003b3d0f, 0x003b5d0f, 0x003b7d0f, - 0x003b9d0f, 0x003bbd0f, 0x003bdd0f, 0x003bfd0f, - 0x003c1d0f, 0x003c3d0f, 0x003c5d0f, 0x003c7d0f, - 0x003c9d0f, 0x003cbd0f, 0x003cdd0f, 0x003cfd0f, - 0x003d1d0f, 0x003d3d0f, 0x003d5d0f, 0x003d7d0f, - 0x003d9d0f, 0x003dbd0f, 0x003ddd0f, 0x003dfd0f, - 0x003e1d0f, 0x003e3d0f, 0x003e5d0f, 0x003e7d0f, - 0x003e9d0f, 0x003ebd0f, 0x003edd0f, 0x003efd0f, - 0x003f1d0f, 0x003f3d0f, 0x003f5d0f, 0x003f7d0f, - 0x003f9d0f, 0x003fbd0f, 0x003fdd0f, 0x003ffd0f, - 0x00401d0f, 0x00403d0f, 0x00405d0f, 0x00407d0f, - 0x00409d0f, 0x0040bd0f, 0x0040dd0f, 0x0040fd0f, - 0x00411d0f, 0x00413d0f, 0x00415d0f, 0x00417d0f, - 0x00419d0f, 0x0041bd0f, 0x0041dd0f, 0x0041fd0f, - 0x00421d0f, 0x00423d0f, 0x00425d0f, 0x00427d0f, - 0x00429d0f, 0x0042bd0f, 0x0042dd0f, 0x0042fd0f, - 0x00431d0f, 0x00433d0f, 0x00435d0f, 0x00437d0f, - 0x00439d0f, 0x0043bd0f, 0x0043dd0f, 0x0043fd0f, - 0x00441d0f, 0x00443d0f, 0x00445d0f, 0x00447d0f, - 0x00449d0f, 0x0044bd0f, 0x0044dd0f, 0x0044fd0f, - 0x00451d0f, 0x00453d0f, 0x00455d0f, 0x00457d0f, - 0x00459d0f, 0x0045bd0f, 0x0045dd0f, 0x0045fd0f, - 0x00461d0f, 0x00463d0f, 0x00465d0f, 0x00467d0f, - 0x00469d0f, 0x0046bd0f, 0x0046dd0f, 0x0046fd0f, - 0x00471d0f, 0x00473d0f, 0x00475d0f, 0x00477d0f, - 0x00479d0f, 0x0047bd0f, 0x0047dd0f, 0x0047fd0f, - 0x00481d0f, 0x00483d0f, 0x00485d0f, 0x00487d0f, - 0x00489d0f, 0x0048bd0f, 0x0048dd0f, 0x0048fd0f, - 0x00491d0f, 0x00493d0f, 0x00495d0f, 0x00497d0f, - 0x00499d0f, 0x0049bd0f, 0x0049dd0f, 0x0049fd0f, - 0x004a1d0f, 0x004a3d0f, 0x004a5d0f, 0x004a7d0f, - 0x004a9d0f, 0x004abd0f, 0x004add0f, 0x004afd0f, - 0x004b1d0f, 0x004b3d0f, 0x004b5d0f, 0x004b7d0f, - 0x004b9d0f, 0x004bbd0f, 0x004bdd0f, 0x004bfd0f, - 0x004c1d0f, 0x004c3d0f, 0x004c5d0f, 0x004c7d0f, - 0x004c9d0f, 0x004cbd0f, 0x004cdd0f, 0x004cfd0f, - 0x004d1d0f, 0x004d3d0f, 0x004d5d0f, 0x004d7d0f, - 0x004d9d0f, 0x004dbd0f, 0x004ddd0f, 0x004dfd0f, - 0x004e1d0f, 0x004e3d0f, 0x004e5d0f, 0x004e7d0f, - 0x004e9d0f, 0x004ebd0f, 0x004edd0f, 0x004efd0f, - 0x004f1d0f, 0x004f3d0f, 0x004f5d0f, 0x004f7d0f, - 0x004f9d0f, 0x004fbd0f, 0x004fdd0f, 0x004ffd0f, - 0x00501d0f, 0x00503d0f, 0x00505d0f, 0x00507d0f, - 0x00509d0f, 0x0050bd0f, 0x0050dd0f, 0x0050fd0f, - 0x00511d0f, 0x00513d0f, 0x00515d0f, 0x00517d0f, - 0x00519d0f, 0x0051bd0f, 0x0051dd0f, 0x0051fd0f, - 0x00521d0f, 0x00523d0f, 0x00525d0f, 0x00527d0f, - 0x00529d0f, 0x0052bd0f, 0x0052dd0f, 0x0052fd0f, - 0x00531d0f, 0x00533d0f, 0x00535d0f, 0x00537d0f, - 0x00539d0f, 0x0053bd0f, 0x0053dd0f, 0x0053fd0f, - 0x00541d0f, 0x00543d0f, 0x00545d0f, 0x00547d0f, - 0x00549d0f, 0x0054bd0f, 0x0054dd0f, 0x0054fd0f, - 0x00551d0f, 0x00553d0f, 0x00555d0f, 0x00557d0f, - 0x00559d0f, 0x0055bd0f, 0x0055dd0f, 0x0055fd0f, - 0x00561d0f, 0x00563d0f, 0x00565d0f, 0x00567d0f, - 0x00569d0f, 0x0056bd0f, 0x0056dd0f, 0x0056fd0f, - 0x00571d0f, 0x00573d0f, 0x00575d0f, 0x00577d0f, - 0x00579d0f, 0x0057bd0f, 0x0057dd0f, 0x0057fd0f, - 0x00581d0f, 0x00583d0f, 0x00585d0f, 0x00587d0f, - 0x00589d0f, 0x0058bd0f, 0x0058dd0f, 0x0058fd0f, - 0x00591d0f, 0x00593d0f, 0x00595d0f, 0x00597d0f, - 0x00599d0f, 0x0059bd0f, 0x0059dd0f, 0x0059fd0f, - 0x005a1d0f, 0x005a3d0f, 0x005a5d0f, 0x005a7d0f, - 0x005a9d0f, 0x005abd0f, 0x005add0f, 0x005afd0f, - 0x005b1d0f, 0x005b3d0f, 0x005b5d0f, 0x005b7d0f, - 0x005b9d0f, 0x005bbd0f, 0x005bdd0f, 0x005bfd0f, - 0x005c1d0f, 0x005c3d0f, 0x005c5d0f, 0x005c7d0f, - 0x005c9d0f, 0x005cbd0f, 0x005cdd0f, 0x005cfd0f, - 0x005d1d0f, 0x005d3d0f, 0x005d5d0f, 0x005d7d0f, - 0x005d9d0f, 0x005dbd0f, 0x005ddd0f, 0x005dfd0f, - 0x005e1d0f, 0x005e3d0f, 0x005e5d0f, 0x005e7d0f, - 0x005e9d0f, 0x005ebd0f, 0x005edd0f, 0x005efd0f, - 0x005f1d0f, 0x005f3d0f, 0x005f5d0f, 0x005f7d0f, - 0x005f9d0f, 0x005fbd0f, 0x005fdd0f, 0x005ffd0f, - 0x00601d0f, 0x00603d0f, 0x00605d0f, 0x00607d0f, - 0x00609d0f, 0x0060bd0f, 0x0060dd0f, 0x0060fd0f, - 0x00611d0f, 0x00613d0f, 0x00615d0f, 0x00617d0f, - 0x00619d0f, 0x0061bd0f, 0x0061dd0f, 0x0061fd0f, - 0x00621d0f, 0x00623d0f, 0x00625d0f, 0x00627d0f, - 0x00629d0f, 0x0062bd0f, 0x0062dd0f, 0x0062fd0f, - 0x00631d0f, 0x00633d0f, 0x00635d0f, 0x00637d0f, - 0x00639d0f, 0x0063bd0f, 0x0063dd0f, 0x0063fd0f, - 0x00641d0f, 0x00643d0f, 0x00645d0f, 0x00647d0f, - 0x00649d0f, 0x0064bd0f, 0x0064dd0f, 0x0064fd0f, - 0x00651d0f, 0x00653d0f, 0x00655d0f, 0x00657d0f, - 0x00659d0f, 0x0065bd0f, 0x0065dd0f, 0x0065fd0f, - 0x00661d0f, 0x00663d0f, 0x00665d0f, 0x00667d0f, - 0x00669d0f, 0x0066bd0f, 0x0066dd0f, 0x0066fd0f, - 0x00671d0f, 0x00673d0f, 0x00675d0f, 0x00677d0f, - 0x00679d0f, 0x0067bd0f, 0x0067dd0f, 0x0067fd0f, - 0x00681d0f, 0x00683d0f, 0x00685d0f, 0x00687d0f, - 0x00689d0f, 0x0068bd0f, 0x0068dd0f, 0x0068fd0f, - 0x00691d0f, 0x00693d0f, 0x00695d0f, 0x00697d0f, - 0x00699d0f, 0x0069bd0f, 0x0069dd0f, 0x0069fd0f, - 0x006a1d0f, 0x006a3d0f, 0x006a5d0f, 0x006a7d0f, - 0x006a9d0f, 0x006abd0f, 0x006add0f, 0x006afd0f, - 0x006b1d0f, 0x006b3d0f, 0x006b5d0f, 0x006b7d0f, - 0x006b9d0f, 0x006bbd0f, 0x006bdd0f, 0x006bfd0f, - 0x006c1d0f, 0x006c3d0f, 0x006c5d0f, 0x006c7d0f, - 0x006c9d0f, 0x006cbd0f, 0x006cdd0f, 0x006cfd0f, - 0x006d1d0f, 0x006d3d0f, 0x006d5d0f, 0x006d7d0f, - 0x006d9d0f, 0x006dbd0f, 0x006ddd0f, 0x006dfd0f, - 0x006e1d0f, 0x006e3d0f, 0x006e5d0f, 0x006e7d0f, - 0x006e9d0f, 0x006ebd0f, 0x006edd0f, 0x006efd0f, - 0x006f1d0f, 0x006f3d0f, 0x006f5d0f, 0x006f7d0f, - 0x006f9d0f, 0x006fbd0f, 0x006fdd0f, 0x006ffd0f, - 0x00701d0f, 0x00703d0f, 0x00705d0f, 0x00707d0f, - 0x00709d0f, 0x0070bd0f, 0x0070dd0f, 0x0070fd0f, - 0x00711d0f, 0x00713d0f, 0x00715d0f, 0x00717d0f, - 0x00719d0f, 0x0071bd0f, 0x0071dd0f, 0x0071fd0f, - 0x00721d0f, 0x00723d0f, 0x00725d0f, 0x00727d0f, - 0x00729d0f, 0x0072bd0f, 0x0072dd0f, 0x0072fd0f, - 0x00731d0f, 0x00733d0f, 0x00735d0f, 0x00737d0f, - 0x00739d0f, 0x0073bd0f, 0x0073dd0f, 0x0073fd0f, - 0x00741d0f, 0x00743d0f, 0x00745d0f, 0x00747d0f, - 0x00749d0f, 0x0074bd0f, 0x0074dd0f, 0x0074fd0f, - 0x00751d0f, 0x00753d0f, 0x00755d0f, 0x00757d0f, - 0x00759d0f, 0x0075bd0f, 0x0075dd0f, 0x0075fd0f, - 0x00761d0f, 0x00763d0f, 0x00765d0f, 0x00767d0f, - 0x00769d0f, 0x0076bd0f, 0x0076dd0f, 0x0076fd0f, - 0x00771d0f, 0x00773d0f, 0x00775d0f, 0x00777d0f, - 0x00779d0f, 0x0077bd0f, 0x0077dd0f, 0x0077fd0f, - 0x00781d0f, 0x00783d0f, 0x00785d0f, 0x00787d0f, - 0x00789d0f, 0x0078bd0f, 0x0078dd0f, 0x0078fd0f, - 0x00791d0f, 0x00793d0f, 0x00795d0f, 0x00797d0f, - 0x00799d0f, 0x0079bd0f, 0x0079dd0f, 0x0079fd0f, - 0x007a1d0f, 0x007a3d0f, 0x007a5d0f, 0x007a7d0f, - 0x007a9d0f, 0x007abd0f, 0x007add0f, 0x007afd0f, - 0x007b1d0f, 0x007b3d0f, 0x007b5d0f, 0x007b7d0f, - 0x007b9d0f, 0x007bbd0f, 0x007bdd0f, 0x007bfd0f, - 0x007c1d0f, 0x007c3d0f, 0x007c5d0f, 0x007c7d0f, - 0x007c9d0f, 0x007cbd0f, 0x007cdd0f, 0x007cfd0f, - 0x007d1d0f, 0x007d3d0f, 0x007d5d0f, 0x007d7d0f, - 0x007d9d0f, 0x007dbd0f, 0x007ddd0f, 0x007dfd0f, - 0x007e1d0f, 0x007e3d0f, 0x007e5d0f, 0x007e7d0f, - 0x007e9d0f, 0x007ebd0f, 0x007edd0f, 0x007efd0f, - 0x007f1d0f, 0x007f3d0f, 0x007f5d0f, 0x007f7d0f, - 0x007f9d0f, 0x007fbd0f, 0x007fdd0f, 0x007ffd0f, - 0x00000310, 0x00002310, 0x00004310, 0x00006310, - 0x00008310, 0x0000a310, 0x0000c310, 0x0000e310, - 0x00010310, 0x00012310, 0x00014310, 0x00016310, - 0x00018310, 0x0001a310, 0x0001c310, 0x0001e310, - 0x00020310, 0x00022310, 0x00024310, 0x00026310, - 0x00028310, 0x0002a310, 0x0002c310, 0x0002e310, - 0x00030310, 0x00032310, 0x00034310, 0x00036310, - 0x00038310, 0x0003a310, 0x0003c310, 0x0003e310, - 0x00040310, 0x00042310, 0x00044310, 0x00046310, - 0x00048310, 0x0004a310, 0x0004c310, 0x0004e310, - 0x00050310, 0x00052310, 0x00054310, 0x00056310, - 0x00058310, 0x0005a310, 0x0005c310, 0x0005e310, - 0x00060310, 0x00062310, 0x00064310, 0x00066310, - 0x00068310, 0x0006a310, 0x0006c310, 0x0006e310, - 0x00070310, 0x00072310, 0x00074310, 0x00076310, - 0x00078310, 0x0007a310, 0x0007c310, 0x0007e310, - 0x00080310, 0x00082310, 0x00084310, 0x00086310, - 0x00088310, 0x0008a310, 0x0008c310, 0x0008e310, - 0x00090310, 0x00092310, 0x00094310, 0x00096310, - 0x00098310, 0x0009a310, 0x0009c310, 0x0009e310, - 0x000a0310, 0x000a2310, 0x000a4310, 0x000a6310, - 0x000a8310, 0x000aa310, 0x000ac310, 0x000ae310, - 0x000b0310, 0x000b2310, 0x000b4310, 0x000b6310, - 0x000b8310, 0x000ba310, 0x000bc310, 0x000be310, - 0x000c0310, 0x000c2310, 0x000c4310, 0x000c6310, - 0x000c8310, 0x000ca310, 0x000cc310, 0x000ce310, - 0x000d0310, 0x000d2310, 0x000d4310, 0x000d6310, - 0x000d8310, 0x000da310, 0x000dc310, 0x000de310, - 0x000e0310, 0x000e2310, 0x000e4310, 0x000e6310, - 0x000e8310, 0x000ea310, 0x000ec310, 0x000ee310, - 0x000f0310, 0x000f2310, 0x000f4310, 0x000f6310, - 0x000f8310, 0x000fa310, 0x000fc310, 0x000fe310, - 0x00100310, 0x00102310, 0x00104310, 0x00106310, - 0x00108310, 0x0010a310, 0x0010c310, 0x0010e310, - 0x00110310, 0x00112310, 0x00114310, 0x00116310, - 0x00118310, 0x0011a310, 0x0011c310, 0x0011e310, - 0x00120310, 0x00122310, 0x00124310, 0x00126310, - 0x00128310, 0x0012a310, 0x0012c310, 0x0012e310, - 0x00130310, 0x00132310, 0x00134310, 0x00136310, - 0x00138310, 0x0013a310, 0x0013c310, 0x0013e310, - 0x00140310, 0x00142310, 0x00144310, 0x00146310, - 0x00148310, 0x0014a310, 0x0014c310, 0x0014e310, - 0x00150310, 0x00152310, 0x00154310, 0x00156310, - 0x00158310, 0x0015a310, 0x0015c310, 0x0015e310, - 0x00160310, 0x00162310, 0x00164310, 0x00166310, - 0x00168310, 0x0016a310, 0x0016c310, 0x0016e310, - 0x00170310, 0x00172310, 0x00174310, 0x00176310, - 0x00178310, 0x0017a310, 0x0017c310, 0x0017e310, - 0x00180310, 0x00182310, 0x00184310, 0x00186310, - 0x00188310, 0x0018a310, 0x0018c310, 0x0018e310, - 0x00190310, 0x00192310, 0x00194310, 0x00196310, - 0x00198310, 0x0019a310, 0x0019c310, 0x0019e310, - 0x001a0310, 0x001a2310, 0x001a4310, 0x001a6310, - 0x001a8310, 0x001aa310, 0x001ac310, 0x001ae310, - 0x001b0310, 0x001b2310, 0x001b4310, 0x001b6310, - 0x001b8310, 0x001ba310, 0x001bc310, 0x001be310, - 0x001c0310, 0x001c2310, 0x001c4310, 0x001c6310, - 0x001c8310, 0x001ca310, 0x001cc310, 0x001ce310, - 0x001d0310, 0x001d2310, 0x001d4310, 0x001d6310, - 0x001d8310, 0x001da310, 0x001dc310, 0x001de310, - 0x001e0310, 0x001e2310, 0x001e4310, 0x001e6310, - 0x001e8310, 0x001ea310, 0x001ec310, 0x001ee310, - 0x001f0310, 0x001f2310, 0x001f4310, 0x001f6310, - 0x001f8310, 0x001fa310, 0x001fc310, 0x001fe310, - 0x00200310, 0x00202310, 0x00204310, 0x00206310, - 0x00208310, 0x0020a310, 0x0020c310, 0x0020e310, - 0x00210310, 0x00212310, 0x00214310, 0x00216310, - 0x00218310, 0x0021a310, 0x0021c310, 0x0021e310, - 0x00220310, 0x00222310, 0x00224310, 0x00226310, - 0x00228310, 0x0022a310, 0x0022c310, 0x0022e310, - 0x00230310, 0x00232310, 0x00234310, 0x00236310, - 0x00238310, 0x0023a310, 0x0023c310, 0x0023e310, - 0x00240310, 0x00242310, 0x00244310, 0x00246310, - 0x00248310, 0x0024a310, 0x0024c310, 0x0024e310, - 0x00250310, 0x00252310, 0x00254310, 0x00256310, - 0x00258310, 0x0025a310, 0x0025c310, 0x0025e310, - 0x00260310, 0x00262310, 0x00264310, 0x00266310, - 0x00268310, 0x0026a310, 0x0026c310, 0x0026e310, - 0x00270310, 0x00272310, 0x00274310, 0x00276310, - 0x00278310, 0x0027a310, 0x0027c310, 0x0027e310, - 0x00280310, 0x00282310, 0x00284310, 0x00286310, - 0x00288310, 0x0028a310, 0x0028c310, 0x0028e310, - 0x00290310, 0x00292310, 0x00294310, 0x00296310, - 0x00298310, 0x0029a310, 0x0029c310, 0x0029e310, - 0x002a0310, 0x002a2310, 0x002a4310, 0x002a6310, - 0x002a8310, 0x002aa310, 0x002ac310, 0x002ae310, - 0x002b0310, 0x002b2310, 0x002b4310, 0x002b6310, - 0x002b8310, 0x002ba310, 0x002bc310, 0x002be310, - 0x002c0310, 0x002c2310, 0x002c4310, 0x002c6310, - 0x002c8310, 0x002ca310, 0x002cc310, 0x002ce310, - 0x002d0310, 0x002d2310, 0x002d4310, 0x002d6310, - 0x002d8310, 0x002da310, 0x002dc310, 0x002de310, - 0x002e0310, 0x002e2310, 0x002e4310, 0x002e6310, - 0x002e8310, 0x002ea310, 0x002ec310, 0x002ee310, - 0x002f0310, 0x002f2310, 0x002f4310, 0x002f6310, - 0x002f8310, 0x002fa310, 0x002fc310, 0x002fe310, - 0x00300310, 0x00302310, 0x00304310, 0x00306310, - 0x00308310, 0x0030a310, 0x0030c310, 0x0030e310, - 0x00310310, 0x00312310, 0x00314310, 0x00316310, - 0x00318310, 0x0031a310, 0x0031c310, 0x0031e310, - 0x00320310, 0x00322310, 0x00324310, 0x00326310, - 0x00328310, 0x0032a310, 0x0032c310, 0x0032e310, - 0x00330310, 0x00332310, 0x00334310, 0x00336310, - 0x00338310, 0x0033a310, 0x0033c310, 0x0033e310, - 0x00340310, 0x00342310, 0x00344310, 0x00346310, - 0x00348310, 0x0034a310, 0x0034c310, 0x0034e310, - 0x00350310, 0x00352310, 0x00354310, 0x00356310, - 0x00358310, 0x0035a310, 0x0035c310, 0x0035e310, - 0x00360310, 0x00362310, 0x00364310, 0x00366310, - 0x00368310, 0x0036a310, 0x0036c310, 0x0036e310, - 0x00370310, 0x00372310, 0x00374310, 0x00376310, - 0x00378310, 0x0037a310, 0x0037c310, 0x0037e310, - 0x00380310, 0x00382310, 0x00384310, 0x00386310, - 0x00388310, 0x0038a310, 0x0038c310, 0x0038e310, - 0x00390310, 0x00392310, 0x00394310, 0x00396310, - 0x00398310, 0x0039a310, 0x0039c310, 0x0039e310, - 0x003a0310, 0x003a2310, 0x003a4310, 0x003a6310, - 0x003a8310, 0x003aa310, 0x003ac310, 0x003ae310, - 0x003b0310, 0x003b2310, 0x003b4310, 0x003b6310, - 0x003b8310, 0x003ba310, 0x003bc310, 0x003be310, - 0x003c0310, 0x003c2310, 0x003c4310, 0x003c6310, - 0x003c8310, 0x003ca310, 0x003cc310, 0x003ce310, - 0x003d0310, 0x003d2310, 0x003d4310, 0x003d6310, - 0x003d8310, 0x003da310, 0x003dc310, 0x003de310, - 0x003e0310, 0x003e2310, 0x003e4310, 0x003e6310, - 0x003e8310, 0x003ea310, 0x003ec310, 0x003ee310, - 0x003f0310, 0x003f2310, 0x003f4310, 0x003f6310, - 0x003f8310, 0x003fa310, 0x003fc310, 0x003fe310, - 0x00400310, 0x00402310, 0x00404310, 0x00406310, - 0x00408310, 0x0040a310, 0x0040c310, 0x0040e310, - 0x00410310, 0x00412310, 0x00414310, 0x00416310, - 0x00418310, 0x0041a310, 0x0041c310, 0x0041e310, - 0x00420310, 0x00422310, 0x00424310, 0x00426310, - 0x00428310, 0x0042a310, 0x0042c310, 0x0042e310, - 0x00430310, 0x00432310, 0x00434310, 0x00436310, - 0x00438310, 0x0043a310, 0x0043c310, 0x0043e310, - 0x00440310, 0x00442310, 0x00444310, 0x00446310, - 0x00448310, 0x0044a310, 0x0044c310, 0x0044e310, - 0x00450310, 0x00452310, 0x00454310, 0x00456310, - 0x00458310, 0x0045a310, 0x0045c310, 0x0045e310, - 0x00460310, 0x00462310, 0x00464310, 0x00466310, - 0x00468310, 0x0046a310, 0x0046c310, 0x0046e310, - 0x00470310, 0x00472310, 0x00474310, 0x00476310, - 0x00478310, 0x0047a310, 0x0047c310, 0x0047e310, - 0x00480310, 0x00482310, 0x00484310, 0x00486310, - 0x00488310, 0x0048a310, 0x0048c310, 0x0048e310, - 0x00490310, 0x00492310, 0x00494310, 0x00496310, - 0x00498310, 0x0049a310, 0x0049c310, 0x0049e310, - 0x004a0310, 0x004a2310, 0x004a4310, 0x004a6310, - 0x004a8310, 0x004aa310, 0x004ac310, 0x004ae310, - 0x004b0310, 0x004b2310, 0x004b4310, 0x004b6310, - 0x004b8310, 0x004ba310, 0x004bc310, 0x004be310, - 0x004c0310, 0x004c2310, 0x004c4310, 0x004c6310, - 0x004c8310, 0x004ca310, 0x004cc310, 0x004ce310, - 0x004d0310, 0x004d2310, 0x004d4310, 0x004d6310, - 0x004d8310, 0x004da310, 0x004dc310, 0x004de310, - 0x004e0310, 0x004e2310, 0x004e4310, 0x004e6310, - 0x004e8310, 0x004ea310, 0x004ec310, 0x004ee310, - 0x004f0310, 0x004f2310, 0x004f4310, 0x004f6310, - 0x004f8310, 0x004fa310, 0x004fc310, 0x004fe310, - 0x00500310, 0x00502310, 0x00504310, 0x00506310, - 0x00508310, 0x0050a310, 0x0050c310, 0x0050e310, - 0x00510310, 0x00512310, 0x00514310, 0x00516310, - 0x00518310, 0x0051a310, 0x0051c310, 0x0051e310, - 0x00520310, 0x00522310, 0x00524310, 0x00526310, - 0x00528310, 0x0052a310, 0x0052c310, 0x0052e310, - 0x00530310, 0x00532310, 0x00534310, 0x00536310, - 0x00538310, 0x0053a310, 0x0053c310, 0x0053e310, - 0x00540310, 0x00542310, 0x00544310, 0x00546310, - 0x00548310, 0x0054a310, 0x0054c310, 0x0054e310, - 0x00550310, 0x00552310, 0x00554310, 0x00556310, - 0x00558310, 0x0055a310, 0x0055c310, 0x0055e310, - 0x00560310, 0x00562310, 0x00564310, 0x00566310, - 0x00568310, 0x0056a310, 0x0056c310, 0x0056e310, - 0x00570310, 0x00572310, 0x00574310, 0x00576310, - 0x00578310, 0x0057a310, 0x0057c310, 0x0057e310, - 0x00580310, 0x00582310, 0x00584310, 0x00586310, - 0x00588310, 0x0058a310, 0x0058c310, 0x0058e310, - 0x00590310, 0x00592310, 0x00594310, 0x00596310, - 0x00598310, 0x0059a310, 0x0059c310, 0x0059e310, - 0x005a0310, 0x005a2310, 0x005a4310, 0x005a6310, - 0x005a8310, 0x005aa310, 0x005ac310, 0x005ae310, - 0x005b0310, 0x005b2310, 0x005b4310, 0x005b6310, - 0x005b8310, 0x005ba310, 0x005bc310, 0x005be310, - 0x005c0310, 0x005c2310, 0x005c4310, 0x005c6310, - 0x005c8310, 0x005ca310, 0x005cc310, 0x005ce310, - 0x005d0310, 0x005d2310, 0x005d4310, 0x005d6310, - 0x005d8310, 0x005da310, 0x005dc310, 0x005de310, - 0x005e0310, 0x005e2310, 0x005e4310, 0x005e6310, - 0x005e8310, 0x005ea310, 0x005ec310, 0x005ee310, - 0x005f0310, 0x005f2310, 0x005f4310, 0x005f6310, - 0x005f8310, 0x005fa310, 0x005fc310, 0x005fe310, - 0x00600310, 0x00602310, 0x00604310, 0x00606310, - 0x00608310, 0x0060a310, 0x0060c310, 0x0060e310, - 0x00610310, 0x00612310, 0x00614310, 0x00616310, - 0x00618310, 0x0061a310, 0x0061c310, 0x0061e310, - 0x00620310, 0x00622310, 0x00624310, 0x00626310, - 0x00628310, 0x0062a310, 0x0062c310, 0x0062e310, - 0x00630310, 0x00632310, 0x00634310, 0x00636310, - 0x00638310, 0x0063a310, 0x0063c310, 0x0063e310, - 0x00640310, 0x00642310, 0x00644310, 0x00646310, - 0x00648310, 0x0064a310, 0x0064c310, 0x0064e310, - 0x00650310, 0x00652310, 0x00654310, 0x00656310, - 0x00658310, 0x0065a310, 0x0065c310, 0x0065e310, - 0x00660310, 0x00662310, 0x00664310, 0x00666310, - 0x00668310, 0x0066a310, 0x0066c310, 0x0066e310, - 0x00670310, 0x00672310, 0x00674310, 0x00676310, - 0x00678310, 0x0067a310, 0x0067c310, 0x0067e310, - 0x00680310, 0x00682310, 0x00684310, 0x00686310, - 0x00688310, 0x0068a310, 0x0068c310, 0x0068e310, - 0x00690310, 0x00692310, 0x00694310, 0x00696310, - 0x00698310, 0x0069a310, 0x0069c310, 0x0069e310, - 0x006a0310, 0x006a2310, 0x006a4310, 0x006a6310, - 0x006a8310, 0x006aa310, 0x006ac310, 0x006ae310, - 0x006b0310, 0x006b2310, 0x006b4310, 0x006b6310, - 0x006b8310, 0x006ba310, 0x006bc310, 0x006be310, - 0x006c0310, 0x006c2310, 0x006c4310, 0x006c6310, - 0x006c8310, 0x006ca310, 0x006cc310, 0x006ce310, - 0x006d0310, 0x006d2310, 0x006d4310, 0x006d6310, - 0x006d8310, 0x006da310, 0x006dc310, 0x006de310, - 0x006e0310, 0x006e2310, 0x006e4310, 0x006e6310, - 0x006e8310, 0x006ea310, 0x006ec310, 0x006ee310, - 0x006f0310, 0x006f2310, 0x006f4310, 0x006f6310, - 0x006f8310, 0x006fa310, 0x006fc310, 0x006fe310, - 0x00700310, 0x00702310, 0x00704310, 0x00706310, - 0x00708310, 0x0070a310, 0x0070c310, 0x0070e310, - 0x00710310, 0x00712310, 0x00714310, 0x00716310, - 0x00718310, 0x0071a310, 0x0071c310, 0x0071e310, - 0x00720310, 0x00722310, 0x00724310, 0x00726310, - 0x00728310, 0x0072a310, 0x0072c310, 0x0072e310, - 0x00730310, 0x00732310, 0x00734310, 0x00736310, - 0x00738310, 0x0073a310, 0x0073c310, 0x0073e310, - 0x00740310, 0x00742310, 0x00744310, 0x00746310, - 0x00748310, 0x0074a310, 0x0074c310, 0x0074e310, - 0x00750310, 0x00752310, 0x00754310, 0x00756310, - 0x00758310, 0x0075a310, 0x0075c310, 0x0075e310, - 0x00760310, 0x00762310, 0x00764310, 0x00766310, - 0x00768310, 0x0076a310, 0x0076c310, 0x0076e310, - 0x00770310, 0x00772310, 0x00774310, 0x00776310, - 0x00778310, 0x0077a310, 0x0077c310, 0x0077e310, - 0x00780310, 0x00782310, 0x00784310, 0x00786310, - 0x00788310, 0x0078a310, 0x0078c310, 0x0078e310, - 0x00790310, 0x00792310, 0x00794310, 0x00796310, - 0x00798310, 0x0079a310, 0x0079c310, 0x0079e310, - 0x007a0310, 0x007a2310, 0x007a4310, 0x007a6310, - 0x007a8310, 0x007aa310, 0x007ac310, 0x007ae310, - 0x007b0310, 0x007b2310, 0x007b4310, 0x007b6310, - 0x007b8310, 0x007ba310, 0x007bc310, 0x007be310, - 0x007c0310, 0x007c2310, 0x007c4310, 0x007c6310, - 0x007c8310, 0x007ca310, 0x007cc310, 0x007ce310, - 0x007d0310, 0x007d2310, 0x007d4310, 0x007d6310, - 0x007d8310, 0x007da310, 0x007dc310, 0x007de310, - 0x007e0310, 0x007e2310, 0x007e4310, 0x007e6310, - 0x007e8310, 0x007ea310, 0x007ec310, 0x007ee310, - 0x007f0310, 0x007f2310, 0x007f4310, 0x007f6310, - 0x007f8310, 0x007fa310, 0x007fc310, 0x007fe310, - 0x00800310, 0x00802310, 0x00804310, 0x00806310, - 0x00808310, 0x0080a310, 0x0080c310, 0x0080e310, - 0x00810310, 0x00812310, 0x00814310, 0x00816310, - 0x00818310, 0x0081a310, 0x0081c310, 0x0081e310, - 0x00820310, 0x00822310, 0x00824310, 0x00826310, - 0x00828310, 0x0082a310, 0x0082c310, 0x0082e310, - 0x00830310, 0x00832310, 0x00834310, 0x00836310, - 0x00838310, 0x0083a310, 0x0083c310, 0x0083e310, - 0x00840310, 0x00842310, 0x00844310, 0x00846310, - 0x00848310, 0x0084a310, 0x0084c310, 0x0084e310, - 0x00850310, 0x00852310, 0x00854310, 0x00856310, - 0x00858310, 0x0085a310, 0x0085c310, 0x0085e310, - 0x00860310, 0x00862310, 0x00864310, 0x00866310, - 0x00868310, 0x0086a310, 0x0086c310, 0x0086e310, - 0x00870310, 0x00872310, 0x00874310, 0x00876310, - 0x00878310, 0x0087a310, 0x0087c310, 0x0087e310, - 0x00880310, 0x00882310, 0x00884310, 0x00886310, - 0x00888310, 0x0088a310, 0x0088c310, 0x0088e310, - 0x00890310, 0x00892310, 0x00894310, 0x00896310, - 0x00898310, 0x0089a310, 0x0089c310, 0x0089e310, - 0x008a0310, 0x008a2310, 0x008a4310, 0x008a6310, - 0x008a8310, 0x008aa310, 0x008ac310, 0x008ae310, - 0x008b0310, 0x008b2310, 0x008b4310, 0x008b6310, - 0x008b8310, 0x008ba310, 0x008bc310, 0x008be310, - 0x008c0310, 0x008c2310, 0x008c4310, 0x008c6310, - 0x008c8310, 0x008ca310, 0x008cc310, 0x008ce310, - 0x008d0310, 0x008d2310, 0x008d4310, 0x008d6310, - 0x008d8310, 0x008da310, 0x008dc310, 0x008de310, - 0x008e0310, 0x008e2310, 0x008e4310, 0x008e6310, - 0x008e8310, 0x008ea310, 0x008ec310, 0x008ee310, - 0x008f0310, 0x008f2310, 0x008f4310, 0x008f6310, - 0x008f8310, 0x008fa310, 0x008fc310, 0x008fe310, - 0x00900310, 0x00902310, 0x00904310, 0x00906310, - 0x00908310, 0x0090a310, 0x0090c310, 0x0090e310, - 0x00910310, 0x00912310, 0x00914310, 0x00916310, - 0x00918310, 0x0091a310, 0x0091c310, 0x0091e310, - 0x00920310, 0x00922310, 0x00924310, 0x00926310, - 0x00928310, 0x0092a310, 0x0092c310, 0x0092e310, - 0x00930310, 0x00932310, 0x00934310, 0x00936310, - 0x00938310, 0x0093a310, 0x0093c310, 0x0093e310, - 0x00940310, 0x00942310, 0x00944310, 0x00946310, - 0x00948310, 0x0094a310, 0x0094c310, 0x0094e310, - 0x00950310, 0x00952310, 0x00954310, 0x00956310, - 0x00958310, 0x0095a310, 0x0095c310, 0x0095e310, - 0x00960310, 0x00962310, 0x00964310, 0x00966310, - 0x00968310, 0x0096a310, 0x0096c310, 0x0096e310, - 0x00970310, 0x00972310, 0x00974310, 0x00976310, - 0x00978310, 0x0097a310, 0x0097c310, 0x0097e310, - 0x00980310, 0x00982310, 0x00984310, 0x00986310, - 0x00988310, 0x0098a310, 0x0098c310, 0x0098e310, - 0x00990310, 0x00992310, 0x00994310, 0x00996310, - 0x00998310, 0x0099a310, 0x0099c310, 0x0099e310, - 0x009a0310, 0x009a2310, 0x009a4310, 0x009a6310, - 0x009a8310, 0x009aa310, 0x009ac310, 0x009ae310, - 0x009b0310, 0x009b2310, 0x009b4310, 0x009b6310, - 0x009b8310, 0x009ba310, 0x009bc310, 0x009be310, - 0x009c0310, 0x009c2310, 0x009c4310, 0x009c6310, - 0x009c8310, 0x009ca310, 0x009cc310, 0x009ce310, - 0x009d0310, 0x009d2310, 0x009d4310, 0x009d6310, - 0x009d8310, 0x009da310, 0x009dc310, 0x009de310, - 0x009e0310, 0x009e2310, 0x009e4310, 0x009e6310, - 0x009e8310, 0x009ea310, 0x009ec310, 0x009ee310, - 0x009f0310, 0x009f2310, 0x009f4310, 0x009f6310, - 0x009f8310, 0x009fa310, 0x009fc310, 0x009fe310, - 0x00a00310, 0x00a02310, 0x00a04310, 0x00a06310, - 0x00a08310, 0x00a0a310, 0x00a0c310, 0x00a0e310, - 0x00a10310, 0x00a12310, 0x00a14310, 0x00a16310, - 0x00a18310, 0x00a1a310, 0x00a1c310, 0x00a1e310, - 0x00a20310, 0x00a22310, 0x00a24310, 0x00a26310, - 0x00a28310, 0x00a2a310, 0x00a2c310, 0x00a2e310, - 0x00a30310, 0x00a32310, 0x00a34310, 0x00a36310, - 0x00a38310, 0x00a3a310, 0x00a3c310, 0x00a3e310, - 0x00a40310, 0x00a42310, 0x00a44310, 0x00a46310, - 0x00a48310, 0x00a4a310, 0x00a4c310, 0x00a4e310, - 0x00a50310, 0x00a52310, 0x00a54310, 0x00a56310, - 0x00a58310, 0x00a5a310, 0x00a5c310, 0x00a5e310, - 0x00a60310, 0x00a62310, 0x00a64310, 0x00a66310, - 0x00a68310, 0x00a6a310, 0x00a6c310, 0x00a6e310, - 0x00a70310, 0x00a72310, 0x00a74310, 0x00a76310, - 0x00a78310, 0x00a7a310, 0x00a7c310, 0x00a7e310, - 0x00a80310, 0x00a82310, 0x00a84310, 0x00a86310, - 0x00a88310, 0x00a8a310, 0x00a8c310, 0x00a8e310, - 0x00a90310, 0x00a92310, 0x00a94310, 0x00a96310, - 0x00a98310, 0x00a9a310, 0x00a9c310, 0x00a9e310, - 0x00aa0310, 0x00aa2310, 0x00aa4310, 0x00aa6310, - 0x00aa8310, 0x00aaa310, 0x00aac310, 0x00aae310, - 0x00ab0310, 0x00ab2310, 0x00ab4310, 0x00ab6310, - 0x00ab8310, 0x00aba310, 0x00abc310, 0x00abe310, - 0x00ac0310, 0x00ac2310, 0x00ac4310, 0x00ac6310, - 0x00ac8310, 0x00aca310, 0x00acc310, 0x00ace310, - 0x00ad0310, 0x00ad2310, 0x00ad4310, 0x00ad6310, - 0x00ad8310, 0x00ada310, 0x00adc310, 0x00ade310, - 0x00ae0310, 0x00ae2310, 0x00ae4310, 0x00ae6310, - 0x00ae8310, 0x00aea310, 0x00aec310, 0x00aee310, - 0x00af0310, 0x00af2310, 0x00af4310, 0x00af6310, - 0x00af8310, 0x00afa310, 0x00afc310, 0x00afe310, - 0x00b00310, 0x00b02310, 0x00b04310, 0x00b06310, - 0x00b08310, 0x00b0a310, 0x00b0c310, 0x00b0e310, - 0x00b10310, 0x00b12310, 0x00b14310, 0x00b16310, - 0x00b18310, 0x00b1a310, 0x00b1c310, 0x00b1e310, - 0x00b20310, 0x00b22310, 0x00b24310, 0x00b26310, - 0x00b28310, 0x00b2a310, 0x00b2c310, 0x00b2e310, - 0x00b30310, 0x00b32310, 0x00b34310, 0x00b36310, - 0x00b38310, 0x00b3a310, 0x00b3c310, 0x00b3e310, - 0x00b40310, 0x00b42310, 0x00b44310, 0x00b46310, - 0x00b48310, 0x00b4a310, 0x00b4c310, 0x00b4e310, - 0x00b50310, 0x00b52310, 0x00b54310, 0x00b56310, - 0x00b58310, 0x00b5a310, 0x00b5c310, 0x00b5e310, - 0x00b60310, 0x00b62310, 0x00b64310, 0x00b66310, - 0x00b68310, 0x00b6a310, 0x00b6c310, 0x00b6e310, - 0x00b70310, 0x00b72310, 0x00b74310, 0x00b76310, - 0x00b78310, 0x00b7a310, 0x00b7c310, 0x00b7e310, - 0x00b80310, 0x00b82310, 0x00b84310, 0x00b86310, - 0x00b88310, 0x00b8a310, 0x00b8c310, 0x00b8e310, - 0x00b90310, 0x00b92310, 0x00b94310, 0x00b96310, - 0x00b98310, 0x00b9a310, 0x00b9c310, 0x00b9e310, - 0x00ba0310, 0x00ba2310, 0x00ba4310, 0x00ba6310, - 0x00ba8310, 0x00baa310, 0x00bac310, 0x00bae310, - 0x00bb0310, 0x00bb2310, 0x00bb4310, 0x00bb6310, - 0x00bb8310, 0x00bba310, 0x00bbc310, 0x00bbe310, - 0x00bc0310, 0x00bc2310, 0x00bc4310, 0x00bc6310, - 0x00bc8310, 0x00bca310, 0x00bcc310, 0x00bce310, - 0x00bd0310, 0x00bd2310, 0x00bd4310, 0x00bd6310, - 0x00bd8310, 0x00bda310, 0x00bdc310, 0x00bde310, - 0x00be0310, 0x00be2310, 0x00be4310, 0x00be6310, - 0x00be8310, 0x00bea310, 0x00bec310, 0x00bee310, - 0x00bf0310, 0x00bf2310, 0x00bf4310, 0x00bf6310, - 0x00bf8310, 0x00bfa310, 0x00bfc310, 0x00bfe310, - 0x00c00310, 0x00c02310, 0x00c04310, 0x00c06310, - 0x00c08310, 0x00c0a310, 0x00c0c310, 0x00c0e310, - 0x00c10310, 0x00c12310, 0x00c14310, 0x00c16310, - 0x00c18310, 0x00c1a310, 0x00c1c310, 0x00c1e310, - 0x00c20310, 0x00c22310, 0x00c24310, 0x00c26310, - 0x00c28310, 0x00c2a310, 0x00c2c310, 0x00c2e310, - 0x00c30310, 0x00c32310, 0x00c34310, 0x00c36310, - 0x00c38310, 0x00c3a310, 0x00c3c310, 0x00c3e310, - 0x00c40310, 0x00c42310, 0x00c44310, 0x00c46310, - 0x00c48310, 0x00c4a310, 0x00c4c310, 0x00c4e310, - 0x00c50310, 0x00c52310, 0x00c54310, 0x00c56310, - 0x00c58310, 0x00c5a310, 0x00c5c310, 0x00c5e310, - 0x00c60310, 0x00c62310, 0x00c64310, 0x00c66310, - 0x00c68310, 0x00c6a310, 0x00c6c310, 0x00c6e310, - 0x00c70310, 0x00c72310, 0x00c74310, 0x00c76310, - 0x00c78310, 0x00c7a310, 0x00c7c310, 0x00c7e310, - 0x00c80310, 0x00c82310, 0x00c84310, 0x00c86310, - 0x00c88310, 0x00c8a310, 0x00c8c310, 0x00c8e310, - 0x00c90310, 0x00c92310, 0x00c94310, 0x00c96310, - 0x00c98310, 0x00c9a310, 0x00c9c310, 0x00c9e310, - 0x00ca0310, 0x00ca2310, 0x00ca4310, 0x00ca6310, - 0x00ca8310, 0x00caa310, 0x00cac310, 0x00cae310, - 0x00cb0310, 0x00cb2310, 0x00cb4310, 0x00cb6310, - 0x00cb8310, 0x00cba310, 0x00cbc310, 0x00cbe310, - 0x00cc0310, 0x00cc2310, 0x00cc4310, 0x00cc6310, - 0x00cc8310, 0x00cca310, 0x00ccc310, 0x00cce310, - 0x00cd0310, 0x00cd2310, 0x00cd4310, 0x00cd6310, - 0x00cd8310, 0x00cda310, 0x00cdc310, 0x00cde310, - 0x00ce0310, 0x00ce2310, 0x00ce4310, 0x00ce6310, - 0x00ce8310, 0x00cea310, 0x00cec310, 0x00cee310, - 0x00cf0310, 0x00cf2310, 0x00cf4310, 0x00cf6310, - 0x00cf8310, 0x00cfa310, 0x00cfc310, 0x00cfe310, - 0x00d00310, 0x00d02310, 0x00d04310, 0x00d06310, - 0x00d08310, 0x00d0a310, 0x00d0c310, 0x00d0e310, - 0x00d10310, 0x00d12310, 0x00d14310, 0x00d16310, - 0x00d18310, 0x00d1a310, 0x00d1c310, 0x00d1e310, - 0x00d20310, 0x00d22310, 0x00d24310, 0x00d26310, - 0x00d28310, 0x00d2a310, 0x00d2c310, 0x00d2e310, - 0x00d30310, 0x00d32310, 0x00d34310, 0x00d36310, - 0x00d38310, 0x00d3a310, 0x00d3c310, 0x00d3e310, - 0x00d40310, 0x00d42310, 0x00d44310, 0x00d46310, - 0x00d48310, 0x00d4a310, 0x00d4c310, 0x00d4e310, - 0x00d50310, 0x00d52310, 0x00d54310, 0x00d56310, - 0x00d58310, 0x00d5a310, 0x00d5c310, 0x00d5e310, - 0x00d60310, 0x00d62310, 0x00d64310, 0x00d66310, - 0x00d68310, 0x00d6a310, 0x00d6c310, 0x00d6e310, - 0x00d70310, 0x00d72310, 0x00d74310, 0x00d76310, - 0x00d78310, 0x00d7a310, 0x00d7c310, 0x00d7e310, - 0x00d80310, 0x00d82310, 0x00d84310, 0x00d86310, - 0x00d88310, 0x00d8a310, 0x00d8c310, 0x00d8e310, - 0x00d90310, 0x00d92310, 0x00d94310, 0x00d96310, - 0x00d98310, 0x00d9a310, 0x00d9c310, 0x00d9e310, - 0x00da0310, 0x00da2310, 0x00da4310, 0x00da6310, - 0x00da8310, 0x00daa310, 0x00dac310, 0x00dae310, - 0x00db0310, 0x00db2310, 0x00db4310, 0x00db6310, - 0x00db8310, 0x00dba310, 0x00dbc310, 0x00dbe310, - 0x00dc0310, 0x00dc2310, 0x00dc4310, 0x00dc6310, - 0x00dc8310, 0x00dca310, 0x00dcc310, 0x00dce310, - 0x00dd0310, 0x00dd2310, 0x00dd4310, 0x00dd6310, - 0x00dd8310, 0x00dda310, 0x00ddc310, 0x00dde310, - 0x00de0310, 0x00de2310, 0x00de4310, 0x00de6310, - 0x00de8310, 0x00dea310, 0x00dec310, 0x00dee310, - 0x00df0310, 0x00df2310, 0x00df4310, 0x00df6310, - 0x00df8310, 0x00dfa310, 0x00dfc310, 0x00dfe310, - 0x00e00310, 0x00e02310, 0x00e04310, 0x00e06310, - 0x00e08310, 0x00e0a310, 0x00e0c310, 0x00e0e310, - 0x00e10310, 0x00e12310, 0x00e14310, 0x00e16310, - 0x00e18310, 0x00e1a310, 0x00e1c310, 0x00e1e310, - 0x00e20310, 0x00e22310, 0x00e24310, 0x00e26310, - 0x00e28310, 0x00e2a310, 0x00e2c310, 0x00e2e310, - 0x00e30310, 0x00e32310, 0x00e34310, 0x00e36310, - 0x00e38310, 0x00e3a310, 0x00e3c310, 0x00e3e310, - 0x00e40310, 0x00e42310, 0x00e44310, 0x00e46310, - 0x00e48310, 0x00e4a310, 0x00e4c310, 0x00e4e310, - 0x00e50310, 0x00e52310, 0x00e54310, 0x00e56310, - 0x00e58310, 0x00e5a310, 0x00e5c310, 0x00e5e310, - 0x00e60310, 0x00e62310, 0x00e64310, 0x00e66310, - 0x00e68310, 0x00e6a310, 0x00e6c310, 0x00e6e310, - 0x00e70310, 0x00e72310, 0x00e74310, 0x00e76310, - 0x00e78310, 0x00e7a310, 0x00e7c310, 0x00e7e310, - 0x00e80310, 0x00e82310, 0x00e84310, 0x00e86310, - 0x00e88310, 0x00e8a310, 0x00e8c310, 0x00e8e310, - 0x00e90310, 0x00e92310, 0x00e94310, 0x00e96310, - 0x00e98310, 0x00e9a310, 0x00e9c310, 0x00e9e310, - 0x00ea0310, 0x00ea2310, 0x00ea4310, 0x00ea6310, - 0x00ea8310, 0x00eaa310, 0x00eac310, 0x00eae310, - 0x00eb0310, 0x00eb2310, 0x00eb4310, 0x00eb6310, - 0x00eb8310, 0x00eba310, 0x00ebc310, 0x00ebe310, - 0x00ec0310, 0x00ec2310, 0x00ec4310, 0x00ec6310, - 0x00ec8310, 0x00eca310, 0x00ecc310, 0x00ece310, - 0x00ed0310, 0x00ed2310, 0x00ed4310, 0x00ed6310, - 0x00ed8310, 0x00eda310, 0x00edc310, 0x00ede310, - 0x00ee0310, 0x00ee2310, 0x00ee4310, 0x00ee6310, - 0x00ee8310, 0x00eea310, 0x00eec310, 0x00eee310, - 0x00ef0310, 0x00ef2310, 0x00ef4310, 0x00ef6310, - 0x00ef8310, 0x00efa310, 0x00efc310, 0x00efe310, - 0x00f00310, 0x00f02310, 0x00f04310, 0x00f06310, - 0x00f08310, 0x00f0a310, 0x00f0c310, 0x00f0e310, - 0x00f10310, 0x00f12310, 0x00f14310, 0x00f16310, - 0x00f18310, 0x00f1a310, 0x00f1c310, 0x00f1e310, - 0x00f20310, 0x00f22310, 0x00f24310, 0x00f26310, - 0x00f28310, 0x00f2a310, 0x00f2c310, 0x00f2e310, - 0x00f30310, 0x00f32310, 0x00f34310, 0x00f36310, - 0x00f38310, 0x00f3a310, 0x00f3c310, 0x00f3e310, - 0x00f40310, 0x00f42310, 0x00f44310, 0x00f46310, - 0x00f48310, 0x00f4a310, 0x00f4c310, 0x00f4e310, - 0x00f50310, 0x00f52310, 0x00f54310, 0x00f56310, - 0x00f58310, 0x00f5a310, 0x00f5c310, 0x00f5e310, - 0x00f60310, 0x00f62310, 0x00f64310, 0x00f66310, - 0x00f68310, 0x00f6a310, 0x00f6c310, 0x00f6e310, - 0x00f70310, 0x00f72310, 0x00f74310, 0x00f76310, - 0x00f78310, 0x00f7a310, 0x00f7c310, 0x00f7e310, - 0x00f80310, 0x00f82310, 0x00f84310, 0x00f86310, - 0x00f88310, 0x00f8a310, 0x00f8c310, 0x00f8e310, - 0x00f90310, 0x00f92310, 0x00f94310, 0x00f96310, - 0x00f98310, 0x00f9a310, 0x00f9c310, 0x00f9e310, - 0x00fa0310, 0x00fa2310, 0x00fa4310, 0x00fa6310, - 0x00fa8310, 0x00faa310, 0x00fac310, 0x00fae310, - 0x00fb0310, 0x00fb2310, 0x00fb4310, 0x00fb6310, - 0x00fb8310, 0x00fba310, 0x00fbc310, 0x00fbe310, - 0x00fc0310, 0x00fc2310, 0x00fc4310, 0x00fc6310, - 0x00fc8310, 0x00fca310, 0x00fcc310, 0x00fce310, - 0x00fd0310, 0x00fd2310, 0x00fd4310, 0x00fd6310, - 0x00fd8310, 0x00fda310, 0x00fdc310, 0x00fde310, - 0x00fe0310, 0x00fe2310, 0x00fe4310, 0x00fe6310, - 0x00fe8310, 0x00fea310, 0x00fec310, 0x00fee310, - 0x00ff0310, 0x00ff2310, 0x00ff4310, 0x00ff6310, - 0x00ff8310, 0x00ffa310, 0x00ffc310, 0x00ffe310, - 0x00001310, 0x00003310, 0x00005310, 0x00007310, - 0x00009310, 0x0000b310, 0x0000d310, 0x0000f310, - 0x00011310, 0x00013310, 0x00015310, 0x00017310, - 0x00019310, 0x0001b310, 0x0001d310, 0x0001f310, - 0x00021310, 0x00023310, 0x00025310, 0x00027310, - 0x00029310, 0x0002b310, 0x0002d310, 0x0002f310, - 0x00031310, 0x00033310, 0x00035310, 0x00037310, - 0x00039310, 0x0003b310, 0x0003d310, 0x0003f310, - 0x00041310, 0x00043310, 0x00045310, 0x00047310, - 0x00049310, 0x0004b310, 0x0004d310, 0x0004f310, - 0x00051310, 0x00053310, 0x00055310, 0x00057310, - 0x00059310, 0x0005b310, 0x0005d310, 0x0005f310, - 0x00061310, 0x00063310, 0x00065310, 0x00067310, - 0x00069310, 0x0006b310, 0x0006d310, 0x0006f310, - 0x00071310, 0x00073310, 0x00075310, 0x00077310, - 0x00079310, 0x0007b310, 0x0007d310, 0x0007f310, - 0x00081310, 0x00083310, 0x00085310, 0x00087310, - 0x00089310, 0x0008b310, 0x0008d310, 0x0008f310, - 0x00091310, 0x00093310, 0x00095310, 0x00097310, - 0x00099310, 0x0009b310, 0x0009d310, 0x0009f310, - 0x000a1310, 0x000a3310, 0x000a5310, 0x000a7310, - 0x000a9310, 0x000ab310, 0x000ad310, 0x000af310, - 0x000b1310, 0x000b3310, 0x000b5310, 0x000b7310, - 0x000b9310, 0x000bb310, 0x000bd310, 0x000bf310, - 0x000c1310, 0x000c3310, 0x000c5310, 0x000c7310, - 0x000c9310, 0x000cb310, 0x000cd310, 0x000cf310, - 0x000d1310, 0x000d3310, 0x000d5310, 0x000d7310, - 0x000d9310, 0x000db310, 0x000dd310, 0x000df310, - 0x000e1310, 0x000e3310, 0x000e5310, 0x000e7310, - 0x000e9310, 0x000eb310, 0x000ed310, 0x000ef310, - 0x000f1310, 0x000f3310, 0x000f5310, 0x000f7310, - 0x000f9310, 0x000fb310, 0x000fd310, 0x000ff310, - 0x00101310, 0x00103310, 0x00105310, 0x00107310, - 0x00109310, 0x0010b310, 0x0010d310, 0x0010f310, - 0x00111310, 0x00113310, 0x00115310, 0x00117310, - 0x00119310, 0x0011b310, 0x0011d310, 0x0011f310, - 0x00121310, 0x00123310, 0x00125310, 0x00127310, - 0x00129310, 0x0012b310, 0x0012d310, 0x0012f310, - 0x00131310, 0x00133310, 0x00135310, 0x00137310, - 0x00139310, 0x0013b310, 0x0013d310, 0x0013f310, - 0x00141310, 0x00143310, 0x00145310, 0x00147310, - 0x00149310, 0x0014b310, 0x0014d310, 0x0014f310, - 0x00151310, 0x00153310, 0x00155310, 0x00157310, - 0x00159310, 0x0015b310, 0x0015d310, 0x0015f310, - 0x00161310, 0x00163310, 0x00165310, 0x00167310, - 0x00169310, 0x0016b310, 0x0016d310, 0x0016f310, - 0x00171310, 0x00173310, 0x00175310, 0x00177310, - 0x00179310, 0x0017b310, 0x0017d310, 0x0017f310, - 0x00181310, 0x00183310, 0x00185310, 0x00187310, - 0x00189310, 0x0018b310, 0x0018d310, 0x0018f310, - 0x00191310, 0x00193310, 0x00195310, 0x00197310, - 0x00199310, 0x0019b310, 0x0019d310, 0x0019f310, - 0x001a1310, 0x001a3310, 0x001a5310, 0x001a7310, - 0x001a9310, 0x001ab310, 0x001ad310, 0x001af310, - 0x001b1310, 0x001b3310, 0x001b5310, 0x001b7310, - 0x001b9310, 0x001bb310, 0x001bd310, 0x001bf310, - 0x001c1310, 0x001c3310, 0x001c5310, 0x001c7310, - 0x001c9310, 0x001cb310, 0x001cd310, 0x001cf310, - 0x001d1310, 0x001d3310, 0x001d5310, 0x001d7310, - 0x001d9310, 0x001db310, 0x001dd310, 0x001df310, - 0x001e1310, 0x001e3310, 0x001e5310, 0x001e7310, - 0x001e9310, 0x001eb310, 0x001ed310, 0x001ef310, - 0x001f1310, 0x001f3310, 0x001f5310, 0x001f7310, - 0x001f9310, 0x001fb310, 0x001fd310, 0x001ff310, - 0x00201310, 0x00203310, 0x00205310, 0x00207310, - 0x00209310, 0x0020b310, 0x0020d310, 0x0020f310, - 0x00211310, 0x00213310, 0x00215310, 0x00217310, - 0x00219310, 0x0021b310, 0x0021d310, 0x0021f310, - 0x00221310, 0x00223310, 0x00225310, 0x00227310, - 0x00229310, 0x0022b310, 0x0022d310, 0x0022f310, - 0x00231310, 0x00233310, 0x00235310, 0x00237310, - 0x00239310, 0x0023b310, 0x0023d310, 0x0023f310, - 0x00241310, 0x00243310, 0x00245310, 0x00247310, - 0x00249310, 0x0024b310, 0x0024d310, 0x0024f310, - 0x00251310, 0x00253310, 0x00255310, 0x00257310, - 0x00259310, 0x0025b310, 0x0025d310, 0x0025f310, - 0x00261310, 0x00263310, 0x00265310, 0x00267310, - 0x00269310, 0x0026b310, 0x0026d310, 0x0026f310, - 0x00271310, 0x00273310, 0x00275310, 0x00277310, - 0x00279310, 0x0027b310, 0x0027d310, 0x0027f310, - 0x00281310, 0x00283310, 0x00285310, 0x00287310, - 0x00289310, 0x0028b310, 0x0028d310, 0x0028f310, - 0x00291310, 0x00293310, 0x00295310, 0x00297310, - 0x00299310, 0x0029b310, 0x0029d310, 0x0029f310, - 0x002a1310, 0x002a3310, 0x002a5310, 0x002a7310, - 0x002a9310, 0x002ab310, 0x002ad310, 0x002af310, - 0x002b1310, 0x002b3310, 0x002b5310, 0x002b7310, - 0x002b9310, 0x002bb310, 0x002bd310, 0x002bf310, - 0x002c1310, 0x002c3310, 0x002c5310, 0x002c7310, - 0x002c9310, 0x002cb310, 0x002cd310, 0x002cf310, - 0x002d1310, 0x002d3310, 0x002d5310, 0x002d7310, - 0x002d9310, 0x002db310, 0x002dd310, 0x002df310, - 0x002e1310, 0x002e3310, 0x002e5310, 0x002e7310, - 0x002e9310, 0x002eb310, 0x002ed310, 0x002ef310, - 0x002f1310, 0x002f3310, 0x002f5310, 0x002f7310, - 0x002f9310, 0x002fb310, 0x002fd310, 0x002ff310, - 0x00301310, 0x00303310, 0x00305310, 0x00307310, - 0x00309310, 0x0030b310, 0x0030d310, 0x0030f310, - 0x00311310, 0x00313310, 0x00315310, 0x00317310, - 0x00319310, 0x0031b310, 0x0031d310, 0x0031f310, - 0x00321310, 0x00323310, 0x00325310, 0x00327310, - 0x00329310, 0x0032b310, 0x0032d310, 0x0032f310, - 0x00331310, 0x00333310, 0x00335310, 0x00337310, - 0x00339310, 0x0033b310, 0x0033d310, 0x0033f310, - 0x00341310, 0x00343310, 0x00345310, 0x00347310, - 0x00349310, 0x0034b310, 0x0034d310, 0x0034f310, - 0x00351310, 0x00353310, 0x00355310, 0x00357310, - 0x00359310, 0x0035b310, 0x0035d310, 0x0035f310, - 0x00361310, 0x00363310, 0x00365310, 0x00367310, - 0x00369310, 0x0036b310, 0x0036d310, 0x0036f310, - 0x00371310, 0x00373310, 0x00375310, 0x00377310, - 0x00379310, 0x0037b310, 0x0037d310, 0x0037f310, - 0x00381310, 0x00383310, 0x00385310, 0x00387310, - 0x00389310, 0x0038b310, 0x0038d310, 0x0038f310, - 0x00391310, 0x00393310, 0x00395310, 0x00397310, - 0x00399310, 0x0039b310, 0x0039d310, 0x0039f310, - 0x003a1310, 0x003a3310, 0x003a5310, 0x003a7310, - 0x003a9310, 0x003ab310, 0x003ad310, 0x003af310, - 0x003b1310, 0x003b3310, 0x003b5310, 0x003b7310, - 0x003b9310, 0x003bb310, 0x003bd310, 0x003bf310, - 0x003c1310, 0x003c3310, 0x003c5310, 0x003c7310, - 0x003c9310, 0x003cb310, 0x003cd310, 0x003cf310, - 0x003d1310, 0x003d3310, 0x003d5310, 0x003d7310, - 0x003d9310, 0x003db310, 0x003dd310, 0x003df310, - 0x003e1310, 0x003e3310, 0x003e5310, 0x003e7310, - 0x003e9310, 0x003eb310, 0x003ed310, 0x003ef310, - 0x003f1310, 0x003f3310, 0x003f5310, 0x003f7310, - 0x003f9310, 0x003fb310, 0x003fd310, 0x003ff310, - 0x00401310, 0x00403310, 0x00405310, 0x00407310, - 0x00409310, 0x0040b310, 0x0040d310, 0x0040f310, - 0x00411310, 0x00413310, 0x00415310, 0x00417310, - 0x00419310, 0x0041b310, 0x0041d310, 0x0041f310, - 0x00421310, 0x00423310, 0x00425310, 0x00427310, - 0x00429310, 0x0042b310, 0x0042d310, 0x0042f310, - 0x00431310, 0x00433310, 0x00435310, 0x00437310, - 0x00439310, 0x0043b310, 0x0043d310, 0x0043f310, - 0x00441310, 0x00443310, 0x00445310, 0x00447310, - 0x00449310, 0x0044b310, 0x0044d310, 0x0044f310, - 0x00451310, 0x00453310, 0x00455310, 0x00457310, - 0x00459310, 0x0045b310, 0x0045d310, 0x0045f310, - 0x00461310, 0x00463310, 0x00465310, 0x00467310, - 0x00469310, 0x0046b310, 0x0046d310, 0x0046f310, - 0x00471310, 0x00473310, 0x00475310, 0x00477310, - 0x00479310, 0x0047b310, 0x0047d310, 0x0047f310, - 0x00481310, 0x00483310, 0x00485310, 0x00487310, - 0x00489310, 0x0048b310, 0x0048d310, 0x0048f310, - 0x00491310, 0x00493310, 0x00495310, 0x00497310, - 0x00499310, 0x0049b310, 0x0049d310, 0x0049f310, - 0x004a1310, 0x004a3310, 0x004a5310, 0x004a7310, - 0x004a9310, 0x004ab310, 0x004ad310, 0x004af310, - 0x004b1310, 0x004b3310, 0x004b5310, 0x004b7310, - 0x004b9310, 0x004bb310, 0x004bd310, 0x004bf310, - 0x004c1310, 0x004c3310, 0x004c5310, 0x004c7310, - 0x004c9310, 0x004cb310, 0x004cd310, 0x004cf310, - 0x004d1310, 0x004d3310, 0x004d5310, 0x004d7310, - 0x004d9310, 0x004db310, 0x004dd310, 0x004df310, - 0x004e1310, 0x004e3310, 0x004e5310, 0x004e7310, - 0x004e9310, 0x004eb310, 0x004ed310, 0x004ef310, - 0x004f1310, 0x004f3310, 0x004f5310, 0x004f7310, - 0x004f9310, 0x004fb310, 0x004fd310, 0x004ff310, - 0x00501310, 0x00503310, 0x00505310, 0x00507310, - 0x00509310, 0x0050b310, 0x0050d310, 0x0050f310, - 0x00511310, 0x00513310, 0x00515310, 0x00517310, - 0x00519310, 0x0051b310, 0x0051d310, 0x0051f310, - 0x00521310, 0x00523310, 0x00525310, 0x00527310, - 0x00529310, 0x0052b310, 0x0052d310, 0x0052f310, - 0x00531310, 0x00533310, 0x00535310, 0x00537310, - 0x00539310, 0x0053b310, 0x0053d310, 0x0053f310, - 0x00541310, 0x00543310, 0x00545310, 0x00547310, - 0x00549310, 0x0054b310, 0x0054d310, 0x0054f310, - 0x00551310, 0x00553310, 0x00555310, 0x00557310, - 0x00559310, 0x0055b310, 0x0055d310, 0x0055f310, - 0x00561310, 0x00563310, 0x00565310, 0x00567310, - 0x00569310, 0x0056b310, 0x0056d310, 0x0056f310, - 0x00571310, 0x00573310, 0x00575310, 0x00577310, - 0x00579310, 0x0057b310, 0x0057d310, 0x0057f310, - 0x00581310, 0x00583310, 0x00585310, 0x00587310, - 0x00589310, 0x0058b310, 0x0058d310, 0x0058f310, - 0x00591310, 0x00593310, 0x00595310, 0x00597310, - 0x00599310, 0x0059b310, 0x0059d310, 0x0059f310, - 0x005a1310, 0x005a3310, 0x005a5310, 0x005a7310, - 0x005a9310, 0x005ab310, 0x005ad310, 0x005af310, - 0x005b1310, 0x005b3310, 0x005b5310, 0x005b7310, - 0x005b9310, 0x005bb310, 0x005bd310, 0x005bf310, - 0x005c1310, 0x005c3310, 0x005c5310, 0x005c7310, - 0x005c9310, 0x005cb310, 0x005cd310, 0x005cf310, - 0x005d1310, 0x005d3310, 0x005d5310, 0x005d7310, - 0x005d9310, 0x005db310, 0x005dd310, 0x005df310, - 0x005e1310, 0x005e3310, 0x005e5310, 0x005e7310, - 0x005e9310, 0x005eb310, 0x005ed310, 0x005ef310, - 0x005f1310, 0x005f3310, 0x005f5310, 0x005f7310, - 0x005f9310, 0x005fb310, 0x005fd310, 0x005ff310, - 0x00601310, 0x00603310, 0x00605310, 0x00607310, - 0x00609310, 0x0060b310, 0x0060d310, 0x0060f310, - 0x00611310, 0x00613310, 0x00615310, 0x00617310, - 0x00619310, 0x0061b310, 0x0061d310, 0x0061f310, - 0x00621310, 0x00623310, 0x00625310, 0x00627310, - 0x00629310, 0x0062b310, 0x0062d310, 0x0062f310, - 0x00631310, 0x00633310, 0x00635310, 0x00637310, - 0x00639310, 0x0063b310, 0x0063d310, 0x0063f310, - 0x00641310, 0x00643310, 0x00645310, 0x00647310, - 0x00649310, 0x0064b310, 0x0064d310, 0x0064f310, - 0x00651310, 0x00653310, 0x00655310, 0x00657310, - 0x00659310, 0x0065b310, 0x0065d310, 0x0065f310, - 0x00661310, 0x00663310, 0x00665310, 0x00667310, - 0x00669310, 0x0066b310, 0x0066d310, 0x0066f310, - 0x00671310, 0x00673310, 0x00675310, 0x00677310, - 0x00679310, 0x0067b310, 0x0067d310, 0x0067f310, - 0x00681310, 0x00683310, 0x00685310, 0x00687310, - 0x00689310, 0x0068b310, 0x0068d310, 0x0068f310, - 0x00691310, 0x00693310, 0x00695310, 0x00697310, - 0x00699310, 0x0069b310, 0x0069d310, 0x0069f310, - 0x006a1310, 0x006a3310, 0x006a5310, 0x006a7310, - 0x006a9310, 0x006ab310, 0x006ad310, 0x006af310, - 0x006b1310, 0x006b3310, 0x006b5310, 0x006b7310, - 0x006b9310, 0x006bb310, 0x006bd310, 0x006bf310, - 0x006c1310, 0x006c3310, 0x006c5310, 0x006c7310, - 0x006c9310, 0x006cb310, 0x006cd310, 0x006cf310, - 0x006d1310, 0x006d3310, 0x006d5310, 0x006d7310, - 0x006d9310, 0x006db310, 0x006dd310, 0x006df310, - 0x006e1310, 0x006e3310, 0x006e5310, 0x006e7310, - 0x006e9310, 0x006eb310, 0x006ed310, 0x006ef310, - 0x006f1310, 0x006f3310, 0x006f5310, 0x006f7310, - 0x006f9310, 0x006fb310, 0x006fd310, 0x006ff310, - 0x00701310, 0x00703310, 0x00705310, 0x00707310, - 0x00709310, 0x0070b310, 0x0070d310, 0x0070f310, - 0x00711310, 0x00713310, 0x00715310, 0x00717310, - 0x00719310, 0x0071b310, 0x0071d310, 0x0071f310, - 0x00721310, 0x00723310, 0x00725310, 0x00727310, - 0x00729310, 0x0072b310, 0x0072d310, 0x0072f310, - 0x00731310, 0x00733310, 0x00735310, 0x00737310, - 0x00739310, 0x0073b310, 0x0073d310, 0x0073f310, - 0x00741310, 0x00743310, 0x00745310, 0x00747310, - 0x00749310, 0x0074b310, 0x0074d310, 0x0074f310, - 0x00751310, 0x00753310, 0x00755310, 0x00757310, - 0x00759310, 0x0075b310, 0x0075d310, 0x0075f310, - 0x00761310, 0x00763310, 0x00765310, 0x00767310, - 0x00769310, 0x0076b310, 0x0076d310, 0x0076f310, - 0x00771310, 0x00773310, 0x00775310, 0x00777310, - 0x00779310, 0x0077b310, 0x0077d310, 0x0077f310, - 0x00781310, 0x00783310, 0x00785310, 0x00787310, - 0x00789310, 0x0078b310, 0x0078d310, 0x0078f310, - 0x00791310, 0x00793310, 0x00795310, 0x00797310, - 0x00799310, 0x0079b310, 0x0079d310, 0x0079f310, - 0x007a1310, 0x007a3310, 0x007a5310, 0x007a7310, - 0x007a9310, 0x007ab310, 0x007ad310, 0x007af310, - 0x007b1310, 0x007b3310, 0x007b5310, 0x007b7310, - 0x007b9310, 0x007bb310, 0x007bd310, 0x007bf310, - 0x007c1310, 0x007c3310, 0x007c5310, 0x007c7310, - 0x007c9310, 0x007cb310, 0x007cd310, 0x007cf310, - 0x007d1310, 0x007d3310, 0x007d5310, 0x007d7310, - 0x007d9310, 0x007db310, 0x007dd310, 0x007df310, - 0x007e1310, 0x007e3310, 0x007e5310, 0x007e7310, - 0x007e9310, 0x007eb310, 0x007ed310, 0x007ef310, - 0x007f1310, 0x007f3310, 0x007f5310, 0x007f7310, - 0x007f9310, 0x007fb310, 0x007fd310, 0x007ff310, - 0x00801310, 0x00803310, 0x00805310, 0x00807310, - 0x00809310, 0x0080b310, 0x0080d310, 0x0080f310, - 0x00811310, 0x00813310, 0x00815310, 0x00817310, - 0x00819310, 0x0081b310, 0x0081d310, 0x0081f310, - 0x00821310, 0x00823310, 0x00825310, 0x00827310, - 0x00829310, 0x0082b310, 0x0082d310, 0x0082f310, - 0x00831310, 0x00833310, 0x00835310, 0x00837310, - 0x00839310, 0x0083b310, 0x0083d310, 0x0083f310, - 0x00841310, 0x00843310, 0x00845310, 0x00847310, - 0x00849310, 0x0084b310, 0x0084d310, 0x0084f310, - 0x00851310, 0x00853310, 0x00855310, 0x00857310, - 0x00859310, 0x0085b310, 0x0085d310, 0x0085f310, - 0x00861310, 0x00863310, 0x00865310, 0x00867310, - 0x00869310, 0x0086b310, 0x0086d310, 0x0086f310, - 0x00871310, 0x00873310, 0x00875310, 0x00877310, - 0x00879310, 0x0087b310, 0x0087d310, 0x0087f310, - 0x00881310, 0x00883310, 0x00885310, 0x00887310, - 0x00889310, 0x0088b310, 0x0088d310, 0x0088f310, - 0x00891310, 0x00893310, 0x00895310, 0x00897310, - 0x00899310, 0x0089b310, 0x0089d310, 0x0089f310, - 0x008a1310, 0x008a3310, 0x008a5310, 0x008a7310, - 0x008a9310, 0x008ab310, 0x008ad310, 0x008af310, - 0x008b1310, 0x008b3310, 0x008b5310, 0x008b7310, - 0x008b9310, 0x008bb310, 0x008bd310, 0x008bf310, - 0x008c1310, 0x008c3310, 0x008c5310, 0x008c7310, - 0x008c9310, 0x008cb310, 0x008cd310, 0x008cf310, - 0x008d1310, 0x008d3310, 0x008d5310, 0x008d7310, - 0x008d9310, 0x008db310, 0x008dd310, 0x008df310, - 0x008e1310, 0x008e3310, 0x008e5310, 0x008e7310, - 0x008e9310, 0x008eb310, 0x008ed310, 0x008ef310, - 0x008f1310, 0x008f3310, 0x008f5310, 0x008f7310, - 0x008f9310, 0x008fb310, 0x008fd310, 0x008ff310, - 0x00901310, 0x00903310, 0x00905310, 0x00907310, - 0x00909310, 0x0090b310, 0x0090d310, 0x0090f310, - 0x00911310, 0x00913310, 0x00915310, 0x00917310, - 0x00919310, 0x0091b310, 0x0091d310, 0x0091f310, - 0x00921310, 0x00923310, 0x00925310, 0x00927310, - 0x00929310, 0x0092b310, 0x0092d310, 0x0092f310, - 0x00931310, 0x00933310, 0x00935310, 0x00937310, - 0x00939310, 0x0093b310, 0x0093d310, 0x0093f310, - 0x00941310, 0x00943310, 0x00945310, 0x00947310, - 0x00949310, 0x0094b310, 0x0094d310, 0x0094f310, - 0x00951310, 0x00953310, 0x00955310, 0x00957310, - 0x00959310, 0x0095b310, 0x0095d310, 0x0095f310, - 0x00961310, 0x00963310, 0x00965310, 0x00967310, - 0x00969310, 0x0096b310, 0x0096d310, 0x0096f310, - 0x00971310, 0x00973310, 0x00975310, 0x00977310, - 0x00979310, 0x0097b310, 0x0097d310, 0x0097f310, - 0x00981310, 0x00983310, 0x00985310, 0x00987310, - 0x00989310, 0x0098b310, 0x0098d310, 0x0098f310, - 0x00991310, 0x00993310, 0x00995310, 0x00997310, - 0x00999310, 0x0099b310, 0x0099d310, 0x0099f310, - 0x009a1310, 0x009a3310, 0x009a5310, 0x009a7310, - 0x009a9310, 0x009ab310, 0x009ad310, 0x009af310, - 0x009b1310, 0x009b3310, 0x009b5310, 0x009b7310, - 0x009b9310, 0x009bb310, 0x009bd310, 0x009bf310, - 0x009c1310, 0x009c3310, 0x009c5310, 0x009c7310, - 0x009c9310, 0x009cb310, 0x009cd310, 0x009cf310, - 0x009d1310, 0x009d3310, 0x009d5310, 0x009d7310, - 0x009d9310, 0x009db310, 0x009dd310, 0x009df310, - 0x009e1310, 0x009e3310, 0x009e5310, 0x009e7310, - 0x009e9310, 0x009eb310, 0x009ed310, 0x009ef310, - 0x009f1310, 0x009f3310, 0x009f5310, 0x009f7310, - 0x009f9310, 0x009fb310, 0x009fd310, 0x009ff310, - 0x00a01310, 0x00a03310, 0x00a05310, 0x00a07310, - 0x00a09310, 0x00a0b310, 0x00a0d310, 0x00a0f310, - 0x00a11310, 0x00a13310, 0x00a15310, 0x00a17310, - 0x00a19310, 0x00a1b310, 0x00a1d310, 0x00a1f310, - 0x00a21310, 0x00a23310, 0x00a25310, 0x00a27310, - 0x00a29310, 0x00a2b310, 0x00a2d310, 0x00a2f310, - 0x00a31310, 0x00a33310, 0x00a35310, 0x00a37310, - 0x00a39310, 0x00a3b310, 0x00a3d310, 0x00a3f310, - 0x00a41310, 0x00a43310, 0x00a45310, 0x00a47310, - 0x00a49310, 0x00a4b310, 0x00a4d310, 0x00a4f310, - 0x00a51310, 0x00a53310, 0x00a55310, 0x00a57310, - 0x00a59310, 0x00a5b310, 0x00a5d310, 0x00a5f310, - 0x00a61310, 0x00a63310, 0x00a65310, 0x00a67310, - 0x00a69310, 0x00a6b310, 0x00a6d310, 0x00a6f310, - 0x00a71310, 0x00a73310, 0x00a75310, 0x00a77310, - 0x00a79310, 0x00a7b310, 0x00a7d310, 0x00a7f310, - 0x00a81310, 0x00a83310, 0x00a85310, 0x00a87310, - 0x00a89310, 0x00a8b310, 0x00a8d310, 0x00a8f310, - 0x00a91310, 0x00a93310, 0x00a95310, 0x00a97310, - 0x00a99310, 0x00a9b310, 0x00a9d310, 0x00a9f310, - 0x00aa1310, 0x00aa3310, 0x00aa5310, 0x00aa7310, - 0x00aa9310, 0x00aab310, 0x00aad310, 0x00aaf310, - 0x00ab1310, 0x00ab3310, 0x00ab5310, 0x00ab7310, - 0x00ab9310, 0x00abb310, 0x00abd310, 0x00abf310, - 0x00ac1310, 0x00ac3310, 0x00ac5310, 0x00ac7310, - 0x00ac9310, 0x00acb310, 0x00acd310, 0x00acf310, - 0x00ad1310, 0x00ad3310, 0x00ad5310, 0x00ad7310, - 0x00ad9310, 0x00adb310, 0x00add310, 0x00adf310, - 0x00ae1310, 0x00ae3310, 0x00ae5310, 0x00ae7310, - 0x00ae9310, 0x00aeb310, 0x00aed310, 0x00aef310, - 0x00af1310, 0x00af3310, 0x00af5310, 0x00af7310, - 0x00af9310, 0x00afb310, 0x00afd310, 0x00aff310, - 0x00b01310, 0x00b03310, 0x00b05310, 0x00b07310, - 0x00b09310, 0x00b0b310, 0x00b0d310, 0x00b0f310, - 0x00b11310, 0x00b13310, 0x00b15310, 0x00b17310, - 0x00b19310, 0x00b1b310, 0x00b1d310, 0x00b1f310, - 0x00b21310, 0x00b23310, 0x00b25310, 0x00b27310, - 0x00b29310, 0x00b2b310, 0x00b2d310, 0x00b2f310, - 0x00b31310, 0x00b33310, 0x00b35310, 0x00b37310, - 0x00b39310, 0x00b3b310, 0x00b3d310, 0x00b3f310, - 0x00b41310, 0x00b43310, 0x00b45310, 0x00b47310, - 0x00b49310, 0x00b4b310, 0x00b4d310, 0x00b4f310, - 0x00b51310, 0x00b53310, 0x00b55310, 0x00b57310, - 0x00b59310, 0x00b5b310, 0x00b5d310, 0x00b5f310, - 0x00b61310, 0x00b63310, 0x00b65310, 0x00b67310, - 0x00b69310, 0x00b6b310, 0x00b6d310, 0x00b6f310, - 0x00b71310, 0x00b73310, 0x00b75310, 0x00b77310, - 0x00b79310, 0x00b7b310, 0x00b7d310, 0x00b7f310, - 0x00b81310, 0x00b83310, 0x00b85310, 0x00b87310, - 0x00b89310, 0x00b8b310, 0x00b8d310, 0x00b8f310, - 0x00b91310, 0x00b93310, 0x00b95310, 0x00b97310, - 0x00b99310, 0x00b9b310, 0x00b9d310, 0x00b9f310, - 0x00ba1310, 0x00ba3310, 0x00ba5310, 0x00ba7310, - 0x00ba9310, 0x00bab310, 0x00bad310, 0x00baf310, - 0x00bb1310, 0x00bb3310, 0x00bb5310, 0x00bb7310, - 0x00bb9310, 0x00bbb310, 0x00bbd310, 0x00bbf310, - 0x00bc1310, 0x00bc3310, 0x00bc5310, 0x00bc7310, - 0x00bc9310, 0x00bcb310, 0x00bcd310, 0x00bcf310, - 0x00bd1310, 0x00bd3310, 0x00bd5310, 0x00bd7310, - 0x00bd9310, 0x00bdb310, 0x00bdd310, 0x00bdf310, - 0x00be1310, 0x00be3310, 0x00be5310, 0x00be7310, - 0x00be9310, 0x00beb310, 0x00bed310, 0x00bef310, - 0x00bf1310, 0x00bf3310, 0x00bf5310, 0x00bf7310, - 0x00bf9310, 0x00bfb310, 0x00bfd310, 0x00bff310, - 0x00c01310, 0x00c03310, 0x00c05310, 0x00c07310, - 0x00c09310, 0x00c0b310, 0x00c0d310, 0x00c0f310, - 0x00c11310, 0x00c13310, 0x00c15310, 0x00c17310, - 0x00c19310, 0x00c1b310, 0x00c1d310, 0x00c1f310, - 0x00c21310, 0x00c23310, 0x00c25310, 0x00c27310, - 0x00c29310, 0x00c2b310, 0x00c2d310, 0x00c2f310, - 0x00c31310, 0x00c33310, 0x00c35310, 0x00c37310, - 0x00c39310, 0x00c3b310, 0x00c3d310, 0x00c3f310, - 0x00c41310, 0x00c43310, 0x00c45310, 0x00c47310, - 0x00c49310, 0x00c4b310, 0x00c4d310, 0x00c4f310, - 0x00c51310, 0x00c53310, 0x00c55310, 0x00c57310, - 0x00c59310, 0x00c5b310, 0x00c5d310, 0x00c5f310, - 0x00c61310, 0x00c63310, 0x00c65310, 0x00c67310, - 0x00c69310, 0x00c6b310, 0x00c6d310, 0x00c6f310, - 0x00c71310, 0x00c73310, 0x00c75310, 0x00c77310, - 0x00c79310, 0x00c7b310, 0x00c7d310, 0x00c7f310, - 0x00c81310, 0x00c83310, 0x00c85310, 0x00c87310, - 0x00c89310, 0x00c8b310, 0x00c8d310, 0x00c8f310, - 0x00c91310, 0x00c93310, 0x00c95310, 0x00c97310, - 0x00c99310, 0x00c9b310, 0x00c9d310, 0x00c9f310, - 0x00ca1310, 0x00ca3310, 0x00ca5310, 0x00ca7310, - 0x00ca9310, 0x00cab310, 0x00cad310, 0x00caf310, - 0x00cb1310, 0x00cb3310, 0x00cb5310, 0x00cb7310, - 0x00cb9310, 0x00cbb310, 0x00cbd310, 0x00cbf310, - 0x00cc1310, 0x00cc3310, 0x00cc5310, 0x00cc7310, - 0x00cc9310, 0x00ccb310, 0x00ccd310, 0x00ccf310, - 0x00cd1310, 0x00cd3310, 0x00cd5310, 0x00cd7310, - 0x00cd9310, 0x00cdb310, 0x00cdd310, 0x00cdf310, - 0x00ce1310, 0x00ce3310, 0x00ce5310, 0x00ce7310, - 0x00ce9310, 0x00ceb310, 0x00ced310, 0x00cef310, - 0x00cf1310, 0x00cf3310, 0x00cf5310, 0x00cf7310, - 0x00cf9310, 0x00cfb310, 0x00cfd310, 0x00cff310, - 0x00d01310, 0x00d03310, 0x00d05310, 0x00d07310, - 0x00d09310, 0x00d0b310, 0x00d0d310, 0x00d0f310, - 0x00d11310, 0x00d13310, 0x00d15310, 0x00d17310, - 0x00d19310, 0x00d1b310, 0x00d1d310, 0x00d1f310, - 0x00d21310, 0x00d23310, 0x00d25310, 0x00d27310, - 0x00d29310, 0x00d2b310, 0x00d2d310, 0x00d2f310, - 0x00d31310, 0x00d33310, 0x00d35310, 0x00d37310, - 0x00d39310, 0x00d3b310, 0x00d3d310, 0x00d3f310, - 0x00d41310, 0x00d43310, 0x00d45310, 0x00d47310, - 0x00d49310, 0x00d4b310, 0x00d4d310, 0x00d4f310, - 0x00d51310, 0x00d53310, 0x00d55310, 0x00d57310, - 0x00d59310, 0x00d5b310, 0x00d5d310, 0x00d5f310, - 0x00d61310, 0x00d63310, 0x00d65310, 0x00d67310, - 0x00d69310, 0x00d6b310, 0x00d6d310, 0x00d6f310, - 0x00d71310, 0x00d73310, 0x00d75310, 0x00d77310, - 0x00d79310, 0x00d7b310, 0x00d7d310, 0x00d7f310, - 0x00d81310, 0x00d83310, 0x00d85310, 0x00d87310, - 0x00d89310, 0x00d8b310, 0x00d8d310, 0x00d8f310, - 0x00d91310, 0x00d93310, 0x00d95310, 0x00d97310, - 0x00d99310, 0x00d9b310, 0x00d9d310, 0x00d9f310, - 0x00da1310, 0x00da3310, 0x00da5310, 0x00da7310, - 0x00da9310, 0x00dab310, 0x00dad310, 0x00daf310, - 0x00db1310, 0x00db3310, 0x00db5310, 0x00db7310, - 0x00db9310, 0x00dbb310, 0x00dbd310, 0x00dbf310, - 0x00dc1310, 0x00dc3310, 0x00dc5310, 0x00dc7310, - 0x00dc9310, 0x00dcb310, 0x00dcd310, 0x00dcf310, - 0x00dd1310, 0x00dd3310, 0x00dd5310, 0x00dd7310, - 0x00dd9310, 0x00ddb310, 0x00ddd310, 0x00ddf310, - 0x00de1310, 0x00de3310, 0x00de5310, 0x00de7310, - 0x00de9310, 0x00deb310, 0x00ded310, 0x00def310, - 0x00df1310, 0x00df3310, 0x00df5310, 0x00df7310, - 0x00df9310, 0x00dfb310, 0x00dfd310, 0x00dff310, - 0x00e01310, 0x00e03310, 0x00e05310, 0x00e07310, - 0x00e09310, 0x00e0b310, 0x00e0d310, 0x00e0f310, - 0x00e11310, 0x00e13310, 0x00e15310, 0x00e17310, - 0x00e19310, 0x00e1b310, 0x00e1d310, 0x00e1f310, - 0x00e21310, 0x00e23310, 0x00e25310, 0x00e27310, - 0x00e29310, 0x00e2b310, 0x00e2d310, 0x00e2f310, - 0x00e31310, 0x00e33310, 0x00e35310, 0x00e37310, - 0x00e39310, 0x00e3b310, 0x00e3d310, 0x00e3f310, - 0x00e41310, 0x00e43310, 0x00e45310, 0x00e47310, - 0x00e49310, 0x00e4b310, 0x00e4d310, 0x00e4f310, - 0x00e51310, 0x00e53310, 0x00e55310, 0x00e57310, - 0x00e59310, 0x00e5b310, 0x00e5d310, 0x00e5f310, - 0x00e61310, 0x00e63310, 0x00e65310, 0x00e67310, - 0x00e69310, 0x00e6b310, 0x00e6d310, 0x00e6f310, - 0x00e71310, 0x00e73310, 0x00e75310, 0x00e77310, - 0x00e79310, 0x00e7b310, 0x00e7d310, 0x00e7f310, - 0x00e81310, 0x00e83310, 0x00e85310, 0x00e87310, - 0x00e89310, 0x00e8b310, 0x00e8d310, 0x00e8f310, - 0x00e91310, 0x00e93310, 0x00e95310, 0x00e97310, - 0x00e99310, 0x00e9b310, 0x00e9d310, 0x00e9f310, - 0x00ea1310, 0x00ea3310, 0x00ea5310, 0x00ea7310, - 0x00ea9310, 0x00eab310, 0x00ead310, 0x00eaf310, - 0x00eb1310, 0x00eb3310, 0x00eb5310, 0x00eb7310, - 0x00eb9310, 0x00ebb310, 0x00ebd310, 0x00ebf310, - 0x00ec1310, 0x00ec3310, 0x00ec5310, 0x00ec7310, - 0x00ec9310, 0x00ecb310, 0x00ecd310, 0x00ecf310, - 0x00ed1310, 0x00ed3310, 0x00ed5310, 0x00ed7310, - 0x00ed9310, 0x00edb310, 0x00edd310, 0x00edf310, - 0x00ee1310, 0x00ee3310, 0x00ee5310, 0x00ee7310, - 0x00ee9310, 0x00eeb310, 0x00eed310, 0x00eef310, - 0x00ef1310, 0x00ef3310, 0x00ef5310, 0x00ef7310, - 0x00ef9310, 0x00efb310, 0x00efd310, 0x00eff310, - 0x00f01310, 0x00f03310, 0x00f05310, 0x00f07310, - 0x00f09310, 0x00f0b310, 0x00f0d310, 0x00f0f310, - 0x00f11310, 0x00f13310, 0x00f15310, 0x00f17310, - 0x00f19310, 0x00f1b310, 0x00f1d310, 0x00f1f310, - 0x00f21310, 0x00f23310, 0x00f25310, 0x00f27310, - 0x00f29310, 0x00f2b310, 0x00f2d310, 0x00f2f310, - 0x00f31310, 0x00f33310, 0x00f35310, 0x00f37310, - 0x00f39310, 0x00f3b310, 0x00f3d310, 0x00f3f310, - 0x00f41310, 0x00f43310, 0x00f45310, 0x00f47310, - 0x00f49310, 0x00f4b310, 0x00f4d310, 0x00f4f310, - 0x00f51310, 0x00f53310, 0x00f55310, 0x00f57310, - 0x00f59310, 0x00f5b310, 0x00f5d310, 0x00f5f310, - 0x00f61310, 0x00f63310, 0x00f65310, 0x00f67310, - 0x00f69310, 0x00f6b310, 0x00f6d310, 0x00f6f310, - 0x00f71310, 0x00f73310, 0x00f75310, 0x00f77310, - 0x00f79310, 0x00f7b310, 0x00f7d310, 0x00f7f310, - 0x00f81310, 0x00f83310, 0x00f85310, 0x00f87310, - 0x00f89310, 0x00f8b310, 0x00f8d310, 0x00f8f310, - 0x00f91310, 0x00f93310, 0x00f95310, 0x00f97310, - 0x00f99310, 0x00f9b310, 0x00f9d310, 0x00f9f310, - 0x00fa1310, 0x00fa3310, 0x00fa5310, 0x00fa7310, - 0x00fa9310, 0x00fab310, 0x00fad310, 0x00faf310, - 0x00fb1310, 0x00fb3310, 0x00fb5310, 0x00fb7310, - 0x00fb9310, 0x00fbb310, 0x00fbd310, 0x00fbf310, - 0x00fc1310, 0x00fc3310, 0x00fc5310, 0x00fc7310, - 0x00fc9310, 0x00fcb310, 0x00fcd310, 0x00fcf310, - 0x00fd1310, 0x00fd3310, 0x00fd5310, 0x00fd7310, - 0x00fd9310, 0x00fdb310, 0x00fdd310, 0x00fdf310, - 0x00fe1310, 0x00fe3310, 0x00fe5310, 0x00fe7310, - 0x00fe9310, 0x00feb310, 0x00fed310, 0x00fef310, - 0x00ff1310, 0x00ff3310, 0x00ff5310, 0x00ff7310, - 0x00ff9310, 0x00ffb310, 0x00ffd310, 0x00fff310, -}; diff --git a/contrib/libzlib-ng/arch/x86/fill_window_sse.c b/contrib/libzlib-ng/arch/x86/fill_window_sse.c deleted file mode 100644 index d82b1d1bcb8..00000000000 --- a/contrib/libzlib-ng/arch/x86/fill_window_sse.c +++ /dev/null @@ -1,165 +0,0 @@ -/* - * Fill Window with SSE2-optimized hash shifting - * - * Copyright (C) 2013 Intel Corporation - * Authors: - * Arjan van de Ven - * Jim Kukunas - * - * For conditions of distribution and use, see copyright notice in zlib.h - */ -#ifdef X86_SSE2_FILL_WINDOW - -#include -#include "deflate.h" - -extern int read_buf(z_stream *strm, unsigned char *buf, unsigned size); - -ZLIB_INTERNAL void fill_window_sse(deflate_state *s) { - const __m128i xmm_wsize = _mm_set1_epi16(s->w_size); - - register unsigned n; - register Pos *p; - unsigned more; /* Amount of free space at the end of the window. */ - unsigned int wsize = s->w_size; - - Assert(s->lookahead < MIN_LOOKAHEAD, "already enough lookahead"); - - do { - more = (unsigned)(s->window_size -(unsigned long)s->lookahead -(unsigned long)s->strstart); - - /* Deal with !@#$% 64K limit: */ - if (sizeof(int) <= 2) { - if (more == 0 && s->strstart == 0 && s->lookahead == 0) { - more = wsize; - - } else if (more == (unsigned)(-1)) { - /* Very unlikely, but possible on 16 bit machine if - * strstart == 0 && lookahead == 1 (input done a byte at time) - */ - more--; - } - } - - /* If the window is almost full and there is insufficient lookahead, - * move the upper half to the lower one to make room in the upper half. - */ - if (s->strstart >= wsize+MAX_DIST(s)) { - memcpy(s->window, s->window+wsize, (unsigned)wsize); - s->match_start -= wsize; - s->strstart -= wsize; /* we now have strstart >= MAX_DIST */ - s->block_start -= (long) wsize; - - /* Slide the hash table (could be avoided with 32 bit values - at the expense of memory usage). We slide even when level == 0 - to keep the hash table consistent if we switch back to level > 0 - later. (Using level 0 permanently is not an optimal usage of - zlib, so we don't care about this pathological case.) - */ - n = s->hash_size; - p = &s->head[n]; - p -= 8; - do { - __m128i value, result; - - value = _mm_loadu_si128((__m128i *)p); - result = _mm_subs_epu16(value, xmm_wsize); - _mm_storeu_si128((__m128i *)p, result); - - p -= 8; - n -= 8; - } while (n > 0); - - n = wsize; - p = &s->prev[n]; - p -= 8; - do { - __m128i value, result; - - value = _mm_loadu_si128((__m128i *)p); - result = _mm_subs_epu16(value, xmm_wsize); - _mm_storeu_si128((__m128i *)p, result); - - p -= 8; - n -= 8; - } while (n > 0); - more += wsize; - } - if (s->strm->avail_in == 0) break; - - /* If there was no sliding: - * strstart <= WSIZE+MAX_DIST-1 && lookahead <= MIN_LOOKAHEAD - 1 && - * more == window_size - lookahead - strstart - * => more >= window_size - (MIN_LOOKAHEAD-1 + WSIZE + MAX_DIST-1) - * => more >= window_size - 2*WSIZE + 2 - * In the BIG_MEM or MMAP case (not yet supported), - * window_size == input_size + MIN_LOOKAHEAD && - * strstart + s->lookahead <= input_size => more >= MIN_LOOKAHEAD. - * Otherwise, window_size == 2*WSIZE so more >= 2. - * If there was sliding, more >= WSIZE. So in all cases, more >= 2. - */ - Assert(more >= 2, "more < 2"); - - n = read_buf(s->strm, s->window + s->strstart + s->lookahead, more); - s->lookahead += n; - - /* Initialize the hash value now that we have some input: */ - if (s->lookahead + s->insert >= MIN_MATCH) { - unsigned int str = s->strstart - s->insert; - s->ins_h = s->window[str]; - if (str >= 1) - UPDATE_HASH(s, s->ins_h, str + 1 - (MIN_MATCH-1)); -#if MIN_MATCH != 3 - Call UPDATE_HASH() MIN_MATCH-3 more times -#endif - while (s->insert) { - UPDATE_HASH(s, s->ins_h, str); - s->prev[str & s->w_mask] = s->head[s->ins_h]; - s->head[s->ins_h] = (Pos)str; - str++; - s->insert--; - if (s->lookahead + s->insert < MIN_MATCH) - break; - } - } - /* If the whole input has less than MIN_MATCH bytes, ins_h is garbage, - * but this is not important since only literal bytes will be emitted. - */ - } while (s->lookahead < MIN_LOOKAHEAD && s->strm->avail_in != 0); - - /* If the WIN_INIT bytes after the end of the current data have never been - * written, then zero those bytes in order to avoid memory check reports of - * the use of uninitialized (or uninitialised as Julian writes) bytes by - * the longest match routines. Update the high water mark for the next - * time through here. WIN_INIT is set to MAX_MATCH since the longest match - * routines allow scanning to strstart + MAX_MATCH, ignoring lookahead. - */ - if (s->high_water < s->window_size) { - unsigned long curr = s->strstart + (unsigned long)(s->lookahead); - unsigned long init; - - if (s->high_water < curr) { - /* Previous high water mark below current data -- zero WIN_INIT - * bytes or up to end of window, whichever is less. - */ - init = s->window_size - curr; - if (init > WIN_INIT) - init = WIN_INIT; - memset(s->window + curr, 0, (unsigned)init); - s->high_water = curr + init; - } else if (s->high_water < (unsigned long)curr + WIN_INIT) { - /* High water mark at or above current data, but below current data - * plus WIN_INIT -- zero out to current data plus WIN_INIT, or up - * to end of window, whichever is less. - */ - init = (unsigned long)curr + WIN_INIT - s->high_water; - if (init > s->window_size - s->high_water) - init = s->window_size - s->high_water; - memset(s->window + s->high_water, 0, (unsigned)init); - s->high_water += init; - } - } - - Assert((unsigned long)s->strstart <= s->window_size - MIN_LOOKAHEAD, "not enough room for search"); -} -#endif diff --git a/contrib/libzlib-ng/arch/x86/insert_string_sse.c b/contrib/libzlib-ng/arch/x86/insert_string_sse.c deleted file mode 100644 index 6d4ddaea519..00000000000 --- a/contrib/libzlib-ng/arch/x86/insert_string_sse.c +++ /dev/null @@ -1,50 +0,0 @@ -/* insert_string_sse -- insert_string variant using SSE4.2's CRC instructions - * - * Copyright (C) 1995-2013 Jean-loup Gailly and Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - * - */ - -#include "deflate.h" - -/* =========================================================================== - * Insert string str in the dictionary and set match_head to the previous head - * of the hash chain (the most recent string with same hash key). Return - * the previous length of the hash chain. - * IN assertion: all calls to to INSERT_STRING are made with consecutive - * input characters and the first MIN_MATCH bytes of str are valid - * (except for the last MIN_MATCH-1 bytes of the input file). - */ -#ifdef X86_SSE4_2_CRC_HASH -Pos insert_string_sse(deflate_state *const s, const Pos str, unsigned int count) { - Pos ret = 0; - unsigned int idx; - unsigned *ip, val, h = 0; - - for (idx = 0; idx < count; idx++) { - ip = (unsigned *)&s->window[str+idx]; - val = *ip; - h = 0; - - if (s->level >= 6) - val &= 0xFFFFFF; - -#ifdef _MSC_VER - h = _mm_crc32_u32(h, val); -#else - __asm__ __volatile__ ( - "crc32 %1,%0\n\t" - : "+r" (h) - : "r" (val) - ); -#endif - - if (s->head[h & s->hash_mask] != str+idx) { - s->prev[(str+idx) & s->w_mask] = s->head[h & s->hash_mask]; - s->head[h & s->hash_mask] = str+idx; - } - } - ret = s->prev[(str+count-1) & s->w_mask]; - return ret; -} -#endif diff --git a/contrib/libzlib-ng/arch/x86/x86.c b/contrib/libzlib-ng/arch/x86/x86.c deleted file mode 100644 index c932627f163..00000000000 --- a/contrib/libzlib-ng/arch/x86/x86.c +++ /dev/null @@ -1,53 +0,0 @@ -/* - * x86 feature check - * - * Copyright (C) 2013 Intel Corporation. All rights reserved. - * Author: - * Jim Kukunas - * - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -#include "x86.h" - -ZLIB_INTERNAL int x86_cpu_has_sse2; -ZLIB_INTERNAL int x86_cpu_has_sse42; -ZLIB_INTERNAL int x86_cpu_has_pclmulqdq; - -#ifdef _MSC_VER -#include -#else -// Newer versions of GCC and clang come with cpuid.h -#include -#endif - -static void cpuid(int info, unsigned* eax, unsigned* ebx, unsigned* ecx, unsigned* edx) { -#ifdef _MSC_VER - unsigned int registers[4]; - __cpuid(registers, info); - - *eax = registers[0]; - *ebx = registers[1]; - *ecx = registers[2]; - *edx = registers[3]; -#else - unsigned int _eax; - unsigned int _ebx; - unsigned int _ecx; - unsigned int _edx; - __cpuid(info, _eax, _ebx, _ecx, _edx); - *eax = _eax; - *ebx = _ebx; - *ecx = _ecx; - *edx = _edx; -#endif -} - -void ZLIB_INTERNAL x86_check_features(void) { - unsigned eax, ebx, ecx, edx; - cpuid(1 /*CPU_PROCINFO_AND_FEATUREBITS*/, &eax, &ebx, &ecx, &edx); - - x86_cpu_has_sse2 = edx & 0x4000000; - x86_cpu_has_sse42 = ecx & 0x100000; - x86_cpu_has_pclmulqdq = ecx & 0x2; -} diff --git a/contrib/libzlib-ng/arch/x86/x86.h b/contrib/libzlib-ng/arch/x86/x86.h deleted file mode 100644 index 78be0a661a1..00000000000 --- a/contrib/libzlib-ng/arch/x86/x86.h +++ /dev/null @@ -1,23 +0,0 @@ - /* cpu.h -- check for CPU features - * Copyright (C) 2013 Intel Corporation Jim Kukunas - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -#ifndef CPU_H_ -#define CPU_H_ - -#if defined(HAVE_INTERNAL) -# define ZLIB_INTERNAL __attribute__((visibility ("internal"))) -#elif defined(HAVE_HIDDEN) -# define ZLIB_INTERNAL __attribute__((visibility ("hidden"))) -#else -# define ZLIB_INTERNAL -#endif - -extern int x86_cpu_has_sse2; -extern int x86_cpu_has_sse42; -extern int x86_cpu_has_pclmulqdq; - -void ZLIB_INTERNAL x86_check_features(void); - -#endif /* CPU_H_ */ diff --git a/contrib/libzlib-ng/compress.c b/contrib/libzlib-ng/compress.c deleted file mode 100644 index 9f6f140298f..00000000000 --- a/contrib/libzlib-ng/compress.c +++ /dev/null @@ -1,74 +0,0 @@ -/* compress.c -- compress a memory buffer - * Copyright (C) 1995-2005, 2014 Jean-loup Gailly, Mark Adler. - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* @(#) $Id$ */ - -#define ZLIB_INTERNAL -#include "zlib.h" - -/* =========================================================================== - Compresses the source buffer into the destination buffer. The level - parameter has the same meaning as in deflateInit. sourceLen is the byte - length of the source buffer. Upon entry, destLen is the total size of the - destination buffer, which must be at least 0.1% larger than sourceLen plus - 12 bytes. Upon exit, destLen is the actual size of the compressed buffer. - - compress2 returns Z_OK if success, Z_MEM_ERROR if there was not enough - memory, Z_BUF_ERROR if there was not enough room in the output buffer, - Z_STREAM_ERROR if the level parameter is invalid. -*/ -int ZEXPORT compress2(unsigned char *dest, size_t *destLen, const unsigned char *source, - size_t sourceLen, int level) { - z_stream stream; - int err; - const unsigned int max = (unsigned int)0 - 1; - size_t left; - - left = *destLen; - *destLen = 0; - - stream.zalloc = (alloc_func)0; - stream.zfree = (free_func)0; - stream.opaque = NULL; - - err = deflateInit(&stream, level); - if (err != Z_OK) - return err; - - stream.next_out = dest; - stream.avail_out = 0; - stream.next_in = (const unsigned char *)source; - stream.avail_in = 0; - - do { - if (stream.avail_out == 0) { - stream.avail_out = left > (unsigned long)max ? max : (unsigned int)left; - left -= stream.avail_out; - } - if (stream.avail_in == 0) { - stream.avail_in = sourceLen > (unsigned long)max ? max : (unsigned int)sourceLen; - sourceLen -= stream.avail_in; - } - err = deflate(&stream, sourceLen ? Z_NO_FLUSH : Z_FINISH); - } while (err == Z_OK); - - *destLen = stream.total_out; - deflateEnd(&stream); - return err == Z_STREAM_END ? Z_OK : err; -} - -/* =========================================================================== - */ -int ZEXPORT compress(unsigned char *dest, size_t *destLen, const unsigned char *source, size_t sourceLen) { - return compress2(dest, destLen, source, sourceLen, Z_DEFAULT_COMPRESSION); -} - -/* =========================================================================== - If the default memLevel or windowBits for deflateInit() is changed, then - this function needs to be updated. - */ -size_t ZEXPORT compressBound(size_t sourceLen) { - return sourceLen + (sourceLen >> 12) + (sourceLen >> 14) + (sourceLen >> 25) + 13; -} diff --git a/contrib/libzlib-ng/configure b/contrib/libzlib-ng/configure deleted file mode 100755 index de55744c6f2..00000000000 --- a/contrib/libzlib-ng/configure +++ /dev/null @@ -1,923 +0,0 @@ -#!/bin/sh -# configure script for zlib. -# -# Normally configure builds both a static and a shared library. -# If you want to build just a static library, use: ./configure --static -# -# To impose specific compiler or flags or install directory, use for example: -# prefix=$HOME CC=cc CFLAGS="-O4" ./configure -# or for csh/tcsh users: -# (setenv prefix $HOME; setenv CC cc; setenv CFLAGS "-O4"; ./configure) - -# Incorrect settings of CC or CFLAGS may prevent creating a shared library. -# If you have problems, try without defining CC and CFLAGS before reporting -# an error. - -# start off configure.log -echo -------------------- >> configure.log -echo $0 $* >> configure.log -date >> configure.log - -SRCDIR=$(cd $(dirname $0); pwd) -BUILDDIR=$(pwd) - -# set command prefix for cross-compilation -if [ -n "${CHOST}" ]; then - uname="`echo "${CHOST}" | sed -e 's/^[^-]*-\([^-]*\)$/\1/' -e 's/^[^-]*-[^-]*-\([^-]*\)$/\1/' -e 's/^[^-]*-[^-]*-\([^-]*\)-.*$/\1/'`" - CROSS_PREFIX="${CHOST}-" - ARCH="`echo "${CHOST}" | sed -e 's/-.*//'`" -else - ARCH="`uname -m`" -fi - -case "${ARCH}" in - x86_64) - case "${CFLAGS}" in - *-m32*) - ARCH=i686 - ;; - esac - ;; - i386 | i486 | i586 | i686) - case "${CFLAGS}" in - *-m64*) - ARCH=x86_64 - ;; - esac - ;; -esac - -# destination name for static library -STATICLIB=libz.a - -# destination name for windows import library -IMPORTLIB= - -# extract zlib version numbers from zlib.h -VER=`sed -n -e '/ZLIB_VERSION "/s/.*"\(.*\)".*/\1/p' < ${SRCDIR}/zlib.h` -VER3=`sed -n -e '/ZLIB_VERSION "/s/.*"\([0-9]*\\.[0-9]*\\.[0-9]*\).*/\1/p' < ${SRCDIR}/zlib.h` -VER2=`sed -n -e '/ZLIB_VERSION "/s/.*"\([0-9]*\\.[0-9]*\)\\..*/\1/p' < ${SRCDIR}/zlib.h` -VER1=`sed -n -e '/ZLIB_VERSION "/s/.*"\([0-9]*\)\\..*/\1/p' < ${SRCDIR}/zlib.h` - -# establish commands for library building -if "${CROSS_PREFIX}ar" --version >/dev/null 2>/dev/null || test $? -lt 126; then - AR=${AR-"${CROSS_PREFIX}ar"} - test -n "${CROSS_PREFIX}" && echo Using ${AR} | tee -a configure.log -else - AR=${AR-"ar"} - test -n "${CROSS_PREFIX}" && echo Using ${AR} | tee -a configure.log -fi -ARFLAGS=${ARFLAGS-"rc"} -if "${CROSS_PREFIX}ranlib" --version >/dev/null 2>/dev/null || test $? -lt 126; then - RANLIB=${RANLIB-"${CROSS_PREFIX}ranlib"} - test -n "${CROSS_PREFIX}" && echo Using ${RANLIB} | tee -a configure.log -else - RANLIB=${RANLIB-"ranlib"} -fi -if "${CROSS_PREFIX}nm" --version >/dev/null 2>/dev/null || test $? -lt 126; then - NM=${NM-"${CROSS_PREFIX}nm"} - test -n "${CROSS_PREFIX}" && echo Using ${NM} | tee -a configure.log -else - NM=${NM-"nm"} -fi - -# set defaults before processing command line options -LDCONFIG=${LDCONFIG-"ldconfig"} -LDSHAREDLIBC="${LDSHAREDLIBC--lc}" -DEFFILE= -RC= -RCFLAGS= -RCOBJS= -STRIP= -ARCHS= -prefix=${prefix-/usr/local} -exec_prefix=${exec_prefix-'${prefix}'} -bindir=${bindir-'${exec_prefix}/bin'} -libdir=${libdir-'${exec_prefix}/lib'} -sharedlibdir=${sharedlibdir-'${libdir}'} -includedir=${includedir-'${prefix}/include'} -mandir=${mandir-'${prefix}/share/man'} -shared_ext='.so' -shared=1 -gzfileops=0 -compat=0 -cover=0 -build32=0 -build64=0 -without_optimizations=0 -without_new_strategies=0 -gcc=0 -old_cc="$CC" -old_cflags="$CFLAGS" -OBJC='$(OBJZ)' -PIC_OBJC='$(PIC_OBJZ)' -INSTALLTARGETS="install-shared install-static" -UNINSTALLTARGETS="uninstall-shared uninstall-static" - -TEST="teststatic" - -# leave this script, optionally in a bad way -leave() -{ - if test "$*" != "0"; then - echo "** $0 aborting." | tee -a configure.log - fi - rm -f $test.[co] $test $test$shared_ext $test.gcno ./--version - echo -------------------- >> configure.log - echo >> configure.log - echo >> configure.log - exit $1 -} - -# process command line options -while test $# -ge 1 -do -case "$1" in - -h* | --help) - echo 'usage:' | tee -a configure.log - echo ' configure [--zlib-compat] [--prefix=PREFIX] [--eprefix=EXPREFIX]' | tee -a configure.log - echo ' [--static] [--32] [--64] [--libdir=LIBDIR] [--sharedlibdir=LIBDIR]' | tee -a configure.log - echo ' [--includedir=INCLUDEDIR] [--archs="-arch i386 -arch x86_64"]' | tee -a configure.log - exit 0 ;; - -p*=* | --prefix=*) prefix=`echo $1 | sed 's/.*=//'`; shift ;; - -e*=* | --eprefix=*) exec_prefix=`echo $1 | sed 's/.*=//'`; shift ;; - -l*=* | --libdir=*) libdir=`echo $1 | sed 's/.*=//'`; shift ;; - --sharedlibdir=*) sharedlibdir=`echo $1 | sed 's/.*=//'`; shift ;; - -i*=* | --includedir=*) includedir=`echo $1 | sed 's/.*=//'`;shift ;; - -u*=* | --uname=*) uname=`echo $1 | sed 's/.*=//'`;shift ;; - -p* | --prefix) prefix="$2"; shift; shift ;; - -e* | --eprefix) exec_prefix="$2"; shift; shift ;; - -l* | --libdir) libdir="$2"; shift; shift ;; - -i* | --includedir) includedir="$2"; shift; shift ;; - -s* | --shared | --enable-shared) shared=1; shift ;; - -t | --static) shared=0; shift ;; - --zlib-compat) compat=1; shift ;; - --cover) cover=1; shift ;; - -3* | --32) build32=1; shift ;; - -6* | --64) build64=1; shift ;; - -a*=* | --archs=*) ARCHS=`echo $1 | sed 's/.*=//'`; shift ;; - --sysconfdir=*) echo "ignored option: --sysconfdir" | tee -a configure.log; shift ;; - --localstatedir=*) echo "ignored option: --localstatedir" | tee -a configure.log; shift ;; - -noopt | --without-optimizations) without_optimizations=1; shift;; - -oldstrat | --without-new-strategies) without_new_strategies=1; shift;; - *) - echo "unknown option: $1" | tee -a configure.log - echo "$0 --help for help" | tee -a configure.log - leave 1;; - esac -done - -# temporary file name -test=ztest$$ - -# put arguments in log, also put test file in log if used in arguments -show() -{ - case "$*" in - *$test.c*) - echo === $test.c === >> configure.log - cat $test.c >> configure.log - echo === >> configure.log;; - esac - echo $* >> configure.log -} - -# check for gcc vs. cc and set compile and link flags based on the system identified by uname -cat > $test.c <&1` in - *gcc*) gcc=1 ;; -esac - -show $cc -c $test.c -if test "$gcc" -eq 1 && ($cc -c $test.c) >> configure.log 2>&1; then - echo ... using gcc >> configure.log - CC="$cc" -# Re-check arch if gcc is a cross-compiler - GCC_ARCH=`$CC -dumpmachine | sed 's/-.*//g'` - case $GCC_ARCH in - i386 | i486 | i586 | i686) -# Honor user choice if gcc is multilib and 64-bit is requested - if test $build64 -eq 1; then - ARCH=x86_64 - else - ARCH=$GCC_ARCH - fi ;; -# Honor user choice if gcc is multilib and 32-bit is requested - x86_64) - if test $build32 -ne 1; then - ARCH=$GCC_ARCH - fi ;; - esac - CFLAGS="${CFLAGS--O3} ${ARCHS} -Wall" - SFLAGS="${CFLAGS--O3} -fPIC" - LDFLAGS="${LDFLAGS} ${ARCHS}" - if test $build64 -eq 1; then - CFLAGS="${CFLAGS} -m64" - SFLAGS="${SFLAGS} -m64" - fi - if test "${ZLIBGCCWARN}" = "YES"; then - CFLAGS="${CFLAGS} -Wextra -pedantic" - fi - if test -z "$uname"; then - uname=`(uname -s || echo unknown) 2>/dev/null` - fi - case "$uname" in - Linux* | linux* | GNU | GNU/* | solaris*) - LDSHARED=${LDSHARED-"$cc -shared -Wl,-soname,libz.so.1,--version-script,${SRCDIR}/zlib.map"} ;; - *BSD | *bsd* | DragonFly) - LDSHARED=${LDSHARED-"$cc -shared -Wl,-soname,libz.so.1,--version-script,${SRCDIR}/zlib.map"} - LDCONFIG="ldconfig -m" ;; - CYGWIN* | Cygwin* | cygwin*) - ARFLAGS="rcs" - CFLAGS="${CFLAGS} -D_LARGEFILE_SOURCE -D_FILE_OFFSET_BITS=64" - SFLAGS="${CFLAGS}" - shared_ext='.dll' - sharedlibdir='${bindir}' - SHAREDLIB=cygz$shared_ext - SHAREDLIBM='' - SHAREDLIBV='' - SHAREDTARGET=$SHAREDLIB - IMPORTLIB='libz.dll.a' - LDSHARED=${LDSHARED-"$cc -shared -Wl,--out-implib,${IMPORTLIB},--version-script,${SRCDIR}/zlib.map"} - LDSHAREDLIBC="" - DEFFILE='win32/zlib.def' - RC='windres' - RCFLAGS='--define GCC_WINDRES' - RCOBJS='zlibrc.o' - STRIP='strip' - EXE='.exe' ;; - MSYS* | msys*) - ARFLAGS="rcs" - SFLAGS="${CFLAGS}" - shared_ext='.dll' - sharedlibdir='${bindir}' - SHAREDLIB=msys-z$shared_ext - SHAREDLIBM='' - SHAREDLIBV='' - SHAREDTARGET=$SHAREDLIB - IMPORTLIB='libz.dll.a' - LDSHARED=${LDSHARED-"$cc -shared -Wl,--out-implib,${IMPORTLIB}"} - LDSHAREDLIBC="" - DEFFILE='win32/zlib.def' - RC='windres' - RCFLAGS='--define GCC_WINDRES' - RCOBJS='zlibrc.o' - STRIP='strip' - EXE='.exe' ;; - MINGW* | mingw*) - ARFLAGS="rcs" - CFLAGS="${CFLAGS} -D_LARGEFILE_SOURCE -D_FILE_OFFSET_BITS=64 -D_POSIX_C_SOURCE=200809L -D_GNU_SOURCE=1" - SFLAGS="${CFLAGS}" - shared_ext='.dll' - sharedlibdir='${bindir}' - SHAREDLIB=libz-$VER1$shared_ext - SHAREDLIBM='' - SHAREDLIBV='' - SHAREDTARGET=$SHAREDLIB - IMPORTLIB='libz.dll.a' - LDSHARED=${LDSHARED-"$cc -shared -Wl,--out-implib=${IMPORTLIB} -Wl,--version-script=${SRCDIR}/zlib.map"} - LDSHAREDLIBC="" - DEFFILE='win32/zlib.def' - RC='windres' - RCFLAGS='--define GCC_WINDRES' - if [ "$CC" == "mingw32-gcc" ]; then - case $ARCH in - i386 | i486 | i586 | i686) RCFLAGS="${RCFLAGS} -F pe-i386";; - esac; - fi - RCOBJS='zlibrc.o' - STRIP='strip' - EXE='.exe' ;; - QNX*) # This is for QNX6. I suppose that the QNX rule below is for QNX2,QNX4 - # (alain.bonnefoy@icbt.com) - LDSHARED=${LDSHARED-"$cc -shared -Wl,-hlibz.so.1"} ;; - HP-UX*) - LDSHARED=${LDSHARED-"$cc -shared $SFLAGS"} - case `(uname -m || echo unknown) 2>/dev/null` in - ia64) - shared_ext='.so' - SHAREDLIB='libz.so' ;; - *) - shared_ext='.sl' - SHAREDLIB='libz.sl' ;; - esac ;; - Darwin* | darwin*) - shared_ext='.dylib' - SHAREDLIB=libz$shared_ext - SHAREDLIBV=libz.$VER$shared_ext - SHAREDLIBM=libz.$VER1$shared_ext - SHAREDTARGET=$SHAREDLIBV - LDSHARED=${LDSHARED-"$cc -dynamiclib -install_name $libdir/$SHAREDLIBM -compatibility_version $VER1 -current_version $VER3"} - if libtool -V 2>&1 | grep Apple > /dev/null; then - AR="libtool" - else - AR="/usr/bin/libtool" - fi - ARFLAGS="-o" ;; - *) LDSHARED=${LDSHARED-"$cc -shared"} ;; - esac -else - # find system name and corresponding cc options - CC=${CC-cc} - gcc=0 - echo ... using $CC >> configure.log - if test -z "$uname"; then - uname=`(uname -sr || echo unknown) 2>/dev/null` - fi - case "$uname" in - HP-UX*) SFLAGS=${CFLAGS-"-O +z"} - CFLAGS=${CFLAGS-"-O"} -# LDSHARED=${LDSHARED-"ld -b +vnocompatwarnings"} - LDSHARED=${LDSHARED-"ld -b"} - case `(uname -m || echo unknown) 2>/dev/null` in - ia64) - shared_ext='.so' - SHAREDLIB='libz.so' ;; - *) - shared_ext='.sl' - SHAREDLIB='libz.sl' ;; - esac ;; - AIX*) # Courtesy of dbakker@arrayasolutions.com - SFLAGS=${CFLAGS-"-O -qmaxmem=8192"} - CFLAGS=${CFLAGS-"-O -qmaxmem=8192"} - LDSHARED=${LDSHARED-"xlc -G"} ;; - # send working options for other systems to zlib@gzip.org - *) SFLAGS=${CFLAGS-"-O"} - CFLAGS=${CFLAGS-"-O"} - LDSHARED=${LDSHARED-"cc -shared"} ;; - esac -fi - -# destination names for shared library if not defined above -SHAREDLIB=${SHAREDLIB-"libz$shared_ext"} -SHAREDLIBV=${SHAREDLIBV-"libz$shared_ext.$VER"} -SHAREDLIBM=${SHAREDLIBM-"libz$shared_ext.$VER1"} -SHAREDTARGET=${SHAREDTARGET-"libz$shared_ext.$VER"} - -echo >> configure.log - -# define functions for testing compiler and library characteristics and logging the results - -cat > $test.c </dev/null; then - try() - { - show $* - test "`( $* ) 2>&1 | tee -a configure.log`" = "" - } - echo - using any output from compiler to indicate an error >> configure.log -else -try() -{ - show $* - ( $* ) >> configure.log 2>&1 - ret=$? - if test $ret -ne 0; then - echo "(exit code "$ret")" >> configure.log - fi - return $ret -} -fi - -tryboth() -{ - show $* - got=`( $* ) 2>&1` - ret=$? - printf %s "$got" >> configure.log - if test $ret -ne 0; then - return $ret - fi - test "$got" = "" -} - -cat > $test.c << EOF -int foo() { return 0; } -EOF -echo "Checking for obsessive-compulsive compiler options..." >> configure.log -if try $CC -c $CFLAGS $test.c; then - : -else - echo "Compiler error reporting is too harsh for $0 (perhaps remove -Werror)." | tee -a configure.log - leave 1 -fi - -echo >> configure.log - -# see if shared library build supported -cat > $test.c <> configure.log - -# check for large file support, and if none, check for fseeko() -cat > $test.c < -#ifdef __MSYS__ -# define off64_t _off64_t -#endif -off64_t dummy = 0; -EOF -if try $CC -c $CFLAGS -D_LARGEFILE64_SOURCE=1 $test.c; then - CFLAGS="${CFLAGS} -D_LARGEFILE64_SOURCE=1" - SFLAGS="${SFLAGS} -D_LARGEFILE64_SOURCE=1" - ALL="${ALL} all64" - TEST="${TEST} test64" - echo "Checking for off64_t... Yes." | tee -a configure.log - echo "Checking for fseeko... Yes." | tee -a configure.log -else - echo "Checking for off64_t... No." | tee -a configure.log - echo >> configure.log - cat > $test.c < -int main(void) { - fseeko(NULL, 0, 0); - return 0; -} -EOF - if try $CC $CFLAGS -o $test $test.c; then - echo "Checking for fseeko... Yes." | tee -a configure.log - else - CFLAGS="${CFLAGS} -DNO_FSEEKO" - SFLAGS="${SFLAGS} -DNO_FSEEKO" - echo "Checking for fseeko... No." | tee -a configure.log - fi -fi - -echo >> configure.log - -# check for strerror() for use by gz* functions -cat > $test.c < -#include -int main() { return strlen(strerror(errno)); } -EOF -if try $CC $CFLAGS -o $test $test.c; then - echo "Checking for strerror... Yes." | tee -a configure.log -else - CFLAGS="${CFLAGS} -DNO_STRERROR" - SFLAGS="${SFLAGS} -DNO_STRERROR" - echo "Checking for strerror... No." | tee -a configure.log -fi - -# We need to remove zconf.h from source directory if building outside of it -if [ "$SRCDIR" != "$BUILDDIR" ]; then rm -f $SRCDIR/zconf.h; fi - -# copy clean zconf.h for subsequent edits -cp -p $SRCDIR/zconf.h.in zconf.h - -echo >> configure.log - -# check for unistd.h and save result in zconf.h -cat > $test.c < -int main() { return 0; } -EOF -if try $CC -c $CFLAGS $test.c; then - sed < zconf.h "/^#ifdef HAVE_UNISTD_H.* may be/s/def HAVE_UNISTD_H\(.*\) may be/ 1\1 was/" > zconf.temp.h - mv zconf.temp.h zconf.h - echo "Checking for unistd.h... Yes." | tee -a configure.log -else - echo "Checking for unistd.h... No." | tee -a configure.log -fi - -echo >> configure.log - -# check for stdarg.h and save result in zconf.h -cat > $test.c < -int main() { return 0; } -EOF -if try $CC -c $CFLAGS $test.c; then - sed < zconf.h "/^#ifdef HAVE_STDARG_H.* may be/s/def HAVE_STDARG_H\(.*\) may be/ 1\1 was/" > zconf.temp.h - mv zconf.temp.h zconf.h - echo "Checking for stdarg.h... Yes." | tee -a configure.log -else - echo "Checking for stdarg.h... No." | tee -a configure.log -fi - -# if --zlib-compat was requested -if test $compat -eq 1; then - CFLAGS="${CFLAGS} -DZLIB_COMPAT -DWITH_GZFILEOP" - SFLAGS="${SFLAGS} -DZLIB_COMPAT -DWITH_GZFILEOP" - OBJC="${OBJC} \$(OBJG)" - PIC_OBJC="${PIC_OBJC} \$(PIC_OBJG)" - case "$uname" in - CYGWIN* | Cygwin* | cygwin* | MSYS* | msys* | MINGW* | mingw*) - DEFFILE="win32/zlibcompat.def" ;; - esac -fi - -# if code coverage testing was requested, use older gcc if defined, e.g. "gcc-4.2" on Mac OS X -if test $cover -eq 1; then - CFLAGS="${CFLAGS} -fprofile-arcs -ftest-coverage" - if test -n "$GCC_CLASSIC"; then - CC=$GCC_CLASSIC - fi -fi - -echo >> configure.log - -# Check for ANSI C compliant compiler -cat > $test.c < -#include -#include "zconf.h" -int main() -{ -#ifdef STDC - return 0; -#endif - return 1; -} -EOF -if try $CC -c $CFLAGS $test.c; then - echo "Checking for ANSI C compliant compiler... Yes." | tee -a configure.log - : -else - echo "Checking for ANSI C compliant compiler... No." | tee -a configure.log - echo "Error: ANSI C compatible compiler needed, cannot continue." | tee -a configure.log - leave 1 -fi - -# see if we can hide zlib internal symbols that are linked between separate source files using hidden -if test "$gcc" -eq 1; then - echo >> configure.log - cat > $test.c <> configure.log - echo "Checking for attribute(visibility(hidden)) support... Yes." | tee -a configure.log - else - echo >> configure.log - echo "Checking for attribute(visibility(hidden)) support... No." | tee -a configure.log - fi -fi - -# see if we can hide zlib internal symbols that are linked between separate source files using internal -if test "$gcc" -eq 1; then - echo >> configure.log - cat > $test.c <> configure.log - echo "Checking for attribute(visibility(internal)) support... Yes." | tee -a configure.log - else - echo >> configure.log - echo "Checking for attribute(visibility(internal)) support... No." | tee -a configure.log - fi -fi - -# Check for __builtin_ctzl() support in compiler -cat > $test.c << EOF -int main(void) -{ - unsigned int zero = 0; - long test = __builtin_ctzl(zero); - (void)test; - return 0; -} -EOF -if try ${CC} ${CFLAGS} $test.c; then - echo "Checking for __builtin_ctzl ... Yes." | tee -a configure.log - CFLAGS="$CFLAGS -DHAVE_BUILTIN_CTZL" - SFLAGS="$SFLAGS -DHAVE_BUILTIN_CTZL" -else - echo "Checking for __builtin_ctzl ... No." | tee -a configure.log -fi - -# Check for SSE2 intrinsics -cat > $test.c << EOF -#include -int main(void) -{ - __m128i zero = _mm_setzero_si128(); - (void)zero; - return 0; -} -EOF -if try ${CC} ${CFLAGS} -msse2 $test.c; then - echo "Checking for SSE2 intrinsics ... Yes." | tee -a configure.log - HAVE_SSE2_INTRIN=1 -else - echo "Checking for SSE2 intrinsics ... No." | tee -a configure.log - HAVE_SSE2_INTRIN=0 -fi - -# Check for PCLMULQDQ intrinsics -cat > $test.c << EOF -#include -#include -int main(void) -{ - __m128i a = _mm_setzero_si128(); - __m128i b = _mm_setzero_si128(); - __m128i c = _mm_clmulepi64_si128(a, b, 0x10); - (void)c; - return 0; -} -EOF -if try ${CC} ${CFLAGS} -mpclmul $test.c; then - echo "Checking for PCLMULQDQ intrinsics ... Yes." | tee -a configure.log - HAVE_PCLMULQDQ_INTRIN=1 -else - echo "Checking for PCLMULQDQ intrinsics ... No." | tee -a configure.log - HAVE_PCLMULQDQ_INTRIN=0 -fi - -# Enable deflate_medium at level 4-6 -if test $without_new_strategies -eq 0; then - CFLAGS="${CFLAGS} -DMEDIUM_STRATEGY" - SFLAGS="${SFLAGS} -DMEDIUM_STRATEGY" -fi - -ARCHDIR='arch/generic' -ARCH_STATIC_OBJS='' -ARCH_SHARED_OBJS='' - -# Set ARCH specific FLAGS -case "${ARCH}" in - - # x86 and x86_64 specific optimizations - i386 | i486 | i586 | i686 | x86_64) - ARCHDIR=arch/x86 - - case "${ARCH}" in - x86_64) - CFLAGS="${CFLAGS} -DX86_64 -DX86_NOCHECK_SSE2" - SFLAGS="${SFLAGS} -DX86_64 -DX86_NOCHECK_SSE2" - ;; - i386 | i486 | i586 | i686) - CFLAGS="${CFLAGS} -DX86" - SFLAGS="${SFLAGS} -DX86" - ;; - esac - - CFLAGS="${CFLAGS} -DUNALIGNED_OK -DUNROLL_LESS -DX86_CPUID" - SFLAGS="${SFLAGS} -DUNALIGNED_OK -DUNROLL_LESS -DX86_CPUID" - - # Enable arch-specific optimizations? - if test $without_optimizations -eq 0; then - ARCH_STATIC_OBJS="${ARCH_STATIC_OBJS} x86.o" - ARCH_SHARED_OBJS="${ARCH_SHARED_OBJS} x86.lo" - - if test ${HAVE_SSE2_INTRIN} -eq 1; then - CFLAGS="${CFLAGS} -DX86_SSE2_FILL_WINDOW" - SFLAGS="${SFLAGS} -DX86_SSE2_FILL_WINDOW" - ARCH_STATIC_OBJS="${ARCH_STATIC_OBJS} fill_window_sse.o" - ARCH_SHARED_OBJS="${ARCH_SHARED_OBJS} fill_window_sse.lo" - fi - - CFLAGS="${CFLAGS} -DX86_SSE4_2_CRC_HASH" - SFLAGS="${SFLAGS} -DX86_SSE4_2_CRC_HASH" - ARCH_STATIC_OBJS="${ARCH_STATIC_OBJS} insert_string_sse.o" - ARCH_SHARED_OBJS="${ARCH_SHARED_OBJS} insert_string_sse.lo" - - if test ${HAVE_PCLMULQDQ_INTRIN} -eq 1; then - CFLAGS="${CFLAGS} -DX86_PCLMULQDQ_CRC" - SFLAGS="${SFLAGS} -DX86_PCLMULQDQ_CRC" - ARCH_STATIC_OBJS="${ARCH_STATIC_OBJS} crc_folding.o" - ARCH_SHARED_OBJS="${ARCH_SHARED_OBJS} crc_folding.lo" - fi - - # Enable deflate_quick at level 1? - # requires SSE2: code uses fill_window_sse - if test ${HAVE_SSE2_INTRIN} -eq 1 && test $without_new_strategies -eq 0; then - CFLAGS="${CFLAGS} -DX86_QUICK_STRATEGY" - SFLAGS="${SFLAGS} -DX86_QUICK_STRATEGY" - - ARCH_STATIC_OBJS="${ARCH_STATIC_OBJS} deflate_quick.o" - ARCH_SHARED_OBJS="${ARCH_SHARED_OBJS} deflate_quick.lo" - fi - fi - ;; - - # ARM specific optimizations - armv3l | armv4b | armv4l | armv4tl | armv5tel | armv5tejl | armv6l | armv6hl | armv7l | armv7hl | armv7hnl) - ARCHDIR=arch/arm - - case "${ARCH}" in - armv6l | armv6hl) - # Tests done on Raspberry pi (armv6hl) indicate that UNALIGNED_OK and UNROLL_LESS both - # provide performance improvements, totaling about 1.5% for the two. - CFLAGS="${CFLAGS} -DUNALIGNED_OK -DUNROLL_LESS" - SFLAGS="${SFLAGS} -DUNALIGNED_OK -DUNROLL_LESS" - ;; - esac - - ;; -esac - -echo "ARCH: ${ARCH}" -echo "Using arch directory: ${ARCHDIR}" - -# show the results in the log -echo >> configure.log -echo ALL = $ALL >> configure.log -echo AR = $AR >> configure.log -echo ARFLAGS = $ARFLAGS >> configure.log -echo CC = $CC >> configure.log -echo CFLAGS = $CFLAGS >> configure.log -echo EXE = $EXE >> configure.log -echo LDCONFIG = $LDCONFIG >> configure.log -echo LDFLAGS = $LDFLAGS >> configure.log -echo LDSHARED = $LDSHARED >> configure.log -echo LDSHAREDLIBC = $LDSHAREDLIBC >> configure.log -echo DEFFILE = $DEFFILE >> configure.log -echo RC = $RC >> configure.log -echo RCFLAGS = $RCFLAGS >> configure.log -echo RCOBJS = $RCOBJS >> configure.log -echo STRIP = $STRIP >> configure.log -echo OBJC = $OBJC >> configure.log -echo PIC_OBJC = $PIC_OBJC >> configure.log -echo RANLIB = $RANLIB >> configure.log -echo SFLAGS = $SFLAGS >> configure.log -echo SHAREDLIB = $SHAREDLIB >> configure.log -echo SHAREDLIBM = $SHAREDLIBM >> configure.log -echo SHAREDLIBV = $SHAREDLIBV >> configure.log -echo SHAREDTARGET = $SHAREDTARGET >> configure.log -echo IMPORTLIB = $IMPORTLIB >> configure.log -echo INSTALLTARGETS = $INSTALLTARGETS >> configure.log -echo UNINSTALLTARGETS = $UNINSTALLTARGETS >> configure.log -echo SRCDIR = $SRCDIR >> configure.log -echo BUILDDIR = $BUILDDIR >> configure.log -echo STATICLIB = $STATICLIB >> configure.log -echo TEST = $TEST >> configure.log -echo VER = $VER >> configure.log -echo exec_prefix = $exec_prefix >> configure.log -echo includedir = $includedir >> configure.log -echo bindir = $bindir >> configure.log -echo libdir = $libdir >> configure.log -echo mandir = $mandir >> configure.log -echo prefix = $prefix >> configure.log -echo sharedlibdir = $sharedlibdir >> configure.log -echo uname = $uname >> configure.log -echo ARCHDIR = ${ARCHDIR} >> configure.log -echo ARCH_STATIC_OBJS = ${ARCH_STATIC_OBJS} >> configure.log -echo ARCH_SHARED_OBJS = ${ARCH_SHARED_OBJS} >> configure.log - -# update Makefile with the configure results - -INCLUDES="-I$SRCDIR" -if [ "$SRCDIR" != "$BUILDDIR" ]; then INCLUDES="-I$BUILDDIR ${INCLUDES}"; fi - -sed < $SRCDIR/Makefile.in " -/^CC *=/s#=.*#=$CC# -/^CFLAGS *=/s#=.*#=$CFLAGS# -/^SFLAGS *=/s#=.*#=$SFLAGS# -/^LDFLAGS *=/s#=.*#=$LDFLAGS# -/^LDSHARED *=/s#=.*#=$LDSHARED# -/^STATICLIB *=/s#=.*#=$STATICLIB# -/^SHAREDLIB *=/s#=.*#=$SHAREDLIB# -/^SHAREDLIBV *=/s#=.*#=$SHAREDLIBV# -/^SHAREDLIBM *=/s#=.*#=$SHAREDLIBM# -/^SHAREDTARGET *=/s#=.*#=$SHAREDTARGET# -/^IMPORTLIB *=/s#=.*#=$IMPORTLIB# -/^AR *=/s#=.*#=$AR# -/^ARFLAGS *=/s#=.*#=$ARFLAGS# -/^RANLIB *=/s#=.*#=$RANLIB# -/^LDCONFIG *=/s#=.*#=$LDCONFIG# -/^LDSHAREDLIBC *=/s#=.*#=$LDSHAREDLIBC# -/^DEFFILE *=/s#=.*#=$DEFFILE# -/^RC *=/s#=.*#=$RC# -/^RCFLAGS *=/s#=.*#=$RCFLAGS# -/^RCOBJS *=/s#=.*#=$RCOBJS# -/^STRIP *=/s#=.*#=$STRIP# -/^EXE *=/s#=.*#=$EXE# -/^prefix *=/s#=.*#= $prefix# -/^exec_prefix *=/s#=.*#= $exec_prefix# -/^bindir *=/s#=.*#= $bindir# -/^libdir *=/s#=.*#= $libdir# -/^sharedlibdir *=/s#=.*#= $sharedlibdir# -/^includedir *=/s#=.*#= $includedir# -/^mandir *=/s#=.*#= $mandir# -/^SRCDIR *=/s#=.*#=$SRCDIR# -/^INCLUDES *=/s#=.*#=$INCLUDES# -/^OBJC *=/s#=.*#= $OBJC# -/^PIC_OBJC *=/s#=.*#= $PIC_OBJC# -/^all: */s#:.*#: $ALL# -/^install-libs: */s#:.*#: $INSTALLTARGETS# -/^uninstall-libs: */s#:.*#: $UNINSTALLTARGETS# -/^ARCHDIR *=/s#=.*#=$ARCHDIR# -/^ARCH_STATIC_OBJS *=/s#=.*#=$ARCH_STATIC_OBJS# -/^ARCH_SHARED_OBJS *=/s#=.*#=$ARCH_SHARED_OBJS# -" > Makefile - -# Generate Makefile in arch dir -mkdir -p $ARCHDIR - -ARCHINCLUDES="-I$SRCDIR/$ARCHDIR -I$SRCDIR" -if [ "$SRCDIR" != "$BUILDDIR" ]; then ARCHINCLUDES="-I$BUILDDIR ${ARCHINCLUDES}"; fi - -sed < $SRCDIR/$ARCHDIR/Makefile.in " -/^CC *=/s#=.*#=$CC# -/^CFLAGS *=/s#=.*#=$CFLAGS# -/^SFLAGS *=/s#=.*#=$SFLAGS# -/^INCLUDES *=/s#=.*#=$ARCHINCLUDES# -/^SRCDIR *=/s#=.*#=$SRCDIR/$ARCHDIR# -/^SRCTOP *=/s#=.*#=$SRCDIR# -" > $ARCHDIR/Makefile - -# Generate Makefile in test dir -mkdir -p test -TESTINCLUDES="-I$SRCDIR" -if [ "$SRCDIR" != "$BUILDDIR" ]; then TESTINCLUDES="${TESTINCLUDES} -I$BUILDDIR"; fi -if test $compat -eq 1; then COMPATTESTS="compattests"; fi -sed < $SRCDIR/test/Makefile.in " -/^CC *=/s#=.*#=$CC# -/^CFLAGS *=/s#=.*#=$CFLAGS# -/^EXE *=/s#=.*#=$EXE# -/^oldtests: */s#:.*#: $TEST# -/^INCLUDES *=/s#=.*#=$TESTINCLUDES# -/^SRCDIR *=/s#=.*#=$SRCDIR/test# -/^SRCTOP *=/s#=.*#=$SRCDIR# -/^COMPATTESTS *=/s#=.*#=$COMPATTESTS# -" > test/Makefile - -# create zlib.pc with the configure results -sed < $SRCDIR/zlib.pc.in " -/^CC *=/s#=.*#=$CC# -/^CFLAGS *=/s#=.*#=$CFLAGS# -/^LDSHARED *=/s#=.*#=$LDSHARED# -/^STATICLIB *=/s#=.*#=$STATICLIB# -/^SHAREDLIB *=/s#=.*#=$SHAREDLIB# -/^SHAREDLIBV *=/s#=.*#=$SHAREDLIBV# -/^SHAREDLIBM *=/s#=.*#=$SHAREDLIBM# -/^IMPORTLIB *=/s#=.*#=$IMPORTLIB# -/^AR *=/s#=.*#=$AR# -/^ARFLAGS *=/s#=.*#=$ARFLAGS# -/^RANLIB *=/s#=.*#=$RANLIB# -/^EXE *=/s#=.*#=$EXE# -/^prefix *=/s#=.*#=$prefix# -/^exec_prefix *=/s#=.*#=$exec_prefix# -/^bindir *=/s#=.*#=$bindir# -/^libdir *=/s#=.*#=$libdir# -/^sharedlibdir *=/s#=.*#=$sharedlibdir# -/^includedir *=/s#=.*#=$includedir# -/^mandir *=/s#=.*#=$mandir# -/^LDFLAGS *=/s#=.*#=$LDFLAGS# -" | sed -e " -s/\@VERSION\@/$VER/g; -" > zlib.pc - -# done -leave 0 diff --git a/contrib/libzlib-ng/crc32.c b/contrib/libzlib-ng/crc32.c deleted file mode 100644 index 937f48d211e..00000000000 --- a/contrib/libzlib-ng/crc32.c +++ /dev/null @@ -1,458 +0,0 @@ -/* crc32.c -- compute the CRC-32 of a data stream - * Copyright (C) 1995-2006, 2010, 2011, 2012 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - * - * Thanks to Rodney Brown for his contribution of faster - * CRC methods: exclusive-oring 32 bits of data at a time, and pre-computing - * tables for updating the shift register in one step with three exclusive-ors - * instead of four steps with four exclusive-ors. This results in about a - * factor of two increase in speed on a Power PC G4 (PPC7455) using gcc -O3. - */ - -/* @(#) $Id$ */ - -#ifdef __MINGW32__ -# include -#elif defined(WIN32) || defined(_WIN32) -# define LITTLE_ENDIAN 1234 -# define BIG_ENDIAN 4321 -# if defined(_M_IX86) || defined(_M_AMD64) || defined(_M_IA64) -# define BYTE_ORDER LITTLE_ENDIAN -# else -# error Unknown endianness! -# endif -#elif __APPLE__ -# include -#elif defined(__FreeBSD__) || defined(__NetBSD__) || defined(__OpenBSD__) || defined(__bsdi__) || defined(__DragonFly__) -# include -#elif defined(__sun) || defined(sun) -# include -# if !defined(LITTLE_ENDIAN) -# define LITTLE_ENDIAN 4321 -# endif -# if !defined(BIG_ENDIAN) -# define BIG_ENDIAN 1234 -# endif -# if !defined(BYTE_ORDER) -# if defined(_BIG_ENDIAN) -# define BYTE_ORDER BIG_ENDIAN -# else -# define BYTE_ORDER LITTLE_ENDIAN -# endif -# endif -#else -# include -#endif - -/* - Note on the use of DYNAMIC_CRC_TABLE: there is no mutex or semaphore - protection on the static variables used to control the first-use generation - of the crc tables. Therefore, if you #define DYNAMIC_CRC_TABLE, you should - first call get_crc_table() to initialize the tables before allowing more than - one thread to use crc32(). - - DYNAMIC_CRC_TABLE and MAKECRCH can be #defined to write out crc32.h. - */ - -#ifdef MAKECRCH -# include -# ifndef DYNAMIC_CRC_TABLE -# define DYNAMIC_CRC_TABLE -# endif /* !DYNAMIC_CRC_TABLE */ -#endif /* MAKECRCH */ - -#include "deflate.h" - -#if BYTE_ORDER == LITTLE_ENDIAN -static uint32_t crc32_little(uint32_t, const unsigned char *, z_off64_t); -#elif BYTE_ORDER == BIG_ENDIAN -static uint32_t crc32_big(uint32_t, const unsigned char *, z_off64_t); -#endif - -/* Local functions for crc concatenation */ -static uint32_t gf2_matrix_times(uint32_t *mat, uint32_t vec); -static void gf2_matrix_square(uint32_t *square, uint32_t *mat); -static uint32_t crc32_combine_(uint32_t crc1, uint32_t crc2, z_off64_t len2); - - -#ifdef DYNAMIC_CRC_TABLE -static volatile int crc_table_empty = 1; -static uint32_t crc_table[8][256]; -static void make_crc_table(void); -#ifdef MAKECRCH -static void write_table(FILE *, const uint32_t *); -#endif /* MAKECRCH */ -/* - Generate tables for a byte-wise 32-bit CRC calculation on the polynomial: - x^32+x^26+x^23+x^22+x^16+x^12+x^11+x^10+x^8+x^7+x^5+x^4+x^2+x+1. - - Polynomials over GF(2) are represented in binary, one bit per coefficient, - with the lowest powers in the most significant bit. Then adding polynomials - is just exclusive-or, and multiplying a polynomial by x is a right shift by - one. If we call the above polynomial p, and represent a byte as the - polynomial q, also with the lowest power in the most significant bit (so the - byte 0xb1 is the polynomial x^7+x^3+x+1), then the CRC is (q*x^32) mod p, - where a mod b means the remainder after dividing a by b. - - This calculation is done using the shift-register method of multiplying and - taking the remainder. The register is initialized to zero, and for each - incoming bit, x^32 is added mod p to the register if the bit is a one (where - x^32 mod p is p+x^32 = x^26+...+1), and the register is multiplied mod p by - x (which is shifting right by one and adding x^32 mod p if the bit shifted - out is a one). We start with the highest power (least significant bit) of - q and repeat for all eight bits of q. - - The first table is simply the CRC of all possible eight bit values. This is - all the information needed to generate CRCs on data a byte at a time for all - combinations of CRC register values and incoming bytes. The remaining tables - allow for word-at-a-time CRC calculation for both big-endian and little- - endian machines, where a word is four bytes. -*/ -static void make_crc_table() { - uint32_t c; - int n, k; - uint32_t poly; /* polynomial exclusive-or pattern */ - /* terms of polynomial defining this crc (except x^32): */ - static volatile int first = 1; /* flag to limit concurrent making */ - static const unsigned char p[] = {0, 1, 2, 4, 5, 7, 8, 10, 11, 12, 16, 22, 23, 26}; - - /* See if another task is already doing this (not thread-safe, but better - than nothing -- significantly reduces duration of vulnerability in - case the advice about DYNAMIC_CRC_TABLE is ignored) */ - if (first) { - first = 0; - - /* make exclusive-or pattern from polynomial (0xedb88320) */ - poly = 0; - for (n = 0; n < (int)(sizeof(p)/sizeof(unsigned char)); n++) - poly |= (uint32_t)1 << (31 - p[n]); - - /* generate a crc for every 8-bit value */ - for (n = 0; n < 256; n++) { - c = (uint32_t)n; - for (k = 0; k < 8; k++) - c = c & 1 ? poly ^ (c >> 1) : c >> 1; - crc_table[0][n] = c; - } - - /* generate crc for each value followed by one, two, and three zeros, - and then the byte reversal of those as well as the first table */ - for (n = 0; n < 256; n++) { - c = crc_table[0][n]; - crc_table[4][n] = ZSWAP32(c); - for (k = 1; k < 4; k++) { - c = crc_table[0][c & 0xff] ^ (c >> 8); - crc_table[k][n] = c; - crc_table[k + 4][n] = ZSWAP32(c); - } - } - - crc_table_empty = 0; - } else { /* not first */ - /* wait for the other guy to finish (not efficient, but rare) */ - while (crc_table_empty) - {} - } - -#ifdef MAKECRCH - /* write out CRC tables to crc32.h */ - { - FILE *out; - - out = fopen("crc32.h", "w"); - if (out == NULL) return; - fprintf(out, "/* crc32.h -- tables for rapid CRC calculation\n"); - fprintf(out, " * Generated automatically by crc32.c\n */\n\n"); - fprintf(out, "static const uint32_t "); - fprintf(out, "crc_table[8][256] =\n{\n {\n"); - write_table(out, crc_table[0]); - for (k = 1; k < 8; k++) { - fprintf(out, " },\n {\n"); - write_table(out, crc_table[k]); - } - fprintf(out, " }\n};\n"); - fclose(out); - } -#endif /* MAKECRCH */ -} - -#ifdef MAKECRCH -static void write_table(FILE *out, const uint32_t *table) { - int n; - - for (n = 0; n < 256; n++) - fprintf(out, "%s0x%08lx%s", n % 5 ? "" : " ", - (uint32_t)(table[n]), - n == 255 ? "\n" : (n % 5 == 4 ? ",\n" : ", ")); -} -#endif /* MAKECRCH */ - -#else /* !DYNAMIC_CRC_TABLE */ -/* ======================================================================== - * Tables of CRC-32s of all single-byte values, made by make_crc_table(). - */ -#include "crc32.h" -#endif /* DYNAMIC_CRC_TABLE */ - -/* ========================================================================= - * This function can be used by asm versions of crc32() - */ -const uint32_t * ZEXPORT get_crc_table(void) { -#ifdef DYNAMIC_CRC_TABLE - if (crc_table_empty) - make_crc_table(); -#endif /* DYNAMIC_CRC_TABLE */ - return (const uint32_t *)crc_table; -} - -/* ========================================================================= */ -#define DO1 crc = crc_table[0][((int)crc ^ (*buf++)) & 0xff] ^ (crc >> 8) -#define DO8 DO1; DO1; DO1; DO1; DO1; DO1; DO1; DO1 -#define DO4 DO1; DO1; DO1; DO1 - -/* ========================================================================= */ -uint32_t ZEXPORT crc32(uint32_t crc, const unsigned char *buf, z_off64_t len) { - if (buf == Z_NULL) return 0; - -#ifdef DYNAMIC_CRC_TABLE - if (crc_table_empty) - make_crc_table(); -#endif /* DYNAMIC_CRC_TABLE */ - - if (sizeof(void *) == sizeof(ptrdiff_t)) { -#if BYTE_ORDER == LITTLE_ENDIAN - return crc32_little(crc, buf, len); -#elif BYTE_ORDER == BIG_ENDIAN - return crc32_big(crc, buf, len); -#endif - } - crc = crc ^ 0xffffffff; - -#ifdef UNROLL_LESS - while (len >= 4) { - DO4; - len -= 4; - } -#else - while (len >= 8) { - DO8; - len -= 8; - } -#endif - - if (len) do { - DO1; - } while (--len); - return crc ^ 0xffffffff; -} - - -/* ========================================================================= */ -#if BYTE_ORDER == LITTLE_ENDIAN -#define DOLIT4 c ^= *buf4++; \ - c = crc_table[3][c & 0xff] ^ crc_table[2][(c >> 8) & 0xff] ^ \ - crc_table[1][(c >> 16) & 0xff] ^ crc_table[0][c >> 24] -#define DOLIT32 DOLIT4; DOLIT4; DOLIT4; DOLIT4; DOLIT4; DOLIT4; DOLIT4; DOLIT4 - -/* ========================================================================= */ -static uint32_t crc32_little(uint32_t crc, const unsigned char *buf, z_off64_t len) { - register uint32_t c; - register const uint32_t *buf4; - - c = crc; - c = ~c; - while (len && ((ptrdiff_t)buf & 3)) { - c = crc_table[0][(c ^ *buf++) & 0xff] ^ (c >> 8); - len--; - } - - buf4 = (const uint32_t *)(const void *)buf; - -#ifndef UNROLL_LESS - while (len >= 32) { - DOLIT32; - len -= 32; - } -#endif - - while (len >= 4) { - DOLIT4; - len -= 4; - } - buf = (const unsigned char *)buf4; - - if (len) do { - c = crc_table[0][(c ^ *buf++) & 0xff] ^ (c >> 8); - } while (--len); - c = ~c; - return c; -} -#endif /* BYTE_ORDER == LITTLE_ENDIAN */ - -/* ========================================================================= */ -#if BYTE_ORDER == BIG_ENDIAN -#define DOBIG4 c ^= *++buf4; \ - c = crc_table[4][c & 0xff] ^ crc_table[5][(c >> 8) & 0xff] ^ \ - crc_table[6][(c >> 16) & 0xff] ^ crc_table[7][c >> 24] -#define DOBIG32 DOBIG4; DOBIG4; DOBIG4; DOBIG4; DOBIG4; DOBIG4; DOBIG4; DOBIG4 - -/* ========================================================================= */ -static uint32_t crc32_big(uint32_t crc, const unsigned char *buf, z_off64_t len) { - register uint32_t c; - register const uint32_t *buf4; - - c = ZSWAP32(crc); - c = ~c; - while (len && ((ptrdiff_t)buf & 3)) { - c = crc_table[4][(c >> 24) ^ *buf++] ^ (c << 8); - len--; - } - - buf4 = (const uint32_t *)(const void *)buf; - buf4--; - -#ifndef UNROLL_LESS - while (len >= 32) { - DOBIG32; - len -= 32; - } -#endif - - while (len >= 4) { - DOBIG4; - len -= 4; - } - buf4++; - buf = (const unsigned char *)buf4; - - if (len) do { - c = crc_table[4][(c >> 24) ^ *buf++] ^ (c << 8); - } while (--len); - c = ~c; - return ZSWAP32(c); -} -#endif /* BYTE_ORDER == BIG_ENDIAN */ - - -#define GF2_DIM 32 /* dimension of GF(2) vectors (length of CRC) */ - -/* ========================================================================= */ -static uint32_t gf2_matrix_times(uint32_t *mat, uint32_t vec) { - uint32_t sum; - - sum = 0; - while (vec) { - if (vec & 1) - sum ^= *mat; - vec >>= 1; - mat++; - } - return sum; -} - -/* ========================================================================= */ -static void gf2_matrix_square(uint32_t *square, uint32_t *mat) { - int n; - - for (n = 0; n < GF2_DIM; n++) - square[n] = gf2_matrix_times(mat, mat[n]); -} - -/* ========================================================================= */ -static uint32_t crc32_combine_(uint32_t crc1, uint32_t crc2, z_off64_t len2) { - int n; - uint32_t row; - uint32_t even[GF2_DIM]; /* even-power-of-two zeros operator */ - uint32_t odd[GF2_DIM]; /* odd-power-of-two zeros operator */ - - /* degenerate case (also disallow negative lengths) */ - if (len2 <= 0) - return crc1; - - /* put operator for one zero bit in odd */ - odd[0] = 0xedb88320; /* CRC-32 polynomial */ - row = 1; - for (n = 1; n < GF2_DIM; n++) { - odd[n] = row; - row <<= 1; - } - - /* put operator for two zero bits in even */ - gf2_matrix_square(even, odd); - - /* put operator for four zero bits in odd */ - gf2_matrix_square(odd, even); - - /* apply len2 zeros to crc1 (first square will put the operator for one - zero byte, eight zero bits, in even) */ - do { - /* apply zeros operator for this bit of len2 */ - gf2_matrix_square(even, odd); - if (len2 & 1) - crc1 = gf2_matrix_times(even, crc1); - len2 >>= 1; - - /* if no more bits set, then done */ - if (len2 == 0) - break; - - /* another iteration of the loop with odd and even swapped */ - gf2_matrix_square(odd, even); - if (len2 & 1) - crc1 = gf2_matrix_times(odd, crc1); - len2 >>= 1; - - /* if no more bits set, then done */ - } while (len2 != 0); - - /* return combined crc */ - crc1 ^= crc2; - return crc1; -} - -/* ========================================================================= */ -uint32_t ZEXPORT crc32_combine(uint32_t crc1, uint32_t crc2, z_off_t len2) { - return crc32_combine_(crc1, crc2, len2); -} - -uint32_t ZEXPORT crc32_combine64(uint32_t crc1, uint32_t crc2, z_off64_t len2) { - return crc32_combine_(crc1, crc2, len2); -} - - -#ifdef X86_PCLMULQDQ_CRC -#include "arch/x86/x86.h" -extern void ZLIB_INTERNAL crc_fold_init(deflate_state *const s); -extern void ZLIB_INTERNAL crc_fold_copy(deflate_state *const s, - unsigned char *dst, const unsigned char *src, long len); -extern uint32_t ZLIB_INTERNAL crc_fold_512to32(deflate_state *const s); -#endif - -ZLIB_INTERNAL void crc_reset(deflate_state *const s) { -#ifdef X86_PCLMULQDQ_CRC - if (x86_cpu_has_pclmulqdq) { - crc_fold_init(s); - return; - } -#endif - s->strm->adler = crc32(0L, Z_NULL, 0); -} - -ZLIB_INTERNAL void crc_finalize(deflate_state *const s) { -#ifdef X86_PCLMULQDQ_CRC - if (x86_cpu_has_pclmulqdq) - s->strm->adler = crc_fold_512to32(s); -#endif -} - -ZLIB_INTERNAL void copy_with_crc(z_stream *strm, unsigned char *dst, long size) { -#ifdef X86_PCLMULQDQ_CRC - if (x86_cpu_has_pclmulqdq) { - crc_fold_copy(strm->state, dst, strm->next_in, size); - return; - } -#endif - memcpy(dst, strm->next_in, size); - strm->adler = crc32(strm->adler, dst, size); -} - diff --git a/contrib/libzlib-ng/crc32.h b/contrib/libzlib-ng/crc32.h deleted file mode 100644 index d194d10b5ea..00000000000 --- a/contrib/libzlib-ng/crc32.h +++ /dev/null @@ -1,444 +0,0 @@ -#ifndef CRC32_H_ -#define CRC32_H_ - -/* crc32.h -- tables for rapid CRC calculation - * Generated automatically by crc32.c - */ - -static const uint32_t crc_table[8][256] = -{ - { - 0x00000000, 0x77073096, 0xee0e612c, 0x990951ba, 0x076dc419, - 0x706af48f, 0xe963a535, 0x9e6495a3, 0x0edb8832, 0x79dcb8a4, - 0xe0d5e91e, 0x97d2d988, 0x09b64c2b, 0x7eb17cbd, 0xe7b82d07, - 0x90bf1d91, 0x1db71064, 0x6ab020f2, 0xf3b97148, 0x84be41de, - 0x1adad47d, 0x6ddde4eb, 0xf4d4b551, 0x83d385c7, 0x136c9856, - 0x646ba8c0, 0xfd62f97a, 0x8a65c9ec, 0x14015c4f, 0x63066cd9, - 0xfa0f3d63, 0x8d080df5, 0x3b6e20c8, 0x4c69105e, 0xd56041e4, - 0xa2677172, 0x3c03e4d1, 0x4b04d447, 0xd20d85fd, 0xa50ab56b, - 0x35b5a8fa, 0x42b2986c, 0xdbbbc9d6, 0xacbcf940, 0x32d86ce3, - 0x45df5c75, 0xdcd60dcf, 0xabd13d59, 0x26d930ac, 0x51de003a, - 0xc8d75180, 0xbfd06116, 0x21b4f4b5, 0x56b3c423, 0xcfba9599, - 0xb8bda50f, 0x2802b89e, 0x5f058808, 0xc60cd9b2, 0xb10be924, - 0x2f6f7c87, 0x58684c11, 0xc1611dab, 0xb6662d3d, 0x76dc4190, - 0x01db7106, 0x98d220bc, 0xefd5102a, 0x71b18589, 0x06b6b51f, - 0x9fbfe4a5, 0xe8b8d433, 0x7807c9a2, 0x0f00f934, 0x9609a88e, - 0xe10e9818, 0x7f6a0dbb, 0x086d3d2d, 0x91646c97, 0xe6635c01, - 0x6b6b51f4, 0x1c6c6162, 0x856530d8, 0xf262004e, 0x6c0695ed, - 0x1b01a57b, 0x8208f4c1, 0xf50fc457, 0x65b0d9c6, 0x12b7e950, - 0x8bbeb8ea, 0xfcb9887c, 0x62dd1ddf, 0x15da2d49, 0x8cd37cf3, - 0xfbd44c65, 0x4db26158, 0x3ab551ce, 0xa3bc0074, 0xd4bb30e2, - 0x4adfa541, 0x3dd895d7, 0xa4d1c46d, 0xd3d6f4fb, 0x4369e96a, - 0x346ed9fc, 0xad678846, 0xda60b8d0, 0x44042d73, 0x33031de5, - 0xaa0a4c5f, 0xdd0d7cc9, 0x5005713c, 0x270241aa, 0xbe0b1010, - 0xc90c2086, 0x5768b525, 0x206f85b3, 0xb966d409, 0xce61e49f, - 0x5edef90e, 0x29d9c998, 0xb0d09822, 0xc7d7a8b4, 0x59b33d17, - 0x2eb40d81, 0xb7bd5c3b, 0xc0ba6cad, 0xedb88320, 0x9abfb3b6, - 0x03b6e20c, 0x74b1d29a, 0xead54739, 0x9dd277af, 0x04db2615, - 0x73dc1683, 0xe3630b12, 0x94643b84, 0x0d6d6a3e, 0x7a6a5aa8, - 0xe40ecf0b, 0x9309ff9d, 0x0a00ae27, 0x7d079eb1, 0xf00f9344, - 0x8708a3d2, 0x1e01f268, 0x6906c2fe, 0xf762575d, 0x806567cb, - 0x196c3671, 0x6e6b06e7, 0xfed41b76, 0x89d32be0, 0x10da7a5a, - 0x67dd4acc, 0xf9b9df6f, 0x8ebeeff9, 0x17b7be43, 0x60b08ed5, - 0xd6d6a3e8, 0xa1d1937e, 0x38d8c2c4, 0x4fdff252, 0xd1bb67f1, - 0xa6bc5767, 0x3fb506dd, 0x48b2364b, 0xd80d2bda, 0xaf0a1b4c, - 0x36034af6, 0x41047a60, 0xdf60efc3, 0xa867df55, 0x316e8eef, - 0x4669be79, 0xcb61b38c, 0xbc66831a, 0x256fd2a0, 0x5268e236, - 0xcc0c7795, 0xbb0b4703, 0x220216b9, 0x5505262f, 0xc5ba3bbe, - 0xb2bd0b28, 0x2bb45a92, 0x5cb36a04, 0xc2d7ffa7, 0xb5d0cf31, - 0x2cd99e8b, 0x5bdeae1d, 0x9b64c2b0, 0xec63f226, 0x756aa39c, - 0x026d930a, 0x9c0906a9, 0xeb0e363f, 0x72076785, 0x05005713, - 0x95bf4a82, 0xe2b87a14, 0x7bb12bae, 0x0cb61b38, 0x92d28e9b, - 0xe5d5be0d, 0x7cdcefb7, 0x0bdbdf21, 0x86d3d2d4, 0xf1d4e242, - 0x68ddb3f8, 0x1fda836e, 0x81be16cd, 0xf6b9265b, 0x6fb077e1, - 0x18b74777, 0x88085ae6, 0xff0f6a70, 0x66063bca, 0x11010b5c, - 0x8f659eff, 0xf862ae69, 0x616bffd3, 0x166ccf45, 0xa00ae278, - 0xd70dd2ee, 0x4e048354, 0x3903b3c2, 0xa7672661, 0xd06016f7, - 0x4969474d, 0x3e6e77db, 0xaed16a4a, 0xd9d65adc, 0x40df0b66, - 0x37d83bf0, 0xa9bcae53, 0xdebb9ec5, 0x47b2cf7f, 0x30b5ffe9, - 0xbdbdf21c, 0xcabac28a, 0x53b39330, 0x24b4a3a6, 0xbad03605, - 0xcdd70693, 0x54de5729, 0x23d967bf, 0xb3667a2e, 0xc4614ab8, - 0x5d681b02, 0x2a6f2b94, 0xb40bbe37, 0xc30c8ea1, 0x5a05df1b, - 0x2d02ef8d - }, - { - 0x00000000, 0x191b3141, 0x32366282, 0x2b2d53c3, 0x646cc504, - 0x7d77f445, 0x565aa786, 0x4f4196c7, 0xc8d98a08, 0xd1c2bb49, - 0xfaefe88a, 0xe3f4d9cb, 0xacb54f0c, 0xb5ae7e4d, 0x9e832d8e, - 0x87981ccf, 0x4ac21251, 0x53d92310, 0x78f470d3, 0x61ef4192, - 0x2eaed755, 0x37b5e614, 0x1c98b5d7, 0x05838496, 0x821b9859, - 0x9b00a918, 0xb02dfadb, 0xa936cb9a, 0xe6775d5d, 0xff6c6c1c, - 0xd4413fdf, 0xcd5a0e9e, 0x958424a2, 0x8c9f15e3, 0xa7b24620, - 0xbea97761, 0xf1e8e1a6, 0xe8f3d0e7, 0xc3de8324, 0xdac5b265, - 0x5d5daeaa, 0x44469feb, 0x6f6bcc28, 0x7670fd69, 0x39316bae, - 0x202a5aef, 0x0b07092c, 0x121c386d, 0xdf4636f3, 0xc65d07b2, - 0xed705471, 0xf46b6530, 0xbb2af3f7, 0xa231c2b6, 0x891c9175, - 0x9007a034, 0x179fbcfb, 0x0e848dba, 0x25a9de79, 0x3cb2ef38, - 0x73f379ff, 0x6ae848be, 0x41c51b7d, 0x58de2a3c, 0xf0794f05, - 0xe9627e44, 0xc24f2d87, 0xdb541cc6, 0x94158a01, 0x8d0ebb40, - 0xa623e883, 0xbf38d9c2, 0x38a0c50d, 0x21bbf44c, 0x0a96a78f, - 0x138d96ce, 0x5ccc0009, 0x45d73148, 0x6efa628b, 0x77e153ca, - 0xbabb5d54, 0xa3a06c15, 0x888d3fd6, 0x91960e97, 0xded79850, - 0xc7cca911, 0xece1fad2, 0xf5facb93, 0x7262d75c, 0x6b79e61d, - 0x4054b5de, 0x594f849f, 0x160e1258, 0x0f152319, 0x243870da, - 0x3d23419b, 0x65fd6ba7, 0x7ce65ae6, 0x57cb0925, 0x4ed03864, - 0x0191aea3, 0x188a9fe2, 0x33a7cc21, 0x2abcfd60, 0xad24e1af, - 0xb43fd0ee, 0x9f12832d, 0x8609b26c, 0xc94824ab, 0xd05315ea, - 0xfb7e4629, 0xe2657768, 0x2f3f79f6, 0x362448b7, 0x1d091b74, - 0x04122a35, 0x4b53bcf2, 0x52488db3, 0x7965de70, 0x607eef31, - 0xe7e6f3fe, 0xfefdc2bf, 0xd5d0917c, 0xcccba03d, 0x838a36fa, - 0x9a9107bb, 0xb1bc5478, 0xa8a76539, 0x3b83984b, 0x2298a90a, - 0x09b5fac9, 0x10aecb88, 0x5fef5d4f, 0x46f46c0e, 0x6dd93fcd, - 0x74c20e8c, 0xf35a1243, 0xea412302, 0xc16c70c1, 0xd8774180, - 0x9736d747, 0x8e2de606, 0xa500b5c5, 0xbc1b8484, 0x71418a1a, - 0x685abb5b, 0x4377e898, 0x5a6cd9d9, 0x152d4f1e, 0x0c367e5f, - 0x271b2d9c, 0x3e001cdd, 0xb9980012, 0xa0833153, 0x8bae6290, - 0x92b553d1, 0xddf4c516, 0xc4eff457, 0xefc2a794, 0xf6d996d5, - 0xae07bce9, 0xb71c8da8, 0x9c31de6b, 0x852aef2a, 0xca6b79ed, - 0xd37048ac, 0xf85d1b6f, 0xe1462a2e, 0x66de36e1, 0x7fc507a0, - 0x54e85463, 0x4df36522, 0x02b2f3e5, 0x1ba9c2a4, 0x30849167, - 0x299fa026, 0xe4c5aeb8, 0xfdde9ff9, 0xd6f3cc3a, 0xcfe8fd7b, - 0x80a96bbc, 0x99b25afd, 0xb29f093e, 0xab84387f, 0x2c1c24b0, - 0x350715f1, 0x1e2a4632, 0x07317773, 0x4870e1b4, 0x516bd0f5, - 0x7a468336, 0x635db277, 0xcbfad74e, 0xd2e1e60f, 0xf9ccb5cc, - 0xe0d7848d, 0xaf96124a, 0xb68d230b, 0x9da070c8, 0x84bb4189, - 0x03235d46, 0x1a386c07, 0x31153fc4, 0x280e0e85, 0x674f9842, - 0x7e54a903, 0x5579fac0, 0x4c62cb81, 0x8138c51f, 0x9823f45e, - 0xb30ea79d, 0xaa1596dc, 0xe554001b, 0xfc4f315a, 0xd7626299, - 0xce7953d8, 0x49e14f17, 0x50fa7e56, 0x7bd72d95, 0x62cc1cd4, - 0x2d8d8a13, 0x3496bb52, 0x1fbbe891, 0x06a0d9d0, 0x5e7ef3ec, - 0x4765c2ad, 0x6c48916e, 0x7553a02f, 0x3a1236e8, 0x230907a9, - 0x0824546a, 0x113f652b, 0x96a779e4, 0x8fbc48a5, 0xa4911b66, - 0xbd8a2a27, 0xf2cbbce0, 0xebd08da1, 0xc0fdde62, 0xd9e6ef23, - 0x14bce1bd, 0x0da7d0fc, 0x268a833f, 0x3f91b27e, 0x70d024b9, - 0x69cb15f8, 0x42e6463b, 0x5bfd777a, 0xdc656bb5, 0xc57e5af4, - 0xee530937, 0xf7483876, 0xb809aeb1, 0xa1129ff0, 0x8a3fcc33, - 0x9324fd72 - }, - { - 0x00000000, 0x01c26a37, 0x0384d46e, 0x0246be59, 0x0709a8dc, - 0x06cbc2eb, 0x048d7cb2, 0x054f1685, 0x0e1351b8, 0x0fd13b8f, - 0x0d9785d6, 0x0c55efe1, 0x091af964, 0x08d89353, 0x0a9e2d0a, - 0x0b5c473d, 0x1c26a370, 0x1de4c947, 0x1fa2771e, 0x1e601d29, - 0x1b2f0bac, 0x1aed619b, 0x18abdfc2, 0x1969b5f5, 0x1235f2c8, - 0x13f798ff, 0x11b126a6, 0x10734c91, 0x153c5a14, 0x14fe3023, - 0x16b88e7a, 0x177ae44d, 0x384d46e0, 0x398f2cd7, 0x3bc9928e, - 0x3a0bf8b9, 0x3f44ee3c, 0x3e86840b, 0x3cc03a52, 0x3d025065, - 0x365e1758, 0x379c7d6f, 0x35dac336, 0x3418a901, 0x3157bf84, - 0x3095d5b3, 0x32d36bea, 0x331101dd, 0x246be590, 0x25a98fa7, - 0x27ef31fe, 0x262d5bc9, 0x23624d4c, 0x22a0277b, 0x20e69922, - 0x2124f315, 0x2a78b428, 0x2bbade1f, 0x29fc6046, 0x283e0a71, - 0x2d711cf4, 0x2cb376c3, 0x2ef5c89a, 0x2f37a2ad, 0x709a8dc0, - 0x7158e7f7, 0x731e59ae, 0x72dc3399, 0x7793251c, 0x76514f2b, - 0x7417f172, 0x75d59b45, 0x7e89dc78, 0x7f4bb64f, 0x7d0d0816, - 0x7ccf6221, 0x798074a4, 0x78421e93, 0x7a04a0ca, 0x7bc6cafd, - 0x6cbc2eb0, 0x6d7e4487, 0x6f38fade, 0x6efa90e9, 0x6bb5866c, - 0x6a77ec5b, 0x68315202, 0x69f33835, 0x62af7f08, 0x636d153f, - 0x612bab66, 0x60e9c151, 0x65a6d7d4, 0x6464bde3, 0x662203ba, - 0x67e0698d, 0x48d7cb20, 0x4915a117, 0x4b531f4e, 0x4a917579, - 0x4fde63fc, 0x4e1c09cb, 0x4c5ab792, 0x4d98dda5, 0x46c49a98, - 0x4706f0af, 0x45404ef6, 0x448224c1, 0x41cd3244, 0x400f5873, - 0x4249e62a, 0x438b8c1d, 0x54f16850, 0x55330267, 0x5775bc3e, - 0x56b7d609, 0x53f8c08c, 0x523aaabb, 0x507c14e2, 0x51be7ed5, - 0x5ae239e8, 0x5b2053df, 0x5966ed86, 0x58a487b1, 0x5deb9134, - 0x5c29fb03, 0x5e6f455a, 0x5fad2f6d, 0xe1351b80, 0xe0f771b7, - 0xe2b1cfee, 0xe373a5d9, 0xe63cb35c, 0xe7fed96b, 0xe5b86732, - 0xe47a0d05, 0xef264a38, 0xeee4200f, 0xeca29e56, 0xed60f461, - 0xe82fe2e4, 0xe9ed88d3, 0xebab368a, 0xea695cbd, 0xfd13b8f0, - 0xfcd1d2c7, 0xfe976c9e, 0xff5506a9, 0xfa1a102c, 0xfbd87a1b, - 0xf99ec442, 0xf85cae75, 0xf300e948, 0xf2c2837f, 0xf0843d26, - 0xf1465711, 0xf4094194, 0xf5cb2ba3, 0xf78d95fa, 0xf64fffcd, - 0xd9785d60, 0xd8ba3757, 0xdafc890e, 0xdb3ee339, 0xde71f5bc, - 0xdfb39f8b, 0xddf521d2, 0xdc374be5, 0xd76b0cd8, 0xd6a966ef, - 0xd4efd8b6, 0xd52db281, 0xd062a404, 0xd1a0ce33, 0xd3e6706a, - 0xd2241a5d, 0xc55efe10, 0xc49c9427, 0xc6da2a7e, 0xc7184049, - 0xc25756cc, 0xc3953cfb, 0xc1d382a2, 0xc011e895, 0xcb4dafa8, - 0xca8fc59f, 0xc8c97bc6, 0xc90b11f1, 0xcc440774, 0xcd866d43, - 0xcfc0d31a, 0xce02b92d, 0x91af9640, 0x906dfc77, 0x922b422e, - 0x93e92819, 0x96a63e9c, 0x976454ab, 0x9522eaf2, 0x94e080c5, - 0x9fbcc7f8, 0x9e7eadcf, 0x9c381396, 0x9dfa79a1, 0x98b56f24, - 0x99770513, 0x9b31bb4a, 0x9af3d17d, 0x8d893530, 0x8c4b5f07, - 0x8e0de15e, 0x8fcf8b69, 0x8a809dec, 0x8b42f7db, 0x89044982, - 0x88c623b5, 0x839a6488, 0x82580ebf, 0x801eb0e6, 0x81dcdad1, - 0x8493cc54, 0x8551a663, 0x8717183a, 0x86d5720d, 0xa9e2d0a0, - 0xa820ba97, 0xaa6604ce, 0xaba46ef9, 0xaeeb787c, 0xaf29124b, - 0xad6fac12, 0xacadc625, 0xa7f18118, 0xa633eb2f, 0xa4755576, - 0xa5b73f41, 0xa0f829c4, 0xa13a43f3, 0xa37cfdaa, 0xa2be979d, - 0xb5c473d0, 0xb40619e7, 0xb640a7be, 0xb782cd89, 0xb2cddb0c, - 0xb30fb13b, 0xb1490f62, 0xb08b6555, 0xbbd72268, 0xba15485f, - 0xb853f606, 0xb9919c31, 0xbcde8ab4, 0xbd1ce083, 0xbf5a5eda, - 0xbe9834ed - }, - { - 0x00000000, 0xb8bc6765, 0xaa09c88b, 0x12b5afee, 0x8f629757, - 0x37def032, 0x256b5fdc, 0x9dd738b9, 0xc5b428ef, 0x7d084f8a, - 0x6fbde064, 0xd7018701, 0x4ad6bfb8, 0xf26ad8dd, 0xe0df7733, - 0x58631056, 0x5019579f, 0xe8a530fa, 0xfa109f14, 0x42acf871, - 0xdf7bc0c8, 0x67c7a7ad, 0x75720843, 0xcdce6f26, 0x95ad7f70, - 0x2d111815, 0x3fa4b7fb, 0x8718d09e, 0x1acfe827, 0xa2738f42, - 0xb0c620ac, 0x087a47c9, 0xa032af3e, 0x188ec85b, 0x0a3b67b5, - 0xb28700d0, 0x2f503869, 0x97ec5f0c, 0x8559f0e2, 0x3de59787, - 0x658687d1, 0xdd3ae0b4, 0xcf8f4f5a, 0x7733283f, 0xeae41086, - 0x525877e3, 0x40edd80d, 0xf851bf68, 0xf02bf8a1, 0x48979fc4, - 0x5a22302a, 0xe29e574f, 0x7f496ff6, 0xc7f50893, 0xd540a77d, - 0x6dfcc018, 0x359fd04e, 0x8d23b72b, 0x9f9618c5, 0x272a7fa0, - 0xbafd4719, 0x0241207c, 0x10f48f92, 0xa848e8f7, 0x9b14583d, - 0x23a83f58, 0x311d90b6, 0x89a1f7d3, 0x1476cf6a, 0xaccaa80f, - 0xbe7f07e1, 0x06c36084, 0x5ea070d2, 0xe61c17b7, 0xf4a9b859, - 0x4c15df3c, 0xd1c2e785, 0x697e80e0, 0x7bcb2f0e, 0xc377486b, - 0xcb0d0fa2, 0x73b168c7, 0x6104c729, 0xd9b8a04c, 0x446f98f5, - 0xfcd3ff90, 0xee66507e, 0x56da371b, 0x0eb9274d, 0xb6054028, - 0xa4b0efc6, 0x1c0c88a3, 0x81dbb01a, 0x3967d77f, 0x2bd27891, - 0x936e1ff4, 0x3b26f703, 0x839a9066, 0x912f3f88, 0x299358ed, - 0xb4446054, 0x0cf80731, 0x1e4da8df, 0xa6f1cfba, 0xfe92dfec, - 0x462eb889, 0x549b1767, 0xec277002, 0x71f048bb, 0xc94c2fde, - 0xdbf98030, 0x6345e755, 0x6b3fa09c, 0xd383c7f9, 0xc1366817, - 0x798a0f72, 0xe45d37cb, 0x5ce150ae, 0x4e54ff40, 0xf6e89825, - 0xae8b8873, 0x1637ef16, 0x048240f8, 0xbc3e279d, 0x21e91f24, - 0x99557841, 0x8be0d7af, 0x335cb0ca, 0xed59b63b, 0x55e5d15e, - 0x47507eb0, 0xffec19d5, 0x623b216c, 0xda874609, 0xc832e9e7, - 0x708e8e82, 0x28ed9ed4, 0x9051f9b1, 0x82e4565f, 0x3a58313a, - 0xa78f0983, 0x1f336ee6, 0x0d86c108, 0xb53aa66d, 0xbd40e1a4, - 0x05fc86c1, 0x1749292f, 0xaff54e4a, 0x322276f3, 0x8a9e1196, - 0x982bbe78, 0x2097d91d, 0x78f4c94b, 0xc048ae2e, 0xd2fd01c0, - 0x6a4166a5, 0xf7965e1c, 0x4f2a3979, 0x5d9f9697, 0xe523f1f2, - 0x4d6b1905, 0xf5d77e60, 0xe762d18e, 0x5fdeb6eb, 0xc2098e52, - 0x7ab5e937, 0x680046d9, 0xd0bc21bc, 0x88df31ea, 0x3063568f, - 0x22d6f961, 0x9a6a9e04, 0x07bda6bd, 0xbf01c1d8, 0xadb46e36, - 0x15080953, 0x1d724e9a, 0xa5ce29ff, 0xb77b8611, 0x0fc7e174, - 0x9210d9cd, 0x2aacbea8, 0x38191146, 0x80a57623, 0xd8c66675, - 0x607a0110, 0x72cfaefe, 0xca73c99b, 0x57a4f122, 0xef189647, - 0xfdad39a9, 0x45115ecc, 0x764dee06, 0xcef18963, 0xdc44268d, - 0x64f841e8, 0xf92f7951, 0x41931e34, 0x5326b1da, 0xeb9ad6bf, - 0xb3f9c6e9, 0x0b45a18c, 0x19f00e62, 0xa14c6907, 0x3c9b51be, - 0x842736db, 0x96929935, 0x2e2efe50, 0x2654b999, 0x9ee8defc, - 0x8c5d7112, 0x34e11677, 0xa9362ece, 0x118a49ab, 0x033fe645, - 0xbb838120, 0xe3e09176, 0x5b5cf613, 0x49e959fd, 0xf1553e98, - 0x6c820621, 0xd43e6144, 0xc68bceaa, 0x7e37a9cf, 0xd67f4138, - 0x6ec3265d, 0x7c7689b3, 0xc4caeed6, 0x591dd66f, 0xe1a1b10a, - 0xf3141ee4, 0x4ba87981, 0x13cb69d7, 0xab770eb2, 0xb9c2a15c, - 0x017ec639, 0x9ca9fe80, 0x241599e5, 0x36a0360b, 0x8e1c516e, - 0x866616a7, 0x3eda71c2, 0x2c6fde2c, 0x94d3b949, 0x090481f0, - 0xb1b8e695, 0xa30d497b, 0x1bb12e1e, 0x43d23e48, 0xfb6e592d, - 0xe9dbf6c3, 0x516791a6, 0xccb0a91f, 0x740cce7a, 0x66b96194, - 0xde0506f1 - }, - { - 0x00000000, 0x96300777, 0x2c610eee, 0xba510999, 0x19c46d07, - 0x8ff46a70, 0x35a563e9, 0xa395649e, 0x3288db0e, 0xa4b8dc79, - 0x1ee9d5e0, 0x88d9d297, 0x2b4cb609, 0xbd7cb17e, 0x072db8e7, - 0x911dbf90, 0x6410b71d, 0xf220b06a, 0x4871b9f3, 0xde41be84, - 0x7dd4da1a, 0xebe4dd6d, 0x51b5d4f4, 0xc785d383, 0x56986c13, - 0xc0a86b64, 0x7af962fd, 0xecc9658a, 0x4f5c0114, 0xd96c0663, - 0x633d0ffa, 0xf50d088d, 0xc8206e3b, 0x5e10694c, 0xe44160d5, - 0x727167a2, 0xd1e4033c, 0x47d4044b, 0xfd850dd2, 0x6bb50aa5, - 0xfaa8b535, 0x6c98b242, 0xd6c9bbdb, 0x40f9bcac, 0xe36cd832, - 0x755cdf45, 0xcf0dd6dc, 0x593dd1ab, 0xac30d926, 0x3a00de51, - 0x8051d7c8, 0x1661d0bf, 0xb5f4b421, 0x23c4b356, 0x9995bacf, - 0x0fa5bdb8, 0x9eb80228, 0x0888055f, 0xb2d90cc6, 0x24e90bb1, - 0x877c6f2f, 0x114c6858, 0xab1d61c1, 0x3d2d66b6, 0x9041dc76, - 0x0671db01, 0xbc20d298, 0x2a10d5ef, 0x8985b171, 0x1fb5b606, - 0xa5e4bf9f, 0x33d4b8e8, 0xa2c90778, 0x34f9000f, 0x8ea80996, - 0x18980ee1, 0xbb0d6a7f, 0x2d3d6d08, 0x976c6491, 0x015c63e6, - 0xf4516b6b, 0x62616c1c, 0xd8306585, 0x4e0062f2, 0xed95066c, - 0x7ba5011b, 0xc1f40882, 0x57c40ff5, 0xc6d9b065, 0x50e9b712, - 0xeab8be8b, 0x7c88b9fc, 0xdf1ddd62, 0x492dda15, 0xf37cd38c, - 0x654cd4fb, 0x5861b24d, 0xce51b53a, 0x7400bca3, 0xe230bbd4, - 0x41a5df4a, 0xd795d83d, 0x6dc4d1a4, 0xfbf4d6d3, 0x6ae96943, - 0xfcd96e34, 0x468867ad, 0xd0b860da, 0x732d0444, 0xe51d0333, - 0x5f4c0aaa, 0xc97c0ddd, 0x3c710550, 0xaa410227, 0x10100bbe, - 0x86200cc9, 0x25b56857, 0xb3856f20, 0x09d466b9, 0x9fe461ce, - 0x0ef9de5e, 0x98c9d929, 0x2298d0b0, 0xb4a8d7c7, 0x173db359, - 0x810db42e, 0x3b5cbdb7, 0xad6cbac0, 0x2083b8ed, 0xb6b3bf9a, - 0x0ce2b603, 0x9ad2b174, 0x3947d5ea, 0xaf77d29d, 0x1526db04, - 0x8316dc73, 0x120b63e3, 0x843b6494, 0x3e6a6d0d, 0xa85a6a7a, - 0x0bcf0ee4, 0x9dff0993, 0x27ae000a, 0xb19e077d, 0x44930ff0, - 0xd2a30887, 0x68f2011e, 0xfec20669, 0x5d5762f7, 0xcb676580, - 0x71366c19, 0xe7066b6e, 0x761bd4fe, 0xe02bd389, 0x5a7ada10, - 0xcc4add67, 0x6fdfb9f9, 0xf9efbe8e, 0x43beb717, 0xd58eb060, - 0xe8a3d6d6, 0x7e93d1a1, 0xc4c2d838, 0x52f2df4f, 0xf167bbd1, - 0x6757bca6, 0xdd06b53f, 0x4b36b248, 0xda2b0dd8, 0x4c1b0aaf, - 0xf64a0336, 0x607a0441, 0xc3ef60df, 0x55df67a8, 0xef8e6e31, - 0x79be6946, 0x8cb361cb, 0x1a8366bc, 0xa0d26f25, 0x36e26852, - 0x95770ccc, 0x03470bbb, 0xb9160222, 0x2f260555, 0xbe3bbac5, - 0x280bbdb2, 0x925ab42b, 0x046ab35c, 0xa7ffd7c2, 0x31cfd0b5, - 0x8b9ed92c, 0x1daede5b, 0xb0c2649b, 0x26f263ec, 0x9ca36a75, - 0x0a936d02, 0xa906099c, 0x3f360eeb, 0x85670772, 0x13570005, - 0x824abf95, 0x147ab8e2, 0xae2bb17b, 0x381bb60c, 0x9b8ed292, - 0x0dbed5e5, 0xb7efdc7c, 0x21dfdb0b, 0xd4d2d386, 0x42e2d4f1, - 0xf8b3dd68, 0x6e83da1f, 0xcd16be81, 0x5b26b9f6, 0xe177b06f, - 0x7747b718, 0xe65a0888, 0x706a0fff, 0xca3b0666, 0x5c0b0111, - 0xff9e658f, 0x69ae62f8, 0xd3ff6b61, 0x45cf6c16, 0x78e20aa0, - 0xeed20dd7, 0x5483044e, 0xc2b30339, 0x612667a7, 0xf71660d0, - 0x4d476949, 0xdb776e3e, 0x4a6ad1ae, 0xdc5ad6d9, 0x660bdf40, - 0xf03bd837, 0x53aebca9, 0xc59ebbde, 0x7fcfb247, 0xe9ffb530, - 0x1cf2bdbd, 0x8ac2baca, 0x3093b353, 0xa6a3b424, 0x0536d0ba, - 0x9306d7cd, 0x2957de54, 0xbf67d923, 0x2e7a66b3, 0xb84a61c4, - 0x021b685d, 0x942b6f2a, 0x37be0bb4, 0xa18e0cc3, 0x1bdf055a, - 0x8def022d - }, - { - 0x00000000, 0x41311b19, 0x82623632, 0xc3532d2b, 0x04c56c64, - 0x45f4777d, 0x86a75a56, 0xc796414f, 0x088ad9c8, 0x49bbc2d1, - 0x8ae8effa, 0xcbd9f4e3, 0x0c4fb5ac, 0x4d7eaeb5, 0x8e2d839e, - 0xcf1c9887, 0x5112c24a, 0x1023d953, 0xd370f478, 0x9241ef61, - 0x55d7ae2e, 0x14e6b537, 0xd7b5981c, 0x96848305, 0x59981b82, - 0x18a9009b, 0xdbfa2db0, 0x9acb36a9, 0x5d5d77e6, 0x1c6c6cff, - 0xdf3f41d4, 0x9e0e5acd, 0xa2248495, 0xe3159f8c, 0x2046b2a7, - 0x6177a9be, 0xa6e1e8f1, 0xe7d0f3e8, 0x2483dec3, 0x65b2c5da, - 0xaaae5d5d, 0xeb9f4644, 0x28cc6b6f, 0x69fd7076, 0xae6b3139, - 0xef5a2a20, 0x2c09070b, 0x6d381c12, 0xf33646df, 0xb2075dc6, - 0x715470ed, 0x30656bf4, 0xf7f32abb, 0xb6c231a2, 0x75911c89, - 0x34a00790, 0xfbbc9f17, 0xba8d840e, 0x79dea925, 0x38efb23c, - 0xff79f373, 0xbe48e86a, 0x7d1bc541, 0x3c2ade58, 0x054f79f0, - 0x447e62e9, 0x872d4fc2, 0xc61c54db, 0x018a1594, 0x40bb0e8d, - 0x83e823a6, 0xc2d938bf, 0x0dc5a038, 0x4cf4bb21, 0x8fa7960a, - 0xce968d13, 0x0900cc5c, 0x4831d745, 0x8b62fa6e, 0xca53e177, - 0x545dbbba, 0x156ca0a3, 0xd63f8d88, 0x970e9691, 0x5098d7de, - 0x11a9ccc7, 0xd2fae1ec, 0x93cbfaf5, 0x5cd76272, 0x1de6796b, - 0xdeb55440, 0x9f844f59, 0x58120e16, 0x1923150f, 0xda703824, - 0x9b41233d, 0xa76bfd65, 0xe65ae67c, 0x2509cb57, 0x6438d04e, - 0xa3ae9101, 0xe29f8a18, 0x21cca733, 0x60fdbc2a, 0xafe124ad, - 0xeed03fb4, 0x2d83129f, 0x6cb20986, 0xab2448c9, 0xea1553d0, - 0x29467efb, 0x687765e2, 0xf6793f2f, 0xb7482436, 0x741b091d, - 0x352a1204, 0xf2bc534b, 0xb38d4852, 0x70de6579, 0x31ef7e60, - 0xfef3e6e7, 0xbfc2fdfe, 0x7c91d0d5, 0x3da0cbcc, 0xfa368a83, - 0xbb07919a, 0x7854bcb1, 0x3965a7a8, 0x4b98833b, 0x0aa99822, - 0xc9fab509, 0x88cbae10, 0x4f5def5f, 0x0e6cf446, 0xcd3fd96d, - 0x8c0ec274, 0x43125af3, 0x022341ea, 0xc1706cc1, 0x804177d8, - 0x47d73697, 0x06e62d8e, 0xc5b500a5, 0x84841bbc, 0x1a8a4171, - 0x5bbb5a68, 0x98e87743, 0xd9d96c5a, 0x1e4f2d15, 0x5f7e360c, - 0x9c2d1b27, 0xdd1c003e, 0x120098b9, 0x533183a0, 0x9062ae8b, - 0xd153b592, 0x16c5f4dd, 0x57f4efc4, 0x94a7c2ef, 0xd596d9f6, - 0xe9bc07ae, 0xa88d1cb7, 0x6bde319c, 0x2aef2a85, 0xed796bca, - 0xac4870d3, 0x6f1b5df8, 0x2e2a46e1, 0xe136de66, 0xa007c57f, - 0x6354e854, 0x2265f34d, 0xe5f3b202, 0xa4c2a91b, 0x67918430, - 0x26a09f29, 0xb8aec5e4, 0xf99fdefd, 0x3accf3d6, 0x7bfde8cf, - 0xbc6ba980, 0xfd5ab299, 0x3e099fb2, 0x7f3884ab, 0xb0241c2c, - 0xf1150735, 0x32462a1e, 0x73773107, 0xb4e17048, 0xf5d06b51, - 0x3683467a, 0x77b25d63, 0x4ed7facb, 0x0fe6e1d2, 0xccb5ccf9, - 0x8d84d7e0, 0x4a1296af, 0x0b238db6, 0xc870a09d, 0x8941bb84, - 0x465d2303, 0x076c381a, 0xc43f1531, 0x850e0e28, 0x42984f67, - 0x03a9547e, 0xc0fa7955, 0x81cb624c, 0x1fc53881, 0x5ef42398, - 0x9da70eb3, 0xdc9615aa, 0x1b0054e5, 0x5a314ffc, 0x996262d7, - 0xd85379ce, 0x174fe149, 0x567efa50, 0x952dd77b, 0xd41ccc62, - 0x138a8d2d, 0x52bb9634, 0x91e8bb1f, 0xd0d9a006, 0xecf37e5e, - 0xadc26547, 0x6e91486c, 0x2fa05375, 0xe836123a, 0xa9070923, - 0x6a542408, 0x2b653f11, 0xe479a796, 0xa548bc8f, 0x661b91a4, - 0x272a8abd, 0xe0bccbf2, 0xa18dd0eb, 0x62defdc0, 0x23efe6d9, - 0xbde1bc14, 0xfcd0a70d, 0x3f838a26, 0x7eb2913f, 0xb924d070, - 0xf815cb69, 0x3b46e642, 0x7a77fd5b, 0xb56b65dc, 0xf45a7ec5, - 0x370953ee, 0x763848f7, 0xb1ae09b8, 0xf09f12a1, 0x33cc3f8a, - 0x72fd2493 - }, - { - 0x00000000, 0x376ac201, 0x6ed48403, 0x59be4602, 0xdca80907, - 0xebc2cb06, 0xb27c8d04, 0x85164f05, 0xb851130e, 0x8f3bd10f, - 0xd685970d, 0xe1ef550c, 0x64f91a09, 0x5393d808, 0x0a2d9e0a, - 0x3d475c0b, 0x70a3261c, 0x47c9e41d, 0x1e77a21f, 0x291d601e, - 0xac0b2f1b, 0x9b61ed1a, 0xc2dfab18, 0xf5b56919, 0xc8f23512, - 0xff98f713, 0xa626b111, 0x914c7310, 0x145a3c15, 0x2330fe14, - 0x7a8eb816, 0x4de47a17, 0xe0464d38, 0xd72c8f39, 0x8e92c93b, - 0xb9f80b3a, 0x3cee443f, 0x0b84863e, 0x523ac03c, 0x6550023d, - 0x58175e36, 0x6f7d9c37, 0x36c3da35, 0x01a91834, 0x84bf5731, - 0xb3d59530, 0xea6bd332, 0xdd011133, 0x90e56b24, 0xa78fa925, - 0xfe31ef27, 0xc95b2d26, 0x4c4d6223, 0x7b27a022, 0x2299e620, - 0x15f32421, 0x28b4782a, 0x1fdeba2b, 0x4660fc29, 0x710a3e28, - 0xf41c712d, 0xc376b32c, 0x9ac8f52e, 0xada2372f, 0xc08d9a70, - 0xf7e75871, 0xae591e73, 0x9933dc72, 0x1c259377, 0x2b4f5176, - 0x72f11774, 0x459bd575, 0x78dc897e, 0x4fb64b7f, 0x16080d7d, - 0x2162cf7c, 0xa4748079, 0x931e4278, 0xcaa0047a, 0xfdcac67b, - 0xb02ebc6c, 0x87447e6d, 0xdefa386f, 0xe990fa6e, 0x6c86b56b, - 0x5bec776a, 0x02523168, 0x3538f369, 0x087faf62, 0x3f156d63, - 0x66ab2b61, 0x51c1e960, 0xd4d7a665, 0xe3bd6464, 0xba032266, - 0x8d69e067, 0x20cbd748, 0x17a11549, 0x4e1f534b, 0x7975914a, - 0xfc63de4f, 0xcb091c4e, 0x92b75a4c, 0xa5dd984d, 0x989ac446, - 0xaff00647, 0xf64e4045, 0xc1248244, 0x4432cd41, 0x73580f40, - 0x2ae64942, 0x1d8c8b43, 0x5068f154, 0x67023355, 0x3ebc7557, - 0x09d6b756, 0x8cc0f853, 0xbbaa3a52, 0xe2147c50, 0xd57ebe51, - 0xe839e25a, 0xdf53205b, 0x86ed6659, 0xb187a458, 0x3491eb5d, - 0x03fb295c, 0x5a456f5e, 0x6d2fad5f, 0x801b35e1, 0xb771f7e0, - 0xeecfb1e2, 0xd9a573e3, 0x5cb33ce6, 0x6bd9fee7, 0x3267b8e5, - 0x050d7ae4, 0x384a26ef, 0x0f20e4ee, 0x569ea2ec, 0x61f460ed, - 0xe4e22fe8, 0xd388ede9, 0x8a36abeb, 0xbd5c69ea, 0xf0b813fd, - 0xc7d2d1fc, 0x9e6c97fe, 0xa90655ff, 0x2c101afa, 0x1b7ad8fb, - 0x42c49ef9, 0x75ae5cf8, 0x48e900f3, 0x7f83c2f2, 0x263d84f0, - 0x115746f1, 0x944109f4, 0xa32bcbf5, 0xfa958df7, 0xcdff4ff6, - 0x605d78d9, 0x5737bad8, 0x0e89fcda, 0x39e33edb, 0xbcf571de, - 0x8b9fb3df, 0xd221f5dd, 0xe54b37dc, 0xd80c6bd7, 0xef66a9d6, - 0xb6d8efd4, 0x81b22dd5, 0x04a462d0, 0x33cea0d1, 0x6a70e6d3, - 0x5d1a24d2, 0x10fe5ec5, 0x27949cc4, 0x7e2adac6, 0x494018c7, - 0xcc5657c2, 0xfb3c95c3, 0xa282d3c1, 0x95e811c0, 0xa8af4dcb, - 0x9fc58fca, 0xc67bc9c8, 0xf1110bc9, 0x740744cc, 0x436d86cd, - 0x1ad3c0cf, 0x2db902ce, 0x4096af91, 0x77fc6d90, 0x2e422b92, - 0x1928e993, 0x9c3ea696, 0xab546497, 0xf2ea2295, 0xc580e094, - 0xf8c7bc9f, 0xcfad7e9e, 0x9613389c, 0xa179fa9d, 0x246fb598, - 0x13057799, 0x4abb319b, 0x7dd1f39a, 0x3035898d, 0x075f4b8c, - 0x5ee10d8e, 0x698bcf8f, 0xec9d808a, 0xdbf7428b, 0x82490489, - 0xb523c688, 0x88649a83, 0xbf0e5882, 0xe6b01e80, 0xd1dadc81, - 0x54cc9384, 0x63a65185, 0x3a181787, 0x0d72d586, 0xa0d0e2a9, - 0x97ba20a8, 0xce0466aa, 0xf96ea4ab, 0x7c78ebae, 0x4b1229af, - 0x12ac6fad, 0x25c6adac, 0x1881f1a7, 0x2feb33a6, 0x765575a4, - 0x413fb7a5, 0xc429f8a0, 0xf3433aa1, 0xaafd7ca3, 0x9d97bea2, - 0xd073c4b5, 0xe71906b4, 0xbea740b6, 0x89cd82b7, 0x0cdbcdb2, - 0x3bb10fb3, 0x620f49b1, 0x55658bb0, 0x6822d7bb, 0x5f4815ba, - 0x06f653b8, 0x319c91b9, 0xb48adebc, 0x83e01cbd, 0xda5e5abf, - 0xed3498be - }, - { - 0x00000000, 0x6567bcb8, 0x8bc809aa, 0xeeafb512, 0x5797628f, - 0x32f0de37, 0xdc5f6b25, 0xb938d79d, 0xef28b4c5, 0x8a4f087d, - 0x64e0bd6f, 0x018701d7, 0xb8bfd64a, 0xddd86af2, 0x3377dfe0, - 0x56106358, 0x9f571950, 0xfa30a5e8, 0x149f10fa, 0x71f8ac42, - 0xc8c07bdf, 0xada7c767, 0x43087275, 0x266fcecd, 0x707fad95, - 0x1518112d, 0xfbb7a43f, 0x9ed01887, 0x27e8cf1a, 0x428f73a2, - 0xac20c6b0, 0xc9477a08, 0x3eaf32a0, 0x5bc88e18, 0xb5673b0a, - 0xd00087b2, 0x6938502f, 0x0c5fec97, 0xe2f05985, 0x8797e53d, - 0xd1878665, 0xb4e03add, 0x5a4f8fcf, 0x3f283377, 0x8610e4ea, - 0xe3775852, 0x0dd8ed40, 0x68bf51f8, 0xa1f82bf0, 0xc49f9748, - 0x2a30225a, 0x4f579ee2, 0xf66f497f, 0x9308f5c7, 0x7da740d5, - 0x18c0fc6d, 0x4ed09f35, 0x2bb7238d, 0xc518969f, 0xa07f2a27, - 0x1947fdba, 0x7c204102, 0x928ff410, 0xf7e848a8, 0x3d58149b, - 0x583fa823, 0xb6901d31, 0xd3f7a189, 0x6acf7614, 0x0fa8caac, - 0xe1077fbe, 0x8460c306, 0xd270a05e, 0xb7171ce6, 0x59b8a9f4, - 0x3cdf154c, 0x85e7c2d1, 0xe0807e69, 0x0e2fcb7b, 0x6b4877c3, - 0xa20f0dcb, 0xc768b173, 0x29c70461, 0x4ca0b8d9, 0xf5986f44, - 0x90ffd3fc, 0x7e5066ee, 0x1b37da56, 0x4d27b90e, 0x284005b6, - 0xc6efb0a4, 0xa3880c1c, 0x1ab0db81, 0x7fd76739, 0x9178d22b, - 0xf41f6e93, 0x03f7263b, 0x66909a83, 0x883f2f91, 0xed589329, - 0x546044b4, 0x3107f80c, 0xdfa84d1e, 0xbacff1a6, 0xecdf92fe, - 0x89b82e46, 0x67179b54, 0x027027ec, 0xbb48f071, 0xde2f4cc9, - 0x3080f9db, 0x55e74563, 0x9ca03f6b, 0xf9c783d3, 0x176836c1, - 0x720f8a79, 0xcb375de4, 0xae50e15c, 0x40ff544e, 0x2598e8f6, - 0x73888bae, 0x16ef3716, 0xf8408204, 0x9d273ebc, 0x241fe921, - 0x41785599, 0xafd7e08b, 0xcab05c33, 0x3bb659ed, 0x5ed1e555, - 0xb07e5047, 0xd519ecff, 0x6c213b62, 0x094687da, 0xe7e932c8, - 0x828e8e70, 0xd49eed28, 0xb1f95190, 0x5f56e482, 0x3a31583a, - 0x83098fa7, 0xe66e331f, 0x08c1860d, 0x6da63ab5, 0xa4e140bd, - 0xc186fc05, 0x2f294917, 0x4a4ef5af, 0xf3762232, 0x96119e8a, - 0x78be2b98, 0x1dd99720, 0x4bc9f478, 0x2eae48c0, 0xc001fdd2, - 0xa566416a, 0x1c5e96f7, 0x79392a4f, 0x97969f5d, 0xf2f123e5, - 0x05196b4d, 0x607ed7f5, 0x8ed162e7, 0xebb6de5f, 0x528e09c2, - 0x37e9b57a, 0xd9460068, 0xbc21bcd0, 0xea31df88, 0x8f566330, - 0x61f9d622, 0x049e6a9a, 0xbda6bd07, 0xd8c101bf, 0x366eb4ad, - 0x53090815, 0x9a4e721d, 0xff29cea5, 0x11867bb7, 0x74e1c70f, - 0xcdd91092, 0xa8beac2a, 0x46111938, 0x2376a580, 0x7566c6d8, - 0x10017a60, 0xfeaecf72, 0x9bc973ca, 0x22f1a457, 0x479618ef, - 0xa939adfd, 0xcc5e1145, 0x06ee4d76, 0x6389f1ce, 0x8d2644dc, - 0xe841f864, 0x51792ff9, 0x341e9341, 0xdab12653, 0xbfd69aeb, - 0xe9c6f9b3, 0x8ca1450b, 0x620ef019, 0x07694ca1, 0xbe519b3c, - 0xdb362784, 0x35999296, 0x50fe2e2e, 0x99b95426, 0xfcdee89e, - 0x12715d8c, 0x7716e134, 0xce2e36a9, 0xab498a11, 0x45e63f03, - 0x208183bb, 0x7691e0e3, 0x13f65c5b, 0xfd59e949, 0x983e55f1, - 0x2106826c, 0x44613ed4, 0xaace8bc6, 0xcfa9377e, 0x38417fd6, - 0x5d26c36e, 0xb389767c, 0xd6eecac4, 0x6fd61d59, 0x0ab1a1e1, - 0xe41e14f3, 0x8179a84b, 0xd769cb13, 0xb20e77ab, 0x5ca1c2b9, - 0x39c67e01, 0x80fea99c, 0xe5991524, 0x0b36a036, 0x6e511c8e, - 0xa7166686, 0xc271da3e, 0x2cde6f2c, 0x49b9d394, 0xf0810409, - 0x95e6b8b1, 0x7b490da3, 0x1e2eb11b, 0x483ed243, 0x2d596efb, - 0xc3f6dbe9, 0xa6916751, 0x1fa9b0cc, 0x7ace0c74, 0x9461b966, - 0xf10605de - } -}; - -#endif /* CRC32_H_ */ diff --git a/contrib/libzlib-ng/deflate.c b/contrib/libzlib-ng/deflate.c deleted file mode 100644 index fb275671777..00000000000 --- a/contrib/libzlib-ng/deflate.c +++ /dev/null @@ -1,1407 +0,0 @@ -/* deflate.c -- compress data using the deflation algorithm - * Copyright (C) 1995-2013 Jean-loup Gailly and Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* - * ALGORITHM - * - * The "deflation" process depends on being able to identify portions - * of the input text which are identical to earlier input (within a - * sliding window trailing behind the input currently being processed). - * - * The most straightforward technique turns out to be the fastest for - * most input files: try all possible matches and select the longest. - * The key feature of this algorithm is that insertions into the string - * dictionary are very simple and thus fast, and deletions are avoided - * completely. Insertions are performed at each input character, whereas - * string matches are performed only when the previous match ends. So it - * is preferable to spend more time in matches to allow very fast string - * insertions and avoid deletions. The matching algorithm for small - * strings is inspired from that of Rabin & Karp. A brute force approach - * is used to find longer strings when a small match has been found. - * A similar algorithm is used in comic (by Jan-Mark Wams) and freeze - * (by Leonid Broukhis). - * A previous version of this file used a more sophisticated algorithm - * (by Fiala and Greene) which is guaranteed to run in linear amortized - * time, but has a larger average cost, uses more memory and is patented. - * However the F&G algorithm may be faster for some highly redundant - * files if the parameter max_chain_length (described below) is too large. - * - * ACKNOWLEDGEMENTS - * - * The idea of lazy evaluation of matches is due to Jan-Mark Wams, and - * I found it in 'freeze' written by Leonid Broukhis. - * Thanks to many people for bug reports and testing. - * - * REFERENCES - * - * Deutsch, L.P.,"DEFLATE Compressed Data Format Specification". - * Available in http://tools.ietf.org/html/rfc1951 - * - * A description of the Rabin and Karp algorithm is given in the book - * "Algorithms" by R. Sedgewick, Addison-Wesley, p252. - * - * Fiala,E.R., and Greene,D.H. - * Data Compression with Finite Windows, Comm.ACM, 32,4 (1989) 490-595 - * - */ - -/* @(#) $Id$ */ - -#include "deflate.h" -#include "deflate_p.h" -#include "match.h" - -const char deflate_copyright[] = " deflate 1.2.8.f Copyright 1995-2013 Jean-loup Gailly and Mark Adler "; -/* - If you use the zlib library in a product, an acknowledgment is welcome - in the documentation of your product. If for some reason you cannot - include such an acknowledgment, I would appreciate that you keep this - copyright string in the executable of your product. - */ - -/* =========================================================================== - * Function prototypes. - */ - -typedef block_state (*compress_func) (deflate_state *s, int flush); -/* Compression function. Returns the block state after the call. */ - -void fill_window (deflate_state *s); -local block_state deflate_stored (deflate_state *s, int flush); -block_state deflate_fast (deflate_state *s, int flush); -block_state deflate_quick (deflate_state *s, int flush); -#ifdef MEDIUM_STRATEGY -block_state deflate_medium (deflate_state *s, int flush); -#endif -block_state deflate_slow (deflate_state *s, int flush); -local block_state deflate_rle (deflate_state *s, int flush); -local block_state deflate_huff (deflate_state *s, int flush); -local void lm_init (deflate_state *s); -local void putShortMSB (deflate_state *s, uint16_t b); -ZLIB_INTERNAL void flush_pending (z_stream *strm); -ZLIB_INTERNAL int read_buf (z_stream *strm, unsigned char *buf, unsigned size); - -extern void crc_reset(deflate_state *const s); -extern void crc_finalize(deflate_state *const s); -extern void copy_with_crc(z_stream *strm, unsigned char *dst, long size); - -/* =========================================================================== - * Local data - */ - -#define NIL 0 -/* Tail of hash chains */ - -/* Values for max_lazy_match, good_match and max_chain_length, depending on - * the desired pack level (0..9). The values given below have been tuned to - * exclude worst case performance for pathological files. Better values may be - * found for specific files. - */ -typedef struct config_s { - uint16_t good_length; /* reduce lazy search above this match length */ - uint16_t max_lazy; /* do not perform lazy search above this match length */ - uint16_t nice_length; /* quit search above this match length */ - uint16_t max_chain; - compress_func func; -} config; - -local const config configuration_table[10] = { -/* good lazy nice chain */ -/* 0 */ {0, 0, 0, 0, deflate_stored}, /* store only */ - -#ifdef X86_QUICK_STRATEGY -/* 1 */ {4, 4, 8, 4, deflate_quick}, -/* 2 */ {4, 4, 8, 4, deflate_fast}, /* max speed, no lazy matches */ -#else -/* 1 */ {4, 4, 8, 4, deflate_fast}, /* max speed, no lazy matches */ -/* 2 */ {4, 5, 16, 8, deflate_fast}, -#endif - -/* 3 */ {4, 6, 32, 32, deflate_fast}, - -#ifdef MEDIUM_STRATEGY -/* 4 */ {4, 4, 16, 16, deflate_medium}, /* lazy matches */ -/* 5 */ {8, 16, 32, 32, deflate_medium}, -/* 6 */ {8, 16, 128, 128, deflate_medium}, -#else -/* 4 */ {4, 4, 16, 16, deflate_slow}, /* lazy matches */ -/* 5 */ {8, 16, 32, 32, deflate_slow}, -/* 6 */ {8, 16, 128, 128, deflate_slow}, -#endif - -/* 7 */ {8, 32, 128, 256, deflate_slow}, -/* 8 */ {32, 128, 258, 1024, deflate_slow}, -/* 9 */ {32, 258, 258, 4096, deflate_slow}}; /* max compression */ - -/* Note: the deflate() code requires max_lazy >= MIN_MATCH and max_chain >= 4 - * For deflate_fast() (levels <= 3) good is ignored and lazy has a different - * meaning. - */ - -#define EQUAL 0 -/* result of memcmp for equal strings */ - -/* rank Z_BLOCK between Z_NO_FLUSH and Z_PARTIAL_FLUSH */ -#define RANK(f) (((f) * 2) - ((f) > 4 ? 9 : 0)) - - -/* =========================================================================== - * Initialize the hash table (avoiding 64K overflow for 16 bit systems). - * prev[] will be initialized on the fly. - */ -#define CLEAR_HASH(s) \ - s->head[s->hash_size-1] = NIL; \ - memset((unsigned char *)s->head, 0, (unsigned)(s->hash_size-1)*sizeof(*s->head)); - -/* ========================================================================= */ -int ZEXPORT deflateInit_(z_stream *strm, int level, const char *version, int stream_size) { - return deflateInit2_(strm, level, Z_DEFLATED, MAX_WBITS, DEF_MEM_LEVEL, Z_DEFAULT_STRATEGY, version, stream_size); - /* Todo: ignore strm->next_in if we use it as window */ -} - -/* ========================================================================= */ -int ZEXPORT deflateInit2_(z_stream *strm, int level, int method, int windowBits, - int memLevel, int strategy, const char *version, int stream_size) { - unsigned window_padding = 0; - deflate_state *s; - int wrap = 1; - static const char my_version[] = ZLIB_VERSION; - - uint16_t *overlay; - /* We overlay pending_buf and d_buf+l_buf. This works since the average - * output size for (length,distance) codes is <= 24 bits. - */ - -#if defined(X86_SSE2_FILL_WINDOW) || defined(X86_SSE4_2_CRC_HASH) - x86_check_features(); -#endif - - if (version == Z_NULL || version[0] != my_version[0] || stream_size != sizeof(z_stream)) { - return Z_VERSION_ERROR; - } - if (strm == Z_NULL) - return Z_STREAM_ERROR; - - strm->msg = Z_NULL; - if (strm->zalloc == (alloc_func)0) { - strm->zalloc = zcalloc; - strm->opaque = NULL; - } - if (strm->zfree == (free_func)0) - strm->zfree = zcfree; - - if (level == Z_DEFAULT_COMPRESSION) - level = 6; - - if (windowBits < 0) { /* suppress zlib wrapper */ - wrap = 0; - windowBits = -windowBits; -#ifdef GZIP - } else if (windowBits > 15) { - wrap = 2; /* write gzip wrapper instead */ - windowBits -= 16; -#endif - } - if (memLevel < 1 || memLevel > MAX_MEM_LEVEL || method != Z_DEFLATED || windowBits < 8 || - windowBits > 15 || level < 0 || level > 9 || strategy < 0 || strategy > Z_FIXED) { - return Z_STREAM_ERROR; - } - if (windowBits == 8) - windowBits = 9; /* until 256-byte window bug fixed */ - -#ifdef X86_QUICK_STRATEGY - if (level == 1) - windowBits = 13; -#endif - - s = (deflate_state *) ZALLOC(strm, 1, sizeof(deflate_state)); - if (s == Z_NULL) - return Z_MEM_ERROR; - strm->state = (struct internal_state *)s; - s->strm = strm; - - s->wrap = wrap; - s->gzhead = Z_NULL; - s->w_bits = windowBits; - s->w_size = 1 << s->w_bits; - s->w_mask = s->w_size - 1; - -#ifdef X86_SSE4_2_CRC_HASH - if (x86_cpu_has_sse42) - s->hash_bits = 15; - else -#endif - s->hash_bits = memLevel + 7; - - s->hash_size = 1 << s->hash_bits; - s->hash_mask = s->hash_size - 1; - s->hash_shift = ((s->hash_bits+MIN_MATCH-1)/MIN_MATCH); - -#ifdef X86_PCLMULQDQ_CRC - window_padding = 8; -#endif - - s->window = (unsigned char *) ZALLOC(strm, s->w_size + window_padding, 2*sizeof(unsigned char)); - s->prev = (Pos *) ZALLOC(strm, s->w_size, sizeof(Pos)); - s->head = (Pos *) ZALLOC(strm, s->hash_size, sizeof(Pos)); - - s->high_water = 0; /* nothing written to s->window yet */ - - s->lit_bufsize = 1 << (memLevel + 6); /* 16K elements by default */ - - overlay = (uint16_t *) ZALLOC(strm, s->lit_bufsize, sizeof(uint16_t)+2); - s->pending_buf = (unsigned char *) overlay; - s->pending_buf_size = (unsigned long)s->lit_bufsize * (sizeof(uint16_t)+2L); - - if (s->window == Z_NULL || s->prev == Z_NULL || s->head == Z_NULL || - s->pending_buf == Z_NULL) { - s->status = FINISH_STATE; - strm->msg = ERR_MSG(Z_MEM_ERROR); - deflateEnd(strm); - return Z_MEM_ERROR; - } - s->d_buf = overlay + s->lit_bufsize/sizeof(uint16_t); - s->l_buf = s->pending_buf + (1+sizeof(uint16_t))*s->lit_bufsize; - - s->level = level; - s->strategy = strategy; - s->method = (unsigned char)method; - - return deflateReset(strm); -} - -/* ========================================================================= */ -int ZEXPORT deflateSetDictionary(z_stream *strm, const unsigned char *dictionary, unsigned int dictLength) { - deflate_state *s; - unsigned int str, n; - int wrap; - uint32_t avail; - const unsigned char *next; - - if (strm == Z_NULL || strm->state == Z_NULL || dictionary == Z_NULL) - return Z_STREAM_ERROR; - s = strm->state; - wrap = s->wrap; - if (wrap == 2 || (wrap == 1 && s->status != INIT_STATE) || s->lookahead) - return Z_STREAM_ERROR; - - /* when using zlib wrappers, compute Adler-32 for provided dictionary */ - if (wrap == 1) - strm->adler = adler32(strm->adler, dictionary, dictLength); - s->wrap = 0; /* avoid computing Adler-32 in read_buf */ - - /* if dictionary would fill window, just replace the history */ - if (dictLength >= s->w_size) { - if (wrap == 0) { /* already empty otherwise */ - CLEAR_HASH(s); - s->strstart = 0; - s->block_start = 0L; - s->insert = 0; - } - dictionary += dictLength - s->w_size; /* use the tail */ - dictLength = s->w_size; - } - - /* insert dictionary into window and hash */ - avail = strm->avail_in; - next = strm->next_in; - strm->avail_in = dictLength; - strm->next_in = (const unsigned char *)dictionary; - fill_window(s); - while (s->lookahead >= MIN_MATCH) { - str = s->strstart; - n = s->lookahead - (MIN_MATCH-1); - bulk_insert_str(s, str, n); - s->strstart = str + n; - s->lookahead = MIN_MATCH-1; - fill_window(s); - } - s->strstart += s->lookahead; - s->block_start = (long)s->strstart; - s->insert = s->lookahead; - s->lookahead = 0; - s->match_length = s->prev_length = MIN_MATCH-1; - s->match_available = 0; - strm->next_in = next; - strm->avail_in = avail; - s->wrap = wrap; - return Z_OK; -} - -/* ========================================================================= */ -int ZEXPORT deflateResetKeep(z_stream *strm) { - deflate_state *s; - - if (strm == Z_NULL || strm->state == Z_NULL || strm->zalloc == (alloc_func)0 || strm->zfree == (free_func)0) { - return Z_STREAM_ERROR; - } - - strm->total_in = strm->total_out = 0; - strm->msg = Z_NULL; /* use zfree if we ever allocate msg dynamically */ - strm->data_type = Z_UNKNOWN; - - s = (deflate_state *)strm->state; - s->pending = 0; - s->pending_out = s->pending_buf; - - if (s->wrap < 0) { - s->wrap = -s->wrap; /* was made negative by deflate(..., Z_FINISH); */ - } - s->status = s->wrap ? INIT_STATE : BUSY_STATE; -#ifdef GZIP - strm->adler = s->wrap == 2 ? crc32(0L, Z_NULL, 0) : adler32(0L, Z_NULL, 0); -#else - strm->adler = adler32(0L, Z_NULL, 0); -#endif - s->last_flush = Z_NO_FLUSH; - - _tr_init(s); - - return Z_OK; -} - -/* ========================================================================= */ -int ZEXPORT deflateReset(z_stream *strm) { - int ret; - - ret = deflateResetKeep(strm); - if (ret == Z_OK) - lm_init(strm->state); - return ret; -} - -/* ========================================================================= */ -int ZEXPORT deflateSetHeader(z_stream *strm, gz_headerp head) { - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - if (strm->state->wrap != 2) - return Z_STREAM_ERROR; - strm->state->gzhead = head; - return Z_OK; -} - -/* ========================================================================= */ -int ZEXPORT deflatePending(z_stream *strm, uint32_t *pending, int *bits) { - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - if (pending != Z_NULL) - *pending = strm->state->pending; - if (bits != Z_NULL) - *bits = strm->state->bi_valid; - return Z_OK; -} - -/* ========================================================================= */ -int ZEXPORT deflatePrime(z_stream *strm, int bits, int value) { - deflate_state *s; - int put; - - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - s = strm->state; - if ((unsigned char *)(s->d_buf) < s->pending_out + ((Buf_size + 7) >> 3)) - return Z_BUF_ERROR; - do { - put = Buf_size - s->bi_valid; - if (put > bits) - put = bits; - s->bi_buf |= (uint16_t)((value & ((1 << put) - 1)) << s->bi_valid); - s->bi_valid += put; - _tr_flush_bits(s); - value >>= put; - bits -= put; - } while (bits); - return Z_OK; -} - -/* ========================================================================= */ -int ZEXPORT deflateParams(z_stream *strm, int level, int strategy) { - deflate_state *s; - compress_func func; - int err = Z_OK; - - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - s = strm->state; - - if (level == Z_DEFAULT_COMPRESSION) - level = 6; - if (level < 0 || level > 9 || strategy < 0 || strategy > Z_FIXED) { - return Z_STREAM_ERROR; - } - func = configuration_table[s->level].func; - - if ((strategy != s->strategy || func != configuration_table[level].func)) { - /* Flush the last buffer: */ - err = deflate(strm, Z_BLOCK); - if (err == Z_BUF_ERROR && s->pending == 0) - err = Z_OK; - } - if (s->level != level) { - s->level = level; - s->max_lazy_match = configuration_table[level].max_lazy; - s->good_match = configuration_table[level].good_length; - s->nice_match = configuration_table[level].nice_length; - s->max_chain_length = configuration_table[level].max_chain; - } - s->strategy = strategy; - return err; -} - -/* ========================================================================= */ -int ZEXPORT deflateTune(z_stream *strm, int good_length, int max_lazy, int nice_length, int max_chain) { - deflate_state *s; - - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - s = strm->state; - s->good_match = good_length; - s->max_lazy_match = max_lazy; - s->nice_match = nice_length; - s->max_chain_length = max_chain; - return Z_OK; -} - -/* ========================================================================= - * For the default windowBits of 15 and memLevel of 8, this function returns - * a close to exact, as well as small, upper bound on the compressed size. - * They are coded as constants here for a reason--if the #define's are - * changed, then this function needs to be changed as well. The return - * value for 15 and 8 only works for those exact settings. - * - * For any setting other than those defaults for windowBits and memLevel, - * the value returned is a conservative worst case for the maximum expansion - * resulting from using fixed blocks instead of stored blocks, which deflate - * can emit on compressed data for some combinations of the parameters. - * - * This function could be more sophisticated to provide closer upper bounds for - * every combination of windowBits and memLevel. But even the conservative - * upper bound of about 14% expansion does not seem onerous for output buffer - * allocation. - */ -unsigned long ZEXPORT deflateBound(z_stream *strm, unsigned long sourceLen) { - deflate_state *s; - unsigned long complen, wraplen; - unsigned char *str; - - /* conservative upper bound for compressed data */ - complen = sourceLen + ((sourceLen + 7) >> 3) + ((sourceLen + 63) >> 6) + 5; - - /* if can't get parameters, return conservative bound plus zlib wrapper */ - if (strm == Z_NULL || strm->state == Z_NULL) - return complen + 6; - - /* compute wrapper length */ - s = strm->state; - switch (s->wrap) { - case 0: /* raw deflate */ - wraplen = 0; - break; - case 1: /* zlib wrapper */ - wraplen = 6 + (s->strstart ? 4 : 0); - break; - case 2: /* gzip wrapper */ - wraplen = 18; - if (s->gzhead != Z_NULL) { /* user-supplied gzip header */ - if (s->gzhead->extra != Z_NULL) { - wraplen += 2 + s->gzhead->extra_len; - } - str = s->gzhead->name; - if (str != Z_NULL) { - do { - wraplen++; - } while (*str++); - } - str = s->gzhead->comment; - if (str != Z_NULL) { - do { - wraplen++; - } while (*str++); - } - if (s->gzhead->hcrc) - wraplen += 2; - } - break; - default: /* for compiler happiness */ - wraplen = 6; - } - - /* if not default parameters, return conservative bound */ - if (s->w_bits != 15 || s->hash_bits != 8 + 7) - return complen + wraplen; - - /* default settings: return tight bound for that case */ - return sourceLen + (sourceLen >> 12) + (sourceLen >> 14) + (sourceLen >> 25) + 13 - 6 + wraplen; -} - -/* ========================================================================= - * Put a short in the pending buffer. The 16-bit value is put in MSB order. - * IN assertion: the stream state is correct and there is enough room in - * pending_buf. - */ -local void putShortMSB(deflate_state *s, uint16_t b) { - put_byte(s, (unsigned char)(b >> 8)); - put_byte(s, (unsigned char)(b & 0xff)); -} - -/* ========================================================================= - * Flush as much pending output as possible. All deflate() output goes - * through this function so some applications may wish to modify it - * to avoid allocating a large strm->next_out buffer and copying into it. - * (See also read_buf()). - */ -ZLIB_INTERNAL void flush_pending(z_stream *strm) { - uint32_t len; - deflate_state *s = strm->state; - - _tr_flush_bits(s); - len = s->pending; - if (len > strm->avail_out) - len = strm->avail_out; - if (len == 0) - return; - - memcpy(strm->next_out, s->pending_out, len); - strm->next_out += len; - s->pending_out += len; - strm->total_out += len; - strm->avail_out -= len; - s->pending -= len; - if (s->pending == 0) { - s->pending_out = s->pending_buf; - } -} - -/* ========================================================================= */ -int ZEXPORT deflate(z_stream *strm, int flush) { - int old_flush; /* value of flush param for previous deflate call */ - deflate_state *s; - - if (strm == Z_NULL || strm->state == Z_NULL || flush > Z_BLOCK || flush < 0) { - return Z_STREAM_ERROR; - } - s = strm->state; - - if (strm->next_out == Z_NULL || (strm->avail_in != 0 && strm->next_in == Z_NULL) || - (s->status == FINISH_STATE && flush != Z_FINISH)) { - ERR_RETURN(strm, Z_STREAM_ERROR); - } - if (strm->avail_out == 0) - ERR_RETURN(strm, Z_BUF_ERROR); - - s->strm = strm; /* just in case */ - old_flush = s->last_flush; - s->last_flush = flush; - - /* Write the header */ - if (s->status == INIT_STATE) { -#ifdef GZIP - if (s->wrap == 2) { - crc_reset(s); - put_byte(s, 31); - put_byte(s, 139); - put_byte(s, 8); - if (s->gzhead == Z_NULL) { - put_byte(s, 0); - put_byte(s, 0); - put_byte(s, 0); - put_byte(s, 0); - put_byte(s, 0); - put_byte(s, s->level == 9 ? 2 : - (s->strategy >= Z_HUFFMAN_ONLY || s->level < 2 ? - 4 : 0)); - put_byte(s, OS_CODE); - s->status = BUSY_STATE; - } else { - put_byte(s, (s->gzhead->text ? 1 : 0) + - (s->gzhead->hcrc ? 2 : 0) + - (s->gzhead->extra == Z_NULL ? 0 : 4) + - (s->gzhead->name == Z_NULL ? 0 : 8) + - (s->gzhead->comment == Z_NULL ? 0 : 16) ); - put_byte(s, (unsigned char)(s->gzhead->time & 0xff)); - put_byte(s, (unsigned char)((s->gzhead->time >> 8) & 0xff)); - put_byte(s, (unsigned char)((s->gzhead->time >> 16) & 0xff)); - put_byte(s, (unsigned char)((s->gzhead->time >> 24) & 0xff)); - put_byte(s, s->level == 9 ? 2 : - (s->strategy >= Z_HUFFMAN_ONLY || s->level < 2 ? - 4 : 0)); - put_byte(s, s->gzhead->os & 0xff); - if (s->gzhead->extra != Z_NULL) { - put_byte(s, s->gzhead->extra_len & 0xff); - put_byte(s, (s->gzhead->extra_len >> 8) & 0xff); - } - if (s->gzhead->hcrc) - strm->adler = crc32(strm->adler, s->pending_buf, s->pending); - s->gzindex = 0; - s->status = EXTRA_STATE; - } - } else -#endif - { - unsigned int header = (Z_DEFLATED + ((s->w_bits-8) << 4)) << 8; - unsigned int level_flags; - - if (s->strategy >= Z_HUFFMAN_ONLY || s->level < 2) - level_flags = 0; - else if (s->level < 6) - level_flags = 1; - else if (s->level == 6) - level_flags = 2; - else - level_flags = 3; - header |= (level_flags << 6); - if (s->strstart != 0) - header |= PRESET_DICT; - header += 31 - (header % 31); - - s->status = BUSY_STATE; - putShortMSB(s, header); - - /* Save the adler32 of the preset dictionary: */ - if (s->strstart != 0) { - putShortMSB(s, (uint16_t)(strm->adler >> 16)); - putShortMSB(s, (uint16_t)strm->adler); - } - strm->adler = adler32(0L, Z_NULL, 0); - } - } -#ifdef GZIP - if (s->status == EXTRA_STATE) { - if (s->gzhead->extra != Z_NULL) { - uint32_t beg = s->pending; /* start of bytes to update crc */ - - while (s->gzindex < (s->gzhead->extra_len & 0xffff)) { - if (s->pending == s->pending_buf_size) { - if (s->gzhead->hcrc && s->pending > beg) - strm->adler = crc32(strm->adler, s->pending_buf + beg, s->pending - beg); - flush_pending(strm); - beg = s->pending; - if (s->pending == s->pending_buf_size) - break; - } - put_byte(s, s->gzhead->extra[s->gzindex]); - s->gzindex++; - } - if (s->gzhead->hcrc && s->pending > beg) - strm->adler = crc32(strm->adler, s->pending_buf + beg, s->pending - beg); - if (s->gzindex == s->gzhead->extra_len) { - s->gzindex = 0; - s->status = NAME_STATE; - } - } else { - s->status = NAME_STATE; - } - } - if (s->status == NAME_STATE) { - if (s->gzhead->name != Z_NULL) { - uint32_t beg = s->pending; /* start of bytes to update crc */ - int val; - - do { - if (s->pending == s->pending_buf_size) { - if (s->gzhead->hcrc && s->pending > beg) - strm->adler = crc32(strm->adler, s->pending_buf + beg, s->pending - beg); - flush_pending(strm); - beg = s->pending; - if (s->pending == s->pending_buf_size) { - val = 1; - break; - } - } - val = s->gzhead->name[s->gzindex++]; - put_byte(s, val); - } while (val != 0); - if (s->gzhead->hcrc && s->pending > beg) - strm->adler = crc32(strm->adler, s->pending_buf + beg, s->pending - beg); - if (val == 0) { - s->gzindex = 0; - s->status = COMMENT_STATE; - } - } else { - s->status = COMMENT_STATE; - } - } - if (s->status == COMMENT_STATE) { - if (s->gzhead->comment != Z_NULL) { - uint32_t beg = s->pending; /* start of bytes to update crc */ - int val; - - do { - if (s->pending == s->pending_buf_size) { - if (s->gzhead->hcrc && s->pending > beg) - strm->adler = crc32(strm->adler, s->pending_buf + beg, s->pending - beg); - flush_pending(strm); - beg = s->pending; - if (s->pending == s->pending_buf_size) { - val = 1; - break; - } - } - val = s->gzhead->comment[s->gzindex++]; - put_byte(s, val); - } while (val != 0); - if (s->gzhead->hcrc && s->pending > beg) - strm->adler = crc32(strm->adler, s->pending_buf + beg, s->pending - beg); - if (val == 0) - s->status = HCRC_STATE; - } else { - s->status = HCRC_STATE; - } - } - if (s->status == HCRC_STATE) { - if (s->gzhead->hcrc) { - if (s->pending + 2 > s->pending_buf_size) - flush_pending(strm); - if (s->pending + 2 <= s->pending_buf_size) { - put_byte(s, (unsigned char)(strm->adler & 0xff)); - put_byte(s, (unsigned char)((strm->adler >> 8) & 0xff)); - strm->adler = crc32(0L, Z_NULL, 0); - s->status = BUSY_STATE; - } - } else { - s->status = BUSY_STATE; - } - } -#endif - - /* Flush as much pending output as possible */ - if (s->pending != 0) { - flush_pending(strm); - if (strm->avail_out == 0) { - /* Since avail_out is 0, deflate will be called again with - * more output space, but possibly with both pending and - * avail_in equal to zero. There won't be anything to do, - * but this is not an error situation so make sure we - * return OK instead of BUF_ERROR at next call of deflate: - */ - s->last_flush = -1; - return Z_OK; - } - - /* Make sure there is something to do and avoid duplicate consecutive - * flushes. For repeated and useless calls with Z_FINISH, we keep - * returning Z_STREAM_END instead of Z_BUF_ERROR. - */ - } else if (strm->avail_in == 0 && RANK(flush) <= RANK(old_flush) && flush != Z_FINISH) { - ERR_RETURN(strm, Z_BUF_ERROR); - } - - /* User must not provide more input after the first FINISH: */ - if (s->status == FINISH_STATE && strm->avail_in != 0) { - ERR_RETURN(strm, Z_BUF_ERROR); - } - - /* Start a new block or continue the current one. - */ - if (strm->avail_in != 0 || s->lookahead != 0 || (flush != Z_NO_FLUSH && s->status != FINISH_STATE)) { - block_state bstate; - -#ifdef X86_QUICK_STRATEGY - if (s->level == 1 && !x86_cpu_has_sse42) - bstate = s->strategy == Z_HUFFMAN_ONLY ? deflate_huff(s, flush) : - (s->strategy == Z_RLE ? deflate_rle(s, flush) : deflate_fast(s, flush)); - else -#endif - bstate = s->strategy == Z_HUFFMAN_ONLY ? deflate_huff(s, flush) : - (s->strategy == Z_RLE ? deflate_rle(s, flush) : (*(configuration_table[s->level].func))(s, flush)); - - if (bstate == finish_started || bstate == finish_done) { - s->status = FINISH_STATE; - } - if (bstate == need_more || bstate == finish_started) { - if (strm->avail_out == 0) { - s->last_flush = -1; /* avoid BUF_ERROR next call, see above */ - } - return Z_OK; - /* If flush != Z_NO_FLUSH && avail_out == 0, the next call - * of deflate should use the same flush parameter to make sure - * that the flush is complete. So we don't have to output an - * empty block here, this will be done at next call. This also - * ensures that for a very small output buffer, we emit at most - * one empty block. - */ - } - if (bstate == block_done) { - if (flush == Z_PARTIAL_FLUSH) { - _tr_align(s); - } else if (flush != Z_BLOCK) { /* FULL_FLUSH or SYNC_FLUSH */ - _tr_stored_block(s, (char*)0, 0L, 0); - /* For a full flush, this empty block will be recognized - * as a special marker by inflate_sync(). - */ - if (flush == Z_FULL_FLUSH) { - CLEAR_HASH(s); /* forget history */ - if (s->lookahead == 0) { - s->strstart = 0; - s->block_start = 0L; - s->insert = 0; - } - } - } - flush_pending(strm); - if (strm->avail_out == 0) { - s->last_flush = -1; /* avoid BUF_ERROR at next call, see above */ - return Z_OK; - } - } - } - Assert(strm->avail_out > 0, "bug2"); - - if (flush != Z_FINISH) - return Z_OK; - if (s->wrap <= 0) - return Z_STREAM_END; - - /* Write the trailer */ -#ifdef GZIP - if (s->wrap == 2) { - crc_finalize(s); - put_byte(s, (unsigned char)(strm->adler & 0xff)); - put_byte(s, (unsigned char)((strm->adler >> 8) & 0xff)); - put_byte(s, (unsigned char)((strm->adler >> 16) & 0xff)); - put_byte(s, (unsigned char)((strm->adler >> 24) & 0xff)); - put_byte(s, (unsigned char)(strm->total_in & 0xff)); - put_byte(s, (unsigned char)((strm->total_in >> 8) & 0xff)); - put_byte(s, (unsigned char)((strm->total_in >> 16) & 0xff)); - put_byte(s, (unsigned char)((strm->total_in >> 24) & 0xff)); - } else -#endif - { - putShortMSB(s, (uint16_t)(strm->adler >> 16)); - putShortMSB(s, (uint16_t)strm->adler); - } - flush_pending(strm); - /* If avail_out is zero, the application will call deflate again - * to flush the rest. - */ - if (s->wrap > 0) - s->wrap = -s->wrap; /* write the trailer only once! */ - return s->pending != 0 ? Z_OK : Z_STREAM_END; -} - -/* ========================================================================= */ -int ZEXPORT deflateEnd(z_stream *strm) { - int status; - - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - - status = strm->state->status; - if (status != INIT_STATE && - status != EXTRA_STATE && - status != NAME_STATE && - status != COMMENT_STATE && - status != HCRC_STATE && - status != BUSY_STATE && - status != FINISH_STATE) { - return Z_STREAM_ERROR; - } - - /* Deallocate in reverse order of allocations: */ - TRY_FREE(strm, strm->state->pending_buf); - TRY_FREE(strm, strm->state->head); - TRY_FREE(strm, strm->state->prev); - TRY_FREE(strm, strm->state->window); - - ZFREE(strm, strm->state); - strm->state = Z_NULL; - - return status == BUSY_STATE ? Z_DATA_ERROR : Z_OK; -} - -/* ========================================================================= - * Copy the source state to the destination state. - */ -int ZEXPORT deflateCopy(z_stream *dest, z_stream *source) { - deflate_state *ds; - deflate_state *ss; - uint16_t *overlay; - - if (source == Z_NULL || dest == Z_NULL || source->state == Z_NULL) { - return Z_STREAM_ERROR; - } - - ss = source->state; - - memcpy((void *)dest, (void *)source, sizeof(z_stream)); - - ds = (deflate_state *) ZALLOC(dest, 1, sizeof(deflate_state)); - if (ds == Z_NULL) - return Z_MEM_ERROR; - dest->state = (struct internal_state *) ds; - memcpy((void *)ds, (void *)ss, sizeof(deflate_state)); - ds->strm = dest; - - ds->window = (unsigned char *) ZALLOC(dest, ds->w_size, 2*sizeof(unsigned char)); - ds->prev = (Pos *) ZALLOC(dest, ds->w_size, sizeof(Pos)); - ds->head = (Pos *) ZALLOC(dest, ds->hash_size, sizeof(Pos)); - overlay = (uint16_t *) ZALLOC(dest, ds->lit_bufsize, sizeof(uint16_t)+2); - ds->pending_buf = (unsigned char *) overlay; - - if (ds->window == Z_NULL || ds->prev == Z_NULL || ds->head == Z_NULL || ds->pending_buf == Z_NULL) { - deflateEnd(dest); - return Z_MEM_ERROR; - } - - memcpy(ds->window, ss->window, ds->w_size * 2 * sizeof(unsigned char)); - memcpy((void *)ds->prev, (void *)ss->prev, ds->w_size * sizeof(Pos)); - memcpy((void *)ds->head, (void *)ss->head, ds->hash_size * sizeof(Pos)); - memcpy(ds->pending_buf, ss->pending_buf, (unsigned int)ds->pending_buf_size); - - ds->pending_out = ds->pending_buf + (ss->pending_out - ss->pending_buf); - ds->d_buf = overlay + ds->lit_bufsize/sizeof(uint16_t); - ds->l_buf = ds->pending_buf + (1+sizeof(uint16_t))*ds->lit_bufsize; - - ds->l_desc.dyn_tree = ds->dyn_ltree; - ds->d_desc.dyn_tree = ds->dyn_dtree; - ds->bl_desc.dyn_tree = ds->bl_tree; - - return Z_OK; -} - -/* =========================================================================== - * Read a new buffer from the current input stream, update the adler32 - * and total number of bytes read. All deflate() input goes through - * this function so some applications may wish to modify it to avoid - * allocating a large strm->next_in buffer and copying from it. - * (See also flush_pending()). - */ -ZLIB_INTERNAL int read_buf(z_stream *strm, unsigned char *buf, unsigned size) { - uint32_t len = strm->avail_in; - - if (len > size) - len = size; - if (len == 0) - return 0; - - strm->avail_in -= len; - -#ifdef GZIP - if (strm->state->wrap == 2) - copy_with_crc(strm, buf, len); - else -#endif - { - memcpy(buf, strm->next_in, len); - if (strm->state->wrap == 1) - strm->adler = adler32(strm->adler, buf, len); - } - strm->next_in += len; - strm->total_in += len; - - return (int)len; -} - -/* =========================================================================== - * Initialize the "longest match" routines for a new zlib stream - */ -local void lm_init(deflate_state *s) { - s->window_size = (unsigned long)2L*s->w_size; - - CLEAR_HASH(s); - - /* Set the default configuration parameters: - */ - s->max_lazy_match = configuration_table[s->level].max_lazy; - s->good_match = configuration_table[s->level].good_length; - s->nice_match = configuration_table[s->level].nice_length; - s->max_chain_length = configuration_table[s->level].max_chain; - - s->strstart = 0; - s->block_start = 0L; - s->lookahead = 0; - s->insert = 0; - s->match_length = s->prev_length = MIN_MATCH-1; - s->match_available = 0; - s->ins_h = 0; -} - -#ifdef DEBUG -/* =========================================================================== - * Check that the match at match_start is indeed a match. - */ -void check_match(deflate_state *s, IPos start, IPos match, int length) { - /* check that the match is indeed a match */ - if (memcmp(s->window + match, s->window + start, length) != EQUAL) { - fprintf(stderr, " start %u, match %u, length %d\n", start, match, length); - do { - fprintf(stderr, "%c%c", s->window[match++], s->window[start++]); - } while (--length != 0); - z_error("invalid match"); - } - if (z_verbose > 1) { - fprintf(stderr, "\\[%u,%d]", start-match, length); - do { - putc(s->window[start++], stderr); - } while (--length != 0); - } -} -#else -# define check_match(s, start, match, length) -#endif /* DEBUG */ - -/* =========================================================================== - * Fill the window when the lookahead becomes insufficient. - * Updates strstart and lookahead. - * - * IN assertion: lookahead < MIN_LOOKAHEAD - * OUT assertions: strstart <= window_size-MIN_LOOKAHEAD - * At least one byte has been read, or avail_in == 0; reads are - * performed for at least two bytes (required for the zip translate_eol - * option -- not supported here). - */ -#ifdef X86_SSE2_FILL_WINDOW -extern void fill_window_sse(deflate_state *s); -#endif -void fill_window_c(deflate_state *s); - -void fill_window(deflate_state *s) { -#ifdef X86_SSE2_FILL_WINDOW -# ifndef X86_NOCHECK_SSE2 - if (x86_cpu_has_sse2) { -# endif - fill_window_sse(s); -# ifndef X86_NOCHECK_SSE2 - } else { - fill_window_c(s); - } -# endif - -#else - fill_window_c(s); -#endif -} - -void fill_window_c(deflate_state *s) { - register unsigned n; - register Pos *p; - unsigned more; /* Amount of free space at the end of the window. */ - unsigned int wsize = s->w_size; - - Assert(s->lookahead < MIN_LOOKAHEAD, "already enough lookahead"); - - do { - more = (unsigned)(s->window_size -(unsigned long)s->lookahead -(unsigned long)s->strstart); - - /* If the window is almost full and there is insufficient lookahead, - * move the upper half to the lower one to make room in the upper half. - */ - if (s->strstart >= wsize+MAX_DIST(s)) { - memcpy(s->window, s->window+wsize, (unsigned)wsize); - s->match_start -= wsize; - s->strstart -= wsize; /* we now have strstart >= MAX_DIST */ - s->block_start -= (long) wsize; - - /* Slide the hash table (could be avoided with 32 bit values - at the expense of memory usage). We slide even when level == 0 - to keep the hash table consistent if we switch back to level > 0 - later. (Using level 0 permanently is not an optimal usage of - zlib, so we don't care about this pathological case.) - */ - n = s->hash_size; - p = &s->head[n]; -#ifdef NOT_TWEAK_COMPILER - do { - unsigned m; - m = *--p; - *p = (Pos)(m >= wsize ? m-wsize : NIL); - } while (--n); -#else - /* As of I make this change, gcc (4.8.*) isn't able to vectorize - * this hot loop using saturated-subtraction on x86-64 architecture. - * To avoid this defect, we can change the loop such that - * o. the pointer advance forward, and - * o. demote the variable 'm' to be local to the loop, and - * choose type "Pos" (instead of 'unsigned int') for the - * variable to avoid unncessary zero-extension. - */ - { - unsigned int i; - Pos *q = p - n; - for (i = 0; i < n; i++) { - Pos m = *q; - Pos t = wsize; - *q++ = (Pos)(m >= t ? m-t: NIL); - } - } - -#endif /* NOT_TWEAK_COMPILER */ - n = wsize; - p = &s->prev[n]; -#ifdef NOT_TWEAK_COMPILER - do { - unsigned m; - m = *--p; - *p = (Pos)(m >= wsize ? m-wsize : NIL); - /* If n is not on any hash chain, prev[n] is garbage but - * its value will never be used. - */ - } while (--n); -#else - { - unsigned int i; - Pos *q = p - n; - for (i = 0; i < n; i++) { - Pos m = *q; - Pos t = wsize; - *q++ = (Pos)(m >= t ? m-t: NIL); - } - } -#endif /* NOT_TWEAK_COMPILER */ - more += wsize; - } - if (s->strm->avail_in == 0) - break; - - /* If there was no sliding: - * strstart <= WSIZE+MAX_DIST-1 && lookahead <= MIN_LOOKAHEAD - 1 && - * more == window_size - lookahead - strstart - * => more >= window_size - (MIN_LOOKAHEAD-1 + WSIZE + MAX_DIST-1) - * => more >= window_size - 2*WSIZE + 2 - * In the BIG_MEM or MMAP case (not yet supported), - * window_size == input_size + MIN_LOOKAHEAD && - * strstart + s->lookahead <= input_size => more >= MIN_LOOKAHEAD. - * Otherwise, window_size == 2*WSIZE so more >= 2. - * If there was sliding, more >= WSIZE. So in all cases, more >= 2. - */ - Assert(more >= 2, "more < 2"); - - n = read_buf(s->strm, s->window + s->strstart + s->lookahead, more); - s->lookahead += n; - - /* Initialize the hash value now that we have some input: */ - if (s->lookahead + s->insert >= MIN_MATCH) { - unsigned int str = s->strstart - s->insert; - s->ins_h = s->window[str]; - if (str >= 1) - UPDATE_HASH(s, s->ins_h, str + 1 - (MIN_MATCH-1)); -#if MIN_MATCH != 3 - Call UPDATE_HASH() MIN_MATCH-3 more times -#endif - while (s->insert) { - UPDATE_HASH(s, s->ins_h, str); - s->prev[str & s->w_mask] = s->head[s->ins_h]; - s->head[s->ins_h] = (Pos)str; - str++; - s->insert--; - if (s->lookahead + s->insert < MIN_MATCH) - break; - } - } - /* If the whole input has less than MIN_MATCH bytes, ins_h is garbage, - * but this is not important since only literal bytes will be emitted. - */ - } while (s->lookahead < MIN_LOOKAHEAD && s->strm->avail_in != 0); - - /* If the WIN_INIT bytes after the end of the current data have never been - * written, then zero those bytes in order to avoid memory check reports of - * the use of uninitialized (or uninitialised as Julian writes) bytes by - * the longest match routines. Update the high water mark for the next - * time through here. WIN_INIT is set to MAX_MATCH since the longest match - * routines allow scanning to strstart + MAX_MATCH, ignoring lookahead. - */ - if (s->high_water < s->window_size) { - unsigned long curr = s->strstart + (unsigned long)(s->lookahead); - unsigned long init; - - if (s->high_water < curr) { - /* Previous high water mark below current data -- zero WIN_INIT - * bytes or up to end of window, whichever is less. - */ - init = s->window_size - curr; - if (init > WIN_INIT) - init = WIN_INIT; - memset(s->window + curr, 0, (unsigned)init); - s->high_water = curr + init; - } else if (s->high_water < (unsigned long)curr + WIN_INIT) { - /* High water mark at or above current data, but below current data - * plus WIN_INIT -- zero out to current data plus WIN_INIT, or up - * to end of window, whichever is less. - */ - init = (unsigned long)curr + WIN_INIT - s->high_water; - if (init > s->window_size - s->high_water) - init = s->window_size - s->high_water; - memset(s->window + s->high_water, 0, (unsigned)init); - s->high_water += init; - } - } - - Assert((unsigned long)s->strstart <= s->window_size - MIN_LOOKAHEAD, - "not enough room for search"); -} - -/* =========================================================================== - * Copy without compression as much as possible from the input stream, return - * the current block state. - * This function does not insert new strings in the dictionary since - * uncompressible data is probably not useful. This function is used - * only for the level=0 compression option. - * NOTE: this function should be optimized to avoid extra copying from - * window to pending_buf. - */ -local block_state deflate_stored(deflate_state *s, int flush) { - /* Stored blocks are limited to 0xffff bytes, pending_buf is limited - * to pending_buf_size, and each stored block has a 5 byte header: - */ - unsigned long max_block_size = 0xffff; - unsigned long max_start; - - if (max_block_size > s->pending_buf_size - 5) { - max_block_size = (uint32_t)(s->pending_buf_size - 5); - } - - /* Copy as much as possible from input to output: */ - for (;;) { - /* Fill the window as much as possible: */ - if (s->lookahead <= 1) { - Assert(s->strstart < s->w_size+MAX_DIST(s) || s->block_start >= (long)s->w_size, "slide too late"); - - fill_window(s); - if (s->lookahead == 0 && flush == Z_NO_FLUSH) - return need_more; - - if (s->lookahead == 0) - break; /* flush the current block */ - } - Assert(s->block_start >= 0L, "block gone"); - - s->strstart += s->lookahead; - s->lookahead = 0; - - /* Emit a stored block if pending_buf will be full: */ - max_start = s->block_start + max_block_size; - if (s->strstart == 0 || (unsigned long)s->strstart >= max_start) { - /* strstart == 0 is possible when wraparound on 16-bit machine */ - s->lookahead = (unsigned int)(s->strstart - max_start); - s->strstart = (unsigned int)max_start; - FLUSH_BLOCK(s, 0); - } - /* Flush if we may have to slide, otherwise block_start may become - * negative and the data will be gone: - */ - if (s->strstart - (unsigned int)s->block_start >= MAX_DIST(s)) { - FLUSH_BLOCK(s, 0); - } - } - s->insert = 0; - if (flush == Z_FINISH) { - FLUSH_BLOCK(s, 1); - return finish_done; - } - if ((long)s->strstart > s->block_start) - FLUSH_BLOCK(s, 0); - return block_done; -} - - -/* =========================================================================== - * For Z_RLE, simply look for runs of bytes, generate matches only of distance - * one. Do not maintain a hash table. (It will be regenerated if this run of - * deflate switches away from Z_RLE.) - */ -local block_state deflate_rle(deflate_state *s, int flush) { - int bflush; /* set if current block must be flushed */ - unsigned int prev; /* byte at distance one to match */ - unsigned char *scan, *strend; /* scan goes up to strend for length of run */ - - for (;;) { - /* Make sure that we always have enough lookahead, except - * at the end of the input file. We need MAX_MATCH bytes - * for the longest run, plus one for the unrolled loop. - */ - if (s->lookahead <= MAX_MATCH) { - fill_window(s); - if (s->lookahead <= MAX_MATCH && flush == Z_NO_FLUSH) { - return need_more; - } - if (s->lookahead == 0) - break; /* flush the current block */ - } - - /* See how many times the previous byte repeats */ - s->match_length = 0; - if (s->lookahead >= MIN_MATCH && s->strstart > 0) { - scan = s->window + s->strstart - 1; - prev = *scan; - if (prev == *++scan && prev == *++scan && prev == *++scan) { - strend = s->window + s->strstart + MAX_MATCH; - do { - } while (prev == *++scan && prev == *++scan && - prev == *++scan && prev == *++scan && - prev == *++scan && prev == *++scan && - prev == *++scan && prev == *++scan && - scan < strend); - s->match_length = MAX_MATCH - (int)(strend - scan); - if (s->match_length > s->lookahead) - s->match_length = s->lookahead; - } - Assert(scan <= s->window+(unsigned int)(s->window_size-1), "wild scan"); - } - - /* Emit match if have run of MIN_MATCH or longer, else emit literal */ - if (s->match_length >= MIN_MATCH) { - check_match(s, s->strstart, s->strstart - 1, s->match_length); - - _tr_tally_dist(s, 1, s->match_length - MIN_MATCH, bflush); - - s->lookahead -= s->match_length; - s->strstart += s->match_length; - s->match_length = 0; - } else { - /* No match, output a literal byte */ - Tracevv((stderr, "%c", s->window[s->strstart])); - _tr_tally_lit(s, s->window[s->strstart], bflush); - s->lookahead--; - s->strstart++; - } - if (bflush) - FLUSH_BLOCK(s, 0); - } - s->insert = 0; - if (flush == Z_FINISH) { - FLUSH_BLOCK(s, 1); - return finish_done; - } - if (s->last_lit) - FLUSH_BLOCK(s, 0); - return block_done; -} - -/* =========================================================================== - * For Z_HUFFMAN_ONLY, do not look for matches. Do not maintain a hash table. - * (It will be regenerated if this run of deflate switches away from Huffman.) - */ -local block_state deflate_huff(deflate_state *s, int flush) { - int bflush; /* set if current block must be flushed */ - - for (;;) { - /* Make sure that we have a literal to write. */ - if (s->lookahead == 0) { - fill_window(s); - if (s->lookahead == 0) { - if (flush == Z_NO_FLUSH) - return need_more; - break; /* flush the current block */ - } - } - - /* Output a literal byte */ - s->match_length = 0; - Tracevv((stderr, "%c", s->window[s->strstart])); - _tr_tally_lit(s, s->window[s->strstart], bflush); - s->lookahead--; - s->strstart++; - if (bflush) - FLUSH_BLOCK(s, 0); - } - s->insert = 0; - if (flush == Z_FINISH) { - FLUSH_BLOCK(s, 1); - return finish_done; - } - if (s->last_lit) - FLUSH_BLOCK(s, 0); - return block_done; -} diff --git a/contrib/libzlib-ng/deflate.h b/contrib/libzlib-ng/deflate.h deleted file mode 100644 index 2d3202641d8..00000000000 --- a/contrib/libzlib-ng/deflate.h +++ /dev/null @@ -1,459 +0,0 @@ -#ifndef DEFLATE_H_ -#define DEFLATE_H_ -/* deflate.h -- internal compression state - * Copyright (C) 1995-2012 Jean-loup Gailly - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* WARNING: this file should *not* be used by applications. It is - part of the implementation of the compression library and is - subject to change. Applications should only use zlib.h. - */ - -/* @(#) $Id$ */ - -#include "zutil.h" - -/* define NO_GZIP when compiling if you want to disable gzip header and - trailer creation by deflate(). NO_GZIP would be used to avoid linking in - the crc code when it is not needed. For shared libraries, gzip encoding - should be left enabled. */ -#ifndef NO_GZIP -# define GZIP -#endif - -#define NIL 0 -/* Tail of hash chains */ - - -/* =========================================================================== - * Internal compression state. - */ - -#define LENGTH_CODES 29 -/* number of length codes, not counting the special END_BLOCK code */ - -#define LITERALS 256 -/* number of literal bytes 0..255 */ - -#define L_CODES (LITERALS+1+LENGTH_CODES) -/* number of Literal or Length codes, including the END_BLOCK code */ - -#define D_CODES 30 -/* number of distance codes */ - -#define BL_CODES 19 -/* number of codes used to transfer the bit lengths */ - -#define HEAP_SIZE (2*L_CODES+1) -/* maximum heap size */ - -#define MAX_BITS 15 -/* All codes must not exceed MAX_BITS bits */ - -#define Buf_size 16 -/* size of bit buffer in bi_buf */ - -#define END_BLOCK 256 -/* end of block literal code */ - -#define INIT_STATE 42 -#define EXTRA_STATE 69 -#define NAME_STATE 73 -#define COMMENT_STATE 91 -#define HCRC_STATE 103 -#define BUSY_STATE 113 -#define FINISH_STATE 666 -/* Stream status */ - - -/* Data structure describing a single value and its code string. */ -typedef struct ct_data_s { - union { - uint16_t freq; /* frequency count */ - uint16_t code; /* bit string */ - } fc; - union { - uint16_t dad; /* father node in Huffman tree */ - uint16_t len; /* length of bit string */ - } dl; -} ct_data; - -#define Freq fc.freq -#define Code fc.code -#define Dad dl.dad -#define Len dl.len - -typedef struct static_tree_desc_s static_tree_desc; - -typedef struct tree_desc_s { - ct_data *dyn_tree; /* the dynamic tree */ - int max_code; /* largest code with non zero frequency */ - const static_tree_desc *stat_desc; /* the corresponding static tree */ -} tree_desc; - -typedef uint16_t Pos; -typedef unsigned IPos; - -/* A Pos is an index in the character window. We use short instead of int to - * save space in the various tables. IPos is used only for parameter passing. - */ - -typedef struct internal_state { - z_stream *strm; /* pointer back to this zlib stream */ - int status; /* as the name implies */ - unsigned char *pending_buf; /* output still pending */ - unsigned long pending_buf_size; /* size of pending_buf */ - unsigned char *pending_out; /* next pending byte to output to the stream */ - unsigned int pending; /* nb of bytes in the pending buffer */ - int wrap; /* bit 0 true for zlib, bit 1 true for gzip */ - gz_headerp gzhead; /* gzip header information to write */ - unsigned int gzindex; /* where in extra, name, or comment */ - unsigned char method; /* can only be DEFLATED */ - int last_flush; /* value of flush param for previous deflate call */ - -#ifdef X86_PCLMULQDQ_CRC - unsigned ALIGNED_(16) crc0[4 * 5]; -#endif - - /* used by deflate.c: */ - - unsigned int w_size; /* LZ77 window size (32K by default) */ - unsigned int w_bits; /* log2(w_size) (8..16) */ - unsigned int w_mask; /* w_size - 1 */ - - unsigned char *window; - /* Sliding window. Input bytes are read into the second half of the window, - * and move to the first half later to keep a dictionary of at least wSize - * bytes. With this organization, matches are limited to a distance of - * wSize-MAX_MATCH bytes, but this ensures that IO is always - * performed with a length multiple of the block size. Also, it limits - * the window size to 64K, which is quite useful on MSDOS. - * To do: use the user input buffer as sliding window. - */ - - unsigned long window_size; - /* Actual size of window: 2*wSize, except when the user input buffer - * is directly used as sliding window. - */ - - Pos *prev; - /* Link to older string with same hash index. To limit the size of this - * array to 64K, this link is maintained only for the last 32K strings. - * An index in this array is thus a window index modulo 32K. - */ - - Pos *head; /* Heads of the hash chains or NIL. */ - - unsigned int ins_h; /* hash index of string to be inserted */ - unsigned int hash_size; /* number of elements in hash table */ - unsigned int hash_bits; /* log2(hash_size) */ - unsigned int hash_mask; /* hash_size-1 */ - - unsigned int hash_shift; - /* Number of bits by which ins_h must be shifted at each input - * step. It must be such that after MIN_MATCH steps, the oldest - * byte no longer takes part in the hash key, that is: - * hash_shift * MIN_MATCH >= hash_bits - */ - - long block_start; - /* Window position at the beginning of the current output block. Gets - * negative when the window is moved backwards. - */ - - unsigned int match_length; /* length of best match */ - IPos prev_match; /* previous match */ - int match_available; /* set if previous match exists */ - unsigned int strstart; /* start of string to insert */ - unsigned int match_start; /* start of matching string */ - unsigned int lookahead; /* number of valid bytes ahead in window */ - - unsigned int prev_length; - /* Length of the best match at previous step. Matches not greater than this - * are discarded. This is used in the lazy match evaluation. - */ - - unsigned int max_chain_length; - /* To speed up deflation, hash chains are never searched beyond this - * length. A higher limit improves compression ratio but degrades the - * speed. - */ - - unsigned int max_lazy_match; - /* Attempt to find a better match only when the current match is strictly - * smaller than this value. This mechanism is used only for compression - * levels >= 4. - */ -# define max_insert_length max_lazy_match - /* Insert new strings in the hash table only if the match length is not - * greater than this length. This saves time but degrades compression. - * max_insert_length is used only for compression levels <= 3. - */ - - int level; /* compression level (1..9) */ - int strategy; /* favor or force Huffman coding*/ - - unsigned int good_match; - /* Use a faster search when the previous match is longer than this */ - - int nice_match; /* Stop searching when current match exceeds this */ - - /* used by trees.c: */ - /* Didn't use ct_data typedef below to suppress compiler warning */ - struct ct_data_s dyn_ltree[HEAP_SIZE]; /* literal and length tree */ - struct ct_data_s dyn_dtree[2*D_CODES+1]; /* distance tree */ - struct ct_data_s bl_tree[2*BL_CODES+1]; /* Huffman tree for bit lengths */ - - struct tree_desc_s l_desc; /* desc. for literal tree */ - struct tree_desc_s d_desc; /* desc. for distance tree */ - struct tree_desc_s bl_desc; /* desc. for bit length tree */ - - uint16_t bl_count[MAX_BITS+1]; - /* number of codes at each bit length for an optimal tree */ - - int heap[2*L_CODES+1]; /* heap used to build the Huffman trees */ - int heap_len; /* number of elements in the heap */ - int heap_max; /* element of largest frequency */ - /* The sons of heap[n] are heap[2*n] and heap[2*n+1]. heap[0] is not used. - * The same heap array is used to build all trees. - */ - - unsigned char depth[2*L_CODES+1]; - /* Depth of each subtree used as tie breaker for trees of equal frequency - */ - - unsigned char *l_buf; /* buffer for literals or lengths */ - - unsigned int lit_bufsize; - /* Size of match buffer for literals/lengths. There are 4 reasons for - * limiting lit_bufsize to 64K: - * - frequencies can be kept in 16 bit counters - * - if compression is not successful for the first block, all input - * data is still in the window so we can still emit a stored block even - * when input comes from standard input. (This can also be done for - * all blocks if lit_bufsize is not greater than 32K.) - * - if compression is not successful for a file smaller than 64K, we can - * even emit a stored file instead of a stored block (saving 5 bytes). - * This is applicable only for zip (not gzip or zlib). - * - creating new Huffman trees less frequently may not provide fast - * adaptation to changes in the input data statistics. (Take for - * example a binary file with poorly compressible code followed by - * a highly compressible string table.) Smaller buffer sizes give - * fast adaptation but have of course the overhead of transmitting - * trees more frequently. - * - I can't count above 4 - */ - - unsigned int last_lit; /* running index in l_buf */ - - uint16_t *d_buf; - /* Buffer for distances. To simplify the code, d_buf and l_buf have - * the same number of elements. To use different lengths, an extra flag - * array would be necessary. - */ - - unsigned long opt_len; /* bit length of current block with optimal trees */ - unsigned long static_len; /* bit length of current block with static trees */ - unsigned int matches; /* number of string matches in current block */ - unsigned int insert; /* bytes at end of window left to insert */ - -#ifdef DEBUG - unsigned long compressed_len; /* total bit length of compressed file mod 2^32 */ - unsigned long bits_sent; /* bit length of compressed data sent mod 2^32 */ -#endif - - uint16_t bi_buf; - /* Output buffer. bits are inserted starting at the bottom (least - * significant bits). - */ - int bi_valid; - /* Number of valid bits in bi_buf. All bits above the last valid bit - * are always zero. - */ - - unsigned long high_water; - /* High water mark offset in window for initialized bytes -- bytes above - * this are set to zero in order to avoid memory check warnings when - * longest match routines access bytes past the input. This is then - * updated to the new high water mark. - */ -} deflate_state; - -typedef enum { - need_more, /* block not completed, need more input or more output */ - block_done, /* block flush performed */ - finish_started, /* finish started, need only more output at next deflate */ - finish_done /* finish done, accept no more input or output */ -} block_state; - -/* Output a byte on the stream. - * IN assertion: there is enough room in pending_buf. - */ -#define put_byte(s, c) {s->pending_buf[s->pending++] = (c);} - -/* =========================================================================== - * Output a short LSB first on the stream. - * IN assertion: there is enough room in pendingBuf. - */ -#if defined(__x86_64) || defined(__i386_) -/* Compared to the else-clause's implementation, there are few advantages: - * - s->pending is loaded only once (else-clause's implementation needs to - * load s->pending twice due to the alias between s->pending and - * s->pending_buf[]. - * - no instructions for extracting bytes from short. - * - needs less registers - * - stores to adjacent bytes are merged into a single store, albeit at the - * cost of penalty of potentially unaligned access. - */ -#define put_short(s, w) { \ - s->pending += 2; \ - *(uint16_t*)(&s->pending_buf[s->pending - 2]) = (w) ; \ -} -#else -#define put_short(s, w) { \ - put_byte(s, (unsigned char)((w) & 0xff)); \ - put_byte(s, (unsigned char)((uint16_t)(w) >> 8)); \ -} -#endif - -#define MIN_LOOKAHEAD (MAX_MATCH+MIN_MATCH+1) -/* Minimum amount of lookahead, except at the end of the input file. - * See deflate.c for comments about the MIN_MATCH+1. - */ - -#define MAX_DIST(s) ((s)->w_size-MIN_LOOKAHEAD) -/* In order to simplify the code, particularly on 16 bit machines, match - * distances are limited to MAX_DIST instead of WSIZE. - */ - -#define WIN_INIT MAX_MATCH -/* Number of bytes after end of data in window to initialize in order to avoid - memory checker errors from longest match routines */ - - /* in trees.c */ -void ZLIB_INTERNAL _tr_init(deflate_state *s); -int ZLIB_INTERNAL _tr_tally(deflate_state *s, unsigned dist, unsigned lc); -void ZLIB_INTERNAL _tr_flush_block(deflate_state *s, char *buf, unsigned long stored_len, int last); -void ZLIB_INTERNAL _tr_flush_bits(deflate_state *s); -void ZLIB_INTERNAL _tr_align(deflate_state *s); -void ZLIB_INTERNAL _tr_stored_block(deflate_state *s, char *buf, unsigned long stored_len, int last); -void ZLIB_INTERNAL bi_windup(deflate_state *s); - -#define d_code(dist) ((dist) < 256 ? _dist_code[dist] : _dist_code[256+((dist)>>7)]) -/* Mapping from a distance to a distance code. dist is the distance - 1 and - * must not have side effects. _dist_code[256] and _dist_code[257] are never - * used. - */ - -#ifndef DEBUG -/* Inline versions of _tr_tally for speed: */ - -# if defined(GEN_TREES_H) - extern unsigned char ZLIB_INTERNAL _length_code[]; - extern unsigned char ZLIB_INTERNAL _dist_code[]; -# else - extern const unsigned char ZLIB_INTERNAL _length_code[]; - extern const unsigned char ZLIB_INTERNAL _dist_code[]; -# endif - -# define _tr_tally_lit(s, c, flush) \ - { unsigned char cc = (c); \ - s->d_buf[s->last_lit] = 0; \ - s->l_buf[s->last_lit++] = cc; \ - s->dyn_ltree[cc].Freq++; \ - flush = (s->last_lit == s->lit_bufsize-1); \ - } -# define _tr_tally_dist(s, distance, length, flush) \ - { unsigned char len = (length); \ - uint16_t dist = (distance); \ - s->d_buf[s->last_lit] = dist; \ - s->l_buf[s->last_lit++] = len; \ - dist--; \ - s->dyn_ltree[_length_code[len]+LITERALS+1].Freq++; \ - s->dyn_dtree[d_code(dist)].Freq++; \ - flush = (s->last_lit == s->lit_bufsize-1); \ - } -#else -# define _tr_tally_lit(s, c, flush) flush = _tr_tally(s, 0, c) -# define _tr_tally_dist(s, distance, length, flush) \ - flush = _tr_tally(s, distance, length) -#endif - -/* =========================================================================== - * Update a hash value with the given input byte - * IN assertion: all calls to to UPDATE_HASH are made with consecutive - * input characters, so that a running hash key can be computed from the - * previous key instead of complete recalculation each time. - */ -#ifdef X86_SSE4_2_CRC_HASH -#define UPDATE_HASH(s, h, i) \ - do {\ - if (s->level < 6) \ - h = (3483 * (s->window[i]) +\ - 23081* (s->window[i+1]) +\ - 6954 * (s->window[i+2]) +\ - 20947* (s->window[i+3])) & s->hash_mask;\ - else\ - h = (25881* (s->window[i]) +\ - 24674* (s->window[i+1]) +\ - 25811* (s->window[i+2])) & s->hash_mask;\ - } while (0) -#else -# define UPDATE_HASH(s, h, i) (h = (((h) << s->hash_shift) ^ (s->window[i + (MIN_MATCH-1)])) & s->hash_mask) -#endif - -#ifndef DEBUG -# define send_code(s, c, tree) send_bits(s, tree[c].Code, tree[c].Len) -/* Send a code of the given tree. c and tree must not have side effects */ - -#else /* DEBUG */ -# define send_code(s, c, tree) \ - { if (z_verbose > 2) { \ - fprintf(stderr, "\ncd %3d ", (c)); \ - } \ - send_bits(s, tree[c].Code, tree[c].Len); \ - } -#endif - -#ifdef DEBUG -/* =========================================================================== - * Send a value on a given number of bits. - * IN assertion: length <= 16 and value fits in length bits. - */ -local void send_bits(deflate_state *s, int value, int length) { - Tracevv((stderr, " l %2d v %4x ", length, value)); - Assert(length > 0 && length <= 15, "invalid length"); - s->bits_sent += (unsigned long)length; - - /* If not enough room in bi_buf, use (valid) bits from bi_buf and - * (16 - bi_valid) bits from value, leaving (width - (16-bi_valid)) - * unused bits in value. - */ - if (s->bi_valid > (int)Buf_size - length) { - s->bi_buf |= (uint16_t)value << s->bi_valid; - put_short(s, s->bi_buf); - s->bi_buf = (uint16_t)value >> (Buf_size - s->bi_valid); - s->bi_valid += length - Buf_size; - } else { - s->bi_buf |= (uint16_t)value << s->bi_valid; - s->bi_valid += length; - } -} -#else -#define send_bits(s, value, length) \ -{ int len = length;\ - if (s->bi_valid > (int)Buf_size - len) {\ - int val = value;\ - s->bi_buf |= (uint16_t)val << s->bi_valid;\ - put_short(s, s->bi_buf);\ - s->bi_buf = (uint16_t)val >> (Buf_size - s->bi_valid);\ - s->bi_valid += len - Buf_size;\ - } else {\ - s->bi_buf |= (uint16_t)(value) << s->bi_valid;\ - s->bi_valid += len;\ - }\ -} -#endif - -#endif /* DEFLATE_H_ */ diff --git a/contrib/libzlib-ng/deflate_fast.c b/contrib/libzlib-ng/deflate_fast.c deleted file mode 100644 index edfe53d7f7d..00000000000 --- a/contrib/libzlib-ng/deflate_fast.c +++ /dev/null @@ -1,114 +0,0 @@ -/* deflate_fast.c -- compress data using the fast strategy of deflation algorithm - * - * Copyright (C) 1995-2013 Jean-loup Gailly and Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -#include "deflate.h" -#include "deflate_p.h" -#include "match.h" - -/* =========================================================================== - * Compress as much as possible from the input stream, return the current - * block state. - * This function does not perform lazy evaluation of matches and inserts - * new strings in the dictionary only for unmatched strings or for short - * matches. It is used only for the fast compression options. - */ -block_state deflate_fast(deflate_state *s, int flush) { - IPos hash_head; /* head of the hash chain */ - int bflush; /* set if current block must be flushed */ - - for (;;) { - /* Make sure that we always have enough lookahead, except - * at the end of the input file. We need MAX_MATCH bytes - * for the next match, plus MIN_MATCH bytes to insert the - * string following the next match. - */ - if (s->lookahead < MIN_LOOKAHEAD) { - fill_window(s); - if (s->lookahead < MIN_LOOKAHEAD && flush == Z_NO_FLUSH) { - return need_more; - } - if (s->lookahead == 0) - break; /* flush the current block */ - } - - /* Insert the string window[strstart .. strstart+2] in the - * dictionary, and set hash_head to the head of the hash chain: - */ - hash_head = NIL; - if (s->lookahead >= MIN_MATCH) { - hash_head = insert_string(s, s->strstart); - } - - /* Find the longest match, discarding those <= prev_length. - * At this point we have always match_length < MIN_MATCH - */ - if (hash_head != NIL && s->strstart - hash_head <= MAX_DIST(s)) { - /* To simplify the code, we prevent matches with the string - * of window index 0 (in particular we have to avoid a match - * of the string with itself at the start of the input file). - */ - s->match_length = longest_match(s, hash_head); - /* longest_match() sets match_start */ - } - if (s->match_length >= MIN_MATCH) { - check_match(s, s->strstart, s->match_start, s->match_length); - - _tr_tally_dist(s, s->strstart - s->match_start, s->match_length - MIN_MATCH, bflush); - - s->lookahead -= s->match_length; - - /* Insert new strings in the hash table only if the match length - * is not too large. This saves time but degrades compression. - */ - if (s->match_length <= s->max_insert_length && s->lookahead >= MIN_MATCH) { - s->match_length--; /* string at strstart already in table */ - s->strstart++; -#ifdef NOT_TWEAK_COMPILER - do { - insert_string(s, s->strstart); - s->strstart++; - /* strstart never exceeds WSIZE-MAX_MATCH, so there are - * always MIN_MATCH bytes ahead. - */ - } while (--s->match_length != 0); -#else - { - bulk_insert_str(s, s->strstart, s->match_length); - s->strstart += s->match_length; - s->match_length = 0; - } -#endif - } else { - s->strstart += s->match_length; - s->match_length = 0; - s->ins_h = s->window[s->strstart]; - UPDATE_HASH(s, s->ins_h, s->strstart+2 - (MIN_MATCH)); -#if MIN_MATCH != 3 - Call UPDATE_HASH() MIN_MATCH-3 more times -#endif - /* If lookahead < MIN_MATCH, ins_h is garbage, but it does not - * matter since it will be recomputed at next deflate call. - */ - } - } else { - /* No match, output a literal byte */ - Tracevv((stderr, "%c", s->window[s->strstart])); - _tr_tally_lit(s, s->window[s->strstart], bflush); - s->lookahead--; - s->strstart++; - } - if (bflush) - FLUSH_BLOCK(s, 0); - } - s->insert = s->strstart < MIN_MATCH-1 ? s->strstart : MIN_MATCH-1; - if (flush == Z_FINISH) { - FLUSH_BLOCK(s, 1); - return finish_done; - } - if (s->last_lit) - FLUSH_BLOCK(s, 0); - return block_done; -} diff --git a/contrib/libzlib-ng/deflate_medium.c b/contrib/libzlib-ng/deflate_medium.c deleted file mode 100644 index 731b8a26850..00000000000 --- a/contrib/libzlib-ng/deflate_medium.c +++ /dev/null @@ -1,322 +0,0 @@ -/* deflate_medium.c -- The deflate_medium deflate strategy - * - * Copyright (C) 2013 Intel Corporation. All rights reserved. - * Authors: - * Arjan van de Ven - * - * For conditions of distribution and use, see copyright notice in zlib.h - */ -#ifdef MEDIUM_STRATEGY -#include "deflate.h" -#include "deflate_p.h" -#include "match.h" - -struct match { - unsigned int match_start; - unsigned int match_length; - unsigned int strstart; - unsigned int orgstart; -}; - -#define MAX_DIST2 ((1 << MAX_WBITS) - MIN_LOOKAHEAD) - -static int tr_tally_dist(deflate_state *s, int distance, int length) { - return _tr_tally(s, distance, length); -} - -static int tr_tally_lit(deflate_state *s, int c) { - return _tr_tally(s, 0, c); -} - -static int emit_match(deflate_state *s, struct match match) { - int flush = 0; - - /* matches that are not long enough we need to emit as litterals */ - if (match.match_length < MIN_MATCH) { - while (match.match_length) { - flush += tr_tally_lit(s, s->window[match.strstart]); - s->lookahead--; - match.strstart++; - match.match_length--; - } - return flush; - } - - check_match(s, match.strstart, match.match_start, match.match_length); - - flush += tr_tally_dist(s, match.strstart - match.match_start, match.match_length - MIN_MATCH); - - s->lookahead -= match.match_length; - return flush; -} - -static void insert_match(deflate_state *s, struct match match) { - if (unlikely(s->lookahead <= match.match_length + MIN_MATCH)) - return; - - /* matches that are not long enough we need to emit as litterals */ - if (match.match_length < MIN_MATCH) { -#ifdef NOT_TWEAK_COMPILER - while (match.match_length) { - match.strstart++; - match.match_length--; - - if (match.match_length) { - if (match.strstart >= match.orgstart) { - insert_string(s, match.strstart); - } - } - } -#else - if (likely(match.match_length == 1)) { - match.strstart++; - match.match_length = 0; - }else{ - match.strstart++; - match.match_length--; - if (match.strstart >= match.orgstart) { - bulk_insert_str(s, match.strstart, match.match_length); - } - match.strstart += match.match_length; - match.match_length = 0; - } -#endif - return; - } - - /* Insert new strings in the hash table only if the match length - * is not too large. This saves time but degrades compression. - */ - if (match.match_length <= 16* s->max_insert_length && s->lookahead >= MIN_MATCH) { - match.match_length--; /* string at strstart already in table */ - match.strstart++; -#ifdef NOT_TWEAK_COMPILER - do { - if (likely(match.strstart >= match.orgstart)) { - insert_string(s, match.strstart); - } - match.strstart++; - /* strstart never exceeds WSIZE-MAX_MATCH, so there are - * always MIN_MATCH bytes ahead. - */ - } while (--match.match_length != 0); -#else - if (likely(match.strstart >= match.orgstart)) { - bulk_insert_str(s, match.strstart, match.match_length); - } - match.strstart += match.match_length; - match.match_length = 0; -#endif - } else { - match.strstart += match.match_length; - match.match_length = 0; - s->ins_h = s->window[match.strstart]; - if (match.strstart >= 1) - UPDATE_HASH(s, s->ins_h, match.strstart+2-MIN_MATCH); -#if MIN_MATCH != 3 -#warning Call UPDATE_HASH() MIN_MATCH-3 more times -#endif - /* If lookahead < MIN_MATCH, ins_h is garbage, but it does not - * matter since it will be recomputed at next deflate call. - */ - } -} - -static void fizzle_matches(deflate_state *s, struct match *current, struct match *next) { - IPos limit; - unsigned char *match, *orig; - int changed = 0; - struct match c, n; - /* step zero: sanity checks */ - - if (current->match_length <= 1) - return; - - if (unlikely(current->match_length > 1 + next->match_start)) - return; - - if (unlikely(current->match_length > 1 + next->strstart)) - return; - - match = s->window - current->match_length + 1 + next->match_start; - orig = s->window - current->match_length + 1 + next->strstart; - - /* quick exit check.. if this fails then don't bother with anything else */ - if (likely(*match != *orig)) - return; - - /* check the overlap case and just give up. We can do better in theory, - * but unlikely to be worth it - */ - if (next->match_start + next->match_length >= current->strstart) - return; - - c = *current; - n = *next; - - /* step one: try to move the "next" match to the left as much as possible */ - limit = next->strstart > MAX_DIST2 ? next->strstart - MAX_DIST2 : 0; - - match = s->window + n.match_start - 1; - orig = s->window + n.strstart - 1; - - while (*match == *orig) { - if (c.match_length < 1) - break; - if (n.strstart <= limit) - break; - if (n.match_length >= 256) - break; - if (n.match_start <= 0) - break; - - n.strstart--; - n.match_start--; - n.match_length++; - c.match_length--; - match--; - orig--; - changed++; - } - - if (!changed) - return; - - if (c.match_length <= 1 && n.match_length != 2) { - n.orgstart++; - *current = c; - *next = n; - } else { - return; - } -} - -block_state deflate_medium(deflate_state *s, int flush) { - struct match current_match, next_match; - - memset(¤t_match, 0, sizeof(struct match)); - memset(&next_match, 0, sizeof(struct match)); - - for (;;) { - IPos hash_head = 0; /* head of the hash chain */ - int bflush; /* set if current block must be flushed */ - - /* Make sure that we always have enough lookahead, except - * at the end of the input file. We need MAX_MATCH bytes - * for the next match, plus MIN_MATCH bytes to insert the - * string following the next current_match. - */ - if (s->lookahead < MIN_LOOKAHEAD) { - fill_window(s); - if (s->lookahead < MIN_LOOKAHEAD && flush == Z_NO_FLUSH) { - return need_more; - } - if (s->lookahead == 0) - break; /* flush the current block */ - next_match.match_length = 0; - } - s->prev_length = 2; - - /* Insert the string window[strstart .. strstart+2] in the - * dictionary, and set hash_head to the head of the hash chain: - */ - - /* If we already have a future match from a previous round, just use that */ - if (next_match.match_length > 0) { - current_match = next_match; - next_match.match_length = 0; - - } else { - hash_head = 0; - if (s->lookahead >= MIN_MATCH) { - hash_head = insert_string(s, s->strstart); - } - - /* set up the initial match to be a 1 byte literal */ - current_match.match_start = 0; - current_match.match_length = 1; - current_match.strstart = s->strstart; - current_match.orgstart = current_match.strstart; - - /* Find the longest match, discarding those <= prev_length. - * At this point we have always match_length < MIN_MATCH - */ - - if (hash_head != 0 && s->strstart - hash_head <= MAX_DIST2) { - /* To simplify the code, we prevent matches with the string - * of window index 0 (in particular we have to avoid a match - * of the string with itself at the start of the input file). - */ - current_match.match_length = longest_match(s, hash_head); - current_match.match_start = s->match_start; - if (current_match.match_length < MIN_MATCH) - current_match.match_length = 1; - if (current_match.match_start >= current_match.strstart) { - /* this can happen due to some restarts */ - current_match.match_length = 1; - } - } - } - - insert_match(s, current_match); - - /* now, look ahead one */ - if (s->lookahead > MIN_LOOKAHEAD) { - s->strstart = current_match.strstart + current_match.match_length; - hash_head = insert_string(s, s->strstart); - - /* set up the initial match to be a 1 byte literal */ - next_match.match_start = 0; - next_match.match_length = 1; - next_match.strstart = s->strstart; - next_match.orgstart = next_match.strstart; - - /* Find the longest match, discarding those <= prev_length. - * At this point we have always match_length < MIN_MATCH - */ - if (hash_head != 0 && s->strstart - hash_head <= MAX_DIST2) { - /* To simplify the code, we prevent matches with the string - * of window index 0 (in particular we have to avoid a match - * of the string with itself at the start of the input file). - */ - next_match.match_length = longest_match(s, hash_head); - next_match.match_start = s->match_start; - if (next_match.match_start >= next_match.strstart) { - /* this can happen due to some restarts */ - next_match.match_length = 1; - } - if (next_match.match_length < MIN_MATCH) - next_match.match_length = 1; - else - fizzle_matches(s, ¤t_match, &next_match); - } - - /* short matches with a very long distance are rarely a good idea encoding wise */ - if (next_match.match_length == 3 && (next_match.strstart - next_match.match_start) > 12000) - next_match.match_length = 1; - s->strstart = current_match.strstart; - - } else { - next_match.match_length = 0; - } - - /* now emit the current match */ - bflush = emit_match(s, current_match); - - /* move the "cursor" forward */ - s->strstart += current_match.match_length; - - if (bflush) - FLUSH_BLOCK(s, 0); - } - s->insert = s->strstart < MIN_MATCH-1 ? s->strstart : MIN_MATCH-1; - if (flush == Z_FINISH) { - FLUSH_BLOCK(s, 1); - return finish_done; - } - if (s->last_lit) - FLUSH_BLOCK(s, 0); - - return block_done; -} -#endif diff --git a/contrib/libzlib-ng/deflate_p.h b/contrib/libzlib-ng/deflate_p.h deleted file mode 100644 index 4b8282d46bc..00000000000 --- a/contrib/libzlib-ng/deflate_p.h +++ /dev/null @@ -1,96 +0,0 @@ -/* deflate_p.h -- Private inline functions and macros shared with more than - * one deflate method - * - * Copyright (C) 1995-2013 Jean-loup Gailly and Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - * - */ - -#ifndef DEFLATE_P_H -#define DEFLATE_P_H - -#if defined(X86_CPUID) -# include "arch/x86/x86.h" -#endif - -/* Forward declare common non-inlined functions declared in deflate.c */ - -#ifdef DEBUG -void check_match(deflate_state *s, IPos start, IPos match, int length); -#else -#define check_match(s, start, match, length) -#endif -void fill_window(deflate_state *s); -void flush_pending(z_stream *strm); - -/* =========================================================================== - * Insert string str in the dictionary and set match_head to the previous head - * of the hash chain (the most recent string with same hash key). Return - * the previous length of the hash chain. - * IN assertion: all calls to to INSERT_STRING are made with consecutive - * input characters and the first MIN_MATCH bytes of str are valid - * (except for the last MIN_MATCH-1 bytes of the input file). - */ - -#ifdef X86_SSE4_2_CRC_HASH -extern Pos insert_string_sse(deflate_state *const s, const Pos str, uInt count); -#endif - -local inline Pos insert_string_c(deflate_state *const s, const Pos str, uInt count) { - Pos ret = 0; - uInt idx; - - for (idx = 0; idx < count; idx++) { - UPDATE_HASH(s, s->ins_h, str+idx); - if (s->head[s->ins_h] != str+idx) { - s->prev[(str+idx) & s->w_mask] = s->head[s->ins_h]; - s->head[s->ins_h] = str+idx; - } - } - ret = s->prev[(str+count-1) & s->w_mask]; - return ret; -} - -local inline Pos insert_string(deflate_state *const s, const Pos str) { -#ifdef X86_SSE4_2_CRC_HASH - if (x86_cpu_has_sse42) - return insert_string_sse(s, str, 1); -#endif - return insert_string_c(s, str, 1); -} - -#ifndef NOT_TWEAK_COMPILER -local inline void bulk_insert_str(deflate_state *const s, Pos startpos, uInt count) { -# ifdef X86_SSE4_2_CRC_HASH - if (x86_cpu_has_sse42) { - insert_string_sse(s, startpos, count); - } else -# endif - { - insert_string_c(s, startpos, count); - } -} -#endif /* NOT_TWEAK_COMPILER */ - -/* =========================================================================== - * Flush the current block, with given end-of-file flag. - * IN assertion: strstart is set to the end of the current match. - */ -#define FLUSH_BLOCK_ONLY(s, last) { \ - _tr_flush_block(s, (s->block_start >= 0L ? \ - (char *)&s->window[(unsigned)s->block_start] : \ - (char *)Z_NULL), \ - (ulg)((long)s->strstart - s->block_start), \ - (last)); \ - s->block_start = s->strstart; \ - flush_pending(s->strm); \ - Tracev((stderr, "[FLUSH]")); \ -} - -/* Same but force premature exit if necessary. */ -#define FLUSH_BLOCK(s, last) { \ - FLUSH_BLOCK_ONLY(s, last); \ - if (s->strm->avail_out == 0) return (last) ? finish_started : need_more; \ -} - -#endif diff --git a/contrib/libzlib-ng/deflate_slow.c b/contrib/libzlib-ng/deflate_slow.c deleted file mode 100644 index 6a855f0c837..00000000000 --- a/contrib/libzlib-ng/deflate_slow.c +++ /dev/null @@ -1,160 +0,0 @@ -/* deflate_slow.c -- compress data using the slow strategy of deflation algorithm - * - * Copyright (C) 1995-2013 Jean-loup Gailly and Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -#include "deflate.h" -#include "deflate_p.h" -#include "match.h" - -/* =========================================================================== - * Local data - */ - -#ifndef TOO_FAR -# define TOO_FAR 4096 -#endif -/* Matches of length 3 are discarded if their distance exceeds TOO_FAR */ - -/* =========================================================================== - * Same as deflate_medium, but achieves better compression. We use a lazy - * evaluation for matches: a match is finally adopted only if there is - * no better match at the next window position. - */ -block_state deflate_slow(deflate_state *s, int flush) { - IPos hash_head; /* head of hash chain */ - int bflush; /* set if current block must be flushed */ - - /* Process the input block. */ - for (;;) { - /* Make sure that we always have enough lookahead, except - * at the end of the input file. We need MAX_MATCH bytes - * for the next match, plus MIN_MATCH bytes to insert the - * string following the next match. - */ - if (s->lookahead < MIN_LOOKAHEAD) { - fill_window(s); - if (s->lookahead < MIN_LOOKAHEAD && flush == Z_NO_FLUSH) { - return need_more; - } - if (s->lookahead == 0) - break; /* flush the current block */ - } - - /* Insert the string window[strstart .. strstart+2] in the - * dictionary, and set hash_head to the head of the hash chain: - */ - hash_head = NIL; - if (s->lookahead >= MIN_MATCH) { - hash_head = insert_string(s, s->strstart); - } - - /* Find the longest match, discarding those <= prev_length. - */ - s->prev_length = s->match_length, s->prev_match = s->match_start; - s->match_length = MIN_MATCH-1; - - if (hash_head != NIL && s->prev_length < s->max_lazy_match && s->strstart - hash_head <= MAX_DIST(s)) { - /* To simplify the code, we prevent matches with the string - * of window index 0 (in particular we have to avoid a match - * of the string with itself at the start of the input file). - */ - s->match_length = longest_match(s, hash_head); - /* longest_match() sets match_start */ - - if (s->match_length <= 5 && (s->strategy == Z_FILTERED -#if TOO_FAR <= 32767 - || (s->match_length == MIN_MATCH && s->strstart - s->match_start > TOO_FAR) -#endif - )) { - - /* If prev_match is also MIN_MATCH, match_start is garbage - * but we will ignore the current match anyway. - */ - s->match_length = MIN_MATCH-1; - } - } - /* If there was a match at the previous step and the current - * match is not better, output the previous match: - */ - if (s->prev_length >= MIN_MATCH && s->match_length <= s->prev_length) { - uInt max_insert = s->strstart + s->lookahead - MIN_MATCH; - /* Do not insert strings in hash table beyond this. */ - - check_match(s, s->strstart-1, s->prev_match, s->prev_length); - - _tr_tally_dist(s, s->strstart -1 - s->prev_match, s->prev_length - MIN_MATCH, bflush); - - /* Insert in hash table all strings up to the end of the match. - * strstart-1 and strstart are already inserted. If there is not - * enough lookahead, the last two strings are not inserted in - * the hash table. - */ - s->lookahead -= s->prev_length-1; - -#ifdef NOT_TWEAK_COMPILER - s->prev_length -= 2; - do { - if (++s->strstart <= max_insert) { - insert_string(s, s->strstart); - } - } while (--s->prev_length != 0); - s->match_available = 0; - s->match_length = MIN_MATCH-1; - s->strstart++; -#else - { - uInt mov_fwd = s->prev_length - 2; - uInt insert_cnt = mov_fwd; - if (unlikely(insert_cnt > max_insert - s->strstart)) - insert_cnt = max_insert - s->strstart; - - bulk_insert_str(s, s->strstart + 1, insert_cnt); - s->prev_length = 0; - s->match_available = 0; - s->match_length = MIN_MATCH-1; - s->strstart += mov_fwd + 1; - } -#endif /*NOT_TWEAK_COMPILER*/ - - if (bflush) FLUSH_BLOCK(s, 0); - - } else if (s->match_available) { - /* If there was no match at the previous position, output a - * single literal. If there was a match but the current match - * is longer, truncate the previous match to a single literal. - */ - Tracevv((stderr, "%c", s->window[s->strstart-1])); - _tr_tally_lit(s, s->window[s->strstart-1], bflush); - if (bflush) { - FLUSH_BLOCK_ONLY(s, 0); - } - s->strstart++; - s->lookahead--; - if (s->strm->avail_out == 0) - return need_more; - } else { - /* There is no previous match to compare with, wait for - * the next step to decide. - */ - s->match_available = 1; - s->strstart++; - s->lookahead--; - } - } - Assert(flush != Z_NO_FLUSH, "no flush?"); - if (s->match_available) { - Tracevv((stderr, "%c", s->window[s->strstart-1])); - _tr_tally_lit(s, s->window[s->strstart-1], bflush); - s->match_available = 0; - } - s->insert = s->strstart < MIN_MATCH-1 ? s->strstart : MIN_MATCH-1; - if (flush == Z_FINISH) { - FLUSH_BLOCK(s, 1); - return finish_done; - } - if (s->last_lit) - FLUSH_BLOCK(s, 0); - return block_done; -} diff --git a/contrib/libzlib-ng/doc/algorithm.txt b/contrib/libzlib-ng/doc/algorithm.txt deleted file mode 100644 index c97f495020b..00000000000 --- a/contrib/libzlib-ng/doc/algorithm.txt +++ /dev/null @@ -1,209 +0,0 @@ -1. Compression algorithm (deflate) - -The deflation algorithm used by gzip (also zip and zlib) is a variation of -LZ77 (Lempel-Ziv 1977, see reference below). It finds duplicated strings in -the input data. The second occurrence of a string is replaced by a -pointer to the previous string, in the form of a pair (distance, -length). Distances are limited to 32K bytes, and lengths are limited -to 258 bytes. When a string does not occur anywhere in the previous -32K bytes, it is emitted as a sequence of literal bytes. (In this -description, `string' must be taken as an arbitrary sequence of bytes, -and is not restricted to printable characters.) - -Literals or match lengths are compressed with one Huffman tree, and -match distances are compressed with another tree. The trees are stored -in a compact form at the start of each block. The blocks can have any -size (except that the compressed data for one block must fit in -available memory). A block is terminated when deflate() determines that -it would be useful to start another block with fresh trees. (This is -somewhat similar to the behavior of LZW-based _compress_.) - -Duplicated strings are found using a hash table. All input strings of -length 3 are inserted in the hash table. A hash index is computed for -the next 3 bytes. If the hash chain for this index is not empty, all -strings in the chain are compared with the current input string, and -the longest match is selected. - -The hash chains are searched starting with the most recent strings, to -favor small distances and thus take advantage of the Huffman encoding. -The hash chains are singly linked. There are no deletions from the -hash chains, the algorithm simply discards matches that are too old. - -To avoid a worst-case situation, very long hash chains are arbitrarily -truncated at a certain length, determined by a runtime option (level -parameter of deflateInit). So deflate() does not always find the longest -possible match but generally finds a match which is long enough. - -deflate() also defers the selection of matches with a lazy evaluation -mechanism. After a match of length N has been found, deflate() searches for -a longer match at the next input byte. If a longer match is found, the -previous match is truncated to a length of one (thus producing a single -literal byte) and the process of lazy evaluation begins again. Otherwise, -the original match is kept, and the next match search is attempted only N -steps later. - -The lazy match evaluation is also subject to a runtime parameter. If -the current match is long enough, deflate() reduces the search for a longer -match, thus speeding up the whole process. If compression ratio is more -important than speed, deflate() attempts a complete second search even if -the first match is already long enough. - -The lazy match evaluation is not performed for the fastest compression -modes (level parameter 1 to 3). For these fast modes, new strings -are inserted in the hash table only when no match was found, or -when the match is not too long. This degrades the compression ratio -but saves time since there are both fewer insertions and fewer searches. - - -2. Decompression algorithm (inflate) - -2.1 Introduction - -The key question is how to represent a Huffman code (or any prefix code) so -that you can decode fast. The most important characteristic is that shorter -codes are much more common than longer codes, so pay attention to decoding the -short codes fast, and let the long codes take longer to decode. - -inflate() sets up a first level table that covers some number of bits of -input less than the length of longest code. It gets that many bits from the -stream, and looks it up in the table. The table will tell if the next -code is that many bits or less and how many, and if it is, it will tell -the value, else it will point to the next level table for which inflate() -grabs more bits and tries to decode a longer code. - -How many bits to make the first lookup is a tradeoff between the time it -takes to decode and the time it takes to build the table. If building the -table took no time (and if you had infinite memory), then there would only -be a first level table to cover all the way to the longest code. However, -building the table ends up taking a lot longer for more bits since short -codes are replicated many times in such a table. What inflate() does is -simply to make the number of bits in the first table a variable, and then -to set that variable for the maximum speed. - -For inflate, which has 286 possible codes for the literal/length tree, the size -of the first table is nine bits. Also the distance trees have 30 possible -values, and the size of the first table is six bits. Note that for each of -those cases, the table ended up one bit longer than the ``average'' code -length, i.e. the code length of an approximately flat code which would be a -little more than eight bits for 286 symbols and a little less than five bits -for 30 symbols. - - -2.2 More details on the inflate table lookup - -Ok, you want to know what this cleverly obfuscated inflate tree actually -looks like. You are correct that it's not a Huffman tree. It is simply a -lookup table for the first, let's say, nine bits of a Huffman symbol. The -symbol could be as short as one bit or as long as 15 bits. If a particular -symbol is shorter than nine bits, then that symbol's translation is duplicated -in all those entries that start with that symbol's bits. For example, if the -symbol is four bits, then it's duplicated 32 times in a nine-bit table. If a -symbol is nine bits long, it appears in the table once. - -If the symbol is longer than nine bits, then that entry in the table points -to another similar table for the remaining bits. Again, there are duplicated -entries as needed. The idea is that most of the time the symbol will be short -and there will only be one table look up. (That's whole idea behind data -compression in the first place.) For the less frequent long symbols, there -will be two lookups. If you had a compression method with really long -symbols, you could have as many levels of lookups as is efficient. For -inflate, two is enough. - -So a table entry either points to another table (in which case nine bits in -the above example are gobbled), or it contains the translation for the symbol -and the number of bits to gobble. Then you start again with the next -ungobbled bit. - -You may wonder: why not just have one lookup table for how ever many bits the -longest symbol is? The reason is that if you do that, you end up spending -more time filling in duplicate symbol entries than you do actually decoding. -At least for deflate's output that generates new trees every several 10's of -kbytes. You can imagine that filling in a 2^15 entry table for a 15-bit code -would take too long if you're only decoding several thousand symbols. At the -other extreme, you could make a new table for every bit in the code. In fact, -that's essentially a Huffman tree. But then you spend too much time -traversing the tree while decoding, even for short symbols. - -So the number of bits for the first lookup table is a trade of the time to -fill out the table vs. the time spent looking at the second level and above of -the table. - -Here is an example, scaled down: - -The code being decoded, with 10 symbols, from 1 to 6 bits long: - -A: 0 -B: 10 -C: 1100 -D: 11010 -E: 11011 -F: 11100 -G: 11101 -H: 11110 -I: 111110 -J: 111111 - -Let's make the first table three bits long (eight entries): - -000: A,1 -001: A,1 -010: A,1 -011: A,1 -100: B,2 -101: B,2 -110: -> table X (gobble 3 bits) -111: -> table Y (gobble 3 bits) - -Each entry is what the bits decode as and how many bits that is, i.e. how -many bits to gobble. Or the entry points to another table, with the number of -bits to gobble implicit in the size of the table. - -Table X is two bits long since the longest code starting with 110 is five bits -long: - -00: C,1 -01: C,1 -10: D,2 -11: E,2 - -Table Y is three bits long since the longest code starting with 111 is six -bits long: - -000: F,2 -001: F,2 -010: G,2 -011: G,2 -100: H,2 -101: H,2 -110: I,3 -111: J,3 - -So what we have here are three tables with a total of 20 entries that had to -be constructed. That's compared to 64 entries for a single table. Or -compared to 16 entries for a Huffman tree (six two entry tables and one four -entry table). Assuming that the code ideally represents the probability of -the symbols, it takes on the average 1.25 lookups per symbol. That's compared -to one lookup for the single table, or 1.66 lookups per symbol for the -Huffman tree. - -There, I think that gives you a picture of what's going on. For inflate, the -meaning of a particular symbol is often more than just a letter. It can be a -byte (a "literal"), or it can be either a length or a distance which -indicates a base value and a number of bits to fetch after the code that is -added to the base value. Or it might be the special end-of-block code. The -data structures created in inftrees.c try to encode all that information -compactly in the tables. - - -Jean-loup Gailly Mark Adler -jloup@gzip.org madler@alumni.caltech.edu - - -References: - -[LZ77] Ziv J., Lempel A., ``A Universal Algorithm for Sequential Data -Compression,'' IEEE Transactions on Information Theory, Vol. 23, No. 3, -pp. 337-343. - -``DEFLATE Compressed Data Format Specification'' available in -http://tools.ietf.org/html/rfc1951 diff --git a/contrib/libzlib-ng/doc/rfc1950.txt b/contrib/libzlib-ng/doc/rfc1950.txt deleted file mode 100644 index ce6428a0f2e..00000000000 --- a/contrib/libzlib-ng/doc/rfc1950.txt +++ /dev/null @@ -1,619 +0,0 @@ - - - - - - -Network Working Group P. Deutsch -Request for Comments: 1950 Aladdin Enterprises -Category: Informational J-L. Gailly - Info-ZIP - May 1996 - - - ZLIB Compressed Data Format Specification version 3.3 - -Status of This Memo - - This memo provides information for the Internet community. This memo - does not specify an Internet standard of any kind. Distribution of - this memo is unlimited. - -IESG Note: - - The IESG takes no position on the validity of any Intellectual - Property Rights statements contained in this document. - -Notices - - Copyright (c) 1996 L. Peter Deutsch and Jean-Loup Gailly - - Permission is granted to copy and distribute this document for any - purpose and without charge, including translations into other - languages and incorporation into compilations, provided that the - copyright notice and this notice are preserved, and that any - substantive changes or deletions from the original are clearly - marked. - - A pointer to the latest version of this and related documentation in - HTML format can be found at the URL - . - -Abstract - - This specification defines a lossless compressed data format. The - data can be produced or consumed, even for an arbitrarily long - sequentially presented input data stream, using only an a priori - bounded amount of intermediate storage. The format presently uses - the DEFLATE compression method but can be easily extended to use - other compression methods. It can be implemented readily in a manner - not covered by patents. This specification also defines the ADLER-32 - checksum (an extension and improvement of the Fletcher checksum), - used for detection of data corruption, and provides an algorithm for - computing it. - - - - -Deutsch & Gailly Informational [Page 1] - -RFC 1950 ZLIB Compressed Data Format Specification May 1996 - - -Table of Contents - - 1. Introduction ................................................... 2 - 1.1. Purpose ................................................... 2 - 1.2. Intended audience ......................................... 3 - 1.3. Scope ..................................................... 3 - 1.4. Compliance ................................................ 3 - 1.5. Definitions of terms and conventions used ................ 3 - 1.6. Changes from previous versions ............................ 3 - 2. Detailed specification ......................................... 3 - 2.1. Overall conventions ....................................... 3 - 2.2. Data format ............................................... 4 - 2.3. Compliance ................................................ 7 - 3. References ..................................................... 7 - 4. Source code .................................................... 8 - 5. Security Considerations ........................................ 8 - 6. Acknowledgements ............................................... 8 - 7. Authors' Addresses ............................................. 8 - 8. Appendix: Rationale ............................................ 9 - 9. Appendix: Sample code ..........................................10 - -1. Introduction - - 1.1. Purpose - - The purpose of this specification is to define a lossless - compressed data format that: - - * Is independent of CPU type, operating system, file system, - and character set, and hence can be used for interchange; - - * Can be produced or consumed, even for an arbitrarily long - sequentially presented input data stream, using only an a - priori bounded amount of intermediate storage, and hence can - be used in data communications or similar structures such as - Unix filters; - - * Can use a number of different compression methods; - - * Can be implemented readily in a manner not covered by - patents, and hence can be practiced freely. - - The data format defined by this specification does not attempt to - allow random access to compressed data. - - - - - - - -Deutsch & Gailly Informational [Page 2] - -RFC 1950 ZLIB Compressed Data Format Specification May 1996 - - - 1.2. Intended audience - - This specification is intended for use by implementors of software - to compress data into zlib format and/or decompress data from zlib - format. - - The text of the specification assumes a basic background in - programming at the level of bits and other primitive data - representations. - - 1.3. Scope - - The specification specifies a compressed data format that can be - used for in-memory compression of a sequence of arbitrary bytes. - - 1.4. Compliance - - Unless otherwise indicated below, a compliant decompressor must be - able to accept and decompress any data set that conforms to all - the specifications presented here; a compliant compressor must - produce data sets that conform to all the specifications presented - here. - - 1.5. Definitions of terms and conventions used - - byte: 8 bits stored or transmitted as a unit (same as an octet). - (For this specification, a byte is exactly 8 bits, even on - machines which store a character on a number of bits different - from 8.) See below, for the numbering of bits within a byte. - - 1.6. Changes from previous versions - - Version 3.1 was the first public release of this specification. - In version 3.2, some terminology was changed and the Adler-32 - sample code was rewritten for clarity. In version 3.3, the - support for a preset dictionary was introduced, and the - specification was converted to RFC style. - -2. Detailed specification - - 2.1. Overall conventions - - In the diagrams below, a box like this: - - +---+ - | | <-- the vertical bars might be missing - +---+ - - - - -Deutsch & Gailly Informational [Page 3] - -RFC 1950 ZLIB Compressed Data Format Specification May 1996 - - - represents one byte; a box like this: - - +==============+ - | | - +==============+ - - represents a variable number of bytes. - - Bytes stored within a computer do not have a "bit order", since - they are always treated as a unit. However, a byte considered as - an integer between 0 and 255 does have a most- and least- - significant bit, and since we write numbers with the most- - significant digit on the left, we also write bytes with the most- - significant bit on the left. In the diagrams below, we number the - bits of a byte so that bit 0 is the least-significant bit, i.e., - the bits are numbered: - - +--------+ - |76543210| - +--------+ - - Within a computer, a number may occupy multiple bytes. All - multi-byte numbers in the format described here are stored with - the MOST-significant byte first (at the lower memory address). - For example, the decimal number 520 is stored as: - - 0 1 - +--------+--------+ - |00000010|00001000| - +--------+--------+ - ^ ^ - | | - | + less significant byte = 8 - + more significant byte = 2 x 256 - - 2.2. Data format - - A zlib stream has the following structure: - - 0 1 - +---+---+ - |CMF|FLG| (more-->) - +---+---+ - - - - - - - - -Deutsch & Gailly Informational [Page 4] - -RFC 1950 ZLIB Compressed Data Format Specification May 1996 - - - (if FLG.FDICT set) - - 0 1 2 3 - +---+---+---+---+ - | DICTID | (more-->) - +---+---+---+---+ - - +=====================+---+---+---+---+ - |...compressed data...| ADLER32 | - +=====================+---+---+---+---+ - - Any data which may appear after ADLER32 are not part of the zlib - stream. - - CMF (Compression Method and flags) - This byte is divided into a 4-bit compression method and a 4- - bit information field depending on the compression method. - - bits 0 to 3 CM Compression method - bits 4 to 7 CINFO Compression info - - CM (Compression method) - This identifies the compression method used in the file. CM = 8 - denotes the "deflate" compression method with a window size up - to 32K. This is the method used by gzip and PNG (see - references [1] and [2] in Chapter 3, below, for the reference - documents). CM = 15 is reserved. It might be used in a future - version of this specification to indicate the presence of an - extra field before the compressed data. - - CINFO (Compression info) - For CM = 8, CINFO is the base-2 logarithm of the LZ77 window - size, minus eight (CINFO=7 indicates a 32K window size). Values - of CINFO above 7 are not allowed in this version of the - specification. CINFO is not defined in this specification for - CM not equal to 8. - - FLG (FLaGs) - This flag byte is divided as follows: - - bits 0 to 4 FCHECK (check bits for CMF and FLG) - bit 5 FDICT (preset dictionary) - bits 6 to 7 FLEVEL (compression level) - - The FCHECK value must be such that CMF and FLG, when viewed as - a 16-bit unsigned integer stored in MSB order (CMF*256 + FLG), - is a multiple of 31. - - - - -Deutsch & Gailly Informational [Page 5] - -RFC 1950 ZLIB Compressed Data Format Specification May 1996 - - - FDICT (Preset dictionary) - If FDICT is set, a DICT dictionary identifier is present - immediately after the FLG byte. The dictionary is a sequence of - bytes which are initially fed to the compressor without - producing any compressed output. DICT is the Adler-32 checksum - of this sequence of bytes (see the definition of ADLER32 - below). The decompressor can use this identifier to determine - which dictionary has been used by the compressor. - - FLEVEL (Compression level) - These flags are available for use by specific compression - methods. The "deflate" method (CM = 8) sets these flags as - follows: - - 0 - compressor used fastest algorithm - 1 - compressor used fast algorithm - 2 - compressor used default algorithm - 3 - compressor used maximum compression, slowest algorithm - - The information in FLEVEL is not needed for decompression; it - is there to indicate if recompression might be worthwhile. - - compressed data - For compression method 8, the compressed data is stored in the - deflate compressed data format as described in the document - "DEFLATE Compressed Data Format Specification" by L. Peter - Deutsch. (See reference [3] in Chapter 3, below) - - Other compressed data formats are not specified in this version - of the zlib specification. - - ADLER32 (Adler-32 checksum) - This contains a checksum value of the uncompressed data - (excluding any dictionary data) computed according to Adler-32 - algorithm. This algorithm is a 32-bit extension and improvement - of the Fletcher algorithm, used in the ITU-T X.224 / ISO 8073 - standard. See references [4] and [5] in Chapter 3, below) - - Adler-32 is composed of two sums accumulated per byte: s1 is - the sum of all bytes, s2 is the sum of all s1 values. Both sums - are done modulo 65521. s1 is initialized to 1, s2 to zero. The - Adler-32 checksum is stored as s2*65536 + s1 in most- - significant-byte first (network) order. - - - - - - - - -Deutsch & Gailly Informational [Page 6] - -RFC 1950 ZLIB Compressed Data Format Specification May 1996 - - - 2.3. Compliance - - A compliant compressor must produce streams with correct CMF, FLG - and ADLER32, but need not support preset dictionaries. When the - zlib data format is used as part of another standard data format, - the compressor may use only preset dictionaries that are specified - by this other data format. If this other format does not use the - preset dictionary feature, the compressor must not set the FDICT - flag. - - A compliant decompressor must check CMF, FLG, and ADLER32, and - provide an error indication if any of these have incorrect values. - A compliant decompressor must give an error indication if CM is - not one of the values defined in this specification (only the - value 8 is permitted in this version), since another value could - indicate the presence of new features that would cause subsequent - data to be interpreted incorrectly. A compliant decompressor must - give an error indication if FDICT is set and DICTID is not the - identifier of a known preset dictionary. A decompressor may - ignore FLEVEL and still be compliant. When the zlib data format - is being used as a part of another standard format, a compliant - decompressor must support all the preset dictionaries specified by - the other format. When the other format does not use the preset - dictionary feature, a compliant decompressor must reject any - stream in which the FDICT flag is set. - -3. References - - [1] Deutsch, L.P.,"GZIP Compressed Data Format Specification", - available in ftp://ftp.uu.net/pub/archiving/zip/doc/ - - [2] Thomas Boutell, "PNG (Portable Network Graphics) specification", - available in ftp://ftp.uu.net/graphics/png/documents/ - - [3] Deutsch, L.P.,"DEFLATE Compressed Data Format Specification", - available in ftp://ftp.uu.net/pub/archiving/zip/doc/ - - [4] Fletcher, J. G., "An Arithmetic Checksum for Serial - Transmissions," IEEE Transactions on Communications, Vol. COM-30, - No. 1, January 1982, pp. 247-252. - - [5] ITU-T Recommendation X.224, Annex D, "Checksum Algorithms," - November, 1993, pp. 144, 145. (Available from - gopher://info.itu.ch). ITU-T X.244 is also the same as ISO 8073. - - - - - - - -Deutsch & Gailly Informational [Page 7] - -RFC 1950 ZLIB Compressed Data Format Specification May 1996 - - -4. Source code - - Source code for a C language implementation of a "zlib" compliant - library is available at ftp://ftp.uu.net/pub/archiving/zip/zlib/. - -5. Security Considerations - - A decoder that fails to check the ADLER32 checksum value may be - subject to undetected data corruption. - -6. Acknowledgements - - Trademarks cited in this document are the property of their - respective owners. - - Jean-Loup Gailly and Mark Adler designed the zlib format and wrote - the related software described in this specification. Glenn - Randers-Pehrson converted this document to RFC and HTML format. - -7. Authors' Addresses - - L. Peter Deutsch - Aladdin Enterprises - 203 Santa Margarita Ave. - Menlo Park, CA 94025 - - Phone: (415) 322-0103 (AM only) - FAX: (415) 322-1734 - EMail: - - - Jean-Loup Gailly - - EMail: - - Questions about the technical content of this specification can be - sent by email to - - Jean-Loup Gailly and - Mark Adler - - Editorial comments on this specification can be sent by email to - - L. Peter Deutsch and - Glenn Randers-Pehrson - - - - - - -Deutsch & Gailly Informational [Page 8] - -RFC 1950 ZLIB Compressed Data Format Specification May 1996 - - -8. Appendix: Rationale - - 8.1. Preset dictionaries - - A preset dictionary is specially useful to compress short input - sequences. The compressor can take advantage of the dictionary - context to encode the input in a more compact manner. The - decompressor can be initialized with the appropriate context by - virtually decompressing a compressed version of the dictionary - without producing any output. However for certain compression - algorithms such as the deflate algorithm this operation can be - achieved without actually performing any decompression. - - The compressor and the decompressor must use exactly the same - dictionary. The dictionary may be fixed or may be chosen among a - certain number of predefined dictionaries, according to the kind - of input data. The decompressor can determine which dictionary has - been chosen by the compressor by checking the dictionary - identifier. This document does not specify the contents of - predefined dictionaries, since the optimal dictionaries are - application specific. Standard data formats using this feature of - the zlib specification must precisely define the allowed - dictionaries. - - 8.2. The Adler-32 algorithm - - The Adler-32 algorithm is much faster than the CRC32 algorithm yet - still provides an extremely low probability of undetected errors. - - The modulo on unsigned long accumulators can be delayed for 5552 - bytes, so the modulo operation time is negligible. If the bytes - are a, b, c, the second sum is 3a + 2b + c + 3, and so is position - and order sensitive, unlike the first sum, which is just a - checksum. That 65521 is prime is important to avoid a possible - large class of two-byte errors that leave the check unchanged. - (The Fletcher checksum uses 255, which is not prime and which also - makes the Fletcher check insensitive to single byte changes 0 <-> - 255.) - - The sum s1 is initialized to 1 instead of zero to make the length - of the sequence part of s2, so that the length does not have to be - checked separately. (Any sequence of zeroes has a Fletcher - checksum of zero.) - - - - - - - - -Deutsch & Gailly Informational [Page 9] - -RFC 1950 ZLIB Compressed Data Format Specification May 1996 - - -9. Appendix: Sample code - - The following C code computes the Adler-32 checksum of a data buffer. - It is written for clarity, not for speed. The sample code is in the - ANSI C programming language. Non C users may find it easier to read - with these hints: - - & Bitwise AND operator. - >> Bitwise right shift operator. When applied to an - unsigned quantity, as here, right shift inserts zero bit(s) - at the left. - << Bitwise left shift operator. Left shift inserts zero - bit(s) at the right. - ++ "n++" increments the variable n. - % modulo operator: a % b is the remainder of a divided by b. - - #define BASE 65521 /* largest prime smaller than 65536 */ - - /* - Update a running Adler-32 checksum with the bytes buf[0..len-1] - and return the updated checksum. The Adler-32 checksum should be - initialized to 1. - - Usage example: - - unsigned long adler = 1L; - - while (read_buffer(buffer, length) != EOF) { - adler = update_adler32(adler, buffer, length); - } - if (adler != original_adler) error(); - */ - unsigned long update_adler32(unsigned long adler, - unsigned char *buf, int len) - { - unsigned long s1 = adler & 0xffff; - unsigned long s2 = (adler >> 16) & 0xffff; - int n; - - for (n = 0; n < len; n++) { - s1 = (s1 + buf[n]) % BASE; - s2 = (s2 + s1) % BASE; - } - return (s2 << 16) + s1; - } - - /* Return the adler32 of the bytes buf[0..len-1] */ - - - - -Deutsch & Gailly Informational [Page 10] - -RFC 1950 ZLIB Compressed Data Format Specification May 1996 - - - unsigned long adler32(unsigned char *buf, int len) - { - return update_adler32(1L, buf, len); - } - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -Deutsch & Gailly Informational [Page 11] - diff --git a/contrib/libzlib-ng/doc/rfc1951.txt b/contrib/libzlib-ng/doc/rfc1951.txt deleted file mode 100644 index 403c8c722ff..00000000000 --- a/contrib/libzlib-ng/doc/rfc1951.txt +++ /dev/null @@ -1,955 +0,0 @@ - - - - - - -Network Working Group P. Deutsch -Request for Comments: 1951 Aladdin Enterprises -Category: Informational May 1996 - - - DEFLATE Compressed Data Format Specification version 1.3 - -Status of This Memo - - This memo provides information for the Internet community. This memo - does not specify an Internet standard of any kind. Distribution of - this memo is unlimited. - -IESG Note: - - The IESG takes no position on the validity of any Intellectual - Property Rights statements contained in this document. - -Notices - - Copyright (c) 1996 L. Peter Deutsch - - Permission is granted to copy and distribute this document for any - purpose and without charge, including translations into other - languages and incorporation into compilations, provided that the - copyright notice and this notice are preserved, and that any - substantive changes or deletions from the original are clearly - marked. - - A pointer to the latest version of this and related documentation in - HTML format can be found at the URL - . - -Abstract - - This specification defines a lossless compressed data format that - compresses data using a combination of the LZ77 algorithm and Huffman - coding, with efficiency comparable to the best currently available - general-purpose compression methods. The data can be produced or - consumed, even for an arbitrarily long sequentially presented input - data stream, using only an a priori bounded amount of intermediate - storage. The format can be implemented readily in a manner not - covered by patents. - - - - - - - - -Deutsch Informational [Page 1] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - -Table of Contents - - 1. Introduction ................................................... 2 - 1.1. Purpose ................................................... 2 - 1.2. Intended audience ......................................... 3 - 1.3. Scope ..................................................... 3 - 1.4. Compliance ................................................ 3 - 1.5. Definitions of terms and conventions used ................ 3 - 1.6. Changes from previous versions ............................ 4 - 2. Compressed representation overview ............................. 4 - 3. Detailed specification ......................................... 5 - 3.1. Overall conventions ....................................... 5 - 3.1.1. Packing into bytes .................................. 5 - 3.2. Compressed block format ................................... 6 - 3.2.1. Synopsis of prefix and Huffman coding ............... 6 - 3.2.2. Use of Huffman coding in the "deflate" format ....... 7 - 3.2.3. Details of block format ............................. 9 - 3.2.4. Non-compressed blocks (BTYPE=00) ................... 11 - 3.2.5. Compressed blocks (length and distance codes) ...... 11 - 3.2.6. Compression with fixed Huffman codes (BTYPE=01) .... 12 - 3.2.7. Compression with dynamic Huffman codes (BTYPE=10) .. 13 - 3.3. Compliance ............................................... 14 - 4. Compression algorithm details ................................. 14 - 5. References .................................................... 16 - 6. Security Considerations ....................................... 16 - 7. Source code ................................................... 16 - 8. Acknowledgements .............................................. 16 - 9. Author's Address .............................................. 17 - -1. Introduction - - 1.1. Purpose - - The purpose of this specification is to define a lossless - compressed data format that: - * Is independent of CPU type, operating system, file system, - and character set, and hence can be used for interchange; - * Can be produced or consumed, even for an arbitrarily long - sequentially presented input data stream, using only an a - priori bounded amount of intermediate storage, and hence - can be used in data communications or similar structures - such as Unix filters; - * Compresses data with efficiency comparable to the best - currently available general-purpose compression methods, - and in particular considerably better than the "compress" - program; - * Can be implemented readily in a manner not covered by - patents, and hence can be practiced freely; - - - -Deutsch Informational [Page 2] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - - * Is compatible with the file format produced by the current - widely used gzip utility, in that conforming decompressors - will be able to read data produced by the existing gzip - compressor. - - The data format defined by this specification does not attempt to: - - * Allow random access to compressed data; - * Compress specialized data (e.g., raster graphics) as well - as the best currently available specialized algorithms. - - A simple counting argument shows that no lossless compression - algorithm can compress every possible input data set. For the - format defined here, the worst case expansion is 5 bytes per 32K- - byte block, i.e., a size increase of 0.015% for large data sets. - English text usually compresses by a factor of 2.5 to 3; - executable files usually compress somewhat less; graphical data - such as raster images may compress much more. - - 1.2. Intended audience - - This specification is intended for use by implementors of software - to compress data into "deflate" format and/or decompress data from - "deflate" format. - - The text of the specification assumes a basic background in - programming at the level of bits and other primitive data - representations. Familiarity with the technique of Huffman coding - is helpful but not required. - - 1.3. Scope - - The specification specifies a method for representing a sequence - of bytes as a (usually shorter) sequence of bits, and a method for - packing the latter bit sequence into bytes. - - 1.4. Compliance - - Unless otherwise indicated below, a compliant decompressor must be - able to accept and decompress any data set that conforms to all - the specifications presented here; a compliant compressor must - produce data sets that conform to all the specifications presented - here. - - 1.5. Definitions of terms and conventions used - - Byte: 8 bits stored or transmitted as a unit (same as an octet). - For this specification, a byte is exactly 8 bits, even on machines - - - -Deutsch Informational [Page 3] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - - which store a character on a number of bits different from eight. - See below, for the numbering of bits within a byte. - - String: a sequence of arbitrary bytes. - - 1.6. Changes from previous versions - - There have been no technical changes to the deflate format since - version 1.1 of this specification. In version 1.2, some - terminology was changed. Version 1.3 is a conversion of the - specification to RFC style. - -2. Compressed representation overview - - A compressed data set consists of a series of blocks, corresponding - to successive blocks of input data. The block sizes are arbitrary, - except that non-compressible blocks are limited to 65,535 bytes. - - Each block is compressed using a combination of the LZ77 algorithm - and Huffman coding. The Huffman trees for each block are independent - of those for previous or subsequent blocks; the LZ77 algorithm may - use a reference to a duplicated string occurring in a previous block, - up to 32K input bytes before. - - Each block consists of two parts: a pair of Huffman code trees that - describe the representation of the compressed data part, and a - compressed data part. (The Huffman trees themselves are compressed - using Huffman encoding.) The compressed data consists of a series of - elements of two types: literal bytes (of strings that have not been - detected as duplicated within the previous 32K input bytes), and - pointers to duplicated strings, where a pointer is represented as a - pair . The representation used in the - "deflate" format limits distances to 32K bytes and lengths to 258 - bytes, but does not limit the size of a block, except for - uncompressible blocks, which are limited as noted above. - - Each type of value (literals, distances, and lengths) in the - compressed data is represented using a Huffman code, using one code - tree for literals and lengths and a separate code tree for distances. - The code trees for each block appear in a compact form just before - the compressed data for that block. - - - - - - - - - - -Deutsch Informational [Page 4] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - -3. Detailed specification - - 3.1. Overall conventions In the diagrams below, a box like this: - - +---+ - | | <-- the vertical bars might be missing - +---+ - - represents one byte; a box like this: - - +==============+ - | | - +==============+ - - represents a variable number of bytes. - - Bytes stored within a computer do not have a "bit order", since - they are always treated as a unit. However, a byte considered as - an integer between 0 and 255 does have a most- and least- - significant bit, and since we write numbers with the most- - significant digit on the left, we also write bytes with the most- - significant bit on the left. In the diagrams below, we number the - bits of a byte so that bit 0 is the least-significant bit, i.e., - the bits are numbered: - - +--------+ - |76543210| - +--------+ - - Within a computer, a number may occupy multiple bytes. All - multi-byte numbers in the format described here are stored with - the least-significant byte first (at the lower memory address). - For example, the decimal number 520 is stored as: - - 0 1 - +--------+--------+ - |00001000|00000010| - +--------+--------+ - ^ ^ - | | - | + more significant byte = 2 x 256 - + less significant byte = 8 - - 3.1.1. Packing into bytes - - This document does not address the issue of the order in which - bits of a byte are transmitted on a bit-sequential medium, - since the final data format described here is byte- rather than - - - -Deutsch Informational [Page 5] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - - bit-oriented. However, we describe the compressed block format - in below, as a sequence of data elements of various bit - lengths, not a sequence of bytes. We must therefore specify - how to pack these data elements into bytes to form the final - compressed byte sequence: - - * Data elements are packed into bytes in order of - increasing bit number within the byte, i.e., starting - with the least-significant bit of the byte. - * Data elements other than Huffman codes are packed - starting with the least-significant bit of the data - element. - * Huffman codes are packed starting with the most- - significant bit of the code. - - In other words, if one were to print out the compressed data as - a sequence of bytes, starting with the first byte at the - *right* margin and proceeding to the *left*, with the most- - significant bit of each byte on the left as usual, one would be - able to parse the result from right to left, with fixed-width - elements in the correct MSB-to-LSB order and Huffman codes in - bit-reversed order (i.e., with the first bit of the code in the - relative LSB position). - - 3.2. Compressed block format - - 3.2.1. Synopsis of prefix and Huffman coding - - Prefix coding represents symbols from an a priori known - alphabet by bit sequences (codes), one code for each symbol, in - a manner such that different symbols may be represented by bit - sequences of different lengths, but a parser can always parse - an encoded string unambiguously symbol-by-symbol. - - We define a prefix code in terms of a binary tree in which the - two edges descending from each non-leaf node are labeled 0 and - 1 and in which the leaf nodes correspond one-for-one with (are - labeled with) the symbols of the alphabet; then the code for a - symbol is the sequence of 0's and 1's on the edges leading from - the root to the leaf labeled with that symbol. For example: - - - - - - - - - - - -Deutsch Informational [Page 6] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - - /\ Symbol Code - 0 1 ------ ---- - / \ A 00 - /\ B B 1 - 0 1 C 011 - / \ D 010 - A /\ - 0 1 - / \ - D C - - A parser can decode the next symbol from an encoded input - stream by walking down the tree from the root, at each step - choosing the edge corresponding to the next input bit. - - Given an alphabet with known symbol frequencies, the Huffman - algorithm allows the construction of an optimal prefix code - (one which represents strings with those symbol frequencies - using the fewest bits of any possible prefix codes for that - alphabet). Such a code is called a Huffman code. (See - reference [1] in Chapter 5, references for additional - information on Huffman codes.) - - Note that in the "deflate" format, the Huffman codes for the - various alphabets must not exceed certain maximum code lengths. - This constraint complicates the algorithm for computing code - lengths from symbol frequencies. Again, see Chapter 5, - references for details. - - 3.2.2. Use of Huffman coding in the "deflate" format - - The Huffman codes used for each alphabet in the "deflate" - format have two additional rules: - - * All codes of a given bit length have lexicographically - consecutive values, in the same order as the symbols - they represent; - - * Shorter codes lexicographically precede longer codes. - - - - - - - - - - - - -Deutsch Informational [Page 7] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - - We could recode the example above to follow this rule as - follows, assuming that the order of the alphabet is ABCD: - - Symbol Code - ------ ---- - A 10 - B 0 - C 110 - D 111 - - I.e., 0 precedes 10 which precedes 11x, and 110 and 111 are - lexicographically consecutive. - - Given this rule, we can define the Huffman code for an alphabet - just by giving the bit lengths of the codes for each symbol of - the alphabet in order; this is sufficient to determine the - actual codes. In our example, the code is completely defined - by the sequence of bit lengths (2, 1, 3, 3). The following - algorithm generates the codes as integers, intended to be read - from most- to least-significant bit. The code lengths are - initially in tree[I].Len; the codes are produced in - tree[I].Code. - - 1) Count the number of codes for each code length. Let - bl_count[N] be the number of codes of length N, N >= 1. - - 2) Find the numerical value of the smallest code for each - code length: - - code = 0; - bl_count[0] = 0; - for (bits = 1; bits <= MAX_BITS; bits++) { - code = (code + bl_count[bits-1]) << 1; - next_code[bits] = code; - } - - 3) Assign numerical values to all codes, using consecutive - values for all codes of the same length with the base - values determined at step 2. Codes that are never used - (which have a bit length of zero) must not be assigned a - value. - - for (n = 0; n <= max_code; n++) { - len = tree[n].Len; - if (len != 0) { - tree[n].Code = next_code[len]; - next_code[len]++; - } - - - -Deutsch Informational [Page 8] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - - } - - Example: - - Consider the alphabet ABCDEFGH, with bit lengths (3, 3, 3, 3, - 3, 2, 4, 4). After step 1, we have: - - N bl_count[N] - - ----------- - 2 1 - 3 5 - 4 2 - - Step 2 computes the following next_code values: - - N next_code[N] - - ------------ - 1 0 - 2 0 - 3 2 - 4 14 - - Step 3 produces the following code values: - - Symbol Length Code - ------ ------ ---- - A 3 010 - B 3 011 - C 3 100 - D 3 101 - E 3 110 - F 2 00 - G 4 1110 - H 4 1111 - - 3.2.3. Details of block format - - Each block of compressed data begins with 3 header bits - containing the following data: - - first bit BFINAL - next 2 bits BTYPE - - Note that the header bits do not necessarily begin on a byte - boundary, since a block does not necessarily occupy an integral - number of bytes. - - - - - -Deutsch Informational [Page 9] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - - BFINAL is set if and only if this is the last block of the data - set. - - BTYPE specifies how the data are compressed, as follows: - - 00 - no compression - 01 - compressed with fixed Huffman codes - 10 - compressed with dynamic Huffman codes - 11 - reserved (error) - - The only difference between the two compressed cases is how the - Huffman codes for the literal/length and distance alphabets are - defined. - - In all cases, the decoding algorithm for the actual data is as - follows: - - do - read block header from input stream. - if stored with no compression - skip any remaining bits in current partially - processed byte - read LEN and NLEN (see next section) - copy LEN bytes of data to output - otherwise - if compressed with dynamic Huffman codes - read representation of code trees (see - subsection below) - loop (until end of block code recognized) - decode literal/length value from input stream - if value < 256 - copy value (literal byte) to output stream - otherwise - if value = end of block (256) - break from loop - otherwise (value = 257..285) - decode distance from input stream - - move backwards distance bytes in the output - stream, and copy length bytes from this - position to the output stream. - end loop - while not last block - - Note that a duplicated string reference may refer to a string - in a previous block; i.e., the backward distance may cross one - or more block boundaries. However a distance cannot refer past - the beginning of the output stream. (An application using a - - - -Deutsch Informational [Page 10] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - - preset dictionary might discard part of the output stream; a - distance can refer to that part of the output stream anyway) - Note also that the referenced string may overlap the current - position; for example, if the last 2 bytes decoded have values - X and Y, a string reference with - adds X,Y,X,Y,X to the output stream. - - We now specify each compression method in turn. - - 3.2.4. Non-compressed blocks (BTYPE=00) - - Any bits of input up to the next byte boundary are ignored. - The rest of the block consists of the following information: - - 0 1 2 3 4... - +---+---+---+---+================================+ - | LEN | NLEN |... LEN bytes of literal data...| - +---+---+---+---+================================+ - - LEN is the number of data bytes in the block. NLEN is the - one's complement of LEN. - - 3.2.5. Compressed blocks (length and distance codes) - - As noted above, encoded data blocks in the "deflate" format - consist of sequences of symbols drawn from three conceptually - distinct alphabets: either literal bytes, from the alphabet of - byte values (0..255), or pairs, - where the length is drawn from (3..258) and the distance is - drawn from (1..32,768). In fact, the literal and length - alphabets are merged into a single alphabet (0..285), where - values 0..255 represent literal bytes, the value 256 indicates - end-of-block, and values 257..285 represent length codes - (possibly in conjunction with extra bits following the symbol - code) as follows: - - - - - - - - - - - - - - - - -Deutsch Informational [Page 11] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - - Extra Extra Extra - Code Bits Length(s) Code Bits Lengths Code Bits Length(s) - ---- ---- ------ ---- ---- ------- ---- ---- ------- - 257 0 3 267 1 15,16 277 4 67-82 - 258 0 4 268 1 17,18 278 4 83-98 - 259 0 5 269 2 19-22 279 4 99-114 - 260 0 6 270 2 23-26 280 4 115-130 - 261 0 7 271 2 27-30 281 5 131-162 - 262 0 8 272 2 31-34 282 5 163-194 - 263 0 9 273 3 35-42 283 5 195-226 - 264 0 10 274 3 43-50 284 5 227-257 - 265 1 11,12 275 3 51-58 285 0 258 - 266 1 13,14 276 3 59-66 - - The extra bits should be interpreted as a machine integer - stored with the most-significant bit first, e.g., bits 1110 - represent the value 14. - - Extra Extra Extra - Code Bits Dist Code Bits Dist Code Bits Distance - ---- ---- ---- ---- ---- ------ ---- ---- -------- - 0 0 1 10 4 33-48 20 9 1025-1536 - 1 0 2 11 4 49-64 21 9 1537-2048 - 2 0 3 12 5 65-96 22 10 2049-3072 - 3 0 4 13 5 97-128 23 10 3073-4096 - 4 1 5,6 14 6 129-192 24 11 4097-6144 - 5 1 7,8 15 6 193-256 25 11 6145-8192 - 6 2 9-12 16 7 257-384 26 12 8193-12288 - 7 2 13-16 17 7 385-512 27 12 12289-16384 - 8 3 17-24 18 8 513-768 28 13 16385-24576 - 9 3 25-32 19 8 769-1024 29 13 24577-32768 - - 3.2.6. Compression with fixed Huffman codes (BTYPE=01) - - The Huffman codes for the two alphabets are fixed, and are not - represented explicitly in the data. The Huffman code lengths - for the literal/length alphabet are: - - Lit Value Bits Codes - --------- ---- ----- - 0 - 143 8 00110000 through - 10111111 - 144 - 255 9 110010000 through - 111111111 - 256 - 279 7 0000000 through - 0010111 - 280 - 287 8 11000000 through - 11000111 - - - -Deutsch Informational [Page 12] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - - The code lengths are sufficient to generate the actual codes, - as described above; we show the codes in the table for added - clarity. Literal/length values 286-287 will never actually - occur in the compressed data, but participate in the code - construction. - - Distance codes 0-31 are represented by (fixed-length) 5-bit - codes, with possible additional bits as shown in the table - shown in Paragraph 3.2.5, above. Note that distance codes 30- - 31 will never actually occur in the compressed data. - - 3.2.7. Compression with dynamic Huffman codes (BTYPE=10) - - The Huffman codes for the two alphabets appear in the block - immediately after the header bits and before the actual - compressed data, first the literal/length code and then the - distance code. Each code is defined by a sequence of code - lengths, as discussed in Paragraph 3.2.2, above. For even - greater compactness, the code length sequences themselves are - compressed using a Huffman code. The alphabet for code lengths - is as follows: - - 0 - 15: Represent code lengths of 0 - 15 - 16: Copy the previous code length 3 - 6 times. - The next 2 bits indicate repeat length - (0 = 3, ... , 3 = 6) - Example: Codes 8, 16 (+2 bits 11), - 16 (+2 bits 10) will expand to - 12 code lengths of 8 (1 + 6 + 5) - 17: Repeat a code length of 0 for 3 - 10 times. - (3 bits of length) - 18: Repeat a code length of 0 for 11 - 138 times - (7 bits of length) - - A code length of 0 indicates that the corresponding symbol in - the literal/length or distance alphabet will not occur in the - block, and should not participate in the Huffman code - construction algorithm given earlier. If only one distance - code is used, it is encoded using one bit, not zero bits; in - this case there is a single code length of one, with one unused - code. One distance code of zero bits means that there are no - distance codes used at all (the data is all literals). - - We can now define the format of the block: - - 5 Bits: HLIT, # of Literal/Length codes - 257 (257 - 286) - 5 Bits: HDIST, # of Distance codes - 1 (1 - 32) - 4 Bits: HCLEN, # of Code Length codes - 4 (4 - 19) - - - -Deutsch Informational [Page 13] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - - (HCLEN + 4) x 3 bits: code lengths for the code length - alphabet given just above, in the order: 16, 17, 18, - 0, 8, 7, 9, 6, 10, 5, 11, 4, 12, 3, 13, 2, 14, 1, 15 - - These code lengths are interpreted as 3-bit integers - (0-7); as above, a code length of 0 means the - corresponding symbol (literal/length or distance code - length) is not used. - - HLIT + 257 code lengths for the literal/length alphabet, - encoded using the code length Huffman code - - HDIST + 1 code lengths for the distance alphabet, - encoded using the code length Huffman code - - The actual compressed data of the block, - encoded using the literal/length and distance Huffman - codes - - The literal/length symbol 256 (end of data), - encoded using the literal/length Huffman code - - The code length repeat codes can cross from HLIT + 257 to the - HDIST + 1 code lengths. In other words, all code lengths form - a single sequence of HLIT + HDIST + 258 values. - - 3.3. Compliance - - A compressor may limit further the ranges of values specified in - the previous section and still be compliant; for example, it may - limit the range of backward pointers to some value smaller than - 32K. Similarly, a compressor may limit the size of blocks so that - a compressible block fits in memory. - - A compliant decompressor must accept the full range of possible - values defined in the previous section, and must accept blocks of - arbitrary size. - -4. Compression algorithm details - - While it is the intent of this document to define the "deflate" - compressed data format without reference to any particular - compression algorithm, the format is related to the compressed - formats produced by LZ77 (Lempel-Ziv 1977, see reference [2] below); - since many variations of LZ77 are patented, it is strongly - recommended that the implementor of a compressor follow the general - algorithm presented here, which is known not to be patented per se. - The material in this section is not part of the definition of the - - - -Deutsch Informational [Page 14] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - - specification per se, and a compressor need not follow it in order to - be compliant. - - The compressor terminates a block when it determines that starting a - new block with fresh trees would be useful, or when the block size - fills up the compressor's block buffer. - - The compressor uses a chained hash table to find duplicated strings, - using a hash function that operates on 3-byte sequences. At any - given point during compression, let XYZ be the next 3 input bytes to - be examined (not necessarily all different, of course). First, the - compressor examines the hash chain for XYZ. If the chain is empty, - the compressor simply writes out X as a literal byte and advances one - byte in the input. If the hash chain is not empty, indicating that - the sequence XYZ (or, if we are unlucky, some other 3 bytes with the - same hash function value) has occurred recently, the compressor - compares all strings on the XYZ hash chain with the actual input data - sequence starting at the current point, and selects the longest - match. - - The compressor searches the hash chains starting with the most recent - strings, to favor small distances and thus take advantage of the - Huffman encoding. The hash chains are singly linked. There are no - deletions from the hash chains; the algorithm simply discards matches - that are too old. To avoid a worst-case situation, very long hash - chains are arbitrarily truncated at a certain length, determined by a - run-time parameter. - - To improve overall compression, the compressor optionally defers the - selection of matches ("lazy matching"): after a match of length N has - been found, the compressor searches for a longer match starting at - the next input byte. If it finds a longer match, it truncates the - previous match to a length of one (thus producing a single literal - byte) and then emits the longer match. Otherwise, it emits the - original match, and, as described above, advances N bytes before - continuing. - - Run-time parameters also control this "lazy match" procedure. If - compression ratio is most important, the compressor attempts a - complete second search regardless of the length of the first match. - In the normal case, if the current match is "long enough", the - compressor reduces the search for a longer match, thus speeding up - the process. If speed is most important, the compressor inserts new - strings in the hash table only when no match was found, or when the - match is not "too long". This degrades the compression ratio but - saves time since there are both fewer insertions and fewer searches. - - - - - -Deutsch Informational [Page 15] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - -5. References - - [1] Huffman, D. A., "A Method for the Construction of Minimum - Redundancy Codes", Proceedings of the Institute of Radio - Engineers, September 1952, Volume 40, Number 9, pp. 1098-1101. - - [2] Ziv J., Lempel A., "A Universal Algorithm for Sequential Data - Compression", IEEE Transactions on Information Theory, Vol. 23, - No. 3, pp. 337-343. - - [3] Gailly, J.-L., and Adler, M., ZLIB documentation and sources, - available in ftp://ftp.uu.net/pub/archiving/zip/doc/ - - [4] Gailly, J.-L., and Adler, M., GZIP documentation and sources, - available as gzip-*.tar in ftp://prep.ai.mit.edu/pub/gnu/ - - [5] Schwartz, E. S., and Kallick, B. "Generating a canonical prefix - encoding." Comm. ACM, 7,3 (Mar. 1964), pp. 166-169. - - [6] Hirschberg and Lelewer, "Efficient decoding of prefix codes," - Comm. ACM, 33,4, April 1990, pp. 449-459. - -6. Security Considerations - - Any data compression method involves the reduction of redundancy in - the data. Consequently, any corruption of the data is likely to have - severe effects and be difficult to correct. Uncompressed text, on - the other hand, will probably still be readable despite the presence - of some corrupted bytes. - - It is recommended that systems using this data format provide some - means of validating the integrity of the compressed data. See - reference [3], for example. - -7. Source code - - Source code for a C language implementation of a "deflate" compliant - compressor and decompressor is available within the zlib package at - ftp://ftp.uu.net/pub/archiving/zip/zlib/. - -8. Acknowledgements - - Trademarks cited in this document are the property of their - respective owners. - - Phil Katz designed the deflate format. Jean-Loup Gailly and Mark - Adler wrote the related software described in this specification. - Glenn Randers-Pehrson converted this document to RFC and HTML format. - - - -Deutsch Informational [Page 16] - -RFC 1951 DEFLATE Compressed Data Format Specification May 1996 - - -9. Author's Address - - L. Peter Deutsch - Aladdin Enterprises - 203 Santa Margarita Ave. - Menlo Park, CA 94025 - - Phone: (415) 322-0103 (AM only) - FAX: (415) 322-1734 - EMail: - - Questions about the technical content of this specification can be - sent by email to: - - Jean-Loup Gailly and - Mark Adler - - Editorial comments on this specification can be sent by email to: - - L. Peter Deutsch and - Glenn Randers-Pehrson - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -Deutsch Informational [Page 17] - diff --git a/contrib/libzlib-ng/doc/rfc1952.txt b/contrib/libzlib-ng/doc/rfc1952.txt deleted file mode 100644 index a8e51b4567f..00000000000 --- a/contrib/libzlib-ng/doc/rfc1952.txt +++ /dev/null @@ -1,675 +0,0 @@ - - - - - - -Network Working Group P. Deutsch -Request for Comments: 1952 Aladdin Enterprises -Category: Informational May 1996 - - - GZIP file format specification version 4.3 - -Status of This Memo - - This memo provides information for the Internet community. This memo - does not specify an Internet standard of any kind. Distribution of - this memo is unlimited. - -IESG Note: - - The IESG takes no position on the validity of any Intellectual - Property Rights statements contained in this document. - -Notices - - Copyright (c) 1996 L. Peter Deutsch - - Permission is granted to copy and distribute this document for any - purpose and without charge, including translations into other - languages and incorporation into compilations, provided that the - copyright notice and this notice are preserved, and that any - substantive changes or deletions from the original are clearly - marked. - - A pointer to the latest version of this and related documentation in - HTML format can be found at the URL - . - -Abstract - - This specification defines a lossless compressed data format that is - compatible with the widely used GZIP utility. The format includes a - cyclic redundancy check value for detecting data corruption. The - format presently uses the DEFLATE method of compression but can be - easily extended to use other compression methods. The format can be - implemented readily in a manner not covered by patents. - - - - - - - - - - -Deutsch Informational [Page 1] - -RFC 1952 GZIP File Format Specification May 1996 - - -Table of Contents - - 1. Introduction ................................................... 2 - 1.1. Purpose ................................................... 2 - 1.2. Intended audience ......................................... 3 - 1.3. Scope ..................................................... 3 - 1.4. Compliance ................................................ 3 - 1.5. Definitions of terms and conventions used ................. 3 - 1.6. Changes from previous versions ............................ 3 - 2. Detailed specification ......................................... 4 - 2.1. Overall conventions ....................................... 4 - 2.2. File format ............................................... 5 - 2.3. Member format ............................................. 5 - 2.3.1. Member header and trailer ........................... 6 - 2.3.1.1. Extra field ................................... 8 - 2.3.1.2. Compliance .................................... 9 - 3. References .................................................. 9 - 4. Security Considerations .................................... 10 - 5. Acknowledgements ........................................... 10 - 6. Author's Address ........................................... 10 - 7. Appendix: Jean-Loup Gailly's gzip utility .................. 11 - 8. Appendix: Sample CRC Code .................................. 11 - -1. Introduction - - 1.1. Purpose - - The purpose of this specification is to define a lossless - compressed data format that: - - * Is independent of CPU type, operating system, file system, - and character set, and hence can be used for interchange; - * Can compress or decompress a data stream (as opposed to a - randomly accessible file) to produce another data stream, - using only an a priori bounded amount of intermediate - storage, and hence can be used in data communications or - similar structures such as Unix filters; - * Compresses data with efficiency comparable to the best - currently available general-purpose compression methods, - and in particular considerably better than the "compress" - program; - * Can be implemented readily in a manner not covered by - patents, and hence can be practiced freely; - * Is compatible with the file format produced by the current - widely used gzip utility, in that conforming decompressors - will be able to read data produced by the existing gzip - compressor. - - - - -Deutsch Informational [Page 2] - -RFC 1952 GZIP File Format Specification May 1996 - - - The data format defined by this specification does not attempt to: - - * Provide random access to compressed data; - * Compress specialized data (e.g., raster graphics) as well as - the best currently available specialized algorithms. - - 1.2. Intended audience - - This specification is intended for use by implementors of software - to compress data into gzip format and/or decompress data from gzip - format. - - The text of the specification assumes a basic background in - programming at the level of bits and other primitive data - representations. - - 1.3. Scope - - The specification specifies a compression method and a file format - (the latter assuming only that a file can store a sequence of - arbitrary bytes). It does not specify any particular interface to - a file system or anything about character sets or encodings - (except for file names and comments, which are optional). - - 1.4. Compliance - - Unless otherwise indicated below, a compliant decompressor must be - able to accept and decompress any file that conforms to all the - specifications presented here; a compliant compressor must produce - files that conform to all the specifications presented here. The - material in the appendices is not part of the specification per se - and is not relevant to compliance. - - 1.5. Definitions of terms and conventions used - - byte: 8 bits stored or transmitted as a unit (same as an octet). - (For this specification, a byte is exactly 8 bits, even on - machines which store a character on a number of bits different - from 8.) See below for the numbering of bits within a byte. - - 1.6. Changes from previous versions - - There have been no technical changes to the gzip format since - version 4.1 of this specification. In version 4.2, some - terminology was changed, and the sample CRC code was rewritten for - clarity and to eliminate the requirement for the caller to do pre- - and post-conditioning. Version 4.3 is a conversion of the - specification to RFC style. - - - -Deutsch Informational [Page 3] - -RFC 1952 GZIP File Format Specification May 1996 - - -2. Detailed specification - - 2.1. Overall conventions - - In the diagrams below, a box like this: - - +---+ - | | <-- the vertical bars might be missing - +---+ - - represents one byte; a box like this: - - +==============+ - | | - +==============+ - - represents a variable number of bytes. - - Bytes stored within a computer do not have a "bit order", since - they are always treated as a unit. However, a byte considered as - an integer between 0 and 255 does have a most- and least- - significant bit, and since we write numbers with the most- - significant digit on the left, we also write bytes with the most- - significant bit on the left. In the diagrams below, we number the - bits of a byte so that bit 0 is the least-significant bit, i.e., - the bits are numbered: - - +--------+ - |76543210| - +--------+ - - This document does not address the issue of the order in which - bits of a byte are transmitted on a bit-sequential medium, since - the data format described here is byte- rather than bit-oriented. - - Within a computer, a number may occupy multiple bytes. All - multi-byte numbers in the format described here are stored with - the least-significant byte first (at the lower memory address). - For example, the decimal number 520 is stored as: - - 0 1 - +--------+--------+ - |00001000|00000010| - +--------+--------+ - ^ ^ - | | - | + more significant byte = 2 x 256 - + less significant byte = 8 - - - -Deutsch Informational [Page 4] - -RFC 1952 GZIP File Format Specification May 1996 - - - 2.2. File format - - A gzip file consists of a series of "members" (compressed data - sets). The format of each member is specified in the following - section. The members simply appear one after another in the file, - with no additional information before, between, or after them. - - 2.3. Member format - - Each member has the following structure: - - +---+---+---+---+---+---+---+---+---+---+ - |ID1|ID2|CM |FLG| MTIME |XFL|OS | (more-->) - +---+---+---+---+---+---+---+---+---+---+ - - (if FLG.FEXTRA set) - - +---+---+=================================+ - | XLEN |...XLEN bytes of "extra field"...| (more-->) - +---+---+=================================+ - - (if FLG.FNAME set) - - +=========================================+ - |...original file name, zero-terminated...| (more-->) - +=========================================+ - - (if FLG.FCOMMENT set) - - +===================================+ - |...file comment, zero-terminated...| (more-->) - +===================================+ - - (if FLG.FHCRC set) - - +---+---+ - | CRC16 | - +---+---+ - - +=======================+ - |...compressed blocks...| (more-->) - +=======================+ - - 0 1 2 3 4 5 6 7 - +---+---+---+---+---+---+---+---+ - | CRC32 | ISIZE | - +---+---+---+---+---+---+---+---+ - - - - -Deutsch Informational [Page 5] - -RFC 1952 GZIP File Format Specification May 1996 - - - 2.3.1. Member header and trailer - - ID1 (IDentification 1) - ID2 (IDentification 2) - These have the fixed values ID1 = 31 (0x1f, \037), ID2 = 139 - (0x8b, \213), to identify the file as being in gzip format. - - CM (Compression Method) - This identifies the compression method used in the file. CM - = 0-7 are reserved. CM = 8 denotes the "deflate" - compression method, which is the one customarily used by - gzip and which is documented elsewhere. - - FLG (FLaGs) - This flag byte is divided into individual bits as follows: - - bit 0 FTEXT - bit 1 FHCRC - bit 2 FEXTRA - bit 3 FNAME - bit 4 FCOMMENT - bit 5 reserved - bit 6 reserved - bit 7 reserved - - If FTEXT is set, the file is probably ASCII text. This is - an optional indication, which the compressor may set by - checking a small amount of the input data to see whether any - non-ASCII characters are present. In case of doubt, FTEXT - is cleared, indicating binary data. For systems which have - different file formats for ascii text and binary data, the - decompressor can use FTEXT to choose the appropriate format. - We deliberately do not specify the algorithm used to set - this bit, since a compressor always has the option of - leaving it cleared and a decompressor always has the option - of ignoring it and letting some other program handle issues - of data conversion. - - If FHCRC is set, a CRC16 for the gzip header is present, - immediately before the compressed data. The CRC16 consists - of the two least significant bytes of the CRC32 for all - bytes of the gzip header up to and not including the CRC16. - [The FHCRC bit was never set by versions of gzip up to - 1.2.4, even though it was documented with a different - meaning in gzip 1.2.4.] - - If FEXTRA is set, optional extra fields are present, as - described in a following section. - - - -Deutsch Informational [Page 6] - -RFC 1952 GZIP File Format Specification May 1996 - - - If FNAME is set, an original file name is present, - terminated by a zero byte. The name must consist of ISO - 8859-1 (LATIN-1) characters; on operating systems using - EBCDIC or any other character set for file names, the name - must be translated to the ISO LATIN-1 character set. This - is the original name of the file being compressed, with any - directory components removed, and, if the file being - compressed is on a file system with case insensitive names, - forced to lower case. There is no original file name if the - data was compressed from a source other than a named file; - for example, if the source was stdin on a Unix system, there - is no file name. - - If FCOMMENT is set, a zero-terminated file comment is - present. This comment is not interpreted; it is only - intended for human consumption. The comment must consist of - ISO 8859-1 (LATIN-1) characters. Line breaks should be - denoted by a single line feed character (10 decimal). - - Reserved FLG bits must be zero. - - MTIME (Modification TIME) - This gives the most recent modification time of the original - file being compressed. The time is in Unix format, i.e., - seconds since 00:00:00 GMT, Jan. 1, 1970. (Note that this - may cause problems for MS-DOS and other systems that use - local rather than Universal time.) If the compressed data - did not come from a file, MTIME is set to the time at which - compression started. MTIME = 0 means no time stamp is - available. - - XFL (eXtra FLags) - These flags are available for use by specific compression - methods. The "deflate" method (CM = 8) sets these flags as - follows: - - XFL = 2 - compressor used maximum compression, - slowest algorithm - XFL = 4 - compressor used fastest algorithm - - OS (Operating System) - This identifies the type of file system on which compression - took place. This may be useful in determining end-of-line - convention for text files. The currently defined values are - as follows: - - - - - - -Deutsch Informational [Page 7] - -RFC 1952 GZIP File Format Specification May 1996 - - - 0 - FAT filesystem (MS-DOS, OS/2, NT/Win32) - 1 - Amiga - 2 - VMS (or OpenVMS) - 3 - Unix - 4 - VM/CMS - 5 - Atari TOS - 6 - HPFS filesystem (OS/2, NT) - 7 - Macintosh - 8 - Z-System - 9 - CP/M - 10 - TOPS-20 - 11 - NTFS filesystem (NT) - 12 - QDOS - 13 - Acorn RISCOS - 255 - unknown - - XLEN (eXtra LENgth) - If FLG.FEXTRA is set, this gives the length of the optional - extra field. See below for details. - - CRC32 (CRC-32) - This contains a Cyclic Redundancy Check value of the - uncompressed data computed according to CRC-32 algorithm - used in the ISO 3309 standard and in section 8.1.1.6.2 of - ITU-T recommendation V.42. (See http://www.iso.ch for - ordering ISO documents. See gopher://info.itu.ch for an - online version of ITU-T V.42.) - - ISIZE (Input SIZE) - This contains the size of the original (uncompressed) input - data modulo 2^32. - - 2.3.1.1. Extra field - - If the FLG.FEXTRA bit is set, an "extra field" is present in - the header, with total length XLEN bytes. It consists of a - series of subfields, each of the form: - - +---+---+---+---+==================================+ - |SI1|SI2| LEN |... LEN bytes of subfield data ...| - +---+---+---+---+==================================+ - - SI1 and SI2 provide a subfield ID, typically two ASCII letters - with some mnemonic value. Jean-Loup Gailly - is maintaining a registry of subfield - IDs; please send him any subfield ID you wish to use. Subfield - IDs with SI2 = 0 are reserved for future use. The following - IDs are currently defined: - - - -Deutsch Informational [Page 8] - -RFC 1952 GZIP File Format Specification May 1996 - - - SI1 SI2 Data - ---------- ---------- ---- - 0x41 ('A') 0x70 ('P') Apollo file type information - - LEN gives the length of the subfield data, excluding the 4 - initial bytes. - - 2.3.1.2. Compliance - - A compliant compressor must produce files with correct ID1, - ID2, CM, CRC32, and ISIZE, but may set all the other fields in - the fixed-length part of the header to default values (255 for - OS, 0 for all others). The compressor must set all reserved - bits to zero. - - A compliant decompressor must check ID1, ID2, and CM, and - provide an error indication if any of these have incorrect - values. It must examine FEXTRA/XLEN, FNAME, FCOMMENT and FHCRC - at least so it can skip over the optional fields if they are - present. It need not examine any other part of the header or - trailer; in particular, a decompressor may ignore FTEXT and OS - and always produce binary output, and still be compliant. A - compliant decompressor must give an error indication if any - reserved bit is non-zero, since such a bit could indicate the - presence of a new field that would cause subsequent data to be - interpreted incorrectly. - -3. References - - [1] "Information Processing - 8-bit single-byte coded graphic - character sets - Part 1: Latin alphabet No.1" (ISO 8859-1:1987). - The ISO 8859-1 (Latin-1) character set is a superset of 7-bit - ASCII. Files defining this character set are available as - iso_8859-1.* in ftp://ftp.uu.net/graphics/png/documents/ - - [2] ISO 3309 - - [3] ITU-T recommendation V.42 - - [4] Deutsch, L.P.,"DEFLATE Compressed Data Format Specification", - available in ftp://ftp.uu.net/pub/archiving/zip/doc/ - - [5] Gailly, J.-L., GZIP documentation, available as gzip-*.tar in - ftp://prep.ai.mit.edu/pub/gnu/ - - [6] Sarwate, D.V., "Computation of Cyclic Redundancy Checks via Table - Look-Up", Communications of the ACM, 31(8), pp.1008-1013. - - - - -Deutsch Informational [Page 9] - -RFC 1952 GZIP File Format Specification May 1996 - - - [7] Schwaderer, W.D., "CRC Calculation", April 85 PC Tech Journal, - pp.118-133. - - [8] ftp://ftp.adelaide.edu.au/pub/rocksoft/papers/crc_v3.txt, - describing the CRC concept. - -4. Security Considerations - - Any data compression method involves the reduction of redundancy in - the data. Consequently, any corruption of the data is likely to have - severe effects and be difficult to correct. Uncompressed text, on - the other hand, will probably still be readable despite the presence - of some corrupted bytes. - - It is recommended that systems using this data format provide some - means of validating the integrity of the compressed data, such as by - setting and checking the CRC-32 check value. - -5. Acknowledgements - - Trademarks cited in this document are the property of their - respective owners. - - Jean-Loup Gailly designed the gzip format and wrote, with Mark Adler, - the related software described in this specification. Glenn - Randers-Pehrson converted this document to RFC and HTML format. - -6. Author's Address - - L. Peter Deutsch - Aladdin Enterprises - 203 Santa Margarita Ave. - Menlo Park, CA 94025 - - Phone: (415) 322-0103 (AM only) - FAX: (415) 322-1734 - EMail: - - Questions about the technical content of this specification can be - sent by email to: - - Jean-Loup Gailly and - Mark Adler - - Editorial comments on this specification can be sent by email to: - - L. Peter Deutsch and - Glenn Randers-Pehrson - - - -Deutsch Informational [Page 10] - -RFC 1952 GZIP File Format Specification May 1996 - - -7. Appendix: Jean-Loup Gailly's gzip utility - - The most widely used implementation of gzip compression, and the - original documentation on which this specification is based, were - created by Jean-Loup Gailly . Since this - implementation is a de facto standard, we mention some more of its - features here. Again, the material in this section is not part of - the specification per se, and implementations need not follow it to - be compliant. - - When compressing or decompressing a file, gzip preserves the - protection, ownership, and modification time attributes on the local - file system, since there is no provision for representing protection - attributes in the gzip file format itself. Since the file format - includes a modification time, the gzip decompressor provides a - command line switch that assigns the modification time from the file, - rather than the local modification time of the compressed input, to - the decompressed output. - -8. Appendix: Sample CRC Code - - The following sample code represents a practical implementation of - the CRC (Cyclic Redundancy Check). (See also ISO 3309 and ITU-T V.42 - for a formal specification.) - - The sample code is in the ANSI C programming language. Non C users - may find it easier to read with these hints: - - & Bitwise AND operator. - ^ Bitwise exclusive-OR operator. - >> Bitwise right shift operator. When applied to an - unsigned quantity, as here, right shift inserts zero - bit(s) at the left. - ! Logical NOT operator. - ++ "n++" increments the variable n. - 0xNNN 0x introduces a hexadecimal (base 16) constant. - Suffix L indicates a long value (at least 32 bits). - - /* Table of CRCs of all 8-bit messages. */ - unsigned long crc_table[256]; - - /* Flag: has the table been computed? Initially false. */ - int crc_table_computed = 0; - - /* Make the table for a fast CRC. */ - void make_crc_table(void) - { - unsigned long c; - - - -Deutsch Informational [Page 11] - -RFC 1952 GZIP File Format Specification May 1996 - - - int n, k; - for (n = 0; n < 256; n++) { - c = (unsigned long) n; - for (k = 0; k < 8; k++) { - if (c & 1) { - c = 0xedb88320L ^ (c >> 1); - } else { - c = c >> 1; - } - } - crc_table[n] = c; - } - crc_table_computed = 1; - } - - /* - Update a running crc with the bytes buf[0..len-1] and return - the updated crc. The crc should be initialized to zero. Pre- and - post-conditioning (one's complement) is performed within this - function so it shouldn't be done by the caller. Usage example: - - unsigned long crc = 0L; - - while (read_buffer(buffer, length) != EOF) { - crc = update_crc(crc, buffer, length); - } - if (crc != original_crc) error(); - */ - unsigned long update_crc(unsigned long crc, - unsigned char *buf, int len) - { - unsigned long c = crc ^ 0xffffffffL; - int n; - - if (!crc_table_computed) - make_crc_table(); - for (n = 0; n < len; n++) { - c = crc_table[(c ^ buf[n]) & 0xff] ^ (c >> 8); - } - return c ^ 0xffffffffL; - } - - /* Return the CRC of the bytes buf[0..len-1]. */ - unsigned long crc(unsigned char *buf, int len) - { - return update_crc(0L, buf, len); - } - - - - -Deutsch Informational [Page 12] - diff --git a/contrib/libzlib-ng/doc/txtvsbin.txt b/contrib/libzlib-ng/doc/txtvsbin.txt deleted file mode 100644 index 3d0f0634f72..00000000000 --- a/contrib/libzlib-ng/doc/txtvsbin.txt +++ /dev/null @@ -1,107 +0,0 @@ -A Fast Method for Identifying Plain Text Files -============================================== - - -Introduction ------------- - -Given a file coming from an unknown source, it is sometimes desirable -to find out whether the format of that file is plain text. Although -this may appear like a simple task, a fully accurate detection of the -file type requires heavy-duty semantic analysis on the file contents. -It is, however, possible to obtain satisfactory results by employing -various heuristics. - -Previous versions of PKZip and other zip-compatible compression tools -were using a crude detection scheme: if more than 80% (4/5) of the bytes -found in a certain buffer are within the range [7..127], the file is -labeled as plain text, otherwise it is labeled as binary. A prominent -limitation of this scheme is the restriction to Latin-based alphabets. -Other alphabets, like Greek, Cyrillic or Asian, make extensive use of -the bytes within the range [128..255], and texts using these alphabets -are most often misidentified by this scheme; in other words, the rate -of false negatives is sometimes too high, which means that the recall -is low. Another weakness of this scheme is a reduced precision, due to -the false positives that may occur when binary files containing large -amounts of textual characters are misidentified as plain text. - -In this article we propose a new, simple detection scheme that features -a much increased precision and a near-100% recall. This scheme is -designed to work on ASCII, Unicode and other ASCII-derived alphabets, -and it handles single-byte encodings (ISO-8859, MacRoman, KOI8, etc.) -and variable-sized encodings (ISO-2022, UTF-8, etc.). Wider encodings -(UCS-2/UTF-16 and UCS-4/UTF-32) are not handled, however. - - -The Algorithm -------------- - -The algorithm works by dividing the set of bytecodes [0..255] into three -categories: -- The white list of textual bytecodes: - 9 (TAB), 10 (LF), 13 (CR), 32 (SPACE) to 255. -- The gray list of tolerated bytecodes: - 7 (BEL), 8 (BS), 11 (VT), 12 (FF), 26 (SUB), 27 (ESC). -- The black list of undesired, non-textual bytecodes: - 0 (NUL) to 6, 14 to 31. - -If a file contains at least one byte that belongs to the white list and -no byte that belongs to the black list, then the file is categorized as -plain text; otherwise, it is categorized as binary. (The boundary case, -when the file is empty, automatically falls into the latter category.) - - -Rationale ---------- - -The idea behind this algorithm relies on two observations. - -The first observation is that, although the full range of 7-bit codes -[0..127] is properly specified by the ASCII standard, most control -characters in the range [0..31] are not used in practice. The only -widely-used, almost universally-portable control codes are 9 (TAB), -10 (LF) and 13 (CR). There are a few more control codes that are -recognized on a reduced range of platforms and text viewers/editors: -7 (BEL), 8 (BS), 11 (VT), 12 (FF), 26 (SUB) and 27 (ESC); but these -codes are rarely (if ever) used alone, without being accompanied by -some printable text. Even the newer, portable text formats such as -XML avoid using control characters outside the list mentioned here. - -The second observation is that most of the binary files tend to contain -control characters, especially 0 (NUL). Even though the older text -detection schemes observe the presence of non-ASCII codes from the range -[128..255], the precision rarely has to suffer if this upper range is -labeled as textual, because the files that are genuinely binary tend to -contain both control characters and codes from the upper range. On the -other hand, the upper range needs to be labeled as textual, because it -is used by virtually all ASCII extensions. In particular, this range is -used for encoding non-Latin scripts. - -Since there is no counting involved, other than simply observing the -presence or the absence of some byte values, the algorithm produces -consistent results, regardless what alphabet encoding is being used. -(If counting were involved, it could be possible to obtain different -results on a text encoded, say, using ISO-8859-16 versus UTF-8.) - -There is an extra category of plain text files that are "polluted" with -one or more black-listed codes, either by mistake or by peculiar design -considerations. In such cases, a scheme that tolerates a small fraction -of black-listed codes would provide an increased recall (i.e. more true -positives). This, however, incurs a reduced precision overall, since -false positives are more likely to appear in binary files that contain -large chunks of textual data. Furthermore, "polluted" plain text should -be regarded as binary by general-purpose text detection schemes, because -general-purpose text processing algorithms might not be applicable. -Under this premise, it is safe to say that our detection method provides -a near-100% recall. - -Experiments have been run on many files coming from various platforms -and applications. We tried plain text files, system logs, source code, -formatted office documents, compiled object code, etc. The results -confirm the optimistic assumptions about the capabilities of this -algorithm. - - --- -Cosmin Truta -Last updated: 2006-May-28 diff --git a/contrib/libzlib-ng/gzclose.c b/contrib/libzlib-ng/gzclose.c deleted file mode 100644 index 48d6a86f04b..00000000000 --- a/contrib/libzlib-ng/gzclose.c +++ /dev/null @@ -1,23 +0,0 @@ -/* gzclose.c -- zlib gzclose() function - * Copyright (C) 2004, 2010 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -#include "gzguts.h" - -/* gzclose() is in a separate file so that it is linked in only if it is used. - That way the other gzclose functions can be used instead to avoid linking in - unneeded compression or decompression routines. */ -int ZEXPORT gzclose(gzFile file) { -#ifndef NO_GZCOMPRESS - gz_statep state; - - if (file == NULL) - return Z_STREAM_ERROR; - state = (gz_statep)file; - - return state->mode == GZ_READ ? gzclose_r(file) : gzclose_w(file); -#else - return gzclose_r(file); -#endif -} diff --git a/contrib/libzlib-ng/gzguts.h b/contrib/libzlib-ng/gzguts.h deleted file mode 100644 index 0921f176abf..00000000000 --- a/contrib/libzlib-ng/gzguts.h +++ /dev/null @@ -1,158 +0,0 @@ -#ifndef GZGUTS_H_ -#define GZGUTS_H_ -/* gzguts.h -- zlib internal header definitions for gz* operations - * Copyright (C) 2004, 2005, 2010, 2011, 2012, 2013 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -#ifdef _LARGEFILE64_SOURCE -# ifndef _LARGEFILE_SOURCE -# define _LARGEFILE_SOURCE 1 -# endif -# ifdef _FILE_OFFSET_BITS -# undef _FILE_OFFSET_BITS -# endif -#endif - -#if defined(HAVE_INTERNAL) -# define ZLIB_INTERNAL __attribute__((visibility ("internal"))) -#elif defined(HAVE_HIDDEN) -# define ZLIB_INTERNAL __attribute__((visibility ("hidden"))) -#else -# define ZLIB_INTERNAL -#endif - -#include -#include -#include -#include -#include -#include "zlib.h" - -#ifdef WIN32 -# include -#endif - -#if defined(_MSC_VER) || defined(WIN32) -# include -#endif - -#if defined(_WIN32) || defined(__CYGWIN__) || defined(__MINGW__) -# define WIDECHAR -#endif - -#ifdef WINAPI_FAMILY -# define open _open -# define read _read -# define write _write -# define close _close -#endif - -/* In Win32, vsnprintf is available as the "non-ANSI" _vsnprintf. */ -#if !defined(STDC99) && !defined(__CYGWIN__) && !defined(__MINGW__) && defined(WIN32) -# if !defined(vsnprintf) -# if !defined(_MSC_VER) || ( defined(_MSC_VER) && _MSC_VER < 1500 ) -# define vsnprintf _vsnprintf -# endif -# endif -#endif - -/* unlike snprintf (which is required in C99), _snprintf does not guarantee - null termination of the result -- however this is only used in gzlib.c - where the result is assured to fit in the space provided */ -#if defined(_MSC_VER) && _MSC_VER < 1900 -# define snprintf _snprintf -#endif - -#ifndef local -# define local static -#endif -/* compile with -Dlocal if your debugger can't find static symbols */ - -/* get errno and strerror definition */ -#ifndef NO_STRERROR -# include -# define zstrerror() strerror(errno) -#else -# define zstrerror() "stdio error (consult errno)" -#endif - -/* provide prototypes for these when building zlib without LFS */ -#if (!defined(_LARGEFILE64_SOURCE) || _LFS64_LARGEFILE-0 == 0) && defined(WITH_GZFILEOP) - ZEXTERN gzFile ZEXPORT gzopen64(const char *, const char *); - ZEXTERN z_off64_t ZEXPORT gzseek64(gzFile, z_off64_t, int); - ZEXTERN z_off64_t ZEXPORT gztell64(gzFile); - ZEXTERN z_off64_t ZEXPORT gzoffset64(gzFile); -#endif - -/* default memLevel */ -#if MAX_MEM_LEVEL >= 8 -# define DEF_MEM_LEVEL 8 -#else -# define DEF_MEM_LEVEL MAX_MEM_LEVEL -#endif - -/* default i/o buffer size -- double this for output when reading (this and - twice this must be able to fit in an unsigned type) */ -#define GZBUFSIZE 8192 - -/* gzip modes, also provide a little integrity check on the passed structure */ -#define GZ_NONE 0 -#define GZ_READ 7247 -#define GZ_WRITE 31153 -#define GZ_APPEND 1 /* mode set to GZ_WRITE after the file is opened */ - -/* values for gz_state how */ -#define LOOK 0 /* look for a gzip header */ -#define COPY 1 /* copy input directly */ -#define GZIP 2 /* decompress a gzip stream */ - -/* internal gzip file state data structure */ -typedef struct { - /* exposed contents for gzgetc() macro */ - struct gzFile_s x; /* "x" for exposed */ - /* x.have: number of bytes available at x.next */ - /* x.next: next output data to deliver or write */ - /* x.pos: current position in uncompressed data */ - /* used for both reading and writing */ - int mode; /* see gzip modes above */ - int fd; /* file descriptor */ - char *path; /* path or fd for error messages */ - unsigned size; /* buffer size, zero if not allocated yet */ - unsigned want; /* requested buffer size, default is GZBUFSIZE */ - unsigned char *in; /* input buffer (double-sized when writing) */ - unsigned char *out; /* output buffer (double-sized when reading) */ - int direct; /* 0 if processing gzip, 1 if transparent */ - /* just for reading */ - int how; /* 0: get header, 1: copy, 2: decompress */ - z_off64_t start; /* where the gzip data started, for rewinding */ - int eof; /* true if end of input file reached */ - int past; /* true if read requested past end */ - /* just for writing */ - int level; /* compression level */ - int strategy; /* compression strategy */ - /* seek request */ - z_off64_t skip; /* amount to skip (already rewound if backwards) */ - int seek; /* true if seek request pending */ - /* error information */ - int err; /* error code */ - char *msg; /* error message */ - /* zlib inflate or deflate stream */ - z_stream strm; /* stream structure in-place (not a pointer) */ -} gz_state; -typedef gz_state *gz_statep; - -/* shared functions */ -void ZLIB_INTERNAL gz_error(gz_statep, int, const char *); - -/* GT_OFF(x), where x is an unsigned value, is true if x > maximum z_off64_t - value -- needed when comparing unsigned to z_off64_t, which is signed - (possible z_off64_t types off_t, off64_t, and long are all signed) */ -#ifdef INT_MAX -# define GT_OFF(x) (sizeof(int) == sizeof(z_off64_t) && (x) > INT_MAX) -#else -unsigned ZLIB_INTERNAL gz_intmax(void); -# define GT_OFF(x) (sizeof(int) == sizeof(z_off64_t) && (x) > gz_intmax()) -#endif - -#endif /* GZGUTS_H_ */ diff --git a/contrib/libzlib-ng/gzlib.c b/contrib/libzlib-ng/gzlib.c deleted file mode 100644 index e5ebe5fa7fa..00000000000 --- a/contrib/libzlib-ng/gzlib.c +++ /dev/null @@ -1,518 +0,0 @@ -/* gzlib.c -- zlib functions common to reading and writing gzip files - * Copyright (C) 2004, 2010, 2011, 2012, 2013 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -#include "gzguts.h" - -#if defined(WIN32) && !defined(__BORLANDC__) && !defined(__MINGW32__) -# define LSEEK _lseeki64 -#else -#if defined(_LARGEFILE64_SOURCE) && _LFS64_LARGEFILE-0 -# define LSEEK lseek64 -#else -# define LSEEK lseek -#endif -#endif - -/* Local functions */ -local void gz_reset(gz_statep); -local gzFile gz_open(const void *, int, const char *); - -/* Reset gzip file state */ -local void gz_reset(gz_statep state) { - state->x.have = 0; /* no output data available */ - if (state->mode == GZ_READ) { /* for reading ... */ - state->eof = 0; /* not at end of file */ - state->past = 0; /* have not read past end yet */ - state->how = LOOK; /* look for gzip header */ - } - state->seek = 0; /* no seek request pending */ - gz_error(state, Z_OK, NULL); /* clear error */ - state->x.pos = 0; /* no uncompressed data yet */ - state->strm.avail_in = 0; /* no input data yet */ -} - -/* Open a gzip file either by name or file descriptor. */ -local gzFile gz_open(const void *path, int fd, const char *mode) { - gz_statep state; - size_t len; - int oflag; -#ifdef O_CLOEXEC - int cloexec = 0; -#endif -#ifdef O_EXCL - int exclusive = 0; -#endif - - /* check input */ - if (path == NULL) - return NULL; - - /* allocate gzFile structure to return */ - state = (gz_statep)malloc(sizeof(gz_state)); - if (state == NULL) - return NULL; - state->size = 0; /* no buffers allocated yet */ - state->want = GZBUFSIZE; /* requested buffer size */ - state->msg = NULL; /* no error message yet */ - - /* interpret mode */ - state->mode = GZ_NONE; - state->level = Z_DEFAULT_COMPRESSION; - state->strategy = Z_DEFAULT_STRATEGY; - state->direct = 0; - while (*mode) { - if (*mode >= '0' && *mode <= '9') { - state->level = *mode - '0'; - } else { - switch (*mode) { - case 'r': - state->mode = GZ_READ; - break; -#ifndef NO_GZCOMPRESS - case 'w': - state->mode = GZ_WRITE; - break; - case 'a': - state->mode = GZ_APPEND; - break; -#endif - case '+': /* can't read and write at the same time */ - free(state); - return NULL; - case 'b': /* ignore -- will request binary anyway */ - break; -#ifdef O_CLOEXEC - case 'e': - cloexec = 1; - break; -#endif -#ifdef O_EXCL - case 'x': - exclusive = 1; - break; -#endif - case 'f': - state->strategy = Z_FILTERED; - break; - case 'h': - state->strategy = Z_HUFFMAN_ONLY; - break; - case 'R': - state->strategy = Z_RLE; - break; - case 'F': - state->strategy = Z_FIXED; - break; - case 'T': - state->direct = 1; - break; - default: /* could consider as an error, but just ignore */ - {} - } - } - mode++; - } - - /* must provide an "r", "w", or "a" */ - if (state->mode == GZ_NONE) { - free(state); - return NULL; - } - - /* can't force transparent read */ - if (state->mode == GZ_READ) { - if (state->direct) { - free(state); - return NULL; - } - state->direct = 1; /* for empty file */ - } - - /* save the path name for error messages */ -#ifdef WIDECHAR - if (fd == -2) { - len = wcstombs(NULL, path, 0); - if (len == (size_t)-1) - len = 0; - } else -#endif - len = strlen((const char *)path); - state->path = (char *)malloc(len + 1); - if (state->path == NULL) { - free(state); - return NULL; - } -#ifdef WIDECHAR - if (fd == -2) - if (len) { - wcstombs(state->path, path, len + 1); - } else { - *(state->path) = 0; - } - else -#endif - snprintf(state->path, len + 1, "%s", (const char *)path); - - /* compute the flags for open() */ - oflag = -#ifdef O_LARGEFILE - O_LARGEFILE | -#endif -#ifdef O_BINARY - O_BINARY | -#endif -#ifdef O_CLOEXEC - (cloexec ? O_CLOEXEC : 0) | -#endif - (state->mode == GZ_READ ? - O_RDONLY : - (O_WRONLY | O_CREAT | -#ifdef O_EXCL - (exclusive ? O_EXCL : 0) | -#endif - (state->mode == GZ_WRITE ? - O_TRUNC : - O_APPEND))); - - /* open the file with the appropriate flags (or just use fd) */ - state->fd = fd > -1 ? fd : ( -#if defined(WIN32) || defined(__MINGW__) - fd == -2 ? _wopen(path, oflag, 0666) : -#elif __CYGWIN__ - fd == -2 ? open(state->path, oflag, 0666) : -#endif - open((const char *)path, oflag, 0666)); - if (state->fd == -1) { - free(state->path); - free(state); - return NULL; - } - if (state->mode == GZ_APPEND) { - LSEEK(state->fd, 0, SEEK_END); /* so gzoffset() is correct */ - state->mode = GZ_WRITE; /* simplify later checks */ - } - - /* save the current position for rewinding (only if reading) */ - if (state->mode == GZ_READ) { - state->start = LSEEK(state->fd, 0, SEEK_CUR); - if (state->start == -1) state->start = 0; - } - - /* initialize stream */ - gz_reset(state); - - /* return stream */ - return (gzFile)state; -} - -/* -- see zlib.h -- */ -gzFile ZEXPORT gzopen(const char *path, const char *mode) { - return gz_open(path, -1, mode); -} - -/* -- see zlib.h -- */ -gzFile ZEXPORT gzopen64(const char *path, const char *mode) { - return gz_open(path, -1, mode); -} - -/* -- see zlib.h -- */ -gzFile ZEXPORT gzdopen(int fd, const char *mode) { - char *path; /* identifier for error messages */ - gzFile gz; - - if (fd == -1 || (path = (char *)malloc(7 + 3 * sizeof(int))) == NULL) - return NULL; - snprintf(path, 7 + 3 * sizeof(int), "", fd); /* for debugging */ - gz = gz_open(path, fd, mode); - free(path); - return gz; -} - -/* -- see zlib.h -- */ -#ifdef WIDECHAR -gzFile ZEXPORT gzopen_w(const wchar_t *path, const char *mode) { - return gz_open(path, -2, mode); -} -#endif - -/* -- see zlib.h -- */ -int ZEXPORT gzbuffer(gzFile file, unsigned size) { - gz_statep state; - - /* get internal structure and check integrity */ - if (file == NULL) - return -1; - state = (gz_statep)file; - if (state->mode != GZ_READ && state->mode != GZ_WRITE) - return -1; - - /* make sure we haven't already allocated memory */ - if (state->size != 0) - return -1; - - /* check and set requested size */ - if ((size << 1) < size) - return -1; /* need to be able to double it */ - if (size < 2) - size = 2; /* need two bytes to check magic header */ - state->want = size; - return 0; -} - -/* -- see zlib.h -- */ -int ZEXPORT gzrewind(gzFile file) { - gz_statep state; - - /* get internal structure */ - if (file == NULL) - return -1; - state = (gz_statep)file; - - /* check that we're reading and that there's no error */ - if (state->mode != GZ_READ || (state->err != Z_OK && state->err != Z_BUF_ERROR)) - return -1; - - /* back up and start over */ - if (LSEEK(state->fd, state->start, SEEK_SET) == -1) - return -1; - gz_reset(state); - return 0; -} - -/* -- see zlib.h -- */ -z_off64_t ZEXPORT gzseek64(gzFile file, z_off64_t offset, int whence) { - unsigned n; - z_off64_t ret; - gz_statep state; - - /* get internal structure and check integrity */ - if (file == NULL) - return -1; - state = (gz_statep)file; - if (state->mode != GZ_READ && state->mode != GZ_WRITE) - return -1; - - /* check that there's no error */ - if (state->err != Z_OK && state->err != Z_BUF_ERROR) - return -1; - - /* can only seek from start or relative to current position */ - if (whence != SEEK_SET && whence != SEEK_CUR) - return -1; - - /* normalize offset to a SEEK_CUR specification */ - if (whence == SEEK_SET) - offset -= state->x.pos; - else if (state->seek) - offset += state->skip; - state->seek = 0; - - /* if within raw area while reading, just go there */ - if (state->mode == GZ_READ && state->how == COPY && state->x.pos + offset >= 0) { - ret = LSEEK(state->fd, offset - state->x.have, SEEK_CUR); - if (ret == -1) - return -1; - state->x.have = 0; - state->eof = 0; - state->past = 0; - state->seek = 0; - gz_error(state, Z_OK, NULL); - state->strm.avail_in = 0; - state->x.pos += offset; - return state->x.pos; - } - - /* calculate skip amount, rewinding if needed for back seek when reading */ - if (offset < 0) { - if (state->mode != GZ_READ) /* writing -- can't go backwards */ - return -1; - offset += state->x.pos; - if (offset < 0) /* before start of file! */ - return -1; - if (gzrewind(file) == -1) /* rewind, then skip to offset */ - return -1; - } - - /* if reading, skip what's in output buffer (one less gzgetc() check) */ - if (state->mode == GZ_READ) { - n = GT_OFF(state->x.have) || (z_off64_t)state->x.have > offset ? (unsigned)offset : state->x.have; - state->x.have -= n; - state->x.next += n; - state->x.pos += n; - offset -= n; - } - - /* request skip (if not zero) */ - if (offset) { - state->seek = 1; - state->skip = offset; - } - return state->x.pos + offset; -} - -/* -- see zlib.h -- */ -z_off_t ZEXPORT gzseek(gzFile file, z_off_t offset, int whence) { - z_off64_t ret; - - ret = gzseek64(file, (z_off64_t)offset, whence); - return ret == (z_off_t)ret ? (z_off_t)ret : -1; -} - -/* -- see zlib.h -- */ -z_off64_t ZEXPORT gztell64(gzFile file) { - gz_statep state; - - /* get internal structure and check integrity */ - if (file == NULL) - return -1; - state = (gz_statep)file; - if (state->mode != GZ_READ && state->mode != GZ_WRITE) - return -1; - - /* return position */ - return state->x.pos + (state->seek ? state->skip : 0); -} - -/* -- see zlib.h -- */ -z_off_t ZEXPORT gztell(gzFile file) { - z_off64_t ret; - - ret = gztell64(file); - return ret == (z_off_t)ret ? (z_off_t)ret : -1; -} - -/* -- see zlib.h -- */ -z_off64_t ZEXPORT gzoffset64(gzFile file) { - z_off64_t offset; - gz_statep state; - - /* get internal structure and check integrity */ - if (file == NULL) - return -1; - state = (gz_statep)file; - if (state->mode != GZ_READ && state->mode != GZ_WRITE) - return -1; - - /* compute and return effective offset in file */ - offset = LSEEK(state->fd, 0, SEEK_CUR); - if (offset == -1) - return -1; - if (state->mode == GZ_READ) /* reading */ - offset -= state->strm.avail_in; /* don't count buffered input */ - return offset; -} - -/* -- see zlib.h -- */ -z_off_t ZEXPORT gzoffset(gzFile file) { - z_off64_t ret; - - ret = gzoffset64(file); - return ret == (z_off_t)ret ? (z_off_t)ret : -1; -} - -/* -- see zlib.h -- */ -int ZEXPORT gzeof(gzFile file) { - gz_statep state; - - /* get internal structure and check integrity */ - if (file == NULL) - return 0; - state = (gz_statep)file; - if (state->mode != GZ_READ && state->mode != GZ_WRITE) - return 0; - - /* return end-of-file state */ - return state->mode == GZ_READ ? state->past : 0; -} - -/* -- see zlib.h -- */ -const char * ZEXPORT gzerror(gzFile file, int *errnum) { - gz_statep state; - - /* get internal structure and check integrity */ - if (file == NULL) - return NULL; - state = (gz_statep)file; - if (state->mode != GZ_READ && state->mode != GZ_WRITE) - return NULL; - - /* return error information */ - if (errnum != NULL) - *errnum = state->err; - return state->err == Z_MEM_ERROR ? "out of memory" : (state->msg == NULL ? "" : state->msg); -} - -/* -- see zlib.h -- */ -void ZEXPORT gzclearerr(gzFile file) { - gz_statep state; - - /* get internal structure and check integrity */ - if (file == NULL) - return; - state = (gz_statep)file; - if (state->mode != GZ_READ && state->mode != GZ_WRITE) - return; - - /* clear error and end-of-file */ - if (state->mode == GZ_READ) { - state->eof = 0; - state->past = 0; - } - gz_error(state, Z_OK, NULL); -} - -/* Create an error message in allocated memory and set state->err and - state->msg accordingly. Free any previous error message already there. Do - not try to free or allocate space if the error is Z_MEM_ERROR (out of - memory). Simply save the error message as a static string. If there is an - allocation failure constructing the error message, then convert the error to - out of memory. */ -void ZLIB_INTERNAL gz_error(gz_statep state, int err, const char *msg) { - /* free previously allocated message and clear */ - if (state->msg != NULL) { - if (state->err != Z_MEM_ERROR) - free(state->msg); - state->msg = NULL; - } - - /* if fatal, set state->x.have to 0 so that the gzgetc() macro fails */ - if (err != Z_OK && err != Z_BUF_ERROR) - state->x.have = 0; - - /* set error code, and if no message, then done */ - state->err = err; - if (msg == NULL) - return; - - /* for an out of memory error, return literal string when requested */ - if (err == Z_MEM_ERROR) - return; - - /* construct error message with path */ - if ((state->msg = (char *)malloc(strlen(state->path) + strlen(msg) + 3)) == NULL) { - state->err = Z_MEM_ERROR; - return; - } - snprintf(state->msg, strlen(state->path) + strlen(msg) + 3, "%s%s%s", state->path, ": ", msg); - return; -} - -#ifndef INT_MAX -/* portably return maximum value for an int (when limits.h presumed not - available) -- we need to do this to cover cases where 2's complement not - used, since C standard permits 1's complement and sign-bit representations, - otherwise we could just use ((unsigned)-1) >> 1 */ -unsigned ZLIB_INTERNAL gz_intmax() { - unsigned p, q; - - p = 1; - do { - q = p; - p <<= 1; - p++; - } while (p > q); - return q >> 1; -} -#endif diff --git a/contrib/libzlib-ng/gzread.c b/contrib/libzlib-ng/gzread.c deleted file mode 100644 index 5db2c5045ba..00000000000 --- a/contrib/libzlib-ng/gzread.c +++ /dev/null @@ -1,538 +0,0 @@ -/* gzread.c -- zlib functions for reading gzip files - * Copyright (C) 2004, 2005, 2010, 2011, 2012, 2013 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -#include "gzguts.h" - -/* Local functions */ -local int gz_load(gz_statep, unsigned char *, unsigned, unsigned *); -local int gz_avail(gz_statep); -local int gz_look(gz_statep); -local int gz_decomp(gz_statep); -local int gz_fetch(gz_statep); -local int gz_skip(gz_statep, z_off64_t); - -/* Use read() to load a buffer -- return -1 on error, otherwise 0. Read from - state->fd, and update state->eof, state->err, and state->msg as appropriate. - This function needs to loop on read(), since read() is not guaranteed to - read the number of bytes requested, depending on the type of descriptor. */ -local int gz_load(gz_statep state, unsigned char *buf, unsigned len, unsigned *have) { - int ret; - - *have = 0; - do { - ret = read(state->fd, buf + *have, len - *have); - if (ret <= 0) - break; - *have += ret; - } while (*have < len); - if (ret < 0) { - gz_error(state, Z_ERRNO, zstrerror()); - return -1; - } - if (ret == 0) - state->eof = 1; - return 0; -} - -/* Load up input buffer and set eof flag if last data loaded -- return -1 on - error, 0 otherwise. Note that the eof flag is set when the end of the input - file is reached, even though there may be unused data in the buffer. Once - that data has been used, no more attempts will be made to read the file. - If strm->avail_in != 0, then the current data is moved to the beginning of - the input buffer, and then the remainder of the buffer is loaded with the - available data from the input file. */ -local int gz_avail(gz_statep state) { - unsigned got; - z_stream *strm = &(state->strm); - - if (state->err != Z_OK && state->err != Z_BUF_ERROR) - return -1; - if (state->eof == 0) { - if (strm->avail_in) { /* copy what's there to the start */ - unsigned char *p = state->in; - unsigned const char *q = strm->next_in; - unsigned n = strm->avail_in; - do { - *p++ = *q++; - } while (--n); - } - if (gz_load(state, state->in + strm->avail_in, state->size - strm->avail_in, &got) == -1) - return -1; - strm->avail_in += got; - strm->next_in = state->in; - } - return 0; -} - -/* Look for gzip header, set up for inflate or copy. state->x.have must be 0. - If this is the first time in, allocate required memory. state->how will be - left unchanged if there is no more input data available, will be set to COPY - if there is no gzip header and direct copying will be performed, or it will - be set to GZIP for decompression. If direct copying, then leftover input - data from the input buffer will be copied to the output buffer. In that - case, all further file reads will be directly to either the output buffer or - a user buffer. If decompressing, the inflate state will be initialized. - gz_look() will return 0 on success or -1 on failure. */ -local int gz_look(gz_statep state) { - z_stream *strm = &(state->strm); - - /* allocate read buffers and inflate memory */ - if (state->size == 0) { - /* allocate buffers */ - state->in = (unsigned char *)malloc(state->want); - state->out = (unsigned char *)malloc(state->want << 1); - if (state->in == NULL || state->out == NULL) { - if (state->out != NULL) - free(state->out); - if (state->in != NULL) - free(state->in); - gz_error(state, Z_MEM_ERROR, "out of memory"); - return -1; - } - state->size = state->want; - - /* allocate inflate memory */ - state->strm.zalloc = Z_NULL; - state->strm.zfree = Z_NULL; - state->strm.opaque = Z_NULL; - state->strm.avail_in = 0; - state->strm.next_in = Z_NULL; - if (inflateInit2(&(state->strm), 15 + 16) != Z_OK) { /* gunzip */ - free(state->out); - free(state->in); - state->size = 0; - gz_error(state, Z_MEM_ERROR, "out of memory"); - return -1; - } - } - - /* get at least the magic bytes in the input buffer */ - if (strm->avail_in < 2) { - if (gz_avail(state) == -1) - return -1; - if (strm->avail_in == 0) - return 0; - } - - /* look for gzip magic bytes -- if there, do gzip decoding (note: there is - a logical dilemma here when considering the case of a partially written - gzip file, to wit, if a single 31 byte is written, then we cannot tell - whether this is a single-byte file, or just a partially written gzip - file -- for here we assume that if a gzip file is being written, then - the header will be written in a single operation, so that reading a - single byte is sufficient indication that it is not a gzip file) */ - if (strm->avail_in > 1 && - strm->next_in[0] == 31 && strm->next_in[1] == 139) { - inflateReset(strm); - state->how = GZIP; - state->direct = 0; - return 0; - } - - /* no gzip header -- if we were decoding gzip before, then this is trailing - garbage. Ignore the trailing garbage and finish. */ - if (state->direct == 0) { - strm->avail_in = 0; - state->eof = 1; - state->x.have = 0; - return 0; - } - - /* doing raw i/o, copy any leftover input to output -- this assumes that - the output buffer is larger than the input buffer, which also assures - space for gzungetc() */ - state->x.next = state->out; - if (strm->avail_in) { - memcpy(state->x.next, strm->next_in, strm->avail_in); - state->x.have = strm->avail_in; - strm->avail_in = 0; - } - state->how = COPY; - state->direct = 1; - return 0; -} - -/* Decompress from input to the provided next_out and avail_out in the state. - On return, state->x.have and state->x.next point to the just decompressed - data. If the gzip stream completes, state->how is reset to LOOK to look for - the next gzip stream or raw data, once state->x.have is depleted. Returns 0 - on success, -1 on failure. */ -local int gz_decomp(gz_statep state) { - int ret = Z_OK; - unsigned had; - z_stream *strm = &(state->strm); - - /* fill output buffer up to end of deflate stream */ - had = strm->avail_out; - do { - /* get more input for inflate() */ - if (strm->avail_in == 0 && gz_avail(state) == -1) - return -1; - if (strm->avail_in == 0) { - gz_error(state, Z_BUF_ERROR, "unexpected end of file"); - break; - } - - /* decompress and handle errors */ - ret = inflate(strm, Z_NO_FLUSH); - if (ret == Z_STREAM_ERROR || ret == Z_NEED_DICT) { - gz_error(state, Z_STREAM_ERROR, "internal error: inflate stream corrupt"); - return -1; - } - if (ret == Z_MEM_ERROR) { - gz_error(state, Z_MEM_ERROR, "out of memory"); - return -1; - } - if (ret == Z_DATA_ERROR) { /* deflate stream invalid */ - gz_error(state, Z_DATA_ERROR, strm->msg == NULL ? "compressed data error" : strm->msg); - return -1; - } - } while (strm->avail_out && ret != Z_STREAM_END); - - /* update available output */ - state->x.have = had - strm->avail_out; - state->x.next = strm->next_out - state->x.have; - - /* if the gzip stream completed successfully, look for another */ - if (ret == Z_STREAM_END) - state->how = LOOK; - - /* good decompression */ - return 0; -} - -/* Fetch data and put it in the output buffer. Assumes state->x.have is 0. - Data is either copied from the input file or decompressed from the input - file depending on state->how. If state->how is LOOK, then a gzip header is - looked for to determine whether to copy or decompress. Returns -1 on error, - otherwise 0. gz_fetch() will leave state->how as COPY or GZIP unless the - end of the input file has been reached and all data has been processed. */ -local int gz_fetch(gz_statep state) { - z_stream *strm = &(state->strm); - - do { - switch (state->how) { - case LOOK: /* -> LOOK, COPY (only if never GZIP), or GZIP */ - if (gz_look(state) == -1) - return -1; - if (state->how == LOOK) - return 0; - break; - case COPY: /* -> COPY */ - if (gz_load(state, state->out, state->size << 1, &(state->x.have)) - == -1) - return -1; - state->x.next = state->out; - return 0; - case GZIP: /* -> GZIP or LOOK (if end of gzip stream) */ - strm->avail_out = state->size << 1; - strm->next_out = state->out; - if (gz_decomp(state) == -1) - return -1; - } - } while (state->x.have == 0 && (!state->eof || strm->avail_in)); - return 0; -} - -/* Skip len uncompressed bytes of output. Return -1 on error, 0 on success. */ -local int gz_skip(gz_statep state, z_off64_t len) { - unsigned n; - - /* skip over len bytes or reach end-of-file, whichever comes first */ - while (len) - /* skip over whatever is in output buffer */ - if (state->x.have) { - n = GT_OFF(state->x.have) || (z_off64_t)state->x.have > len ? - (unsigned)len : state->x.have; - state->x.have -= n; - state->x.next += n; - state->x.pos += n; - len -= n; - } else if (state->eof && state->strm.avail_in == 0) { - /* output buffer empty -- return if we're at the end of the input */ - break; - } else { - /* need more data to skip -- load up output buffer */ - /* get more output, looking for header if required */ - if (gz_fetch(state) == -1) - return -1; - } - return 0; -} - -/* -- see zlib.h -- */ -int ZEXPORT gzread(gzFile file, void *buf, unsigned len) { - unsigned got, n; - gz_statep state; - z_stream *strm; - - /* get internal structure */ - if (file == NULL) - return -1; - state = (gz_statep)file; - strm = &(state->strm); - - /* check that we're reading and that there's no (serious) error */ - if (state->mode != GZ_READ || (state->err != Z_OK && state->err != Z_BUF_ERROR)) - return -1; - - /* since an int is returned, make sure len fits in one, otherwise return - with an error (this avoids the flaw in the interface) */ - if ((int)len < 0) { - gz_error(state, Z_DATA_ERROR, "requested length does not fit in int"); - return -1; - } - - /* if len is zero, avoid unnecessary operations */ - if (len == 0) - return 0; - - /* process a skip request */ - if (state->seek) { - state->seek = 0; - if (gz_skip(state, state->skip) == -1) - return -1; - } - - /* get len bytes to buf, or less than len if at the end */ - got = 0; - do { - /* first just try copying data from the output buffer */ - if (state->x.have) { - n = state->x.have > len ? len : state->x.have; - memcpy(buf, state->x.next, n); - state->x.next += n; - state->x.have -= n; - } else if (state->eof && strm->avail_in == 0) { - /* output buffer empty -- return if we're at the end of the input */ - state->past = 1; /* tried to read past end */ - break; - } else if (state->how == LOOK || len < (state->size << 1)) { - /* need output data -- for small len or new stream load up our output buffer */ - /* get more output, looking for header if required */ - if (gz_fetch(state) == -1) - return -1; - continue; /* no progress yet -- go back to copy above */ - /* the copy above assures that we will leave with space in the - output buffer, allowing at least one gzungetc() to succeed */ - } else if (state->how == COPY) { /* read directly */ - /* large len -- read directly into user buffer */ - if (gz_load(state, (unsigned char *)buf, len, &n) == -1) - return -1; - } else { /* state->how == GZIP */ - /* large len -- decompress directly into user buffer */ - strm->avail_out = len; - strm->next_out = (unsigned char *)buf; - if (gz_decomp(state) == -1) - return -1; - n = state->x.have; - state->x.have = 0; - } - - /* update progress */ - len -= n; - buf = (char *)buf + n; - got += n; - state->x.pos += n; - } while (len); - - /* return number of bytes read into user buffer (will fit in int) */ - return (int)got; -} - -/* -- see zlib.h -- */ -#undef gzgetc -int ZEXPORT gzgetc(gzFile file) { - int ret; - unsigned char buf[1]; - gz_statep state; - - /* get internal structure */ - if (file == NULL) - return -1; - state = (gz_statep)file; - - /* check that we're reading and that there's no (serious) error */ - if (state->mode != GZ_READ || (state->err != Z_OK && state->err != Z_BUF_ERROR)) - return -1; - - /* try output buffer (no need to check for skip request) */ - if (state->x.have) { - state->x.have--; - state->x.pos++; - return *(state->x.next)++; - } - - /* nothing there -- try gzread() */ - ret = gzread(file, buf, 1); - return ret < 1 ? -1 : buf[0]; -} - -int ZEXPORT gzgetc_(gzFile file) { - return gzgetc(file); -} - -/* -- see zlib.h -- */ -int ZEXPORT gzungetc(int c, gzFile file) { - gz_statep state; - - /* get internal structure */ - if (file == NULL) - return -1; - state = (gz_statep)file; - - /* check that we're reading and that there's no (serious) error */ - if (state->mode != GZ_READ || (state->err != Z_OK && state->err != Z_BUF_ERROR)) - return -1; - - /* process a skip request */ - if (state->seek) { - state->seek = 0; - if (gz_skip(state, state->skip) == -1) - return -1; - } - - /* can't push EOF */ - if (c < 0) - return -1; - - /* if output buffer empty, put byte at end (allows more pushing) */ - if (state->x.have == 0) { - state->x.have = 1; - state->x.next = state->out + (state->size << 1) - 1; - state->x.next[0] = c; - state->x.pos--; - state->past = 0; - return c; - } - - /* if no room, give up (must have already done a gzungetc()) */ - if (state->x.have == (state->size << 1)) { - gz_error(state, Z_DATA_ERROR, "out of room to push characters"); - return -1; - } - - /* slide output data if needed and insert byte before existing data */ - if (state->x.next == state->out) { - unsigned char *src = state->out + state->x.have; - unsigned char *dest = state->out + (state->size << 1); - while (src > state->out) - *--dest = *--src; - state->x.next = dest; - } - state->x.have++; - state->x.next--; - state->x.next[0] = c; - state->x.pos--; - state->past = 0; - return c; -} - -/* -- see zlib.h -- */ -char * ZEXPORT gzgets(gzFile file, char *buf, int len) { - unsigned left, n; - char *str; - unsigned char *eol; - gz_statep state; - - /* check parameters and get internal structure */ - if (file == NULL || buf == NULL || len < 1) - return NULL; - state = (gz_statep)file; - - /* check that we're reading and that there's no (serious) error */ - if (state->mode != GZ_READ || (state->err != Z_OK && state->err != Z_BUF_ERROR)) - return NULL; - - /* process a skip request */ - if (state->seek) { - state->seek = 0; - if (gz_skip(state, state->skip) == -1) - return NULL; - } - - /* copy output bytes up to new line or len - 1, whichever comes first -- - append a terminating zero to the string (we don't check for a zero in - the contents, let the user worry about that) */ - str = buf; - left = (unsigned)len - 1; - if (left) do { - /* assure that something is in the output buffer */ - if (state->x.have == 0 && gz_fetch(state) == -1) - return NULL; /* error */ - if (state->x.have == 0) { /* end of file */ - state->past = 1; /* read past end */ - break; /* return what we have */ - } - - /* look for end-of-line in current output buffer */ - n = state->x.have > left ? left : state->x.have; - eol = (unsigned char *)memchr(state->x.next, '\n', n); - if (eol != NULL) - n = (unsigned)(eol - state->x.next) + 1; - - /* copy through end-of-line, or remainder if not found */ - memcpy(buf, state->x.next, n); - state->x.have -= n; - state->x.next += n; - state->x.pos += n; - left -= n; - buf += n; - } while (left && eol == NULL); - - /* return terminated string, or if nothing, end of file */ - if (buf == str) - return NULL; - buf[0] = 0; - return str; -} - -/* -- see zlib.h -- */ -int ZEXPORT gzdirect(gzFile file) { - gz_statep state; - - /* get internal structure */ - if (file == NULL) - return 0; - - state = (gz_statep)file; - - /* if the state is not known, but we can find out, then do so (this is - mainly for right after a gzopen() or gzdopen()) */ - if (state->mode == GZ_READ && state->how == LOOK && state->x.have == 0) - (void)gz_look(state); - - /* return 1 if transparent, 0 if processing a gzip stream */ - return state->direct; -} - -/* -- see zlib.h -- */ -int ZEXPORT gzclose_r(gzFile file) { - int ret, err; - gz_statep state; - - /* get internal structure */ - if (file == NULL) - return Z_STREAM_ERROR; - - state = (gz_statep)file; - - /* check that we're reading */ - if (state->mode != GZ_READ) - return Z_STREAM_ERROR; - - /* free memory and close file */ - if (state->size) { - inflateEnd(&(state->strm)); - free(state->out); - free(state->in); - } - err = state->err == Z_BUF_ERROR ? Z_BUF_ERROR : Z_OK; - gz_error(state, Z_OK, NULL); - free(state->path); - ret = close(state->fd); - free(state); - return ret ? Z_ERRNO : err; -} diff --git a/contrib/libzlib-ng/gzwrite.c b/contrib/libzlib-ng/gzwrite.c deleted file mode 100644 index dedaee0cd45..00000000000 --- a/contrib/libzlib-ng/gzwrite.c +++ /dev/null @@ -1,457 +0,0 @@ -/* gzwrite.c -- zlib functions for writing gzip files - * Copyright (C) 2004, 2005, 2010, 2011, 2012, 2013 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -#include -#include "gzguts.h" - -/* Local functions */ -local int gz_init(gz_statep); -local int gz_comp(gz_statep, int); -local int gz_zero(gz_statep, z_off64_t); - -/* Initialize state for writing a gzip file. Mark initialization by setting - state->size to non-zero. Return -1 on failure or 0 on success. */ -local int gz_init(gz_statep state) { - int ret; - z_stream *strm = &(state->strm); - - /* allocate input buffer (double size for gzprintf) */ - state->in = (unsigned char *)malloc(state->want << 1); - if (state->in == NULL) { - gz_error(state, Z_MEM_ERROR, "out of memory"); - return -1; - } - - /* only need output buffer and deflate state if compressing */ - if (!state->direct) { - /* allocate output buffer */ - state->out = (unsigned char *)malloc(state->want); - if (state->out == NULL) { - free(state->in); - gz_error(state, Z_MEM_ERROR, "out of memory"); - return -1; - } - - /* allocate deflate memory, set up for gzip compression */ - strm->zalloc = Z_NULL; - strm->zfree = Z_NULL; - strm->opaque = Z_NULL; - ret = deflateInit2(strm, state->level, Z_DEFLATED, MAX_WBITS + 16, DEF_MEM_LEVEL, state->strategy); - if (ret != Z_OK) { - free(state->out); - free(state->in); - gz_error(state, Z_MEM_ERROR, "out of memory"); - return -1; - } - strm->next_in = NULL; - } - - /* mark state as initialized */ - state->size = state->want; - - /* initialize write buffer if compressing */ - if (!state->direct) { - strm->avail_out = state->size; - strm->next_out = state->out; - state->x.next = strm->next_out; - } - return 0; -} - -/* Compress whatever is at avail_in and next_in and write to the output file. - Return -1 if there is an error writing to the output file, otherwise 0. - flush is assumed to be a valid deflate() flush value. If flush is Z_FINISH, - then the deflate() state is reset to start a new gzip stream. If gz->direct - is true, then simply write to the output file without compressing, and - ignore flush. */ -local int gz_comp(gz_statep state, int flush) { - int ret, got; - unsigned have; - z_stream *strm = &(state->strm); - - /* allocate memory if this is the first time through */ - if (state->size == 0 && gz_init(state) == -1) - return -1; - - /* write directly if requested */ - if (state->direct) { - while (strm->avail_in) { - if ((got = write(state->fd, strm->next_in, strm->avail_in)) < 0) { - gz_error(state, Z_ERRNO, zstrerror()); - return -1; - } - strm->avail_in -= got; - strm->next_in += got; - } - return 0; - } - - /* run deflate() on provided input until it produces no more output */ - ret = Z_OK; - do { - /* write out current buffer contents if full, or if flushing, but if - doing Z_FINISH then don't write until we get to Z_STREAM_END */ - if (strm->avail_out == 0 || (flush != Z_NO_FLUSH && (flush != Z_FINISH || ret == Z_STREAM_END))) { - while (strm->next_out > state->x.next) { - if ((got = write(state->fd, state->x.next, strm->next_out - state->x.next)) < 0) { - gz_error(state, Z_ERRNO, zstrerror()); - return -1; - } - state->x.next += got; - } - if (strm->avail_out == 0) { - strm->avail_out = state->size; - strm->next_out = state->out; - } - } - - /* compress */ - have = strm->avail_out; - ret = deflate(strm, flush); - if (ret == Z_STREAM_ERROR) { - gz_error(state, Z_STREAM_ERROR, "internal error: deflate stream corrupt"); - return -1; - } - have -= strm->avail_out; - } while (have); - - /* if that completed a deflate stream, allow another to start */ - if (flush == Z_FINISH) - deflateReset(strm); - - /* all done, no errors */ - return 0; -} - -/* Compress len zeros to output. Return -1 on error, 0 on success. */ -local int gz_zero(gz_statep state, z_off64_t len) { - int first; - unsigned n; - z_stream *strm = &(state->strm); - - /* consume whatever's left in the input buffer */ - if (strm->avail_in && gz_comp(state, Z_NO_FLUSH) == -1) - return -1; - - /* compress len zeros (len guaranteed > 0) */ - first = 1; - while (len) { - n = GT_OFF(state->size) || (z_off64_t)state->size > len ? (unsigned)len : state->size; - if (first) { - memset(state->in, 0, n); - first = 0; - } - strm->avail_in = n; - strm->next_in = state->in; - state->x.pos += n; - if (gz_comp(state, Z_NO_FLUSH) == -1) - return -1; - len -= n; - } - return 0; -} - -/* -- see zlib.h -- */ -int ZEXPORT gzwrite(gzFile file, void const *buf, unsigned len) { - unsigned put = len; - gz_statep state; - z_stream *strm; - - /* get internal structure */ - if (file == NULL) - return 0; - state = (gz_statep)file; - strm = &(state->strm); - - /* check that we're writing and that there's no error */ - if (state->mode != GZ_WRITE || state->err != Z_OK) - return 0; - - /* since an int is returned, make sure len fits in one, otherwise return - with an error (this avoids the flaw in the interface) */ - if ((int)len < 0) { - gz_error(state, Z_DATA_ERROR, "requested length does not fit in int"); - return 0; - } - - /* if len is zero, avoid unnecessary operations */ - if (len == 0) - return 0; - - /* allocate memory if this is the first time through */ - if (state->size == 0 && gz_init(state) == -1) - return 0; - - /* check for seek request */ - if (state->seek) { - state->seek = 0; - if (gz_zero(state, state->skip) == -1) - return 0; - } - - /* for small len, copy to input buffer, otherwise compress directly */ - if (len < state->size) { - /* copy to input buffer, compress when full */ - do { - unsigned have, copy; - - if (strm->avail_in == 0) - strm->next_in = state->in; - have = (unsigned)((strm->next_in + strm->avail_in) - state->in); - copy = state->size - have; - if (copy > len) - copy = len; - memcpy(state->in + have, buf, copy); - strm->avail_in += copy; - state->x.pos += copy; - buf = (const char *)buf + copy; - len -= copy; - if (len && gz_comp(state, Z_NO_FLUSH) == -1) - return 0; - } while (len); - } else { - /* consume whatever's left in the input buffer */ - if (strm->avail_in && gz_comp(state, Z_NO_FLUSH) == -1) - return 0; - - /* directly compress user buffer to file */ - strm->avail_in = len; - strm->next_in = (const unsigned char *)buf; - state->x.pos += len; - if (gz_comp(state, Z_NO_FLUSH) == -1) - return 0; - } - - /* input was all buffered or compressed (put will fit in int) */ - return (int)put; -} - -/* -- see zlib.h -- */ -int ZEXPORT gzputc(gzFile file, int c) { - unsigned have; - unsigned char buf[1]; - gz_statep state; - z_stream *strm; - - /* get internal structure */ - if (file == NULL) - return -1; - state = (gz_statep)file; - strm = &(state->strm); - - /* check that we're writing and that there's no error */ - if (state->mode != GZ_WRITE || state->err != Z_OK) - return -1; - - /* check for seek request */ - if (state->seek) { - state->seek = 0; - if (gz_zero(state, state->skip) == -1) - return -1; - } - - /* try writing to input buffer for speed (state->size == 0 if buffer not - initialized) */ - if (state->size) { - if (strm->avail_in == 0) - strm->next_in = state->in; - have = (unsigned)((strm->next_in + strm->avail_in) - state->in); - if (have < state->size) { - state->in[have] = c; - strm->avail_in++; - state->x.pos++; - return c & 0xff; - } - } - - /* no room in buffer or not initialized, use gz_write() */ - buf[0] = c; - if (gzwrite(file, buf, 1) != 1) - return -1; - return c & 0xff; -} - -/* -- see zlib.h -- */ -int ZEXPORT gzputs(gzFile file, const char *str) { - int ret; - unsigned len; - - /* write string */ - len = (unsigned)strlen(str); - ret = gzwrite(file, str, len); - return ret == 0 && len != 0 ? -1 : ret; -} - -/* -- see zlib.h -- */ -int ZEXPORTVA gzvprintf(gzFile file, const char *format, va_list va) { - int len, left; - char *next; - gz_statep state; - z_stream *strm; - - /* get internal structure */ - if (file == NULL) - return -1; - state = (gz_statep)file; - strm = &(state->strm); - - /* check that we're writing and that there's no error */ - if (state->mode != GZ_WRITE || state->err != Z_OK) - return 0; - - /* make sure we have some buffer space */ - if (state->size == 0 && gz_init(state) == -1) - return 0; - - /* check for seek request */ - if (state->seek) { - state->seek = 0; - if (gz_zero(state, state->skip) == -1) - return 0; - } - - /* do the printf() into the input buffer, put length in len -- the input - buffer is double-sized just for this function, so there is guaranteed to - be state->size bytes available after the current contents */ - if (strm->avail_in == 0) - strm->next_in = state->in; - next = (char *)(strm->next_in + strm->avail_in); - next[state->size - 1] = 0; - len = vsnprintf(next, state->size, format, va); - - /* check that printf() results fit in buffer */ - if (len == 0 || len >= state->size || next[state->size - 1] != 0) - return 0; - - /* update buffer and position, compress first half if past that */ - strm->avail_in += len; - state->x.pos += len; - if (strm->avail_in >= state->size) { - left = strm->avail_in - state->size; - strm->avail_in = state->size; - if (gz_comp(state, Z_NO_FLUSH) == -1) - return 0; - memcpy(state->in, state->in + state->size, left); - strm->next_in = state->in; - strm->avail_in = left; - } - return (int)len; -} - -int ZEXPORTVA gzprintf(gzFile file, const char *format, ...) { - va_list va; - int ret; - - va_start(va, format); - ret = gzvprintf(file, format, va); - va_end(va); - return ret; -} - -/* -- see zlib.h -- */ -int ZEXPORT gzflush(gzFile file, int flush) { - gz_statep state; - - /* get internal structure */ - if (file == NULL) - return -1; - state = (gz_statep)file; - - /* check that we're writing and that there's no error */ - if (state->mode != GZ_WRITE || state->err != Z_OK) - return Z_STREAM_ERROR; - - /* check flush parameter */ - if (flush < 0 || flush > Z_FINISH) - return Z_STREAM_ERROR; - - /* check for seek request */ - if (state->seek) { - state->seek = 0; - if (gz_zero(state, state->skip) == -1) - return -1; - } - - /* compress remaining data with requested flush */ - gz_comp(state, flush); - return state->err; -} - -/* -- see zlib.h -- */ -int ZEXPORT gzsetparams(gzFile file, int level, int strategy) { - gz_statep state; - z_stream *strm; - - /* get internal structure */ - if (file == NULL) - return Z_STREAM_ERROR; - state = (gz_statep)file; - strm = &(state->strm); - - /* check that we're writing and that there's no error */ - if (state->mode != GZ_WRITE || state->err != Z_OK) - return Z_STREAM_ERROR; - - /* if no change is requested, then do nothing */ - if (level == state->level && strategy == state->strategy) - return Z_OK; - - /* check for seek request */ - if (state->seek) { - state->seek = 0; - if (gz_zero(state, state->skip) == -1) - return -1; - } - - /* change compression parameters for subsequent input */ - if (state->size) { - /* flush previous input with previous parameters before changing */ - if (strm->avail_in && gz_comp(state, Z_PARTIAL_FLUSH) == -1) - return state->err; - deflateParams(strm, level, strategy); - } - state->level = level; - state->strategy = strategy; - return Z_OK; -} - -/* -- see zlib.h -- */ -int ZEXPORT gzclose_w(gzFile file) { - int ret = Z_OK; - gz_statep state; - - /* get internal structure */ - if (file == NULL) - return Z_STREAM_ERROR; - state = (gz_statep)file; - - /* check that we're writing */ - if (state->mode != GZ_WRITE) - return Z_STREAM_ERROR; - - /* check for seek request */ - if (state->seek) { - state->seek = 0; - if (gz_zero(state, state->skip) == -1) - ret = state->err; - } - - /* flush, free memory, and close file */ - if (gz_comp(state, Z_FINISH) == -1) - ret = state->err; - if (state->size) { - if (!state->direct) { - (void)deflateEnd(&(state->strm)); - free(state->out); - } - free(state->in); - } - gz_error(state, Z_OK, NULL); - free(state->path); - if (close(state->fd) == -1) - ret = Z_ERRNO; - free(state); - return ret; -} diff --git a/contrib/libzlib-ng/infback.c b/contrib/libzlib-ng/infback.c deleted file mode 100644 index 9f3dc4645e8..00000000000 --- a/contrib/libzlib-ng/infback.c +++ /dev/null @@ -1,612 +0,0 @@ -/* infback.c -- inflate using a call-back interface - * Copyright (C) 1995-2011 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* - This code is largely copied from inflate.c. Normally either infback.o or - inflate.o would be linked into an application--not both. The interface - with inffast.c is retained so that optimized assembler-coded versions of - inflate_fast() can be used with either inflate.c or infback.c. - */ - -#include "zutil.h" -#include "inftrees.h" -#include "inflate.h" -#include "inffast.h" - -/* function prototypes */ -local void fixedtables(struct inflate_state *state); - -/* - strm provides memory allocation functions in zalloc and zfree, or - Z_NULL to use the library memory allocation functions. - - windowBits is in the range 8..15, and window is a user-supplied - window and output buffer that is 2**windowBits bytes. - */ -int ZEXPORT inflateBackInit_(z_stream *strm, int windowBits, unsigned char *window, - const char *version, int stream_size) { - struct inflate_state *state; - - if (version == Z_NULL || version[0] != ZLIB_VERSION[0] || stream_size != (int)(sizeof(z_stream))) - return Z_VERSION_ERROR; - if (strm == Z_NULL || window == Z_NULL || windowBits < 8 || windowBits > 15) - return Z_STREAM_ERROR; - strm->msg = Z_NULL; /* in case we return an error */ - if (strm->zalloc == (alloc_func)0) { - strm->zalloc = zcalloc; - strm->opaque = NULL; - } - if (strm->zfree == (free_func)0) - strm->zfree = zcfree; - state = (struct inflate_state *)ZALLOC(strm, 1, sizeof(struct inflate_state)); - if (state == Z_NULL) - return Z_MEM_ERROR; - Tracev((stderr, "inflate: allocated\n")); - strm->state = (struct internal_state *)state; - state->dmax = 32768U; - state->wbits = windowBits; - state->wsize = 1U << windowBits; - state->window = window; - state->wnext = 0; - state->whave = 0; - return Z_OK; -} - -/* - Return state with length and distance decoding tables and index sizes set to - fixed code decoding. Normally this returns fixed tables from inffixed.h. - If BUILDFIXED is defined, then instead this routine builds the tables the - first time it's called, and returns those tables the first time and - thereafter. This reduces the size of the code by about 2K bytes, in - exchange for a little execution time. However, BUILDFIXED should not be - used for threaded applications, since the rewriting of the tables and virgin - may not be thread-safe. - */ -local void fixedtables(struct inflate_state *state) { -#ifdef BUILDFIXED - static int virgin = 1; - static code *lenfix, *distfix; - static code fixed[544]; - - /* build fixed huffman tables if first call (may not be thread safe) */ - if (virgin) { - unsigned sym, bits; - static code *next; - - /* literal/length table */ - sym = 0; - while (sym < 144) state->lens[sym++] = 8; - while (sym < 256) state->lens[sym++] = 9; - while (sym < 280) state->lens[sym++] = 7; - while (sym < 288) state->lens[sym++] = 8; - next = fixed; - lenfix = next; - bits = 9; - inflate_table(LENS, state->lens, 288, &(next), &(bits), state->work); - - /* distance table */ - sym = 0; - while (sym < 32) state->lens[sym++] = 5; - distfix = next; - bits = 5; - inflate_table(DISTS, state->lens, 32, &(next), &(bits), state->work); - - /* do this just once */ - virgin = 0; - } -#else /* !BUILDFIXED */ -# include "inffixed.h" -#endif /* BUILDFIXED */ - state->lencode = lenfix; - state->lenbits = 9; - state->distcode = distfix; - state->distbits = 5; -} - -/* Macros for inflateBack(): */ - -/* Load returned state from inflate_fast() */ -#define LOAD() \ - do { \ - put = strm->next_out; \ - left = strm->avail_out; \ - next = strm->next_in; \ - have = strm->avail_in; \ - hold = state->hold; \ - bits = state->bits; \ - } while (0) - -/* Set state from registers for inflate_fast() */ -#define RESTORE() \ - do { \ - strm->next_out = put; \ - strm->avail_out = left; \ - strm->next_in = next; \ - strm->avail_in = have; \ - state->hold = hold; \ - state->bits = bits; \ - } while (0) - -/* Clear the input bit accumulator */ -#define INITBITS() \ - do { \ - hold = 0; \ - bits = 0; \ - } while (0) - -/* Assure that some input is available. If input is requested, but denied, - then return a Z_BUF_ERROR from inflateBack(). */ -#define PULL() \ - do { \ - if (have == 0) { \ - have = in(in_desc, &next); \ - if (have == 0) { \ - next = Z_NULL; \ - ret = Z_BUF_ERROR; \ - goto inf_leave; \ - } \ - } \ - } while (0) - -/* Get a byte of input into the bit accumulator, or return from inflateBack() - with an error if there is no input available. */ -#define PULLBYTE() \ - do { \ - PULL(); \ - have--; \ - hold += (*next++ << bits); \ - bits += 8; \ - } while (0) - -/* Assure that there are at least n bits in the bit accumulator. If there is - not enough available input to do that, then return from inflateBack() with - an error. */ -#define NEEDBITS(n) \ - do { \ - while (bits < (unsigned)(n)) \ - PULLBYTE(); \ - } while (0) - -/* Return the low n bits of the bit accumulator (n < 16) */ -#define BITS(n) \ - (hold & ((1U << (n)) - 1)) - -/* Remove n bits from the bit accumulator */ -#define DROPBITS(n) \ - do { \ - hold >>= (n); \ - bits -= (unsigned)(n); \ - } while (0) - -/* Remove zero to seven bits as needed to go to a byte boundary */ -#define BYTEBITS() \ - do { \ - hold >>= bits & 7; \ - bits -= bits & 7; \ - } while (0) - -/* Assure that some output space is available, by writing out the window - if it's full. If the write fails, return from inflateBack() with a - Z_BUF_ERROR. */ -#define ROOM() \ - do { \ - if (left == 0) { \ - put = state->window; \ - left = state->wsize; \ - state->whave = left; \ - if (out(out_desc, put, left)) { \ - ret = Z_BUF_ERROR; \ - goto inf_leave; \ - } \ - } \ - } while (0) - -/* - strm provides the memory allocation functions and window buffer on input, - and provides information on the unused input on return. For Z_DATA_ERROR - returns, strm will also provide an error message. - - in() and out() are the call-back input and output functions. When - inflateBack() needs more input, it calls in(). When inflateBack() has - filled the window with output, or when it completes with data in the - window, it calls out() to write out the data. The application must not - change the provided input until in() is called again or inflateBack() - returns. The application must not change the window/output buffer until - inflateBack() returns. - - in() and out() are called with a descriptor parameter provided in the - inflateBack() call. This parameter can be a structure that provides the - information required to do the read or write, as well as accumulated - information on the input and output such as totals and check values. - - in() should return zero on failure. out() should return non-zero on - failure. If either in() or out() fails, than inflateBack() returns a - Z_BUF_ERROR. strm->next_in can be checked for Z_NULL to see whether it - was in() or out() that caused in the error. Otherwise, inflateBack() - returns Z_STREAM_END on success, Z_DATA_ERROR for an deflate format - error, or Z_MEM_ERROR if it could not allocate memory for the state. - inflateBack() can also return Z_STREAM_ERROR if the input parameters - are not correct, i.e. strm is Z_NULL or the state was not initialized. - */ -int ZEXPORT inflateBack(z_stream *strm, in_func in, void *in_desc, out_func out, void *out_desc) { - struct inflate_state *state; - const unsigned char *next; /* next input */ - unsigned char *put; /* next output */ - unsigned have, left; /* available input and output */ - uint32_t hold; /* bit buffer */ - unsigned bits; /* bits in bit buffer */ - unsigned copy; /* number of stored or match bytes to copy */ - unsigned char *from; /* where to copy match bytes from */ - code here; /* current decoding table entry */ - code last; /* parent table entry */ - unsigned len; /* length to copy for repeats, bits to drop */ - int ret; /* return code */ - static const uint16_t order[19] = /* permutation of code lengths */ - {16, 17, 18, 0, 8, 7, 9, 6, 10, 5, 11, 4, 12, 3, 13, 2, 14, 1, 15}; - - /* Check that the strm exists and that the state was initialized */ - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - state = (struct inflate_state *)strm->state; - - /* Reset the state */ - strm->msg = Z_NULL; - state->mode = TYPE; - state->last = 0; - state->whave = 0; - next = strm->next_in; - have = next != Z_NULL ? strm->avail_in : 0; - hold = 0; - bits = 0; - put = state->window; - left = state->wsize; - - /* Inflate until end of block marked as last */ - for (;;) - switch (state->mode) { - case TYPE: - /* determine and dispatch block type */ - if (state->last) { - BYTEBITS(); - state->mode = DONE; - break; - } - NEEDBITS(3); - state->last = BITS(1); - DROPBITS(1); - switch (BITS(2)) { - case 0: /* stored block */ - Tracev((stderr, "inflate: stored block%s\n", state->last ? " (last)" : "")); - state->mode = STORED; - break; - case 1: /* fixed block */ - fixedtables(state); - Tracev((stderr, "inflate: fixed codes block%s\n", state->last ? " (last)" : "")); - state->mode = LEN; /* decode codes */ - break; - case 2: /* dynamic block */ - Tracev((stderr, "inflate: dynamic codes block%s\n", state->last ? " (last)" : "")); - state->mode = TABLE; - break; - case 3: - strm->msg = (char *)"invalid block type"; - state->mode = BAD; - } - DROPBITS(2); - break; - - case STORED: - /* get and verify stored block length */ - BYTEBITS(); /* go to byte boundary */ - NEEDBITS(32); - if ((hold & 0xffff) != ((hold >> 16) ^ 0xffff)) { - strm->msg = (char *)"invalid stored block lengths"; - state->mode = BAD; - break; - } - state->length = (uint16_t)hold; - Tracev((stderr, "inflate: stored length %u\n", state->length)); - INITBITS(); - - /* copy stored block from input to output */ - while (state->length != 0) { - copy = state->length; - PULL(); - ROOM(); - if (copy > have) - copy = have; - if (copy > left) - copy = left; - memcpy(put, next, copy); - have -= copy; - next += copy; - left -= copy; - put += copy; - state->length -= copy; - } - Tracev((stderr, "inflate: stored end\n")); - state->mode = TYPE; - break; - - case TABLE: - /* get dynamic table entries descriptor */ - NEEDBITS(14); - state->nlen = BITS(5) + 257; - DROPBITS(5); - state->ndist = BITS(5) + 1; - DROPBITS(5); - state->ncode = BITS(4) + 4; - DROPBITS(4); -#ifndef PKZIP_BUG_WORKAROUND - if (state->nlen > 286 || state->ndist > 30) { - strm->msg = (char *)"too many length or distance symbols"; - state->mode = BAD; - break; - } -#endif - Tracev((stderr, "inflate: table sizes ok\n")); - - /* get code length code lengths (not a typo) */ - state->have = 0; - while (state->have < state->ncode) { - NEEDBITS(3); - state->lens[order[state->have++]] = (uint16_t)BITS(3); - DROPBITS(3); - } - while (state->have < 19) - state->lens[order[state->have++]] = 0; - state->next = state->codes; - state->lencode = (code const *)(state->next); - state->lenbits = 7; - ret = inflate_table(CODES, state->lens, 19, &(state->next), &(state->lenbits), state->work); - if (ret) { - strm->msg = (char *)"invalid code lengths set"; - state->mode = BAD; - break; - } - Tracev((stderr, "inflate: code lengths ok\n")); - - /* get length and distance code code lengths */ - state->have = 0; - while (state->have < state->nlen + state->ndist) { - for (;;) { - here = state->lencode[BITS(state->lenbits)]; - if (here.bits <= bits) - break; - PULLBYTE(); - } - if (here.val < 16) { - DROPBITS(here.bits); - state->lens[state->have++] = here.val; - } else { - if (here.val == 16) { - NEEDBITS(here.bits + 2); - DROPBITS(here.bits); - if (state->have == 0) { - strm->msg = (char *)"invalid bit length repeat"; - state->mode = BAD; - break; - } - len = (unsigned)(state->lens[state->have - 1]); - copy = 3 + BITS(2); - DROPBITS(2); - } else if (here.val == 17) { - NEEDBITS(here.bits + 3); - DROPBITS(here.bits); - len = 0; - copy = 3 + BITS(3); - DROPBITS(3); - } else { - NEEDBITS(here.bits + 7); - DROPBITS(here.bits); - len = 0; - copy = 11 + BITS(7); - DROPBITS(7); - } - if (state->have + copy > state->nlen + state->ndist) { - strm->msg = (char *)"invalid bit length repeat"; - state->mode = BAD; - break; - } - while (copy--) - state->lens[state->have++] = (uint16_t)len; - } - } - - /* handle error breaks in while */ - if (state->mode == BAD) - break; - - /* check for end-of-block code (better have one) */ - if (state->lens[256] == 0) { - strm->msg = (char *)"invalid code -- missing end-of-block"; - state->mode = BAD; - break; - } - - /* build code tables -- note: do not change the lenbits or distbits - values here (9 and 6) without reading the comments in inftrees.h - concerning the ENOUGH constants, which depend on those values */ - state->next = state->codes; - state->lencode = (code const *)(state->next); - state->lenbits = 9; - ret = inflate_table(LENS, state->lens, state->nlen, &(state->next), &(state->lenbits), state->work); - if (ret) { - strm->msg = (char *)"invalid literal/lengths set"; - state->mode = BAD; - break; - } - state->distcode = (code const *)(state->next); - state->distbits = 6; - ret = inflate_table(DISTS, state->lens + state->nlen, state->ndist, - &(state->next), &(state->distbits), state->work); - if (ret) { - strm->msg = (char *)"invalid distances set"; - state->mode = BAD; - break; - } - Tracev((stderr, "inflate: codes ok\n")); - state->mode = LEN; - - case LEN: - /* use inflate_fast() if we have enough input and output */ - if (have >= 6 && left >= 258) { - RESTORE(); - if (state->whave < state->wsize) - state->whave = state->wsize - left; - inflate_fast(strm, state->wsize); - LOAD(); - break; - } - - /* get a literal, length, or end-of-block code */ - for (;;) { - here = state->lencode[BITS(state->lenbits)]; - if (here.bits <= bits) - break; - PULLBYTE(); - } - if (here.op && (here.op & 0xf0) == 0) { - last = here; - for (;;) { - here = state->lencode[last.val + - (BITS(last.bits + last.op) >> last.bits)]; - if ((unsigned)last.bits + (unsigned)here.bits <= bits) - break; - PULLBYTE(); - } - DROPBITS(last.bits); - } - DROPBITS(here.bits); - state->length = here.val; - - /* process literal */ - if (here.op == 0) { - Tracevv((stderr, here.val >= 0x20 && here.val < 0x7f ? - "inflate: literal '%c'\n" : - "inflate: literal 0x%02x\n", here.val)); - ROOM(); - *put++ = (unsigned char)(state->length); - left--; - state->mode = LEN; - break; - } - - /* process end of block */ - if (here.op & 32) { - Tracevv((stderr, "inflate: end of block\n")); - state->mode = TYPE; - break; - } - - /* invalid code */ - if (here.op & 64) { - strm->msg = (char *)"invalid literal/length code"; - state->mode = BAD; - break; - } - - /* length code -- get extra bits, if any */ - state->extra = (here.op & 15); - if (state->extra != 0) { - NEEDBITS(state->extra); - state->length += BITS(state->extra); - DROPBITS(state->extra); - } - Tracevv((stderr, "inflate: length %u\n", state->length)); - - /* get distance code */ - for (;;) { - here = state->distcode[BITS(state->distbits)]; - if (here.bits <= bits) - break; - PULLBYTE(); - } - if ((here.op & 0xf0) == 0) { - last = here; - for (;;) { - here = state->distcode[last.val + (BITS(last.bits + last.op) >> last.bits)]; - if ((unsigned)last.bits + (unsigned)here.bits <= bits) - break; - PULLBYTE(); - } - DROPBITS(last.bits); - } - DROPBITS(here.bits); - if (here.op & 64) { - strm->msg = (char *)"invalid distance code"; - state->mode = BAD; - break; - } - state->offset = here.val; - - /* get distance extra bits, if any */ - state->extra = (here.op & 15); - if (state->extra != 0) { - NEEDBITS(state->extra); - state->offset += BITS(state->extra); - DROPBITS(state->extra); - } - if (state->offset > state->wsize - (state->whave < state->wsize ? left : 0)) { - strm->msg = (char *)"invalid distance too far back"; - state->mode = BAD; - break; - } - Tracevv((stderr, "inflate: distance %u\n", state->offset)); - - /* copy match from window to output */ - do { - ROOM(); - copy = state->wsize - state->offset; - if (copy < left) { - from = put + copy; - copy = left - copy; - } else { - from = put - state->offset; - copy = left; - } - if (copy > state->length) - copy = state->length; - state->length -= copy; - left -= copy; - do { - *put++ = *from++; - } while (--copy); - } while (state->length != 0); - break; - - case DONE: - /* inflate stream terminated properly -- write leftover output */ - ret = Z_STREAM_END; - if (left < state->wsize) { - if (out(out_desc, state->window, state->wsize - left)) - ret = Z_BUF_ERROR; - } - goto inf_leave; - - case BAD: - ret = Z_DATA_ERROR; - goto inf_leave; - - default: /* can't happen, but makes compilers happy */ - ret = Z_STREAM_ERROR; - goto inf_leave; - } - - /* Return unused input */ - inf_leave: - strm->next_in = next; - strm->avail_in = have; - return ret; -} - -int ZEXPORT inflateBackEnd(z_stream *strm) { - if (strm == Z_NULL || strm->state == Z_NULL || strm->zfree == (free_func)0) - return Z_STREAM_ERROR; - ZFREE(strm, strm->state); - strm->state = Z_NULL; - Tracev((stderr, "inflate: end\n")); - return Z_OK; -} diff --git a/contrib/libzlib-ng/inffast.c b/contrib/libzlib-ng/inffast.c deleted file mode 100644 index 8acb261d008..00000000000 --- a/contrib/libzlib-ng/inffast.c +++ /dev/null @@ -1,328 +0,0 @@ -/* inffast.c -- fast decoding - * Copyright (C) 1995-2008, 2010, 2013 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -#include "zutil.h" -#include "inftrees.h" -#include "inflate.h" -#include "inffast.h" - -/* Allow machine dependent optimization for post-increment or pre-increment. - Based on testing to date, - Pre-increment preferred for: - - PowerPC G3 (Adler) - - MIPS R5000 (Randers-Pehrson) - Post-increment preferred for: - - none - No measurable difference: - - Pentium III (Anderson) - - M68060 (Nikl) - */ -#ifdef POSTINC -# define OFF 0 -# define PUP(a) *(a)++ -#else -# define OFF 1 -# define PUP(a) *++(a) -#endif - -/* Return the low n bits of the bit accumulator (n < 16) */ -#define BITS(n) \ - (hold & ((1U << (n)) - 1)) - -/* Remove n bits from the bit accumulator */ -#define DROPBITS(n) \ - do { \ - hold >>= (n); \ - bits -= (unsigned)(n); \ - } while (0) - -/* - Decode literal, length, and distance codes and write out the resulting - literal and match bytes until either not enough input or output is - available, an end-of-block is encountered, or a data error is encountered. - When large enough input and output buffers are supplied to inflate(), for - example, a 16K input buffer and a 64K output buffer, more than 95% of the - inflate execution time is spent in this routine. - - Entry assumptions: - - state->mode == LEN - strm->avail_in >= 6 - strm->avail_out >= 258 - start >= strm->avail_out - state->bits < 8 - - On return, state->mode is one of: - - LEN -- ran out of enough output space or enough available input - TYPE -- reached end of block code, inflate() to interpret next block - BAD -- error in block data - - Notes: - - - The maximum input bits used by a length/distance pair is 15 bits for the - length code, 5 bits for the length extra, 15 bits for the distance code, - and 13 bits for the distance extra. This totals 48 bits, or six bytes. - Therefore if strm->avail_in >= 6, then there is enough input to avoid - checking for available input while decoding. - - - The maximum bytes that a single length/distance pair can output is 258 - bytes, which is the maximum length that can be coded. inflate_fast() - requires strm->avail_out >= 258 for each loop to avoid checking for - output space. - */ -void ZLIB_INTERNAL inflate_fast(z_stream *strm, unsigned long start) { - /* start: inflate()'s starting value for strm->avail_out */ - struct inflate_state *state; - const unsigned char *in; /* local strm->next_in */ - const unsigned char *last; /* have enough input while in < last */ - unsigned char *out; /* local strm->next_out */ - unsigned char *beg; /* inflate()'s initial strm->next_out */ - unsigned char *end; /* while out < end, enough space available */ -#ifdef INFLATE_STRICT - unsigned dmax; /* maximum distance from zlib header */ -#endif - unsigned wsize; /* window size or zero if not using window */ - unsigned whave; /* valid bytes in the window */ - unsigned wnext; /* window write index */ - unsigned char *window; /* allocated sliding window, if wsize != 0 */ - uint32_t hold; /* local strm->hold */ - unsigned bits; /* local strm->bits */ - code const *lcode; /* local strm->lencode */ - code const *dcode; /* local strm->distcode */ - unsigned lmask; /* mask for first level of length codes */ - unsigned dmask; /* mask for first level of distance codes */ - code here; /* retrieved table entry */ - unsigned op; /* code bits, operation, extra bits, or */ - /* window position, window bytes to copy */ - unsigned len; /* match length, unused bytes */ - unsigned dist; /* match distance */ - unsigned char *from; /* where to copy match from */ - - /* copy state to local variables */ - state = (struct inflate_state *)strm->state; - in = strm->next_in - OFF; - last = in + (strm->avail_in - 5); - out = strm->next_out - OFF; - beg = out - (start - strm->avail_out); - end = out + (strm->avail_out - 257); -#ifdef INFLATE_STRICT - dmax = state->dmax; -#endif - wsize = state->wsize; - whave = state->whave; - wnext = state->wnext; - window = state->window; - hold = state->hold; - bits = state->bits; - lcode = state->lencode; - dcode = state->distcode; - lmask = (1U << state->lenbits) - 1; - dmask = (1U << state->distbits) - 1; - - /* decode literals and length/distances until end-of-block or not enough - input data or output space */ - do { - if (bits < 15) { - hold += (PUP(in) << bits); - bits += 8; - hold += (PUP(in) << bits); - bits += 8; - } - here = lcode[hold & lmask]; - dolen: - DROPBITS(here.bits); - op = here.op; - if (op == 0) { /* literal */ - Tracevv((stderr, here.val >= 0x20 && here.val < 0x7f ? - "inflate: literal '%c'\n" : - "inflate: literal 0x%02x\n", here.val)); - PUP(out) = (unsigned char)(here.val); - } else if (op & 16) { /* length base */ - len = here.val; - op &= 15; /* number of extra bits */ - if (op) { - if (bits < op) { - hold += (PUP(in) << bits); - bits += 8; - } - len += BITS(op); - DROPBITS(op); - } - Tracevv((stderr, "inflate: length %u\n", len)); - if (bits < 15) { - hold += (PUP(in) << bits); - bits += 8; - hold += (PUP(in) << bits); - bits += 8; - } - here = dcode[hold & dmask]; - dodist: - DROPBITS(here.bits); - op = here.op; - if (op & 16) { /* distance base */ - dist = here.val; - op &= 15; /* number of extra bits */ - if (bits < op) { - hold += (PUP(in) << bits); - bits += 8; - if (bits < op) { - hold += (PUP(in) << bits); - bits += 8; - } - } - dist += BITS(op); -#ifdef INFLATE_STRICT - if (dist > dmax) { - strm->msg = (char *)"invalid distance too far back"; - state->mode = BAD; - break; - } -#endif - DROPBITS(op); - Tracevv((stderr, "inflate: distance %u\n", dist)); - op = (unsigned)(out - beg); /* max distance in output */ - if (dist > op) { /* see if copy from window */ - op = dist - op; /* distance back in window */ - if (op > whave) { - if (state->sane) { - strm->msg = (char *)"invalid distance too far back"; - state->mode = BAD; - break; - } -#ifdef INFLATE_ALLOW_INVALID_DISTANCE_TOOFAR_ARRR - if (len <= op - whave) { - do { - PUP(out) = 0; - } while (--len); - continue; - } - len -= op - whave; - do { - PUP(out) = 0; - } while (--op > whave); - if (op == 0) { - from = out - dist; - do { - PUP(out) = PUP(from); - } while (--len); - continue; - } -#endif - } - from = window - OFF; - if (wnext == 0) { /* very common case */ - from += wsize - op; - if (op < len) { /* some from window */ - len -= op; - do { - PUP(out) = PUP(from); - } while (--op); - from = out - dist; /* rest from output */ - } - } else if (wnext < op) { /* wrap around window */ - from += wsize + wnext - op; - op -= wnext; - if (op < len) { /* some from end of window */ - len -= op; - do { - PUP(out) = PUP(from); - } while (--op); - from = window - OFF; - if (wnext < len) { /* some from start of window */ - op = wnext; - len -= op; - do { - PUP(out) = PUP(from); - } while (--op); - from = out - dist; /* rest from output */ - } - } - } else { /* contiguous in window */ - from += wnext - op; - if (op < len) { /* some from window */ - len -= op; - do { - PUP(out) = PUP(from); - } while (--op); - from = out - dist; /* rest from output */ - } - } - while (len > 2) { - PUP(out) = PUP(from); - PUP(out) = PUP(from); - PUP(out) = PUP(from); - len -= 3; - } - if (len) { - PUP(out) = PUP(from); - if (len > 1) - PUP(out) = PUP(from); - } - } else { - from = out - dist; /* copy direct from output */ - do { /* minimum length is three */ - PUP(out) = PUP(from); - PUP(out) = PUP(from); - PUP(out) = PUP(from); - len -= 3; - } while (len > 2); - if (len) { - PUP(out) = PUP(from); - if (len > 1) - PUP(out) = PUP(from); - } - } - } else if ((op & 64) == 0) { /* 2nd level distance code */ - here = dcode[here.val + BITS(op)]; - goto dodist; - } else { - strm->msg = (char *)"invalid distance code"; - state->mode = BAD; - break; - } - } else if ((op & 64) == 0) { /* 2nd level length code */ - here = lcode[here.val + BITS(op)]; - goto dolen; - } else if (op & 32) { /* end-of-block */ - Tracevv((stderr, "inflate: end of block\n")); - state->mode = TYPE; - break; - } else { - strm->msg = (char *)"invalid literal/length code"; - state->mode = BAD; - break; - } - } while (in < last && out < end); - - /* return unused bytes (on entry, bits < 8, so in won't go too far back) */ - len = bits >> 3; - in -= len; - bits -= len << 3; - hold &= (1U << bits) - 1; - - /* update state and return */ - strm->next_in = in + OFF; - strm->next_out = out + OFF; - strm->avail_in = (unsigned)(in < last ? 5 + (last - in) : 5 - (in - last)); - strm->avail_out = (unsigned)(out < end ? 257 + (end - out) : 257 - (out - end)); - state->hold = hold; - state->bits = bits; - return; -} - -/* - inflate_fast() speedups that turned out slower (on a PowerPC G3 750CXe): - - Using bit fields for code structure - - Different op definition to avoid & for extra bits (do & for table bits) - - Three separate decoding do-loops for direct, window, and wnext == 0 - - Special case for distance > 1 copies to do overlapped load and store copy - - Explicit branch predictions (based on measured branch probabilities) - - Deferring match copy and interspersed it with decoding subsequent codes - - Swapping literal/length else - - Swapping window/direct else - - Larger unrolled copy loops (three is about right) - - Moving len -= 3 statement into middle of loop - */ diff --git a/contrib/libzlib-ng/inffast.h b/contrib/libzlib-ng/inffast.h deleted file mode 100644 index 0d75614772a..00000000000 --- a/contrib/libzlib-ng/inffast.h +++ /dev/null @@ -1,15 +0,0 @@ -#ifndef INFFAST_H_ -#define INFFAST_H_ -/* inffast.h -- header to use inffast.c - * Copyright (C) 1995-2003, 2010 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* WARNING: this file should *not* be used by applications. It is - part of the implementation of the compression library and is - subject to change. Applications should only use zlib.h. - */ - -void ZLIB_INTERNAL inflate_fast(z_stream *strm, unsigned long start); - -#endif /* INFFAST_H_ */ diff --git a/contrib/libzlib-ng/inffixed.h b/contrib/libzlib-ng/inffixed.h deleted file mode 100644 index d6283277694..00000000000 --- a/contrib/libzlib-ng/inffixed.h +++ /dev/null @@ -1,94 +0,0 @@ - /* inffixed.h -- table for decoding fixed codes - * Generated automatically by makefixed(). - */ - - /* WARNING: this file should *not* be used by applications. - It is part of the implementation of this library and is - subject to change. Applications should only use zlib.h. - */ - - static const code lenfix[512] = { - {96,7,0},{0,8,80},{0,8,16},{20,8,115},{18,7,31},{0,8,112},{0,8,48}, - {0,9,192},{16,7,10},{0,8,96},{0,8,32},{0,9,160},{0,8,0},{0,8,128}, - {0,8,64},{0,9,224},{16,7,6},{0,8,88},{0,8,24},{0,9,144},{19,7,59}, - {0,8,120},{0,8,56},{0,9,208},{17,7,17},{0,8,104},{0,8,40},{0,9,176}, - {0,8,8},{0,8,136},{0,8,72},{0,9,240},{16,7,4},{0,8,84},{0,8,20}, - {21,8,227},{19,7,43},{0,8,116},{0,8,52},{0,9,200},{17,7,13},{0,8,100}, - {0,8,36},{0,9,168},{0,8,4},{0,8,132},{0,8,68},{0,9,232},{16,7,8}, - {0,8,92},{0,8,28},{0,9,152},{20,7,83},{0,8,124},{0,8,60},{0,9,216}, - {18,7,23},{0,8,108},{0,8,44},{0,9,184},{0,8,12},{0,8,140},{0,8,76}, - {0,9,248},{16,7,3},{0,8,82},{0,8,18},{21,8,163},{19,7,35},{0,8,114}, - {0,8,50},{0,9,196},{17,7,11},{0,8,98},{0,8,34},{0,9,164},{0,8,2}, - {0,8,130},{0,8,66},{0,9,228},{16,7,7},{0,8,90},{0,8,26},{0,9,148}, - {20,7,67},{0,8,122},{0,8,58},{0,9,212},{18,7,19},{0,8,106},{0,8,42}, - {0,9,180},{0,8,10},{0,8,138},{0,8,74},{0,9,244},{16,7,5},{0,8,86}, - {0,8,22},{64,8,0},{19,7,51},{0,8,118},{0,8,54},{0,9,204},{17,7,15}, - {0,8,102},{0,8,38},{0,9,172},{0,8,6},{0,8,134},{0,8,70},{0,9,236}, - {16,7,9},{0,8,94},{0,8,30},{0,9,156},{20,7,99},{0,8,126},{0,8,62}, - {0,9,220},{18,7,27},{0,8,110},{0,8,46},{0,9,188},{0,8,14},{0,8,142}, - {0,8,78},{0,9,252},{96,7,0},{0,8,81},{0,8,17},{21,8,131},{18,7,31}, - {0,8,113},{0,8,49},{0,9,194},{16,7,10},{0,8,97},{0,8,33},{0,9,162}, - {0,8,1},{0,8,129},{0,8,65},{0,9,226},{16,7,6},{0,8,89},{0,8,25}, - {0,9,146},{19,7,59},{0,8,121},{0,8,57},{0,9,210},{17,7,17},{0,8,105}, - {0,8,41},{0,9,178},{0,8,9},{0,8,137},{0,8,73},{0,9,242},{16,7,4}, - {0,8,85},{0,8,21},{16,8,258},{19,7,43},{0,8,117},{0,8,53},{0,9,202}, - {17,7,13},{0,8,101},{0,8,37},{0,9,170},{0,8,5},{0,8,133},{0,8,69}, - {0,9,234},{16,7,8},{0,8,93},{0,8,29},{0,9,154},{20,7,83},{0,8,125}, - {0,8,61},{0,9,218},{18,7,23},{0,8,109},{0,8,45},{0,9,186},{0,8,13}, - {0,8,141},{0,8,77},{0,9,250},{16,7,3},{0,8,83},{0,8,19},{21,8,195}, - {19,7,35},{0,8,115},{0,8,51},{0,9,198},{17,7,11},{0,8,99},{0,8,35}, - {0,9,166},{0,8,3},{0,8,131},{0,8,67},{0,9,230},{16,7,7},{0,8,91}, - {0,8,27},{0,9,150},{20,7,67},{0,8,123},{0,8,59},{0,9,214},{18,7,19}, - {0,8,107},{0,8,43},{0,9,182},{0,8,11},{0,8,139},{0,8,75},{0,9,246}, - {16,7,5},{0,8,87},{0,8,23},{64,8,0},{19,7,51},{0,8,119},{0,8,55}, - {0,9,206},{17,7,15},{0,8,103},{0,8,39},{0,9,174},{0,8,7},{0,8,135}, - {0,8,71},{0,9,238},{16,7,9},{0,8,95},{0,8,31},{0,9,158},{20,7,99}, - {0,8,127},{0,8,63},{0,9,222},{18,7,27},{0,8,111},{0,8,47},{0,9,190}, - {0,8,15},{0,8,143},{0,8,79},{0,9,254},{96,7,0},{0,8,80},{0,8,16}, - {20,8,115},{18,7,31},{0,8,112},{0,8,48},{0,9,193},{16,7,10},{0,8,96}, - {0,8,32},{0,9,161},{0,8,0},{0,8,128},{0,8,64},{0,9,225},{16,7,6}, - {0,8,88},{0,8,24},{0,9,145},{19,7,59},{0,8,120},{0,8,56},{0,9,209}, - {17,7,17},{0,8,104},{0,8,40},{0,9,177},{0,8,8},{0,8,136},{0,8,72}, - {0,9,241},{16,7,4},{0,8,84},{0,8,20},{21,8,227},{19,7,43},{0,8,116}, - {0,8,52},{0,9,201},{17,7,13},{0,8,100},{0,8,36},{0,9,169},{0,8,4}, - {0,8,132},{0,8,68},{0,9,233},{16,7,8},{0,8,92},{0,8,28},{0,9,153}, - {20,7,83},{0,8,124},{0,8,60},{0,9,217},{18,7,23},{0,8,108},{0,8,44}, - {0,9,185},{0,8,12},{0,8,140},{0,8,76},{0,9,249},{16,7,3},{0,8,82}, - {0,8,18},{21,8,163},{19,7,35},{0,8,114},{0,8,50},{0,9,197},{17,7,11}, - {0,8,98},{0,8,34},{0,9,165},{0,8,2},{0,8,130},{0,8,66},{0,9,229}, - {16,7,7},{0,8,90},{0,8,26},{0,9,149},{20,7,67},{0,8,122},{0,8,58}, - {0,9,213},{18,7,19},{0,8,106},{0,8,42},{0,9,181},{0,8,10},{0,8,138}, - {0,8,74},{0,9,245},{16,7,5},{0,8,86},{0,8,22},{64,8,0},{19,7,51}, - {0,8,118},{0,8,54},{0,9,205},{17,7,15},{0,8,102},{0,8,38},{0,9,173}, - {0,8,6},{0,8,134},{0,8,70},{0,9,237},{16,7,9},{0,8,94},{0,8,30}, - {0,9,157},{20,7,99},{0,8,126},{0,8,62},{0,9,221},{18,7,27},{0,8,110}, - {0,8,46},{0,9,189},{0,8,14},{0,8,142},{0,8,78},{0,9,253},{96,7,0}, - {0,8,81},{0,8,17},{21,8,131},{18,7,31},{0,8,113},{0,8,49},{0,9,195}, - {16,7,10},{0,8,97},{0,8,33},{0,9,163},{0,8,1},{0,8,129},{0,8,65}, - {0,9,227},{16,7,6},{0,8,89},{0,8,25},{0,9,147},{19,7,59},{0,8,121}, - {0,8,57},{0,9,211},{17,7,17},{0,8,105},{0,8,41},{0,9,179},{0,8,9}, - {0,8,137},{0,8,73},{0,9,243},{16,7,4},{0,8,85},{0,8,21},{16,8,258}, - {19,7,43},{0,8,117},{0,8,53},{0,9,203},{17,7,13},{0,8,101},{0,8,37}, - {0,9,171},{0,8,5},{0,8,133},{0,8,69},{0,9,235},{16,7,8},{0,8,93}, - {0,8,29},{0,9,155},{20,7,83},{0,8,125},{0,8,61},{0,9,219},{18,7,23}, - {0,8,109},{0,8,45},{0,9,187},{0,8,13},{0,8,141},{0,8,77},{0,9,251}, - {16,7,3},{0,8,83},{0,8,19},{21,8,195},{19,7,35},{0,8,115},{0,8,51}, - {0,9,199},{17,7,11},{0,8,99},{0,8,35},{0,9,167},{0,8,3},{0,8,131}, - {0,8,67},{0,9,231},{16,7,7},{0,8,91},{0,8,27},{0,9,151},{20,7,67}, - {0,8,123},{0,8,59},{0,9,215},{18,7,19},{0,8,107},{0,8,43},{0,9,183}, - {0,8,11},{0,8,139},{0,8,75},{0,9,247},{16,7,5},{0,8,87},{0,8,23}, - {64,8,0},{19,7,51},{0,8,119},{0,8,55},{0,9,207},{17,7,15},{0,8,103}, - {0,8,39},{0,9,175},{0,8,7},{0,8,135},{0,8,71},{0,9,239},{16,7,9}, - {0,8,95},{0,8,31},{0,9,159},{20,7,99},{0,8,127},{0,8,63},{0,9,223}, - {18,7,27},{0,8,111},{0,8,47},{0,9,191},{0,8,15},{0,8,143},{0,8,79}, - {0,9,255} - }; - - static const code distfix[32] = { - {16,5,1},{23,5,257},{19,5,17},{27,5,4097},{17,5,5},{25,5,1025}, - {21,5,65},{29,5,16385},{16,5,3},{24,5,513},{20,5,33},{28,5,8193}, - {18,5,9},{26,5,2049},{22,5,129},{64,5,0},{16,5,2},{23,5,385}, - {19,5,25},{27,5,6145},{17,5,7},{25,5,1537},{21,5,97},{29,5,24577}, - {16,5,4},{24,5,769},{20,5,49},{28,5,12289},{18,5,13},{26,5,3073}, - {22,5,193},{64,5,0} - }; diff --git a/contrib/libzlib-ng/inflate.c b/contrib/libzlib-ng/inflate.c deleted file mode 100644 index de1f0124e55..00000000000 --- a/contrib/libzlib-ng/inflate.c +++ /dev/null @@ -1,1467 +0,0 @@ -/* inflate.c -- zlib decompression - * Copyright (C) 1995-2012 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* - * Change history: - * - * 1.2.beta0 24 Nov 2002 - * - First version -- complete rewrite of inflate to simplify code, avoid - * creation of window when not needed, minimize use of window when it is - * needed, make inffast.c even faster, implement gzip decoding, and to - * improve code readability and style over the previous zlib inflate code - * - * 1.2.beta1 25 Nov 2002 - * - Use pointers for available input and output checking in inffast.c - * - Remove input and output counters in inffast.c - * - Change inffast.c entry and loop from avail_in >= 7 to >= 6 - * - Remove unnecessary second byte pull from length extra in inffast.c - * - Unroll direct copy to three copies per loop in inffast.c - * - * 1.2.beta2 4 Dec 2002 - * - Change external routine names to reduce potential conflicts - * - Correct filename to inffixed.h for fixed tables in inflate.c - * - Make hbuf[] unsigned char to match parameter type in inflate.c - * - Change strm->next_out[-state->offset] to *(strm->next_out - state->offset) - * to avoid negation problem on Alphas (64 bit) in inflate.c - * - * 1.2.beta3 22 Dec 2002 - * - Add comments on state->bits assertion in inffast.c - * - Add comments on op field in inftrees.h - * - Fix bug in reuse of allocated window after inflateReset() - * - Remove bit fields--back to byte structure for speed - * - Remove distance extra == 0 check in inflate_fast()--only helps for lengths - * - Change post-increments to pre-increments in inflate_fast(), PPC biased? - * - Add compile time option, POSTINC, to use post-increments instead (Intel?) - * - Make MATCH copy in inflate() much faster for when inflate_fast() not used - * - Use local copies of stream next and avail values, as well as local bit - * buffer and bit count in inflate()--for speed when inflate_fast() not used - * - * 1.2.beta4 1 Jan 2003 - * - Split ptr - 257 statements in inflate_table() to avoid compiler warnings - * - Move a comment on output buffer sizes from inffast.c to inflate.c - * - Add comments in inffast.c to introduce the inflate_fast() routine - * - Rearrange window copies in inflate_fast() for speed and simplification - * - Unroll last copy for window match in inflate_fast() - * - Use local copies of window variables in inflate_fast() for speed - * - Pull out common wnext == 0 case for speed in inflate_fast() - * - Make op and len in inflate_fast() unsigned for consistency - * - Add FAR to lcode and dcode declarations in inflate_fast() - * - Simplified bad distance check in inflate_fast() - * - Added inflateBackInit(), inflateBack(), and inflateBackEnd() in new - * source file infback.c to provide a call-back interface to inflate for - * programs like gzip and unzip -- uses window as output buffer to avoid - * window copying - * - * 1.2.beta5 1 Jan 2003 - * - Improved inflateBack() interface to allow the caller to provide initial - * input in strm. - * - Fixed stored blocks bug in inflateBack() - * - * 1.2.beta6 4 Jan 2003 - * - Added comments in inffast.c on effectiveness of POSTINC - * - Typecasting all around to reduce compiler warnings - * - Changed loops from while (1) or do {} while (1) to for (;;), again to - * make compilers happy - * - Changed type of window in inflateBackInit() to unsigned char * - * - * 1.2.beta7 27 Jan 2003 - * - Changed many types to unsigned or unsigned short to avoid warnings - * - Added inflateCopy() function - * - * 1.2.0 9 Mar 2003 - * - Changed inflateBack() interface to provide separate opaque descriptors - * for the in() and out() functions - * - Changed inflateBack() argument and in_func typedef to swap the length - * and buffer address return values for the input function - * - Check next_in and next_out for Z_NULL on entry to inflate() - * - * The history for versions after 1.2.0 are in ChangeLog in zlib distribution. - */ - -#include "zutil.h" -#include "inftrees.h" -#include "inflate.h" -#include "inffast.h" - -#ifdef MAKEFIXED -# ifndef BUILDFIXED -# define BUILDFIXED -# endif -#endif - -/* function prototypes */ -local void fixedtables(struct inflate_state *state); -local int updatewindow(z_stream *strm, const unsigned char *end, uint32_t copy); -#ifdef BUILDFIXED - void makefixed(void); -#endif -local uint32_t syncsearch(uint32_t *have, const unsigned char *buf, uint32_t len); - -int ZEXPORT inflateResetKeep(z_stream *strm) { - struct inflate_state *state; - - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - state = (struct inflate_state *)strm->state; - strm->total_in = strm->total_out = state->total = 0; - strm->msg = Z_NULL; - if (state->wrap) /* to support ill-conceived Java test suite */ - strm->adler = state->wrap & 1; - state->mode = HEAD; - state->last = 0; - state->havedict = 0; - state->dmax = 32768U; - state->head = Z_NULL; - state->hold = 0; - state->bits = 0; - state->lencode = state->distcode = state->next = state->codes; - state->sane = 1; - state->back = -1; - Tracev((stderr, "inflate: reset\n")); - return Z_OK; -} - -int ZEXPORT inflateReset(z_stream *strm) { - struct inflate_state *state; - - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - state = (struct inflate_state *)strm->state; - state->wsize = 0; - state->whave = 0; - state->wnext = 0; - return inflateResetKeep(strm); -} - -int ZEXPORT inflateReset2(z_stream *strm, int windowBits) { - int wrap; - struct inflate_state *state; - - /* get the state */ - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - state = (struct inflate_state *)strm->state; - - /* extract wrap request from windowBits parameter */ - if (windowBits < 0) { - wrap = 0; - windowBits = -windowBits; - } else { - wrap = (windowBits >> 4) + 1; -#ifdef GUNZIP - if (windowBits < 48) - windowBits &= 15; -#endif - } - - /* set number of window bits, free window if different */ - if (windowBits && (windowBits < 8 || windowBits > 15)) - return Z_STREAM_ERROR; - if (state->window != Z_NULL && state->wbits != (unsigned)windowBits) { - ZFREE(strm, state->window); - state->window = Z_NULL; - } - - /* update state and reset the rest of it */ - state->wrap = wrap; - state->wbits = (unsigned)windowBits; - return inflateReset(strm); -} - -int ZEXPORT inflateInit2_(z_stream *strm, int windowBits, const char *version, int stream_size) { - int ret; - struct inflate_state *state; - - if (version == Z_NULL || version[0] != ZLIB_VERSION[0] || stream_size != (int)(sizeof(z_stream))) - return Z_VERSION_ERROR; - if (strm == Z_NULL) - return Z_STREAM_ERROR; - strm->msg = Z_NULL; /* in case we return an error */ - if (strm->zalloc == (alloc_func)0) { - strm->zalloc = zcalloc; - strm->opaque = NULL; - } - if (strm->zfree == (free_func)0) - strm->zfree = zcfree; - state = (struct inflate_state *) ZALLOC(strm, 1, sizeof(struct inflate_state)); - if (state == Z_NULL) - return Z_MEM_ERROR; - Tracev((stderr, "inflate: allocated\n")); - strm->state = (struct internal_state *)state; - state->window = Z_NULL; - ret = inflateReset2(strm, windowBits); - if (ret != Z_OK) { - ZFREE(strm, state); - strm->state = Z_NULL; - } - return ret; -} - -int ZEXPORT inflateInit_(z_stream *strm, const char *version, int stream_size) { - return inflateInit2_(strm, DEF_WBITS, version, stream_size); -} - -int ZEXPORT inflatePrime(z_stream *strm, int bits, int value) { - struct inflate_state *state; - - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - state = (struct inflate_state *)strm->state; - if (bits < 0) { - state->hold = 0; - state->bits = 0; - return Z_OK; - } - if (bits > 16 || state->bits + bits > 32) - return Z_STREAM_ERROR; - value &= (1L << bits) - 1; - state->hold += (unsigned)value << state->bits; - state->bits += bits; - return Z_OK; -} - -/* - Return state with length and distance decoding tables and index sizes set to - fixed code decoding. Normally this returns fixed tables from inffixed.h. - If BUILDFIXED is defined, then instead this routine builds the tables the - first time it's called, and returns those tables the first time and - thereafter. This reduces the size of the code by about 2K bytes, in - exchange for a little execution time. However, BUILDFIXED should not be - used for threaded applications, since the rewriting of the tables and virgin - may not be thread-safe. - */ -local void fixedtables(struct inflate_state *state) { -#ifdef BUILDFIXED - static int virgin = 1; - static code *lenfix, *distfix; - static code fixed[544]; - - /* build fixed huffman tables if first call (may not be thread safe) */ - if (virgin) { - unsigned sym, bits; - static code *next; - - /* literal/length table */ - sym = 0; - while (sym < 144) state->lens[sym++] = 8; - while (sym < 256) state->lens[sym++] = 9; - while (sym < 280) state->lens[sym++] = 7; - while (sym < 288) state->lens[sym++] = 8; - next = fixed; - lenfix = next; - bits = 9; - inflate_table(LENS, state->lens, 288, &(next), &(bits), state->work); - - /* distance table */ - sym = 0; - while (sym < 32) state->lens[sym++] = 5; - distfix = next; - bits = 5; - inflate_table(DISTS, state->lens, 32, &(next), &(bits), state->work); - - /* do this just once */ - virgin = 0; - } -#else /* !BUILDFIXED */ -# include "inffixed.h" -#endif /* BUILDFIXED */ - state->lencode = lenfix; - state->lenbits = 9; - state->distcode = distfix; - state->distbits = 5; -} - -#ifdef MAKEFIXED -#include - -/* - Write out the inffixed.h that is #include'd above. Defining MAKEFIXED also - defines BUILDFIXED, so the tables are built on the fly. makefixed() writes - those tables to stdout, which would be piped to inffixed.h. A small program - can simply call makefixed to do this: - - void makefixed(void); - - int main(void) - { - makefixed(); - return 0; - } - - Then that can be linked with zlib built with MAKEFIXED defined and run: - - a.out > inffixed.h - */ -void makefixed(void) { - unsigned low, size; - struct inflate_state state; - - fixedtables(&state); - puts(" /* inffixed.h -- table for decoding fixed codes"); - puts(" * Generated automatically by makefixed()."); - puts(" */"); - puts(""); - puts(" /* WARNING: this file should *not* be used by applications."); - puts(" It is part of the implementation of this library and is"); - puts(" subject to change. Applications should only use zlib.h."); - puts(" */"); - puts(""); - size = 1U << 9; - printf(" static const code lenfix[%u] = {", size); - low = 0; - for (;;) { - if ((low % 7) == 0) - printf("\n "); - printf("{%u,%u,%d}", (low & 127) == 99 ? 64 : state.lencode[low].op, - state.lencode[low].bits, state.lencode[low].val); - if (++low == size) - break; - putchar(','); - } - puts("\n };"); - size = 1U << 5; - printf("\n static const code distfix[%u] = {", size); - low = 0; - for (;;) { - if ((low % 6) == 0) - printf("\n "); - printf("{%u,%u,%d}", state.distcode[low].op, state.distcode[low].bits, state.distcode[low].val); - if (++low == size) - break; - putchar(','); - } - puts("\n };"); -} -#endif /* MAKEFIXED */ - -/* - Update the window with the last wsize (normally 32K) bytes written before - returning. If window does not exist yet, create it. This is only called - when a window is already in use, or when output has been written during this - inflate call, but the end of the deflate stream has not been reached yet. - It is also called to create a window for dictionary data when a dictionary - is loaded. - - Providing output buffers larger than 32K to inflate() should provide a speed - advantage, since only the last 32K of output is copied to the sliding window - upon return from inflate(), and since all distances after the first 32K of - output will fall in the output data, making match copies simpler and faster. - The advantage may be dependent on the size of the processor's data caches. - */ -local int updatewindow(z_stream *strm, const unsigned char *end, uint32_t copy) { - struct inflate_state *state; - uint32_t dist; - - state = (struct inflate_state *)strm->state; - - /* if it hasn't been done already, allocate space for the window */ - if (state->window == Z_NULL) { - state->window = (unsigned char *) ZALLOC(strm, 1U << state->wbits, sizeof(unsigned char)); - if (state->window == Z_NULL) - return 1; - } - - /* if window not in use yet, initialize */ - if (state->wsize == 0) { - state->wsize = 1U << state->wbits; - state->wnext = 0; - state->whave = 0; - } - - /* copy state->wsize or less output bytes into the circular window */ - if (copy >= state->wsize) { - memcpy(state->window, end - state->wsize, state->wsize); - state->wnext = 0; - state->whave = state->wsize; - } else { - dist = state->wsize - state->wnext; - if (dist > copy) - dist = copy; - memcpy(state->window + state->wnext, end - copy, dist); - copy -= dist; - if (copy) { - memcpy(state->window, end - copy, copy); - state->wnext = copy; - state->whave = state->wsize; - } else { - state->wnext += dist; - if (state->wnext == state->wsize) - state->wnext = 0; - if (state->whave < state->wsize) - state->whave += dist; - } - } - return 0; -} - -/* Macros for inflate(): */ - -/* check function to use adler32() for zlib or crc32() for gzip */ -#ifdef GUNZIP -# define UPDATE(check, buf, len) \ - (state->flags ? crc32(check, buf, len) : adler32(check, buf, len)) -#else -# define UPDATE(check, buf, len) adler32(check, buf, len) -#endif - -/* check macros for header crc */ -#ifdef GUNZIP -# define CRC2(check, word) \ - do { \ - hbuf[0] = (unsigned char)(word); \ - hbuf[1] = (unsigned char)((word) >> 8); \ - check = crc32(check, hbuf, 2); \ - } while (0) - -# define CRC4(check, word) \ - do { \ - hbuf[0] = (unsigned char)(word); \ - hbuf[1] = (unsigned char)((word) >> 8); \ - hbuf[2] = (unsigned char)((word) >> 16); \ - hbuf[3] = (unsigned char)((word) >> 24); \ - check = crc32(check, hbuf, 4); \ - } while (0) -#endif - -/* Load registers with state in inflate() for speed */ -#define LOAD() \ - do { \ - put = strm->next_out; \ - left = strm->avail_out; \ - next = strm->next_in; \ - have = strm->avail_in; \ - hold = state->hold; \ - bits = state->bits; \ - } while (0) - -/* Restore state from registers in inflate() */ -#define RESTORE() \ - do { \ - strm->next_out = put; \ - strm->avail_out = left; \ - strm->next_in = next; \ - strm->avail_in = have; \ - state->hold = hold; \ - state->bits = bits; \ - } while (0) - -/* Clear the input bit accumulator */ -#define INITBITS() \ - do { \ - hold = 0; \ - bits = 0; \ - } while (0) - -/* Get a byte of input into the bit accumulator, or return from inflate() - if there is no input available. */ -#define PULLBYTE() \ - do { \ - if (have == 0) goto inf_leave; \ - have--; \ - hold += (*next++ << bits); \ - bits += 8; \ - } while (0) - -/* Assure that there are at least n bits in the bit accumulator. If there is - not enough available input to do that, then return from inflate(). */ -#define NEEDBITS(n) \ - do { \ - while (bits < (unsigned)(n)) \ - PULLBYTE(); \ - } while (0) - -/* Return the low n bits of the bit accumulator (n < 16) */ -#define BITS(n) \ - (hold & ((1U << (n)) - 1)) - -/* Remove n bits from the bit accumulator */ -#define DROPBITS(n) \ - do { \ - hold >>= (n); \ - bits -= (unsigned)(n); \ - } while (0) - -/* Remove zero to seven bits as needed to go to a byte boundary */ -#define BYTEBITS() \ - do { \ - hold >>= bits & 7; \ - bits -= bits & 7; \ - } while (0) - -/* - inflate() uses a state machine to process as much input data and generate as - much output data as possible before returning. The state machine is - structured roughly as follows: - - for (;;) switch (state) { - ... - case STATEn: - if (not enough input data or output space to make progress) - return; - ... make progress ... - state = STATEm; - break; - ... - } - - so when inflate() is called again, the same case is attempted again, and - if the appropriate resources are provided, the machine proceeds to the - next state. The NEEDBITS() macro is usually the way the state evaluates - whether it can proceed or should return. NEEDBITS() does the return if - the requested bits are not available. The typical use of the BITS macros - is: - - NEEDBITS(n); - ... do something with BITS(n) ... - DROPBITS(n); - - where NEEDBITS(n) either returns from inflate() if there isn't enough - input left to load n bits into the accumulator, or it continues. BITS(n) - gives the low n bits in the accumulator. When done, DROPBITS(n) drops - the low n bits off the accumulator. INITBITS() clears the accumulator - and sets the number of available bits to zero. BYTEBITS() discards just - enough bits to put the accumulator on a byte boundary. After BYTEBITS() - and a NEEDBITS(8), then BITS(8) would return the next byte in the stream. - - NEEDBITS(n) uses PULLBYTE() to get an available byte of input, or to return - if there is no input available. The decoding of variable length codes uses - PULLBYTE() directly in order to pull just enough bytes to decode the next - code, and no more. - - Some states loop until they get enough input, making sure that enough - state information is maintained to continue the loop where it left off - if NEEDBITS() returns in the loop. For example, want, need, and keep - would all have to actually be part of the saved state in case NEEDBITS() - returns: - - case STATEw: - while (want < need) { - NEEDBITS(n); - keep[want++] = BITS(n); - DROPBITS(n); - } - state = STATEx; - case STATEx: - - As shown above, if the next state is also the next case, then the break - is omitted. - - A state may also return if there is not enough output space available to - complete that state. Those states are copying stored data, writing a - literal byte, and copying a matching string. - - When returning, a "goto inf_leave" is used to update the total counters, - update the check value, and determine whether any progress has been made - during that inflate() call in order to return the proper return code. - Progress is defined as a change in either strm->avail_in or strm->avail_out. - When there is a window, goto inf_leave will update the window with the last - output written. If a goto inf_leave occurs in the middle of decompression - and there is no window currently, goto inf_leave will create one and copy - output to the window for the next call of inflate(). - - In this implementation, the flush parameter of inflate() only affects the - return code (per zlib.h). inflate() always writes as much as possible to - strm->next_out, given the space available and the provided input--the effect - documented in zlib.h of Z_SYNC_FLUSH. Furthermore, inflate() always defers - the allocation of and copying into a sliding window until necessary, which - provides the effect documented in zlib.h for Z_FINISH when the entire input - stream available. So the only thing the flush parameter actually does is: - when flush is set to Z_FINISH, inflate() cannot return Z_OK. Instead it - will return Z_BUF_ERROR if it has not reached the end of the stream. - */ - -int ZEXPORT inflate(z_stream *strm, int flush) { - struct inflate_state *state; - const unsigned char *next; /* next input */ - unsigned char *put; /* next output */ - unsigned have, left; /* available input and output */ - uint32_t hold; /* bit buffer */ - unsigned bits; /* bits in bit buffer */ - uint32_t in, out; /* save starting available input and output */ - unsigned copy; /* number of stored or match bytes to copy */ - unsigned char *from; /* where to copy match bytes from */ - code here; /* current decoding table entry */ - code last; /* parent table entry */ - unsigned len; /* length to copy for repeats, bits to drop */ - int ret; /* return code */ -#ifdef GUNZIP - unsigned char hbuf[4]; /* buffer for gzip header crc calculation */ -#endif - static const uint16_t order[19] = /* permutation of code lengths */ - {16, 17, 18, 0, 8, 7, 9, 6, 10, 5, 11, 4, 12, 3, 13, 2, 14, 1, 15}; - - if (strm == Z_NULL || strm->state == Z_NULL || strm->next_out == Z_NULL || - (strm->next_in == Z_NULL && strm->avail_in != 0)) - return Z_STREAM_ERROR; - - state = (struct inflate_state *)strm->state; - if (state->mode == TYPE) /* skip check */ - state->mode = TYPEDO; - LOAD(); - in = have; - out = left; - ret = Z_OK; - for (;;) - switch (state->mode) { - case HEAD: - if (state->wrap == 0) { - state->mode = TYPEDO; - break; - } - NEEDBITS(16); -#ifdef GUNZIP - if ((state->wrap & 2) && hold == 0x8b1f) { /* gzip header */ - if (state->wbits == 0) - state->wbits = 15; - state->check = crc32(0L, Z_NULL, 0); - CRC2(state->check, hold); - INITBITS(); - state->mode = FLAGS; - break; - } - state->flags = 0; /* expect zlib header */ - if (state->head != Z_NULL) - state->head->done = -1; - if (!(state->wrap & 1) || /* check if zlib header allowed */ -#else - if ( -#endif - ((BITS(8) << 8) + (hold >> 8)) % 31) { - strm->msg = (char *)"incorrect header check"; - state->mode = BAD; - break; - } - if (BITS(4) != Z_DEFLATED) { - strm->msg = (char *)"unknown compression method"; - state->mode = BAD; - break; - } - DROPBITS(4); - len = BITS(4) + 8; - if (state->wbits == 0) - state->wbits = len; - if (len > 15 || len > state->wbits) { - strm->msg = (char *)"invalid window size"; - state->mode = BAD; - break; - } - state->dmax = 1U << len; - Tracev((stderr, "inflate: zlib header ok\n")); - strm->adler = state->check = adler32(0L, Z_NULL, 0); - state->mode = hold & 0x200 ? DICTID : TYPE; - INITBITS(); - break; -#ifdef GUNZIP - case FLAGS: - NEEDBITS(16); - state->flags = (int)(hold); - if ((state->flags & 0xff) != Z_DEFLATED) { - strm->msg = (char *)"unknown compression method"; - state->mode = BAD; - break; - } - if (state->flags & 0xe000) { - strm->msg = (char *)"unknown header flags set"; - state->mode = BAD; - break; - } - if (state->head != Z_NULL) - state->head->text = (int)((hold >> 8) & 1); - if (state->flags & 0x0200) - CRC2(state->check, hold); - INITBITS(); - state->mode = TIME; - case TIME: - NEEDBITS(32); - if (state->head != Z_NULL) - state->head->time = hold; - if (state->flags & 0x0200) - CRC4(state->check, hold); - INITBITS(); - state->mode = OS; - case OS: - NEEDBITS(16); - if (state->head != Z_NULL) { - state->head->xflags = (int)(hold & 0xff); - state->head->os = (int)(hold >> 8); - } - if (state->flags & 0x0200) - CRC2(state->check, hold); - INITBITS(); - state->mode = EXLEN; - case EXLEN: - if (state->flags & 0x0400) { - NEEDBITS(16); - state->length = (uint16_t)hold; - if (state->head != Z_NULL) - state->head->extra_len = (uint16_t)hold; - if (state->flags & 0x0200) - CRC2(state->check, hold); - INITBITS(); - } else if (state->head != Z_NULL) { - state->head->extra = Z_NULL; - } - state->mode = EXTRA; - case EXTRA: - if (state->flags & 0x0400) { - copy = state->length; - if (copy > have) - copy = have; - if (copy) { - if (state->head != Z_NULL && - state->head->extra != Z_NULL) { - len = state->head->extra_len - state->length; - memcpy(state->head->extra + len, next, - len + copy > state->head->extra_max ? - state->head->extra_max - len : copy); - } - if (state->flags & 0x0200) - state->check = crc32(state->check, next, copy); - have -= copy; - next += copy; - state->length -= copy; - } - if (state->length) - goto inf_leave; - } - state->length = 0; - state->mode = NAME; - case NAME: - if (state->flags & 0x0800) { - if (have == 0) goto inf_leave; - copy = 0; - do { - len = (unsigned)(next[copy++]); - if (state->head != Z_NULL && state->head->name != Z_NULL && state->length < state->head->name_max) - state->head->name[state->length++] = len; - } while (len && copy < have); - if (state->flags & 0x0200) - state->check = crc32(state->check, next, copy); - have -= copy; - next += copy; - if (len) - goto inf_leave; - } else if (state->head != Z_NULL) { - state->head->name = Z_NULL; - } - state->length = 0; - state->mode = COMMENT; - case COMMENT: - if (state->flags & 0x1000) { - if (have == 0) goto inf_leave; - copy = 0; - do { - len = (unsigned)(next[copy++]); - if (state->head != Z_NULL && state->head->comment != Z_NULL - && state->length < state->head->comm_max) - state->head->comment[state->length++] = len; - } while (len && copy < have); - if (state->flags & 0x0200) - state->check = crc32(state->check, next, copy); - have -= copy; - next += copy; - if (len) - goto inf_leave; - } else if (state->head != Z_NULL) { - state->head->comment = Z_NULL; - } - state->mode = HCRC; - case HCRC: - if (state->flags & 0x0200) { - NEEDBITS(16); - if (hold != (state->check & 0xffff)) { - strm->msg = (char *)"header crc mismatch"; - state->mode = BAD; - break; - } - INITBITS(); - } - if (state->head != Z_NULL) { - state->head->hcrc = (int)((state->flags >> 9) & 1); - state->head->done = 1; - } - strm->adler = state->check = crc32(0L, Z_NULL, 0); - state->mode = TYPE; - break; -#endif - case DICTID: - NEEDBITS(32); - strm->adler = state->check = ZSWAP32(hold); - INITBITS(); - state->mode = DICT; - case DICT: - if (state->havedict == 0) { - RESTORE(); - return Z_NEED_DICT; - } - strm->adler = state->check = adler32(0L, Z_NULL, 0); - state->mode = TYPE; - case TYPE: - if (flush == Z_BLOCK || flush == Z_TREES) - goto inf_leave; - case TYPEDO: - if (state->last) { - BYTEBITS(); - state->mode = CHECK; - break; - } - NEEDBITS(3); - state->last = BITS(1); - DROPBITS(1); - switch (BITS(2)) { - case 0: /* stored block */ - Tracev((stderr, "inflate: stored block%s\n", state->last ? " (last)" : "")); - state->mode = STORED; - break; - case 1: /* fixed block */ - fixedtables(state); - Tracev((stderr, "inflate: fixed codes block%s\n", state->last ? " (last)" : "")); - state->mode = LEN_; /* decode codes */ - if (flush == Z_TREES) { - DROPBITS(2); - goto inf_leave; - } - break; - case 2: /* dynamic block */ - Tracev((stderr, "inflate: dynamic codes block%s\n", state->last ? " (last)" : "")); - state->mode = TABLE; - break; - case 3: - strm->msg = (char *)"invalid block type"; - state->mode = BAD; - } - DROPBITS(2); - break; - case STORED: - BYTEBITS(); /* go to byte boundary */ - NEEDBITS(32); - if ((hold & 0xffff) != ((hold >> 16) ^ 0xffff)) { - strm->msg = (char *)"invalid stored block lengths"; - state->mode = BAD; - break; - } - state->length = (uint16_t)hold; - Tracev((stderr, "inflate: stored length %u\n", state->length)); - INITBITS(); - state->mode = COPY_; - if (flush == Z_TREES) - goto inf_leave; - case COPY_: - state->mode = COPY; - case COPY: - copy = state->length; - if (copy) { - if (copy > have) copy = have; - if (copy > left) copy = left; - if (copy == 0) goto inf_leave; - memcpy(put, next, copy); - have -= copy; - next += copy; - left -= copy; - put += copy; - state->length -= copy; - break; - } - Tracev((stderr, "inflate: stored end\n")); - state->mode = TYPE; - break; - case TABLE: - NEEDBITS(14); - state->nlen = BITS(5) + 257; - DROPBITS(5); - state->ndist = BITS(5) + 1; - DROPBITS(5); - state->ncode = BITS(4) + 4; - DROPBITS(4); -#ifndef PKZIP_BUG_WORKAROUND - if (state->nlen > 286 || state->ndist > 30) { - strm->msg = (char *)"too many length or distance symbols"; - state->mode = BAD; - break; - } -#endif - Tracev((stderr, "inflate: table sizes ok\n")); - state->have = 0; - state->mode = LENLENS; - case LENLENS: - while (state->have < state->ncode) { - NEEDBITS(3); - state->lens[order[state->have++]] = (uint16_t)BITS(3); - DROPBITS(3); - } - while (state->have < 19) - state->lens[order[state->have++]] = 0; - state->next = state->codes; - state->lencode = (const code *)(state->next); - state->lenbits = 7; - ret = inflate_table(CODES, state->lens, 19, &(state->next), &(state->lenbits), state->work); - if (ret) { - strm->msg = (char *)"invalid code lengths set"; - state->mode = BAD; - break; - } - Tracev((stderr, "inflate: code lengths ok\n")); - state->have = 0; - state->mode = CODELENS; - case CODELENS: - while (state->have < state->nlen + state->ndist) { - for (;;) { - here = state->lencode[BITS(state->lenbits)]; - if (here.bits <= bits) break; - PULLBYTE(); - } - if (here.val < 16) { - DROPBITS(here.bits); - state->lens[state->have++] = here.val; - } else { - if (here.val == 16) { - NEEDBITS(here.bits + 2); - DROPBITS(here.bits); - if (state->have == 0) { - strm->msg = (char *)"invalid bit length repeat"; - state->mode = BAD; - break; - } - len = state->lens[state->have - 1]; - copy = 3 + BITS(2); - DROPBITS(2); - } else if (here.val == 17) { - NEEDBITS(here.bits + 3); - DROPBITS(here.bits); - len = 0; - copy = 3 + BITS(3); - DROPBITS(3); - } else { - NEEDBITS(here.bits + 7); - DROPBITS(here.bits); - len = 0; - copy = 11 + BITS(7); - DROPBITS(7); - } - if (state->have + copy > state->nlen + state->ndist) { - strm->msg = (char *)"invalid bit length repeat"; - state->mode = BAD; - break; - } - while (copy--) - state->lens[state->have++] = (uint16_t)len; - } - } - - /* handle error breaks in while */ - if (state->mode == BAD) - break; - - /* check for end-of-block code (better have one) */ - if (state->lens[256] == 0) { - strm->msg = (char *)"invalid code -- missing end-of-block"; - state->mode = BAD; - break; - } - - /* build code tables -- note: do not change the lenbits or distbits - values here (9 and 6) without reading the comments in inftrees.h - concerning the ENOUGH constants, which depend on those values */ - state->next = state->codes; - state->lencode = (const code *)(state->next); - state->lenbits = 9; - ret = inflate_table(LENS, state->lens, state->nlen, &(state->next), &(state->lenbits), state->work); - if (ret) { - strm->msg = (char *)"invalid literal/lengths set"; - state->mode = BAD; - break; - } - state->distcode = (const code *)(state->next); - state->distbits = 6; - ret = inflate_table(DISTS, state->lens + state->nlen, state->ndist, - &(state->next), &(state->distbits), state->work); - if (ret) { - strm->msg = (char *)"invalid distances set"; - state->mode = BAD; - break; - } - Tracev((stderr, "inflate: codes ok\n")); - state->mode = LEN_; - if (flush == Z_TREES) - goto inf_leave; - case LEN_: - state->mode = LEN; - case LEN: - if (have >= 6 && left >= 258) { - RESTORE(); - inflate_fast(strm, out); - LOAD(); - if (state->mode == TYPE) - state->back = -1; - break; - } - state->back = 0; - for (;;) { - here = state->lencode[BITS(state->lenbits)]; - if (here.bits <= bits) - break; - PULLBYTE(); - } - if (here.op && (here.op & 0xf0) == 0) { - last = here; - for (;;) { - here = state->lencode[last.val + (BITS(last.bits + last.op) >> last.bits)]; - if ((unsigned)last.bits + (unsigned)here.bits <= bits) - break; - PULLBYTE(); - } - DROPBITS(last.bits); - state->back += last.bits; - } - DROPBITS(here.bits); - state->back += here.bits; - state->length = here.val; - if ((int)(here.op) == 0) { - Tracevv((stderr, here.val >= 0x20 && here.val < 0x7f ? - "inflate: literal '%c'\n" : - "inflate: literal 0x%02x\n", here.val)); - state->mode = LIT; - break; - } - if (here.op & 32) { - Tracevv((stderr, "inflate: end of block\n")); - state->back = -1; - state->mode = TYPE; - break; - } - if (here.op & 64) { - strm->msg = (char *)"invalid literal/length code"; - state->mode = BAD; - break; - } - state->extra = (here.op & 15); - state->mode = LENEXT; - case LENEXT: - if (state->extra) { - NEEDBITS(state->extra); - state->length += BITS(state->extra); - DROPBITS(state->extra); - state->back += state->extra; - } - Tracevv((stderr, "inflate: length %u\n", state->length)); - state->was = state->length; - state->mode = DIST; - case DIST: - for (;;) { - here = state->distcode[BITS(state->distbits)]; - if (here.bits <= bits) - break; - PULLBYTE(); - } - if ((here.op & 0xf0) == 0) { - last = here; - for (;;) { - here = state->distcode[last.val + (BITS(last.bits + last.op) >> last.bits)]; - if ((unsigned)last.bits + (unsigned)here.bits <= bits) - break; - PULLBYTE(); - } - DROPBITS(last.bits); - state->back += last.bits; - } - DROPBITS(here.bits); - state->back += here.bits; - if (here.op & 64) { - strm->msg = (char *)"invalid distance code"; - state->mode = BAD; - break; - } - state->offset = here.val; - state->extra = (here.op & 15); - state->mode = DISTEXT; - case DISTEXT: - if (state->extra) { - NEEDBITS(state->extra); - state->offset += BITS(state->extra); - DROPBITS(state->extra); - state->back += state->extra; - } -#ifdef INFLATE_STRICT - if (state->offset > state->dmax) { - strm->msg = (char *)"invalid distance too far back"; - state->mode = BAD; - break; - } -#endif - Tracevv((stderr, "inflate: distance %u\n", state->offset)); - state->mode = MATCH; - case MATCH: - if (left == 0) goto inf_leave; - copy = out - left; - if (state->offset > copy) { /* copy from window */ - copy = state->offset - copy; - if (copy > state->whave) { - if (state->sane) { - strm->msg = (char *)"invalid distance too far back"; - state->mode = BAD; - break; - } -#ifdef INFLATE_ALLOW_INVALID_DISTANCE_TOOFAR_ARRR - Trace((stderr, "inflate.c too far\n")); - copy -= state->whave; - if (copy > state->length) - copy = state->length; - if (copy > left) - copy = left; - left -= copy; - state->length -= copy; - do { - *put++ = 0; - } while (--copy); - if (state->length == 0) - state->mode = LEN; - break; -#endif - } - if (copy > state->wnext) { - copy -= state->wnext; - from = state->window + (state->wsize - copy); - } else { - from = state->window + (state->wnext - copy); - } - if (copy > state->length) - copy = state->length; - } else { /* copy from output */ - from = put - state->offset; - copy = state->length; - } - if (copy > left) - copy = left; - left -= copy; - state->length -= copy; - do { - *put++ = *from++; - } while (--copy); - if (state->length == 0) - state->mode = LEN; - break; - case LIT: - if (left == 0) - goto inf_leave; - *put++ = (unsigned char)(state->length); - left--; - state->mode = LEN; - break; - case CHECK: - if (state->wrap) { - NEEDBITS(32); - out -= left; - strm->total_out += out; - state->total += out; - if (out) - strm->adler = state->check = UPDATE(state->check, put - out, out); - out = left; - if (( -#ifdef GUNZIP - state->flags ? hold : -#endif - ZSWAP32(hold)) != state->check) { - strm->msg = (char *)"incorrect data check"; - state->mode = BAD; - break; - } - INITBITS(); - Tracev((stderr, "inflate: check matches trailer\n")); - } -#ifdef GUNZIP - state->mode = LENGTH; - case LENGTH: - if (state->wrap && state->flags) { - NEEDBITS(32); - if (hold != (state->total & 0xffffffffUL)) { - strm->msg = (char *)"incorrect length check"; - state->mode = BAD; - break; - } - INITBITS(); - Tracev((stderr, "inflate: length matches trailer\n")); - } -#endif - state->mode = DONE; - case DONE: - ret = Z_STREAM_END; - goto inf_leave; - case BAD: - ret = Z_DATA_ERROR; - goto inf_leave; - case MEM: - return Z_MEM_ERROR; - case SYNC: - default: - return Z_STREAM_ERROR; - } - - /* - Return from inflate(), updating the total counts and the check value. - If there was no progress during the inflate() call, return a buffer - error. Call updatewindow() to create and/or update the window state. - Note: a memory error from inflate() is non-recoverable. - */ - inf_leave: - RESTORE(); - if (state->wsize || (out != strm->avail_out && state->mode < BAD && (state->mode < CHECK || flush != Z_FINISH))) { - if (updatewindow(strm, strm->next_out, out - strm->avail_out)) { - state->mode = MEM; - return Z_MEM_ERROR; - } - } - in -= strm->avail_in; - out -= strm->avail_out; - strm->total_in += in; - strm->total_out += out; - state->total += out; - if (state->wrap && out) - strm->adler = state->check = UPDATE(state->check, strm->next_out - out, out); - strm->data_type = state->bits + (state->last ? 64 : 0) + - (state->mode == TYPE ? 128 : 0) + (state->mode == LEN_ || state->mode == COPY_ ? 256 : 0); - if (((in == 0 && out == 0) || flush == Z_FINISH) && ret == Z_OK) - ret = Z_BUF_ERROR; - return ret; -} - -int ZEXPORT inflateEnd(z_stream *strm) { - struct inflate_state *state; - if (strm == Z_NULL || strm->state == Z_NULL || strm->zfree == (free_func)0) - return Z_STREAM_ERROR; - state = (struct inflate_state *)strm->state; - if (state->window != Z_NULL) - ZFREE(strm, state->window); - ZFREE(strm, strm->state); - strm->state = Z_NULL; - Tracev((stderr, "inflate: end\n")); - return Z_OK; -} - -int ZEXPORT inflateGetDictionary(z_stream *strm, unsigned char *dictionary, unsigned int *dictLength) { - struct inflate_state *state; - - /* check state */ - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - state = (struct inflate_state *)strm->state; - - /* copy dictionary */ - if (state->whave && dictionary != Z_NULL) { - memcpy(dictionary, state->window + state->wnext, state->whave - state->wnext); - memcpy(dictionary + state->whave - state->wnext, state->window, state->wnext); - } - if (dictLength != Z_NULL) - *dictLength = state->whave; - return Z_OK; -} - -int ZEXPORT inflateSetDictionary(z_stream *strm, const unsigned char *dictionary, unsigned int dictLength) { - struct inflate_state *state; - unsigned long dictid; - int ret; - - /* check state */ - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - state = (struct inflate_state *)strm->state; - if (state->wrap != 0 && state->mode != DICT) - return Z_STREAM_ERROR; - - /* check for correct dictionary identifier */ - if (state->mode == DICT) { - dictid = adler32(0L, Z_NULL, 0); - dictid = adler32(dictid, dictionary, dictLength); - if (dictid != state->check) - return Z_DATA_ERROR; - } - - /* copy dictionary to window using updatewindow(), which will amend the - existing dictionary if appropriate */ - ret = updatewindow(strm, dictionary + dictLength, dictLength); - if (ret) { - state->mode = MEM; - return Z_MEM_ERROR; - } - state->havedict = 1; - Tracev((stderr, "inflate: dictionary set\n")); - return Z_OK; -} - -int ZEXPORT inflateGetHeader(z_stream *strm, gz_headerp head) { - struct inflate_state *state; - - /* check state */ - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - state = (struct inflate_state *)strm->state; - if ((state->wrap & 2) == 0) - return Z_STREAM_ERROR; - - /* save header structure */ - state->head = head; - head->done = 0; - return Z_OK; -} - -/* - Search buf[0..len-1] for the pattern: 0, 0, 0xff, 0xff. Return when found - or when out of input. When called, *have is the number of pattern bytes - found in order so far, in 0..3. On return *have is updated to the new - state. If on return *have equals four, then the pattern was found and the - return value is how many bytes were read including the last byte of the - pattern. If *have is less than four, then the pattern has not been found - yet and the return value is len. In the latter case, syncsearch() can be - called again with more data and the *have state. *have is initialized to - zero for the first call. - */ -local unsigned syncsearch(uint32_t *have, const unsigned char *buf, uint32_t len) { - uint32_t got; - uint32_t next; - - got = *have; - next = 0; - while (next < len && got < 4) { - if ((int)(buf[next]) == (got < 2 ? 0 : 0xff)) - got++; - else if (buf[next]) - got = 0; - else - got = 4 - got; - next++; - } - *have = got; - return next; -} - -int ZEXPORT inflateSync(z_stream *strm) { - unsigned len; /* number of bytes to look at or looked at */ - size_t in, out; /* temporary to save total_in and total_out */ - unsigned char buf[4]; /* to restore bit buffer to byte string */ - struct inflate_state *state; - - /* check parameters */ - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - state = (struct inflate_state *)strm->state; - if (strm->avail_in == 0 && state->bits < 8) - return Z_BUF_ERROR; - - /* if first time, start search in bit buffer */ - if (state->mode != SYNC) { - state->mode = SYNC; - state->hold <<= state->bits & 7; - state->bits -= state->bits & 7; - len = 0; - while (state->bits >= 8) { - buf[len++] = (unsigned char)(state->hold); - state->hold >>= 8; - state->bits -= 8; - } - state->have = 0; - syncsearch(&(state->have), buf, len); - } - - /* search available input */ - len = syncsearch(&(state->have), strm->next_in, strm->avail_in); - strm->avail_in -= len; - strm->next_in += len; - strm->total_in += len; - - /* return no joy or set up to restart inflate() on a new block */ - if (state->have != 4) - return Z_DATA_ERROR; - in = strm->total_in; - out = strm->total_out; - inflateReset(strm); - strm->total_in = in; - strm->total_out = out; - state->mode = TYPE; - return Z_OK; -} - -/* - Returns true if inflate is currently at the end of a block generated by - Z_SYNC_FLUSH or Z_FULL_FLUSH. This function is used by one PPP - implementation to provide an additional safety check. PPP uses - Z_SYNC_FLUSH but removes the length bytes of the resulting empty stored - block. When decompressing, PPP checks that at the end of input packet, - inflate is waiting for these length bytes. - */ -int ZEXPORT inflateSyncPoint(z_stream *strm) { - struct inflate_state *state; - - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - state = (struct inflate_state *)strm->state; - return state->mode == STORED && state->bits == 0; -} - -int ZEXPORT inflateCopy(z_stream *dest, z_stream *source) { - struct inflate_state *state; - struct inflate_state *copy; - unsigned char *window; - unsigned wsize; - - /* check input */ - if (dest == Z_NULL || source == Z_NULL || source->state == Z_NULL || - source->zalloc == (alloc_func)0 || source->zfree == (free_func)0) - return Z_STREAM_ERROR; - state = (struct inflate_state *)source->state; - - /* allocate space */ - copy = (struct inflate_state *) - ZALLOC(source, 1, sizeof(struct inflate_state)); - if (copy == Z_NULL) - return Z_MEM_ERROR; - window = Z_NULL; - if (state->window != Z_NULL) { - window = (unsigned char *) ZALLOC(source, 1U << state->wbits, sizeof(unsigned char)); - if (window == Z_NULL) { - ZFREE(source, copy); - return Z_MEM_ERROR; - } - } - - /* copy state */ - memcpy((void *)dest, (void *)source, sizeof(z_stream)); - memcpy((void *)copy, (void *)state, sizeof(struct inflate_state)); - if (state->lencode >= state->codes && state->lencode <= state->codes + ENOUGH - 1) { - copy->lencode = copy->codes + (state->lencode - state->codes); - copy->distcode = copy->codes + (state->distcode - state->codes); - } - copy->next = copy->codes + (state->next - state->codes); - if (window != Z_NULL) { - wsize = 1U << state->wbits; - memcpy(window, state->window, wsize); - } - copy->window = window; - dest->state = (struct internal_state *)copy; - return Z_OK; -} - -int ZEXPORT inflateUndermine(z_stream *strm, int subvert) { - struct inflate_state *state; - - if (strm == Z_NULL || strm->state == Z_NULL) - return Z_STREAM_ERROR; - state = (struct inflate_state *)strm->state; -#ifdef INFLATE_ALLOW_INVALID_DISTANCE_TOOFAR_ARRR - state->sane = !subvert; - return Z_OK; -#else - state->sane = 1; - return Z_DATA_ERROR; -#endif -} - -long ZEXPORT inflateMark(z_stream *strm) { - struct inflate_state *state; - - if (strm == Z_NULL || strm->state == Z_NULL) - return 0xFFFFFFFFFFFF0000L; - state = (struct inflate_state *)strm->state; - return ((long)(state->back) << 16) + (state->mode == COPY ? state->length : - (state->mode == MATCH ? state->was - state->length : 0)); -} diff --git a/contrib/libzlib-ng/inflate.h b/contrib/libzlib-ng/inflate.h deleted file mode 100644 index 2bf129db153..00000000000 --- a/contrib/libzlib-ng/inflate.h +++ /dev/null @@ -1,127 +0,0 @@ -/* inflate.h -- internal inflate state definition - * Copyright (C) 1995-2009 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* WARNING: this file should *not* be used by applications. It is - part of the implementation of the compression library and is - subject to change. Applications should only use zlib.h. - */ - -#ifndef INFLATE_H_ -#define INFLATE_H_ - -/* define NO_GZIP when compiling if you want to disable gzip header and - trailer decoding by inflate(). NO_GZIP would be used to avoid linking in - the crc code when it is not needed. For shared libraries, gzip decoding - should be left enabled. */ -#ifndef NO_GZIP -# define GUNZIP -#endif - -/* Possible inflate modes between inflate() calls */ -typedef enum { - HEAD, /* i: waiting for magic header */ - FLAGS, /* i: waiting for method and flags (gzip) */ - TIME, /* i: waiting for modification time (gzip) */ - OS, /* i: waiting for extra flags and operating system (gzip) */ - EXLEN, /* i: waiting for extra length (gzip) */ - EXTRA, /* i: waiting for extra bytes (gzip) */ - NAME, /* i: waiting for end of file name (gzip) */ - COMMENT, /* i: waiting for end of comment (gzip) */ - HCRC, /* i: waiting for header crc (gzip) */ - DICTID, /* i: waiting for dictionary check value */ - DICT, /* waiting for inflateSetDictionary() call */ - TYPE, /* i: waiting for type bits, including last-flag bit */ - TYPEDO, /* i: same, but skip check to exit inflate on new block */ - STORED, /* i: waiting for stored size (length and complement) */ - COPY_, /* i/o: same as COPY below, but only first time in */ - COPY, /* i/o: waiting for input or output to copy stored block */ - TABLE, /* i: waiting for dynamic block table lengths */ - LENLENS, /* i: waiting for code length code lengths */ - CODELENS, /* i: waiting for length/lit and distance code lengths */ - LEN_, /* i: same as LEN below, but only first time in */ - LEN, /* i: waiting for length/lit/eob code */ - LENEXT, /* i: waiting for length extra bits */ - DIST, /* i: waiting for distance code */ - DISTEXT, /* i: waiting for distance extra bits */ - MATCH, /* o: waiting for output space to copy string */ - LIT, /* o: waiting for output space to write literal */ - CHECK, /* i: waiting for 32-bit check value */ - LENGTH, /* i: waiting for 32-bit length (gzip) */ - DONE, /* finished check, done -- remain here until reset */ - BAD, /* got a data error -- remain here until reset */ - MEM, /* got an inflate() memory error -- remain here until reset */ - SYNC /* looking for synchronization bytes to restart inflate() */ -} inflate_mode; - -/* - State transitions between above modes - - - (most modes can go to BAD or MEM on error -- not shown for clarity) - - Process header: - HEAD -> (gzip) or (zlib) or (raw) - (gzip) -> FLAGS -> TIME -> OS -> EXLEN -> EXTRA -> NAME -> COMMENT -> - HCRC -> TYPE - (zlib) -> DICTID or TYPE - DICTID -> DICT -> TYPE - (raw) -> TYPEDO - Read deflate blocks: - TYPE -> TYPEDO -> STORED or TABLE or LEN_ or CHECK - STORED -> COPY_ -> COPY -> TYPE - TABLE -> LENLENS -> CODELENS -> LEN_ - LEN_ -> LEN - Read deflate codes in fixed or dynamic block: - LEN -> LENEXT or LIT or TYPE - LENEXT -> DIST -> DISTEXT -> MATCH -> LEN - LIT -> LEN - Process trailer: - CHECK -> LENGTH -> DONE - */ - -/* state maintained between inflate() calls. Approximately 10K bytes. */ -struct inflate_state { - inflate_mode mode; /* current inflate mode */ - int last; /* true if processing last block */ - int wrap; /* bit 0 true for zlib, bit 1 true for gzip */ - int havedict; /* true if dictionary provided */ - int flags; /* gzip header method and flags (0 if zlib) */ - unsigned dmax; /* zlib header max distance (INFLATE_STRICT) */ - unsigned long check; /* protected copy of check value */ - unsigned long total; /* protected copy of output count */ - gz_headerp head; /* where to save gzip header information */ - /* sliding window */ - unsigned wbits; /* log base 2 of requested window size */ - uint32_t wsize; /* window size or zero if not using window */ - uint32_t whave; /* valid bytes in the window */ - uint32_t wnext; /* window write index */ - unsigned char *window; /* allocated sliding window, if needed */ - /* bit accumulator */ - uint32_t hold; /* input bit accumulator */ - unsigned bits; /* number of bits in "in" */ - /* for string and stored block copying */ - uint32_t length; /* literal or length of data to copy */ - unsigned offset; /* distance back to copy string from */ - /* for table and code decoding */ - unsigned extra; /* extra bits needed */ - /* fixed and dynamic code tables */ - code const *lencode; /* starting table for length/literal codes */ - code const *distcode; /* starting table for distance codes */ - unsigned lenbits; /* index bits for lencode */ - unsigned distbits; /* index bits for distcode */ - /* dynamic table building */ - unsigned ncode; /* number of code length code lengths */ - unsigned nlen; /* number of length code lengths */ - unsigned ndist; /* number of distance code lengths */ - uint32_t have; /* number of code lengths in lens[] */ - code *next; /* next available space in codes[] */ - uint16_t lens[320]; /* temporary storage for code lengths */ - uint16_t work[288]; /* work area for code table building */ - code codes[ENOUGH]; /* space for code tables */ - int sane; /* if false, allow invalid distance too far */ - int back; /* bits back of last unprocessed length/lit */ - unsigned was; /* initial length of match */ -}; - -#endif /* INFLATE_H_ */ diff --git a/contrib/libzlib-ng/inftrees.c b/contrib/libzlib-ng/inftrees.c deleted file mode 100644 index e02272cf864..00000000000 --- a/contrib/libzlib-ng/inftrees.c +++ /dev/null @@ -1,298 +0,0 @@ -/* inftrees.c -- generate Huffman trees for efficient decoding - * Copyright (C) 1995-2013 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -#include "zutil.h" -#include "inftrees.h" - -#define MAXBITS 15 - -const char inflate_copyright[] = " inflate 1.2.8.f Copyright 1995-2013 Mark Adler "; -/* - If you use the zlib library in a product, an acknowledgment is welcome - in the documentation of your product. If for some reason you cannot - include such an acknowledgment, I would appreciate that you keep this - copyright string in the executable of your product. - */ - -/* - Build a set of tables to decode the provided canonical Huffman code. - The code lengths are lens[0..codes-1]. The result starts at *table, - whose indices are 0..2^bits-1. work is a writable array of at least - lens shorts, which is used as a work area. type is the type of code - to be generated, CODES, LENS, or DISTS. On return, zero is success, - -1 is an invalid code, and +1 means that ENOUGH isn't enough. table - on return points to the next available entry's address. bits is the - requested root table index bits, and on return it is the actual root - table index bits. It will differ if the request is greater than the - longest code or if it is less than the shortest code. - */ -int ZLIB_INTERNAL inflate_table(codetype type, uint16_t *lens, unsigned codes, - code * *table, unsigned *bits, uint16_t *work) { - unsigned len; /* a code's length in bits */ - unsigned sym; /* index of code symbols */ - unsigned min, max; /* minimum and maximum code lengths */ - unsigned root; /* number of index bits for root table */ - unsigned curr; /* number of index bits for current table */ - unsigned drop; /* code bits to drop for sub-table */ - int left; /* number of prefix codes available */ - unsigned used; /* code entries in table used */ - unsigned huff; /* Huffman code */ - unsigned incr; /* for incrementing code, index */ - unsigned fill; /* index for replicating entries */ - unsigned low; /* low bits for current root entry */ - unsigned mask; /* mask for low root bits */ - code here; /* table entry for duplication */ - code *next; /* next available space in table */ - const uint16_t *base; /* base value table to use */ - const uint16_t *extra; /* extra bits table to use */ - int end; /* use base and extra for symbol > end */ - uint16_t count[MAXBITS+1]; /* number of codes of each length */ - uint16_t offs[MAXBITS+1]; /* offsets in table for each length */ - static const uint16_t lbase[31] = { /* Length codes 257..285 base */ - 3, 4, 5, 6, 7, 8, 9, 10, 11, 13, 15, 17, 19, 23, 27, 31, - 35, 43, 51, 59, 67, 83, 99, 115, 131, 163, 195, 227, 258, 0, 0}; - static const uint16_t lext[31] = { /* Length codes 257..285 extra */ - 16, 16, 16, 16, 16, 16, 16, 16, 17, 17, 17, 17, 18, 18, 18, 18, - 19, 19, 19, 19, 20, 20, 20, 20, 21, 21, 21, 21, 16, 72, 78}; - static const uint16_t dbase[32] = { /* Distance codes 0..29 base */ - 1, 2, 3, 4, 5, 7, 9, 13, 17, 25, 33, 49, 65, 97, 129, 193, - 257, 385, 513, 769, 1025, 1537, 2049, 3073, 4097, 6145, - 8193, 12289, 16385, 24577, 0, 0}; - static const uint16_t dext[32] = { /* Distance codes 0..29 extra */ - 16, 16, 16, 16, 17, 17, 18, 18, 19, 19, 20, 20, 21, 21, 22, 22, - 23, 23, 24, 24, 25, 25, 26, 26, 27, 27, - 28, 28, 29, 29, 64, 64}; - - /* - Process a set of code lengths to create a canonical Huffman code. The - code lengths are lens[0..codes-1]. Each length corresponds to the - symbols 0..codes-1. The Huffman code is generated by first sorting the - symbols by length from short to long, and retaining the symbol order - for codes with equal lengths. Then the code starts with all zero bits - for the first code of the shortest length, and the codes are integer - increments for the same length, and zeros are appended as the length - increases. For the deflate format, these bits are stored backwards - from their more natural integer increment ordering, and so when the - decoding tables are built in the large loop below, the integer codes - are incremented backwards. - - This routine assumes, but does not check, that all of the entries in - lens[] are in the range 0..MAXBITS. The caller must assure this. - 1..MAXBITS is interpreted as that code length. zero means that that - symbol does not occur in this code. - - The codes are sorted by computing a count of codes for each length, - creating from that a table of starting indices for each length in the - sorted table, and then entering the symbols in order in the sorted - table. The sorted table is work[], with that space being provided by - the caller. - - The length counts are used for other purposes as well, i.e. finding - the minimum and maximum length codes, determining if there are any - codes at all, checking for a valid set of lengths, and looking ahead - at length counts to determine sub-table sizes when building the - decoding tables. - */ - - /* accumulate lengths for codes (assumes lens[] all in 0..MAXBITS) */ - for (len = 0; len <= MAXBITS; len++) - count[len] = 0; - for (sym = 0; sym < codes; sym++) - count[lens[sym]]++; - - /* bound code lengths, force root to be within code lengths */ - root = *bits; - for (max = MAXBITS; max >= 1; max--) - if (count[max] != 0) break; - if (root > max) root = max; - if (max == 0) { /* no symbols to code at all */ - here.op = (unsigned char)64; /* invalid code marker */ - here.bits = (unsigned char)1; - here.val = (uint16_t)0; - *(*table)++ = here; /* make a table to force an error */ - *(*table)++ = here; - *bits = 1; - return 0; /* no symbols, but wait for decoding to report error */ - } - for (min = 1; min < max; min++) - if (count[min] != 0) break; - if (root < min) root = min; - - /* check for an over-subscribed or incomplete set of lengths */ - left = 1; - for (len = 1; len <= MAXBITS; len++) { - left <<= 1; - left -= count[len]; - if (left < 0) return -1; /* over-subscribed */ - } - if (left > 0 && (type == CODES || max != 1)) - return -1; /* incomplete set */ - - /* generate offsets into symbol table for each length for sorting */ - offs[1] = 0; - for (len = 1; len < MAXBITS; len++) - offs[len + 1] = offs[len] + count[len]; - - /* sort symbols by length, by symbol order within each length */ - for (sym = 0; sym < codes; sym++) - if (lens[sym] != 0) work[offs[lens[sym]]++] = (uint16_t)sym; - - /* - Create and fill in decoding tables. In this loop, the table being - filled is at next and has curr index bits. The code being used is huff - with length len. That code is converted to an index by dropping drop - bits off of the bottom. For codes where len is less than drop + curr, - those top drop + curr - len bits are incremented through all values to - fill the table with replicated entries. - - root is the number of index bits for the root table. When len exceeds - root, sub-tables are created pointed to by the root entry with an index - of the low root bits of huff. This is saved in low to check for when a - new sub-table should be started. drop is zero when the root table is - being filled, and drop is root when sub-tables are being filled. - - When a new sub-table is needed, it is necessary to look ahead in the - code lengths to determine what size sub-table is needed. The length - counts are used for this, and so count[] is decremented as codes are - entered in the tables. - - used keeps track of how many table entries have been allocated from the - provided *table space. It is checked for LENS and DIST tables against - the constants ENOUGH_LENS and ENOUGH_DISTS to guard against changes in - the initial root table size constants. See the comments in inftrees.h - for more information. - - sym increments through all symbols, and the loop terminates when - all codes of length max, i.e. all codes, have been processed. This - routine permits incomplete codes, so another loop after this one fills - in the rest of the decoding tables with invalid code markers. - */ - - /* set up for code type */ - switch (type) { - case CODES: - base = extra = work; /* dummy value--not used */ - end = 19; - break; - case LENS: - base = lbase; - base -= 257; - extra = lext; - extra -= 257; - end = 256; - break; - default: /* DISTS */ - base = dbase; - extra = dext; - end = -1; - } - - /* initialize state for loop */ - huff = 0; /* starting code */ - sym = 0; /* starting code symbol */ - len = min; /* starting code length */ - next = *table; /* current table to fill in */ - curr = root; /* current table index bits */ - drop = 0; /* current bits to drop from code for index */ - low = (unsigned)(-1); /* trigger new sub-table when len > root */ - used = 1U << root; /* use root table entries */ - mask = used - 1; /* mask for comparing low */ - - /* check available table space */ - if ((type == LENS && used > ENOUGH_LENS) || - (type == DISTS && used > ENOUGH_DISTS)) - return 1; - - /* process all codes and make table entries */ - for (;;) { - /* create table entry */ - here.bits = (unsigned char)(len - drop); - if ((int)(work[sym]) < end) { - here.op = (unsigned char)0; - here.val = work[sym]; - } else if ((int)(work[sym]) > end) { - here.op = (unsigned char)(extra[work[sym]]); - here.val = base[work[sym]]; - } else { - here.op = (unsigned char)(32 + 64); /* end of block */ - here.val = 0; - } - - /* replicate for those indices with low len bits equal to huff */ - incr = 1U << (len - drop); - fill = 1U << curr; - min = fill; /* save offset to next table */ - do { - fill -= incr; - next[(huff >> drop) + fill] = here; - } while (fill != 0); - - /* backwards increment the len-bit code huff */ - incr = 1U << (len - 1); - while (huff & incr) - incr >>= 1; - if (incr != 0) { - huff &= incr - 1; - huff += incr; - } else { - huff = 0; - } - - /* go to next symbol, update count, len */ - sym++; - if (--(count[len]) == 0) { - if (len == max) - break; - len = lens[work[sym]]; - } - - /* create new sub-table if needed */ - if (len > root && (huff & mask) != low) { - /* if first time, transition to sub-tables */ - if (drop == 0) - drop = root; - - /* increment past last table */ - next += min; /* here min is 1 << curr */ - - /* determine length of next table */ - curr = len - drop; - left = (int)(1 << curr); - while (curr + drop < max) { - left -= count[curr + drop]; - if (left <= 0) - break; - curr++; - left <<= 1; - } - - /* check for enough space */ - used += 1U << curr; - if ((type == LENS && used > ENOUGH_LENS) || (type == DISTS && used > ENOUGH_DISTS)) - return 1; - - /* point entry in root table to sub-table */ - low = huff & mask; - (*table)[low].op = (unsigned char)curr; - (*table)[low].bits = (unsigned char)root; - (*table)[low].val = (uint16_t)(next - *table); - } - } - - /* fill in remaining table entry if code is incomplete (guaranteed to have - at most one remaining entry, since if the code is incomplete, the - maximum code length that was allowed to get this far is one bit) */ - if (huff != 0) { - here.op = (unsigned char)64; /* invalid code marker */ - here.bits = (unsigned char)(len - drop); - here.val = (uint16_t)0; - next[huff] = here; - } - - /* set return parameters */ - *table += used; - *bits = root; - return 0; -} diff --git a/contrib/libzlib-ng/inftrees.h b/contrib/libzlib-ng/inftrees.h deleted file mode 100644 index eaf3df1ca66..00000000000 --- a/contrib/libzlib-ng/inftrees.h +++ /dev/null @@ -1,66 +0,0 @@ -#ifndef INFTREES_H_ -#define INFTREES_H_ - -/* inftrees.h -- header to use inftrees.c - * Copyright (C) 1995-2005, 2010 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* WARNING: this file should *not* be used by applications. It is - part of the implementation of the compression library and is - subject to change. Applications should only use zlib.h. - */ - -/* Structure for decoding tables. Each entry provides either the - information needed to do the operation requested by the code that - indexed that table entry, or it provides a pointer to another - table that indexes more bits of the code. op indicates whether - the entry is a pointer to another table, a literal, a length or - distance, an end-of-block, or an invalid code. For a table - pointer, the low four bits of op is the number of index bits of - that table. For a length or distance, the low four bits of op - is the number of extra bits to get after the code. bits is - the number of bits in this code or part of the code to drop off - of the bit buffer. val is the actual byte to output in the case - of a literal, the base length or distance, or the offset from - the current table to the next table. Each entry is four bytes. */ -typedef struct { - unsigned char op; /* operation, extra bits, table bits */ - unsigned char bits; /* bits in this part of the code */ - uint16_t val; /* offset in table or code value */ -} code; - -/* op values as set by inflate_table(): - 00000000 - literal - 0000tttt - table link, tttt != 0 is the number of table index bits - 0001eeee - length or distance, eeee is the number of extra bits - 01100000 - end of block - 01000000 - invalid code - */ - -/* Maximum size of the dynamic table. The maximum number of code structures is - 1444, which is the sum of 852 for literal/length codes and 592 for distance - codes. These values were found by exhaustive searches using the program - examples/enough.c found in the zlib distribtution. The arguments to that - program are the number of symbols, the initial root table size, and the - maximum bit length of a code. "enough 286 9 15" for literal/length codes - returns returns 852, and "enough 30 6 15" for distance codes returns 592. - The initial root table size (9 or 6) is found in the fifth argument of the - inflate_table() calls in inflate.c and infback.c. If the root table size is - changed, then these maximum sizes would be need to be recalculated and - updated. */ -#define ENOUGH_LENS 852 -#define ENOUGH_DISTS 592 -#define ENOUGH (ENOUGH_LENS+ENOUGH_DISTS) - -/* Type of code to build for inflate_table() */ -typedef enum { - CODES, - LENS, - DISTS -} codetype; - -int ZLIB_INTERNAL inflate_table (codetype type, uint16_t *lens, unsigned codes, - code * *table, unsigned *bits, uint16_t *work); - -#endif /* INFTREES_H_ */ diff --git a/contrib/libzlib-ng/match.c b/contrib/libzlib-ng/match.c deleted file mode 100644 index 25b3b14ce07..00000000000 --- a/contrib/libzlib-ng/match.c +++ /dev/null @@ -1,471 +0,0 @@ -/* - * Set match_start to the longest match starting at the given string and - * return its length. Matches shorter or equal to prev_length are discarded, - * in which case the result is equal to prev_length and match_start is garbage. - * - * IN assertions: cur_match is the head of the hash chain for the current - * string (strstart) and its distance is <= MAX_DIST, and prev_length >=1 - * OUT assertion: the match length is not greater than s->lookahead - */ - -#include "deflate.h" - -#if (defined(UNALIGNED_OK) && MAX_MATCH == 258) - - /* Only use std3_longest_match for little_endian systems, also avoid using it with - non-gcc compilers since the __builtin_ctzl() function might not be optimized. */ -# if defined(__GNUC__) && defined(HAVE_BUILTIN_CTZL) && ((__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__) \ - || defined(__LITTLE_ENDIAN__)) -# define std3_longest_match -# elif(defined(_MSC_VER) && defined(_WIN32)) -# define std3_longest_match -# else -# define std2_longest_match -# endif - -#else -# define std1_longest_match -#endif - - -#if defined(_MSC_VER) && !defined(__clang__) -#include -/* This is not a general purpose replacement for __builtin_ctzl. The function expects that value is != 0 - * Because of that assumption trailing_zero is not initialized and the return value of _BitScanForward is not checked - */ -static __forceinline unsigned long __builtin_ctzl(unsigned long value) -{ - unsigned long trailing_zero; - _BitScanForward(&trailing_zero, value); - return trailing_zero; -} -#endif - - - -#ifdef std1_longest_match - -/* - * Standard longest_match - * - */ -ZLIB_INTERNAL unsigned longest_match(deflate_state *const s, IPos cur_match) { - const unsigned wmask = s->w_mask; - const Pos *prev = s->prev; - - unsigned chain_length; - IPos limit; - unsigned int len, best_len, nice_match; - unsigned char *scan, *match, *strend, scan_end, scan_end1; - - /* - * The code is optimized for HASH_BITS >= 8 and MAX_MATCH-2 multiple - * of 16. It is easy to get rid of this optimization if necessary. - */ - Assert(s->hash_bits >= 8 && MAX_MATCH == 258, "Code too clever"); - - /* - * Do not waste too much time if we already have a good match - */ - best_len = s->prev_length; - chain_length = s->max_chain_length; - if (best_len >= s->good_match) - chain_length >>= 2; - - /* - * Do not looks for matches beyond the end of the input. This is - * necessary to make deflate deterministic - */ - nice_match = (unsigned int)s->nice_match > s->lookahead ? s->lookahead : s->nice_match; - - /* - * Stop when cur_match becomes <= limit. To simplify the code, - * we prevent matches with the string of window index 0 - */ - limit = s->strstart > MAX_DIST(s) ? s->strstart - MAX_DIST(s) : 0; - - scan = s->window + s->strstart; - strend = s->window + s->strstart + MAX_MATCH; - scan_end1 = scan[best_len-1]; - scan_end = scan[best_len]; - - Assert((unsigned long)s->strstart <= s->window_size - MIN_LOOKAHEAD, "need lookahead"); - do { - Assert(cur_match < s->strstart, "no future"); - match = s->window + cur_match; - - /* - * Skip to next match if the match length cannot increase - * or if the match length is less than 2. Note that the checks - * below for insufficient lookahead only occur occasionally - * for performance reasons. Therefore uninitialized memory - * will be accessed and conditional jumps will be made that - * depend on those values. However the length of the match - * is limited to the lookahead, so the output of deflate is not - * affected by the uninitialized values. - */ - if (match[best_len] != scan_end || - match[best_len-1] != scan_end1 || - *match != *scan || - *++match != scan[1]) - continue; - - /* - * The check at best_len-1 can be removed because it will - * be made again later. (This heuristic is not always a win.) - * It is not necessary to compare scan[2] and match[2] since - * they are always equal when the other bytes match, given - * that the hash keys are equal and that HASH_BITS >= 8. - */ - scan += 2; - match++; - Assert(*scan == *match, "match[2]?"); - - /* - * We check for insufficient lookahead only every 8th - * comparision; the 256th check will be made at strstart + 258. - */ - do { - } while (*++scan == *++match && *++scan == *++match && - *++scan == *++match && *++scan == *++match && - *++scan == *++match && *++scan == *++match && - *++scan == *++match && *++scan == *++match && - scan < strend); - - Assert(scan <= s->window+(unsigned int)(s->window_size-1), "wild scan"); - - len = MAX_MATCH - (int)(strend - scan); - scan = strend - MAX_MATCH; - - if (len > best_len) { - s->match_start = cur_match; - best_len = len; - if (len >= nice_match) - break; - scan_end1 = scan[best_len-1]; - scan_end = scan[best_len]; - } else { - /* - * The probability of finding a match later if we here - * is pretty low, so for performance it's best to - * outright stop here for the lower compression levels - */ - if (s->level < 6) - break; - } - } while ((cur_match = prev[cur_match & wmask]) > limit && --chain_length); - - if ((unsigned int)best_len <= s->lookahead) - return best_len; - return s->lookahead; -} -#endif - -#ifdef std2_longest_match -/* - * UNALIGNED_OK longest_match - * - */ -ZLIB_INTERNAL unsigned longest_match(deflate_state *const s, IPos cur_match) { - const unsigned wmask = s->w_mask; - const Pos *prev = s->prev; - - uint16_t scan_start, scan_end; - unsigned chain_length; - IPos limit; - unsigned int len, best_len, nice_match; - unsigned char *scan, *strend; - - /* - * The code is optimized for HASH_BITS >= 8 and MAX_MATCH-2 multiple - * of 16. It is easy to get rid of this optimization if necessary. - */ - Assert(s->hash_bits >= 8 && MAX_MATCH == 258, "Code too clever"); - - /* - * Do not waste too much time if we already have a good match - */ - best_len = s->prev_length; - chain_length = s->max_chain_length; - if (best_len >= s->good_match) - chain_length >>= 2; - - /* - * Do not looks for matches beyond the end of the input. This is - * necessary to make deflate deterministic - */ - nice_match = (unsigned int)s->nice_match > s->lookahead ? s->lookahead : s->nice_match; - - /* - * Stop when cur_match becomes <= limit. To simplify the code, - * we prevent matches with the string of window index 0 - */ - limit = s->strstart > MAX_DIST(s) ? s->strstart - MAX_DIST(s) : 0; - - scan = s->window + s->strstart; - strend = s->window + s->strstart + MAX_MATCH - 1; - scan_start = *(uint16_t *)scan; - scan_end = *(uint16_t *)(scan + best_len-1); - - Assert((unsigned long)s->strstart <= s->window_size - MIN_LOOKAHEAD, "need lookahead"); - do { - unsigned char *match; - Assert(cur_match < s->strstart, "no future"); - match = s->window + cur_match; - - /* - * Skip to next match if the match length cannot increase - * or if the match length is less than 2. Note that the checks - * below for insufficient lookahead only occur occasionally - * for performance reasons. Therefore uninitialized memory - * will be accessed and conditional jumps will be made that - * depend on those values. However the length of the match - * is limited to the lookahead, so the output of deflate is not - * affected by the uninitialized values. - */ - if (likely((*(uint16_t *)(match + best_len - 1) != scan_end))) - continue; - if (*(uint16_t *)match != scan_start) - continue; - - /* It is not necessary to compare scan[2] and match[2] since - * they are always equal when the other bytes match, given that - * the hash keys are equal and that HASH_BITS >= 8. Compare 2 - * bytes at a time at strstart+3, +5, ... up to strstart+257. - * We check for insufficient lookahead only every 4th - * comparison; the 128th check will be made at strstart+257. - * If MAX_MATCH-2 is not a multiple of 8, it is necessary to - * put more guard bytes at the end of the window, or to check - * more often for insufficient lookahead. - */ - Assert(scan[2] == match[2], "scan[2]?"); - scan++; - match++; - - do { - } while (*(uint16_t *)(scan += 2)== *(uint16_t *)(match += 2) && - *(uint16_t *)(scan += 2)== *(uint16_t *)(match += 2) && - *(uint16_t *)(scan += 2)== *(uint16_t *)(match += 2) && - *(uint16_t *)(scan += 2)== *(uint16_t *)(match += 2) && - scan < strend); - - /* - * Here, scan <= window + strstart + 257 - */ - Assert(scan <= s->window+(unsigned)(s->window_size-1), "wild scan"); - if (*scan == *match) - scan++; - - len = (MAX_MATCH -1) - (int)(strend-scan); - scan = strend - (MAX_MATCH-1); - - if (len > best_len) { - s->match_start = cur_match; - best_len = len; - if (len >= nice_match) - break; - scan_end = *(uint16_t *)(scan + best_len - 1); - } else { - /* - * The probability of finding a match later if we here - * is pretty low, so for performance it's best to - * outright stop here for the lower compression levels - */ - if (s->level < 6) - break; - } - } while (--chain_length && (cur_match = prev[cur_match & wmask]) > limit); - - if ((unsigned)best_len <= s->lookahead) - return best_len; - return s->lookahead; -} -#endif - -#ifdef std3_longest_match -/* longest_match() with minor change to improve performance (in terms of - * execution time). - * - * The pristine longest_match() function is sketched bellow (strip the - * then-clause of the "#ifdef UNALIGNED_OK"-directive) - * - * ------------------------------------------------------------ - * unsigned int longest_match(...) { - * ... - * do { - * match = s->window + cur_match; //s0 - * if (*(ushf*)(match+best_len-1) != scan_end || //s1 - * *(ushf*)match != scan_start) continue; //s2 - * ... - * - * do { - * } while (*(ushf*)(scan+=2) == *(ushf*)(match+=2) && - * *(ushf*)(scan+=2) == *(ushf*)(match+=2) && - * *(ushf*)(scan+=2) == *(ushf*)(match+=2) && - * *(ushf*)(scan+=2) == *(ushf*)(match+=2) && - * scan < strend); //s3 - * - * ... - * } while(cond); //s4 - * - * ------------------------------------------------------------- - * - * The change include: - * - * 1) The hottest statements of the function is: s0, s1 and s4. Pull them - * together to form a new loop. The benefit is two-fold: - * - * o. Ease the compiler to yield good code layout: the conditional-branch - * corresponding to s1 and its biased target s4 become very close (likely, - * fit in the same cache-line), hence improving instruction-fetching - * efficiency. - * - * o. Ease the compiler to promote "s->window" into register. "s->window" - * is loop-invariant; it is supposed to be promoted into register and keep - * the value throughout the entire loop. However, there are many such - * loop-invariant, and x86-family has small register file; "s->window" is - * likely to be chosen as register-allocation victim such that its value - * is reloaded from memory in every single iteration. By forming a new loop, - * "s->window" is loop-invariant of that newly created tight loop. It is - * lot easier for compiler to promote this quantity to register and keep - * its value throughout the entire small loop. - * - * 2) Transfrom s3 such that it examines sizeof(long)-byte-match at a time. - * This is done by: - * ------------------------------------------------ - * v1 = load from "scan" by sizeof(long) bytes - * v2 = load from "match" by sizeof(lnog) bytes - * v3 = v1 xor v2 - * match-bit = little-endian-machine(yes-for-x86) ? - * count-trailing-zero(v3) : - * count-leading-zero(v3); - * - * match-byte = match-bit/8 - * - * "scan" and "match" advance if necessary - * ------------------------------------------------- - */ - -ZLIB_INTERNAL unsigned longest_match(deflate_state *const s, IPos cur_match) { - unsigned chain_length = s->max_chain_length;/* max hash chain length */ - register unsigned char *scan = s->window + s->strstart; /* current string */ - register unsigned char *match; /* matched string */ - register unsigned int len; /* length of current match */ - unsigned int best_len = s->prev_length; /* best match length so far */ - unsigned int nice_match = s->nice_match; /* stop if match long enough */ - IPos limit = s->strstart > (IPos)MAX_DIST(s) ? - s->strstart - (IPos)MAX_DIST(s) : NIL; - /* Stop when cur_match becomes <= limit. To simplify the code, - * we prevent matches with the string of window index 0. - */ - Pos *prev = s->prev; - unsigned int wmask = s->w_mask; - - register unsigned char *strend = s->window + s->strstart + MAX_MATCH; - register uint16_t scan_start = *(uint16_t*)scan; - register uint16_t scan_end = *(uint16_t*)(scan+best_len-1); - - /* The code is optimized for HASH_BITS >= 8 and MAX_MATCH-2 multiple of 16. - * It is easy to get rid of this optimization if necessary. - */ - Assert(s->hash_bits >= 8 && MAX_MATCH == 258, "Code too clever"); - - /* Do not waste too much time if we already have a good match: */ - if (s->prev_length >= s->good_match) { - chain_length >>= 2; - } - /* Do not look for matches beyond the end of the input. This is necessary - * to make deflate deterministic. - */ - if ((unsigned int)nice_match > s->lookahead) nice_match = s->lookahead; - - Assert((unsigned long)s->strstart <= s->window_size-MIN_LOOKAHEAD, "need lookahead"); - - do { - Assert(cur_match < s->strstart, "no future"); - - /* Skip to next match if the match length cannot increase - * or if the match length is less than 2. Note that the checks below - * for insufficient lookahead only occur occasionally for performance - * reasons. Therefore uninitialized memory will be accessed, and - * conditional jumps will be made that depend on those values. - * However the length of the match is limited to the lookahead, so - * the output of deflate is not affected by the uninitialized values. - */ - unsigned char *win = s->window; - int cont = 1; - do { - match = win + cur_match; - if (likely(*(uint16_t*)(match+best_len-1) != scan_end)) { - if ((cur_match = prev[cur_match & wmask]) > limit - && --chain_length != 0) { - continue; - } else { - cont = 0; - } - } - break; - } while (1); - - if (!cont) - break; - - if (*(uint16_t*)match != scan_start) - continue; - - /* It is not necessary to compare scan[2] and match[2] since they are - * always equal when the other bytes match, given that the hash keys - * are equal and that HASH_BITS >= 8. Compare 2 bytes at a time at - * strstart+3, +5, ... up to strstart+257. We check for insufficient - * lookahead only every 4th comparison; the 128th check will be made - * at strstart+257. If MAX_MATCH-2 is not a multiple of 8, it is - * necessary to put more guard bytes at the end of the window, or - * to check more often for insufficient lookahead. - */ - scan += 2, match+=2; - Assert(*scan == *match, "match[2]?"); - do { - unsigned long sv = *(unsigned long*)(void*)scan; - unsigned long mv = *(unsigned long*)(void*)match; - unsigned long xor = sv ^ mv; - if (xor) { - int match_byte = __builtin_ctzl(xor) / 8; - scan += match_byte; - match += match_byte; - break; - } else { - scan += sizeof(unsigned long); - match += sizeof(unsigned long); - } - } while (scan < strend); - - if (scan > strend) - scan = strend; - - Assert(scan <= s->window+(unsigned)(s->window_size-1), "wild scan"); - - len = MAX_MATCH - (int)(strend - scan); - scan = strend - MAX_MATCH; - - if (len > best_len) { - s->match_start = cur_match; - best_len = len; - if (len >= nice_match) - break; - scan_end = *(uint16_t*)(scan+best_len-1); - } else { - /* - * The probability of finding a match later if we here - * is pretty low, so for performance it's best to - * outright stop here for the lower compression levels - */ - if (s->level < 6) - break; - } - } while ((cur_match = prev[cur_match & wmask]) > limit && --chain_length != 0); - - if ((unsigned int)best_len <= s->lookahead) - return (unsigned int)best_len; - return s->lookahead; -} -#endif diff --git a/contrib/libzlib-ng/match.h b/contrib/libzlib-ng/match.h deleted file mode 100644 index 70842019849..00000000000 --- a/contrib/libzlib-ng/match.h +++ /dev/null @@ -1,6 +0,0 @@ -#ifndef MATCH_H_ -#define MATCH_H_ - -unsigned int longest_match (deflate_state *s, IPos cur_match); - -#endif /* MATCH_H_ */ diff --git a/contrib/libzlib-ng/test/.gitignore b/contrib/libzlib-ng/test/.gitignore deleted file mode 100644 index 2c3af0a08cb..00000000000 --- a/contrib/libzlib-ng/test/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -# ignore Makefiles; they're all automatically generated -Makefile diff --git a/contrib/libzlib-ng/test/CVE-2002-0059/test.gz b/contrib/libzlib-ng/test/CVE-2002-0059/test.gz deleted file mode 100644 index c5c3e184b1a90692f1c2dc729eb106476d231378..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4610 zcmb2|=3oE==C>CV8G$T;1@EN)&o6EfRlAUMpn;K@jYq;DVU#f%2%{-sG#8BKg3(+s vnhQpA!DucR%>|>mU^Ewu=7P~&Fq#WSbHQjX7-G5L2b -#include -#include -#include - -int main(void) { -gzFile f; -int ret; - -if(!(f = gzopen("/dev/null", "w"))) { -perror("/dev/null"); -exit(1); -} -ret = gzprintf(f, "%10240s", ""); -printf("gzprintf -> %d\n", ret); -ret = gzclose(f); -printf("gzclose -> %d [%d]\n", ret, errno); -exit(0); -} diff --git a/contrib/libzlib-ng/test/CVE-2004-0797/test.gz b/contrib/libzlib-ng/test/CVE-2004-0797/test.gz deleted file mode 100644 index 62dcf34bddbfe2c2e861c7929c21dc2f731dd000..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 52 ocmb2|=3oE==C>CV85!7kBn%P`G%zxAfEl}iz!@q6kpjs906!lM=l}o! diff --git a/contrib/libzlib-ng/test/CVE-2005-1849/test.gz b/contrib/libzlib-ng/test/CVE-2005-1849/test.gz deleted file mode 100644 index b28f278263c0a3154faeb3c3dd7b083a3a593c8d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 52 kcmb2|=3oE==C@ZA85!7kBn%P`G%zxAz!-252m>Ss01LtkQvd(} diff --git a/contrib/libzlib-ng/test/CVE-2005-2096/test.gz b/contrib/libzlib-ng/test/CVE-2005-2096/test.gz deleted file mode 100644 index 11590aeab9ac844087776ab66eb2f5d2c9f6d013..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 52 pcmb2|=3oE==C>CV84oltGP4!m+X - -#include -#include -#include - -#define TESTFILE "foo.gz" - -#define CHECK_ERR(err, msg) { \ - if (err != Z_OK) { \ - fprintf(stderr, "%s error: %d\n", msg, err); \ - exit(1); \ - } \ -} - -const char hello[] = "hello, hello!"; -/* "hello world" would be more standard, but the repeated "hello" - * stresses the compression code better, sorry... - */ - -const char dictionary[] = "hello"; -unsigned long dictId; /* Adler32 value of the dictionary */ - -void test_deflate (unsigned char *compr, size_t comprLen); -void test_inflate (unsigned char *compr, size_t comprLen, unsigned char *uncompr, size_t uncomprLen); -void test_large_deflate (unsigned char *compr, size_t comprLen, unsigned char *uncompr, size_t uncomprLen); -void test_large_inflate (unsigned char *compr, size_t comprLen, unsigned char *uncompr, size_t uncomprLen); -void test_flush (unsigned char *compr, size_t *comprLen); -void test_sync (unsigned char *compr, size_t comprLen, unsigned char *uncompr, size_t uncomprLen); -void test_dict_deflate (unsigned char *compr, size_t comprLen); -void test_dict_inflate (unsigned char *compr, size_t comprLen, unsigned char *uncompr, size_t uncomprLen); -int main (int argc, char *argv[]); - - -static alloc_func zalloc = (alloc_func)0; -static free_func zfree = (free_func)0; - -void test_compress (unsigned char *compr, size_t comprLen, - unsigned char *uncompr, size_t uncomprLen); - -/* =========================================================================== - * Test compress() and uncompress() - */ -void test_compress(unsigned char *compr, size_t comprLen, unsigned char *uncompr, size_t uncomprLen) -{ - int err; - size_t len = strlen(hello)+1; - - err = compress(compr, &comprLen, (const unsigned char*)hello, len); - CHECK_ERR(err, "compress"); - - strcpy((char*)uncompr, "garbage"); - - err = uncompress(uncompr, &uncomprLen, compr, comprLen); - CHECK_ERR(err, "uncompress"); - - if (strcmp((char*)uncompr, hello)) { - fprintf(stderr, "bad uncompress\n"); - exit(1); - } else { - printf("uncompress(): %s\n", (char *)uncompr); - } -} - -#ifdef WITH_GZFILEOP -void test_gzio (const char *fname, - unsigned char *uncompr, unsigned long uncomprLen); - -/* =========================================================================== - * Test read/write of .gz files - */ -void test_gzio(const char *fname, unsigned char *uncompr, unsigned long uncomprLen) -{ -#ifdef NO_GZCOMPRESS - fprintf(stderr, "NO_GZCOMPRESS -- gz* functions cannot compress\n"); -#else - int err; - int len = (int)strlen(hello)+1; - gzFile file; - z_off_t pos; - - file = gzopen(fname, "wb"); - if (file == NULL) { - fprintf(stderr, "gzopen error\n"); - exit(1); - } - gzputc(file, 'h'); - if (gzputs(file, "ello") != 4) { - fprintf(stderr, "gzputs err: %s\n", gzerror(file, &err)); - exit(1); - } - if (gzprintf(file, ", %s!", "hello") != 8) { - fprintf(stderr, "gzprintf err: %s\n", gzerror(file, &err)); - exit(1); - } - gzseek(file, 1L, SEEK_CUR); /* add one zero byte */ - gzclose(file); - - file = gzopen(fname, "rb"); - if (file == NULL) { - fprintf(stderr, "gzopen error\n"); - exit(1); - } - strcpy((char*)uncompr, "garbage"); - - if (gzread(file, uncompr, (unsigned)uncomprLen) != len) { - fprintf(stderr, "gzread err: %s\n", gzerror(file, &err)); - exit(1); - } - if (strcmp((char*)uncompr, hello)) { - fprintf(stderr, "bad gzread: %s\n", (char*)uncompr); - exit(1); - } else { - printf("gzread(): %s\n", (char*)uncompr); - } - - pos = gzseek(file, -8L, SEEK_CUR); - if (pos != 6 || gztell(file) != pos) { - fprintf(stderr, "gzseek error, pos=%ld, gztell=%ld\n", - (long)pos, (long)gztell(file)); - exit(1); - } - - if (gzgetc(file) != ' ') { - fprintf(stderr, "gzgetc error\n"); - exit(1); - } - - if (gzungetc(' ', file) != ' ') { - fprintf(stderr, "gzungetc error\n"); - exit(1); - } - - gzgets(file, (char*)uncompr, (int)uncomprLen); - if (strlen((char*)uncompr) != 7) { /* " hello!" */ - fprintf(stderr, "gzgets err after gzseek: %s\n", gzerror(file, &err)); - exit(1); - } - if (strcmp((char*)uncompr, hello + 6)) { - fprintf(stderr, "bad gzgets after gzseek\n"); - exit(1); - } else { - printf("gzgets() after gzseek: %s\n", (char*)uncompr); - } - - gzclose(file); -#endif -} - -#endif /* WITH_GZFILEOP */ - -/* =========================================================================== - * Test deflate() with small buffers - */ -void test_deflate(unsigned char *compr, size_t comprLen) -{ - z_stream c_stream; /* compression stream */ - int err; - unsigned long len = (unsigned long)strlen(hello)+1; - - c_stream.zalloc = zalloc; - c_stream.zfree = zfree; - c_stream.opaque = (void *)0; - - err = deflateInit(&c_stream, Z_DEFAULT_COMPRESSION); - CHECK_ERR(err, "deflateInit"); - - c_stream.next_in = (const unsigned char *)hello; - c_stream.next_out = compr; - - while (c_stream.total_in != len && c_stream.total_out < comprLen) { - c_stream.avail_in = c_stream.avail_out = 1; /* force small buffers */ - err = deflate(&c_stream, Z_NO_FLUSH); - CHECK_ERR(err, "deflate"); - } - /* Finish the stream, still forcing small buffers: */ - for (;;) { - c_stream.avail_out = 1; - err = deflate(&c_stream, Z_FINISH); - if (err == Z_STREAM_END) break; - CHECK_ERR(err, "deflate"); - } - - err = deflateEnd(&c_stream); - CHECK_ERR(err, "deflateEnd"); -} - -/* =========================================================================== - * Test inflate() with small buffers - */ -void test_inflate(unsigned char *compr, size_t comprLen, unsigned char *uncompr, size_t uncomprLen) -{ - int err; - z_stream d_stream; /* decompression stream */ - - strcpy((char*)uncompr, "garbage"); - - d_stream.zalloc = zalloc; - d_stream.zfree = zfree; - d_stream.opaque = (void *)0; - - d_stream.next_in = compr; - d_stream.avail_in = 0; - d_stream.next_out = uncompr; - - err = inflateInit(&d_stream); - CHECK_ERR(err, "inflateInit"); - - while (d_stream.total_out < uncomprLen && d_stream.total_in < comprLen) { - d_stream.avail_in = d_stream.avail_out = 1; /* force small buffers */ - err = inflate(&d_stream, Z_NO_FLUSH); - if (err == Z_STREAM_END) break; - CHECK_ERR(err, "inflate"); - } - - err = inflateEnd(&d_stream); - CHECK_ERR(err, "inflateEnd"); - - if (strcmp((char*)uncompr, hello)) { - fprintf(stderr, "bad inflate\n"); - exit(1); - } else { - printf("inflate(): %s\n", (char *)uncompr); - } -} - -/* =========================================================================== - * Test deflate() with large buffers and dynamic change of compression level - */ -void test_large_deflate(unsigned char *compr, size_t comprLen, unsigned char *uncompr, size_t uncomprLen) -{ - z_stream c_stream; /* compression stream */ - int err; - - c_stream.zalloc = zalloc; - c_stream.zfree = zfree; - c_stream.opaque = (void *)0; - - err = deflateInit(&c_stream, Z_BEST_SPEED); - CHECK_ERR(err, "deflateInit"); - - c_stream.next_out = compr; - c_stream.avail_out = (unsigned int)comprLen; - - /* At this point, uncompr is still mostly zeroes, so it should compress - * very well: - */ - c_stream.next_in = uncompr; - c_stream.avail_in = (unsigned int)uncomprLen; - err = deflate(&c_stream, Z_NO_FLUSH); - CHECK_ERR(err, "deflate"); - if (c_stream.avail_in != 0) { - fprintf(stderr, "deflate not greedy\n"); - exit(1); - } - - /* Feed in already compressed data and switch to no compression: */ - deflateParams(&c_stream, Z_NO_COMPRESSION, Z_DEFAULT_STRATEGY); - c_stream.next_in = compr; - c_stream.avail_in = (unsigned int)comprLen/2; - err = deflate(&c_stream, Z_NO_FLUSH); - CHECK_ERR(err, "deflate"); - - /* Switch back to compressing mode: */ - deflateParams(&c_stream, Z_BEST_COMPRESSION, Z_FILTERED); - c_stream.next_in = uncompr; - c_stream.avail_in = (unsigned int)uncomprLen; - err = deflate(&c_stream, Z_NO_FLUSH); - CHECK_ERR(err, "deflate"); - - err = deflate(&c_stream, Z_FINISH); - if (err != Z_STREAM_END) { - fprintf(stderr, "deflate should report Z_STREAM_END\n"); - exit(1); - } - err = deflateEnd(&c_stream); - CHECK_ERR(err, "deflateEnd"); -} - -/* =========================================================================== - * Test inflate() with large buffers - */ -void test_large_inflate(unsigned char *compr, size_t comprLen, unsigned char *uncompr, size_t uncomprLen) -{ - int err; - z_stream d_stream; /* decompression stream */ - - strcpy((char*)uncompr, "garbage"); - - d_stream.zalloc = zalloc; - d_stream.zfree = zfree; - d_stream.opaque = (void *)0; - - d_stream.next_in = compr; - d_stream.avail_in = (unsigned int)comprLen; - - err = inflateInit(&d_stream); - CHECK_ERR(err, "inflateInit"); - - for (;;) { - d_stream.next_out = uncompr; /* discard the output */ - d_stream.avail_out = (unsigned int)uncomprLen; - err = inflate(&d_stream, Z_NO_FLUSH); - if (err == Z_STREAM_END) break; - CHECK_ERR(err, "large inflate"); - } - - err = inflateEnd(&d_stream); - CHECK_ERR(err, "inflateEnd"); - - if (d_stream.total_out != 2*uncomprLen + comprLen/2) { - fprintf(stderr, "bad large inflate: %zu\n", d_stream.total_out); - exit(1); - } else { - printf("large_inflate(): OK\n"); - } -} - -/* =========================================================================== - * Test deflate() with full flush - */ -void test_flush(unsigned char *compr, size_t *comprLen) -{ - z_stream c_stream; /* compression stream */ - int err; - unsigned int len = (unsigned int)strlen(hello)+1; - - c_stream.zalloc = zalloc; - c_stream.zfree = zfree; - c_stream.opaque = (void *)0; - - err = deflateInit(&c_stream, Z_DEFAULT_COMPRESSION); - CHECK_ERR(err, "deflateInit"); - - c_stream.next_in = (const unsigned char *)hello; - c_stream.next_out = compr; - c_stream.avail_in = 3; - c_stream.avail_out = (unsigned int)*comprLen; - err = deflate(&c_stream, Z_FULL_FLUSH); - CHECK_ERR(err, "deflate"); - - compr[3]++; /* force an error in first compressed block */ - c_stream.avail_in = len - 3; - - err = deflate(&c_stream, Z_FINISH); - if (err != Z_STREAM_END) { - CHECK_ERR(err, "deflate"); - } - err = deflateEnd(&c_stream); - CHECK_ERR(err, "deflateEnd"); - - *comprLen = c_stream.total_out; -} - -/* =========================================================================== - * Test inflateSync() - */ -void test_sync(unsigned char *compr, size_t comprLen, unsigned char *uncompr, size_t uncomprLen) -{ - int err; - z_stream d_stream; /* decompression stream */ - - strcpy((char*)uncompr, "garbage"); - - d_stream.zalloc = zalloc; - d_stream.zfree = zfree; - d_stream.opaque = (void *)0; - - d_stream.next_in = compr; - d_stream.avail_in = 2; /* just read the zlib header */ - - err = inflateInit(&d_stream); - CHECK_ERR(err, "inflateInit"); - - d_stream.next_out = uncompr; - d_stream.avail_out = (unsigned int)uncomprLen; - - err = inflate(&d_stream, Z_NO_FLUSH); - CHECK_ERR(err, "inflate"); - - d_stream.avail_in = (unsigned int)comprLen-2; /* read all compressed data */ - err = inflateSync(&d_stream); /* but skip the damaged part */ - CHECK_ERR(err, "inflateSync"); - - err = inflate(&d_stream, Z_FINISH); - if (err != Z_DATA_ERROR) { - fprintf(stderr, "inflate should report DATA_ERROR\n"); - /* Because of incorrect adler32 */ - exit(1); - } - err = inflateEnd(&d_stream); - CHECK_ERR(err, "inflateEnd"); - - printf("after inflateSync(): hel%s\n", (char *)uncompr); -} - -/* =========================================================================== - * Test deflate() with preset dictionary - */ -void test_dict_deflate(unsigned char *compr, size_t comprLen) -{ - z_stream c_stream; /* compression stream */ - int err; - - c_stream.zalloc = zalloc; - c_stream.zfree = zfree; - c_stream.opaque = (void *)0; - - err = deflateInit(&c_stream, Z_BEST_COMPRESSION); - CHECK_ERR(err, "deflateInit"); - - err = deflateSetDictionary(&c_stream, - (const unsigned char*)dictionary, (int)sizeof(dictionary)); - CHECK_ERR(err, "deflateSetDictionary"); - - dictId = c_stream.adler; - c_stream.next_out = compr; - c_stream.avail_out = (unsigned int)comprLen; - - c_stream.next_in = (const unsigned char *)hello; - c_stream.avail_in = (unsigned int)strlen(hello)+1; - - err = deflate(&c_stream, Z_FINISH); - if (err != Z_STREAM_END) { - fprintf(stderr, "deflate should report Z_STREAM_END\n"); - exit(1); - } - err = deflateEnd(&c_stream); - CHECK_ERR(err, "deflateEnd"); -} - -/* =========================================================================== - * Test inflate() with a preset dictionary - */ -void test_dict_inflate(unsigned char *compr, size_t comprLen, unsigned char *uncompr, size_t uncomprLen) -{ - int err; - z_stream d_stream; /* decompression stream */ - - strcpy((char*)uncompr, "garbage"); - - d_stream.zalloc = zalloc; - d_stream.zfree = zfree; - d_stream.opaque = (void *)0; - - d_stream.next_in = compr; - d_stream.avail_in = (unsigned int)comprLen; - - err = inflateInit(&d_stream); - CHECK_ERR(err, "inflateInit"); - - d_stream.next_out = uncompr; - d_stream.avail_out = (unsigned int)uncomprLen; - - for (;;) { - err = inflate(&d_stream, Z_NO_FLUSH); - if (err == Z_STREAM_END) break; - if (err == Z_NEED_DICT) { - if (d_stream.adler != dictId) { - fprintf(stderr, "unexpected dictionary"); - exit(1); - } - err = inflateSetDictionary(&d_stream, (const unsigned char*)dictionary, - (int)sizeof(dictionary)); - } - CHECK_ERR(err, "inflate with dict"); - } - - err = inflateEnd(&d_stream); - CHECK_ERR(err, "inflateEnd"); - - if (strcmp((char*)uncompr, hello)) { - fprintf(stderr, "bad inflate with dict\n"); - exit(1); - } else { - printf("inflate with dictionary: %s\n", (char *)uncompr); - } -} - -/* =========================================================================== - * Usage: example [output.gz [input.gz]] - */ - -int main(int argc, char *argv[]) -{ - unsigned char *compr, *uncompr; - size_t comprLen = 10000*sizeof(int); /* don't overflow on MSDOS */ - size_t uncomprLen = comprLen; - static const char* myVersion = ZLIB_VERSION; - - if (zlibVersion()[0] != myVersion[0]) { - fprintf(stderr, "incompatible zlib version\n"); - exit(1); - - } else if (strcmp(zlibVersion(), ZLIB_VERSION) != 0) { - fprintf(stderr, "warning: different zlib version\n"); - } - - printf("zlib version %s = 0x%04x, compile flags = 0x%lx\n", - ZLIB_VERSION, ZLIB_VERNUM, zlibCompileFlags()); - - compr = (unsigned char*)calloc((unsigned int)comprLen, 1); - uncompr = (unsigned char*)calloc((unsigned int)uncomprLen, 1); - /* compr and uncompr are cleared to avoid reading uninitialized - * data and to ensure that uncompr compresses well. - */ - if (compr == Z_NULL || uncompr == Z_NULL) { - printf("out of memory\n"); - exit(1); - } - - test_compress(compr, comprLen, uncompr, uncomprLen); - -#ifdef WITH_GZFILEOP - test_gzio((argc > 1 ? argv[1] : TESTFILE), - uncompr, uncomprLen); -#endif - - test_deflate(compr, comprLen); - test_inflate(compr, comprLen, uncompr, uncomprLen); - - test_large_deflate(compr, comprLen, uncompr, uncomprLen); - test_large_inflate(compr, comprLen, uncompr, uncomprLen); - - test_flush(compr, &comprLen); - test_sync(compr, comprLen, uncompr, uncomprLen); - comprLen = uncomprLen; - - test_dict_deflate(compr, comprLen); - test_dict_inflate(compr, comprLen, uncompr, uncomprLen); - - free(compr); - free(uncompr); - - return 0; -} diff --git a/contrib/libzlib-ng/test/infcover.c b/contrib/libzlib-ng/test/infcover.c deleted file mode 100644 index 5555a155fc3..00000000000 --- a/contrib/libzlib-ng/test/infcover.c +++ /dev/null @@ -1,668 +0,0 @@ -/* infcover.c -- test zlib's inflate routines with full code coverage - * Copyright (C) 2011 Mark Adler - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* to use, do: ./configure --cover && make cover */ - -#include -#include -#include -#include -#include "zlib.h" - -/* get definition of internal structure so we can mess with it (see pull()), - and so we can call inflate_trees() (see cover5()) */ -#define ZLIB_INTERNAL -#include "inftrees.h" -#include "inflate.h" - -/* -- memory tracking routines -- */ - -/* - These memory tracking routines are provided to zlib and track all of zlib's - allocations and deallocations, check for LIFO operations, keep a current - and high water mark of total bytes requested, optionally set a limit on the - total memory that can be allocated, and when done check for memory leaks. - - They are used as follows: - - z_stream strm; - mem_setup(&strm) initializes the memory tracking and sets the - zalloc, zfree, and opaque members of strm to use - memory tracking for all zlib operations on strm - mem_limit(&strm, limit) sets a limit on the total bytes requested -- a - request that exceeds this limit will result in an - allocation failure (returns NULL) -- setting the - limit to zero means no limit, which is the default - after mem_setup() - mem_used(&strm, "msg") prints to stderr "msg" and the total bytes used - mem_high(&strm, "msg") prints to stderr "msg" and the high water mark - mem_done(&strm, "msg") ends memory tracking, releases all allocations - for the tracking as well as leaked zlib blocks, if - any. If there was anything unusual, such as leaked - blocks, non-FIFO frees, or frees of addresses not - allocated, then "msg" and information about the - problem is printed to stderr. If everything is - normal, nothing is printed. mem_done resets the - strm members to Z_NULL to use the default memory - allocation routines on the next zlib initialization - using strm. - */ - -/* these items are strung together in a linked list, one for each allocation */ -struct mem_item { - void *ptr; /* pointer to allocated memory */ - size_t size; /* requested size of allocation */ - struct mem_item *next; /* pointer to next item in list, or NULL */ -}; - -/* this structure is at the root of the linked list, and tracks statistics */ -struct mem_zone { - struct mem_item *first; /* pointer to first item in list, or NULL */ - size_t total, highwater; /* total allocations, and largest total */ - size_t limit; /* memory allocation limit, or 0 if no limit */ - int notlifo, rogue; /* counts of non-LIFO frees and rogue frees */ -}; - -/* memory allocation routine to pass to zlib */ -static void *mem_alloc(void *mem, unsigned count, unsigned size) -{ - void *ptr; - struct mem_item *item; - struct mem_zone *zone = mem; - size_t len = count * (size_t)size; - - /* induced allocation failure */ - if (zone == NULL || (zone->limit && zone->total + len > zone->limit)) - return NULL; - - /* perform allocation using the standard library, fill memory with a - non-zero value to make sure that the code isn't depending on zeros */ - ptr = malloc(len); - if (ptr == NULL) - return NULL; - memset(ptr, 0xa5, len); - - /* create a new item for the list */ - item = malloc(sizeof(struct mem_item)); - if (item == NULL) { - free(ptr); - return NULL; - } - item->ptr = ptr; - item->size = len; - - /* insert item at the beginning of the list */ - item->next = zone->first; - zone->first = item; - - /* update the statistics */ - zone->total += item->size; - if (zone->total > zone->highwater) - zone->highwater = zone->total; - - /* return the allocated memory */ - return ptr; -} - -/* memory free routine to pass to zlib */ -static void mem_free(void *mem, void *ptr) -{ - struct mem_item *item, *next; - struct mem_zone *zone = mem; - - /* if no zone, just do a free */ - if (zone == NULL) { - free(ptr); - return; - } - - /* point next to the item that matches ptr, or NULL if not found -- remove - the item from the linked list if found */ - next = zone->first; - if (next) { - if (next->ptr == ptr) - zone->first = next->next; /* first one is it, remove from list */ - else { - do { /* search the linked list */ - item = next; - next = item->next; - } while (next != NULL && next->ptr != ptr); - if (next) { /* if found, remove from linked list */ - item->next = next->next; - zone->notlifo++; /* not a LIFO free */ - } - - } - } - - /* if found, update the statistics and free the item */ - if (next) { - zone->total -= next->size; - free(next); - } - - /* if not found, update the rogue count */ - else - zone->rogue++; - - /* in any case, do the requested free with the standard library function */ - free(ptr); -} - -/* set up a controlled memory allocation space for monitoring, set the stream - parameters to the controlled routines, with opaque pointing to the space */ -static void mem_setup(z_stream *strm) -{ - struct mem_zone *zone; - - zone = malloc(sizeof(struct mem_zone)); - assert(zone != NULL); - zone->first = NULL; - zone->total = 0; - zone->highwater = 0; - zone->limit = 0; - zone->notlifo = 0; - zone->rogue = 0; - strm->opaque = zone; - strm->zalloc = mem_alloc; - strm->zfree = mem_free; -} - -/* set a limit on the total memory allocation, or 0 to remove the limit */ -static void mem_limit(z_stream *strm, size_t limit) -{ - struct mem_zone *zone = strm->opaque; - - zone->limit = limit; -} - -/* show the current total requested allocations in bytes */ -static void mem_used(z_stream *strm, char *prefix) -{ - struct mem_zone *zone = strm->opaque; - - fprintf(stderr, "%s: %zu allocated\n", prefix, zone->total); -} - -/* show the high water allocation in bytes */ -static void mem_high(z_stream *strm, char *prefix) -{ - struct mem_zone *zone = strm->opaque; - - fprintf(stderr, "%s: %zu high water mark\n", prefix, zone->highwater); -} - -/* release the memory allocation zone -- if there are any surprises, notify */ -static void mem_done(z_stream *strm, char *prefix) -{ - int count = 0; - struct mem_item *item, *next; - struct mem_zone *zone = strm->opaque; - - /* show high water mark */ - mem_high(strm, prefix); - - /* free leftover allocations and item structures, if any */ - item = zone->first; - while (item != NULL) { - free(item->ptr); - next = item->next; - free(item); - item = next; - count++; - } - - /* issue alerts about anything unexpected */ - if (count || zone->total) - fprintf(stderr, "** %s: %zu bytes in %d blocks not freed\n", - prefix, zone->total, count); - if (zone->notlifo) - fprintf(stderr, "** %s: %d frees not LIFO\n", prefix, zone->notlifo); - if (zone->rogue) - fprintf(stderr, "** %s: %d frees not recognized\n", - prefix, zone->rogue); - - /* free the zone and delete from the stream */ - free(zone); - strm->opaque = Z_NULL; - strm->zalloc = Z_NULL; - strm->zfree = Z_NULL; -} - -/* -- inflate test routines -- */ - -/* Decode a hexadecimal string, set *len to length, in[] to the bytes. This - decodes liberally, in that hex digits can be adjacent, in which case two in - a row writes a byte. Or they can be delimited by any non-hex character, - where the delimiters are ignored except when a single hex digit is followed - by a delimiter, where that single digit writes a byte. The returned data is - allocated and must eventually be freed. NULL is returned if out of memory. - If the length is not needed, then len can be NULL. */ -static unsigned char *h2b(const char *hex, unsigned *len) -{ - unsigned char *in, *re; - unsigned next, val; - - in = malloc((strlen(hex) + 1) >> 1); - if (in == NULL) - return NULL; - next = 0; - val = 1; - do { - if (*hex >= '0' && *hex <= '9') - val = (val << 4) + *hex - '0'; - else if (*hex >= 'A' && *hex <= 'F') - val = (val << 4) + *hex - 'A' + 10; - else if (*hex >= 'a' && *hex <= 'f') - val = (val << 4) + *hex - 'a' + 10; - else if (val != 1 && val < 32) /* one digit followed by delimiter */ - val += 240; /* make it look like two digits */ - if (val > 255) { /* have two digits */ - in[next++] = val & 0xff; /* save the decoded byte */ - val = 1; /* start over */ - } - } while (*hex++); /* go through the loop with the terminating null */ - if (len != NULL) - *len = next; - re = realloc(in, next); - return re == NULL ? in : re; -} - -/* generic inflate() run, where hex is the hexadecimal input data, what is the - text to include in an error message, step is how much input data to feed - inflate() on each call, or zero to feed it all, win is the window bits - parameter to inflateInit2(), len is the size of the output buffer, and err - is the error code expected from the first inflate() call (the second - inflate() call is expected to return Z_STREAM_END). If win is 47, then - header information is collected with inflateGetHeader(). If a zlib stream - is looking for a dictionary, then an empty dictionary is provided. - inflate() is run until all of the input data is consumed. */ -static void inf(char *hex, char *what, unsigned step, int win, unsigned len, int err) -{ - int ret; - unsigned have; - unsigned char *in, *out; - z_stream strm, copy; - gz_header head; - - mem_setup(&strm); - strm.avail_in = 0; - strm.next_in = Z_NULL; - ret = inflateInit2(&strm, win); - if (ret != Z_OK) { - mem_done(&strm, what); - return; - } - out = malloc(len); assert(out != NULL); - if (win == 47) { - head.extra = out; - head.extra_max = len; - head.name = out; - head.name_max = len; - head.comment = out; - head.comm_max = len; - ret = inflateGetHeader(&strm, &head); assert(ret == Z_OK); - } - in = h2b(hex, &have); assert(in != NULL); - if (step == 0 || step > have) - step = have; - strm.avail_in = step; - have -= step; - strm.next_in = in; - do { - strm.avail_out = len; - strm.next_out = out; - ret = inflate(&strm, Z_NO_FLUSH); assert(err == 9 || ret == err); - if (ret != Z_OK && ret != Z_BUF_ERROR && ret != Z_NEED_DICT) - break; - if (ret == Z_NEED_DICT) { - ret = inflateSetDictionary(&strm, in, 1); - assert(ret == Z_DATA_ERROR); - mem_limit(&strm, 1); - ret = inflateSetDictionary(&strm, out, 0); - assert(ret == Z_MEM_ERROR); - mem_limit(&strm, 0); - ((struct inflate_state *)strm.state)->mode = DICT; - ret = inflateSetDictionary(&strm, out, 0); - assert(ret == Z_OK); - ret = inflate(&strm, Z_NO_FLUSH); assert(ret == Z_BUF_ERROR); - } - ret = inflateCopy(©, &strm); assert(ret == Z_OK); - ret = inflateEnd(©); assert(ret == Z_OK); - err = 9; /* don't care next time around */ - have += strm.avail_in; - strm.avail_in = step > have ? have : step; - have -= strm.avail_in; - } while (strm.avail_in); - free(in); - free(out); - ret = inflateReset2(&strm, -8); assert(ret == Z_OK); - ret = inflateEnd(&strm); assert(ret == Z_OK); - mem_done(&strm, what); -} - -/* cover all of the lines in inflate.c up to inflate() */ -static void cover_support(void) -{ - int ret; - z_stream strm; - - mem_setup(&strm); - strm.avail_in = 0; - strm.next_in = Z_NULL; - ret = inflateInit(&strm); assert(ret == Z_OK); - mem_used(&strm, "inflate init"); - ret = inflatePrime(&strm, 5, 31); assert(ret == Z_OK); - ret = inflatePrime(&strm, -1, 0); assert(ret == Z_OK); - ret = inflateSetDictionary(&strm, Z_NULL, 0); - assert(ret == Z_STREAM_ERROR); - ret = inflateEnd(&strm); assert(ret == Z_OK); - mem_done(&strm, "prime"); - - inf("63 0", "force window allocation", 0, -15, 1, Z_OK); - inf("63 18 5", "force window replacement", 0, -8, 259, Z_OK); - inf("63 18 68 30 d0 0 0", "force split window update", 4, -8, 259, Z_OK); - inf("3 0", "use fixed blocks", 0, -15, 1, Z_STREAM_END); - inf("", "bad window size", 0, 1, 0, Z_STREAM_ERROR); - - mem_setup(&strm); - strm.avail_in = 0; - strm.next_in = Z_NULL; - ret = inflateInit_(&strm, ZLIB_VERSION + 1, (int)sizeof(z_stream)); - assert(ret == Z_VERSION_ERROR); - mem_done(&strm, "wrong version"); - - strm.avail_in = 0; - strm.next_in = Z_NULL; - ret = inflateInit(&strm); assert(ret == Z_OK); - ret = inflateEnd(&strm); assert(ret == Z_OK); - fputs("inflate built-in memory routines\n", stderr); -} - -/* cover all inflate() header and trailer cases and code after inflate() */ -static void cover_wrap(void) -{ - int ret; - z_stream strm, copy; - unsigned char dict[257]; - - ret = inflate(Z_NULL, 0); assert(ret == Z_STREAM_ERROR); - ret = inflateEnd(Z_NULL); assert(ret == Z_STREAM_ERROR); - ret = inflateCopy(Z_NULL, Z_NULL); assert(ret == Z_STREAM_ERROR); - fputs("inflate bad parameters\n", stderr); - - inf("1f 8b 0 0", "bad gzip method", 0, 31, 0, Z_DATA_ERROR); - inf("1f 8b 8 80", "bad gzip flags", 0, 31, 0, Z_DATA_ERROR); - inf("77 85", "bad zlib method", 0, 15, 0, Z_DATA_ERROR); - inf("8 99", "set window size from header", 0, 0, 0, Z_OK); - inf("78 9c", "bad zlib window size", 0, 8, 0, Z_DATA_ERROR); - inf("78 9c 63 0 0 0 1 0 1", "check adler32", 0, 15, 1, Z_STREAM_END); - inf("1f 8b 8 1e 0 0 0 0 0 0 1 0 0 0 0 0 0", "bad header crc", 0, 47, 1, - Z_DATA_ERROR); - inf("1f 8b 8 2 0 0 0 0 0 0 1d 26 3 0 0 0 0 0 0 0 0 0", "check gzip length", - 0, 47, 0, Z_STREAM_END); - inf("78 90", "bad zlib header check", 0, 47, 0, Z_DATA_ERROR); - inf("8 b8 0 0 0 1", "need dictionary", 0, 8, 0, Z_NEED_DICT); - inf("78 9c 63 0", "compute adler32", 0, 15, 1, Z_OK); - - mem_setup(&strm); - strm.avail_in = 0; - strm.next_in = Z_NULL; - ret = inflateInit2(&strm, -8); - strm.avail_in = 2; - strm.next_in = (void *)"\x63"; - strm.avail_out = 1; - strm.next_out = (void *)&ret; - mem_limit(&strm, 1); - ret = inflate(&strm, Z_NO_FLUSH); assert(ret == Z_MEM_ERROR); - ret = inflate(&strm, Z_NO_FLUSH); assert(ret == Z_MEM_ERROR); - mem_limit(&strm, 0); - memset(dict, 0, 257); - ret = inflateSetDictionary(&strm, dict, 257); - assert(ret == Z_OK); - mem_limit(&strm, (sizeof(struct inflate_state) << 1) + 256); - ret = inflatePrime(&strm, 16, 0); assert(ret == Z_OK); - strm.avail_in = 2; - strm.next_in = (void *)"\x80"; - ret = inflateSync(&strm); assert(ret == Z_DATA_ERROR); - ret = inflate(&strm, Z_NO_FLUSH); assert(ret == Z_STREAM_ERROR); - strm.avail_in = 4; - strm.next_in = (void *)"\0\0\xff\xff"; - ret = inflateSync(&strm); assert(ret == Z_OK); - (void)inflateSyncPoint(&strm); - ret = inflateCopy(©, &strm); assert(ret == Z_MEM_ERROR); - mem_limit(&strm, 0); - ret = inflateUndermine(&strm, 1); assert(ret == Z_DATA_ERROR); - (void)inflateMark(&strm); - ret = inflateEnd(&strm); assert(ret == Z_OK); - mem_done(&strm, "miscellaneous, force memory errors"); -} - -/* input and output functions for inflateBack() */ -static unsigned pull(void *desc, const unsigned char **buf) -{ - static unsigned int next = 0; - static unsigned char dat[] = {0x63, 0, 2, 0}; - struct inflate_state *state; - - if (desc == Z_NULL) { - next = 0; - return 0; /* no input (already provided at next_in) */ - } - state = (void *)((z_stream *)desc)->state; - if (state != Z_NULL) - state->mode = SYNC; /* force an otherwise impossible situation */ - return next < sizeof(dat) ? (*buf = dat + next++, 1) : 0; -} - -static int push(void *desc, unsigned char *buf, unsigned len) -{ - buf += len; - return desc != Z_NULL; /* force error if desc not null */ -} - -/* cover inflateBack() up to common deflate data cases and after those */ -static void cover_back(void) -{ - int ret; - z_stream strm; - unsigned char win[32768]; - - ret = inflateBackInit_(Z_NULL, 0, win, 0, 0); - assert(ret == Z_VERSION_ERROR); - ret = inflateBackInit(Z_NULL, 0, win); assert(ret == Z_STREAM_ERROR); - ret = inflateBack(Z_NULL, Z_NULL, Z_NULL, Z_NULL, Z_NULL); - assert(ret == Z_STREAM_ERROR); - ret = inflateBackEnd(Z_NULL); assert(ret == Z_STREAM_ERROR); - fputs("inflateBack bad parameters\n", stderr); - - mem_setup(&strm); - ret = inflateBackInit(&strm, 15, win); assert(ret == Z_OK); - strm.avail_in = 2; - strm.next_in = (void *)"\x03"; - ret = inflateBack(&strm, pull, Z_NULL, push, Z_NULL); - assert(ret == Z_STREAM_END); - /* force output error */ - strm.avail_in = 3; - strm.next_in = (void *)"\x63\x00"; - ret = inflateBack(&strm, pull, Z_NULL, push, &strm); - assert(ret == Z_BUF_ERROR); - /* force mode error by mucking with state */ - ret = inflateBack(&strm, pull, &strm, push, Z_NULL); - assert(ret == Z_STREAM_ERROR); - ret = inflateBackEnd(&strm); assert(ret == Z_OK); - mem_done(&strm, "inflateBack bad state"); - - ret = inflateBackInit(&strm, 15, win); assert(ret == Z_OK); - ret = inflateBackEnd(&strm); assert(ret == Z_OK); - fputs("inflateBack built-in memory routines\n", stderr); -} - -/* do a raw inflate of data in hexadecimal with both inflate and inflateBack */ -static int try(char *hex, char *id, int err) -{ - int ret; - unsigned len, size; - unsigned char *in, *out, *win; - char *prefix; - z_stream strm; - - /* convert to hex */ - in = h2b(hex, &len); - assert(in != NULL); - - /* allocate work areas */ - size = len << 3; - out = malloc(size); - assert(out != NULL); - win = malloc(32768); - assert(win != NULL); - prefix = malloc(strlen(id) + 6); - assert(prefix != NULL); - - /* first with inflate */ - strcpy(prefix, id); - strcat(prefix, "-late"); - mem_setup(&strm); - strm.avail_in = 0; - strm.next_in = Z_NULL; - ret = inflateInit2(&strm, err < 0 ? 47 : -15); - assert(ret == Z_OK); - strm.avail_in = len; - strm.next_in = in; - do { - strm.avail_out = size; - strm.next_out = out; - ret = inflate(&strm, Z_TREES); - assert(ret != Z_STREAM_ERROR && ret != Z_MEM_ERROR); - if (ret == Z_DATA_ERROR || ret == Z_NEED_DICT) - break; - } while (strm.avail_in || strm.avail_out == 0); - if (err) { - assert(ret == Z_DATA_ERROR); - assert(strcmp(id, strm.msg) == 0); - } - inflateEnd(&strm); - mem_done(&strm, prefix); - - /* then with inflateBack */ - if (err >= 0) { - strcpy(prefix, id); - strcat(prefix, "-back"); - mem_setup(&strm); - ret = inflateBackInit(&strm, 15, win); - assert(ret == Z_OK); - strm.avail_in = len; - strm.next_in = in; - ret = inflateBack(&strm, pull, Z_NULL, push, Z_NULL); - assert(ret != Z_STREAM_ERROR); - if (err) { - assert(ret == Z_DATA_ERROR); - assert(strcmp(id, strm.msg) == 0); - } - inflateBackEnd(&strm); - mem_done(&strm, prefix); - } - - /* clean up */ - free(prefix); - free(win); - free(out); - free(in); - return ret; -} - -/* cover deflate data cases in both inflate() and inflateBack() */ -static void cover_inflate(void) -{ - try("0 0 0 0 0", "invalid stored block lengths", 1); - try("3 0", "fixed", 0); - try("6", "invalid block type", 1); - try("1 1 0 fe ff 0", "stored", 0); - try("fc 0 0", "too many length or distance symbols", 1); - try("4 0 fe ff", "invalid code lengths set", 1); - try("4 0 24 49 0", "invalid bit length repeat", 1); - try("4 0 24 e9 ff ff", "invalid bit length repeat", 1); - try("4 0 24 e9 ff 6d", "invalid code -- missing end-of-block", 1); - try("4 80 49 92 24 49 92 24 71 ff ff 93 11 0", - "invalid literal/lengths set", 1); - try("4 80 49 92 24 49 92 24 f b4 ff ff c3 84", "invalid distances set", 1); - try("4 c0 81 8 0 0 0 0 20 7f eb b 0 0", "invalid literal/length code", 1); - try("2 7e ff ff", "invalid distance code", 1); - try("c c0 81 0 0 0 0 0 90 ff 6b 4 0", "invalid distance too far back", 1); - - /* also trailer mismatch just in inflate() */ - try("1f 8b 8 0 0 0 0 0 0 0 3 0 0 0 0 1", "incorrect data check", -1); - try("1f 8b 8 0 0 0 0 0 0 0 3 0 0 0 0 0 0 0 0 1", - "incorrect length check", -1); - try("5 c0 21 d 0 0 0 80 b0 fe 6d 2f 91 6c", "pull 17", 0); - try("5 e0 81 91 24 cb b2 2c 49 e2 f 2e 8b 9a 47 56 9f fb fe ec d2 ff 1f", - "long code", 0); - try("ed c0 1 1 0 0 0 40 20 ff 57 1b 42 2c 4f", "length extra", 0); - try("ed cf c1 b1 2c 47 10 c4 30 fa 6f 35 1d 1 82 59 3d fb be 2e 2a fc f c", - "long distance and extra", 0); - try("ed c0 81 0 0 0 0 80 a0 fd a9 17 a9 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 " - "0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 6", "window end", 0); - inf("2 8 20 80 0 3 0", "inflate_fast TYPE return", 0, -15, 258, - Z_STREAM_END); - inf("63 18 5 40 c 0", "window wrap", 3, -8, 300, Z_OK); -} - -/* cover remaining lines in inftrees.c */ -static void cover_trees(void) -{ - int ret; - unsigned bits; - uint16_t lens[16], work[16]; - code *next, table[ENOUGH_DISTS]; - - /* we need to call inflate_table() directly in order to manifest not- - enough errors, since zlib insures that enough is always enough */ - for (bits = 0; bits < 15; bits++) - lens[bits] = (uint16_t)(bits + 1); - lens[15] = 15; - next = table; - bits = 15; - ret = inflate_table(DISTS, lens, 16, &next, &bits, work); - assert(ret == 1); - next = table; - bits = 1; - ret = inflate_table(DISTS, lens, 16, &next, &bits, work); - assert(ret == 1); - fputs("inflate_table not enough errors\n", stderr); -} - -/* cover remaining inffast.c decoding and window copying */ -static void cover_fast(void) -{ - inf("e5 e0 81 ad 6d cb b2 2c c9 01 1e 59 63 ae 7d ee fb 4d fd b5 35 41 68" - " ff 7f 0f 0 0 0", "fast length extra bits", 0, -8, 258, Z_DATA_ERROR); - inf("25 fd 81 b5 6d 59 b6 6a 49 ea af 35 6 34 eb 8c b9 f6 b9 1e ef 67 49" - " 50 fe ff ff 3f 0 0", "fast distance extra bits", 0, -8, 258, - Z_DATA_ERROR); - inf("3 7e 0 0 0 0 0", "fast invalid distance code", 0, -8, 258, - Z_DATA_ERROR); - inf("1b 7 0 0 0 0 0", "fast invalid literal/length code", 0, -8, 258, - Z_DATA_ERROR); - inf("d c7 1 ae eb 38 c 4 41 a0 87 72 de df fb 1f b8 36 b1 38 5d ff ff 0", - "fast 2nd level codes and too far back", 0, -8, 258, Z_DATA_ERROR); - inf("63 18 5 8c 10 8 0 0 0 0", "very common case", 0, -8, 259, Z_OK); - inf("63 60 60 18 c9 0 8 18 18 18 26 c0 28 0 29 0 0 0", - "contiguous and wrap around window", 6, -8, 259, Z_OK); - inf("63 0 3 0 0 0 0 0", "copy direct from output", 0, -8, 259, - Z_STREAM_END); -} - -int main(void) -{ - fprintf(stderr, "%s\n", zlibVersion()); - cover_support(); - cover_wrap(); - cover_back(); - cover_inflate(); - cover_trees(); - cover_fast(); - return 0; -} diff --git a/contrib/libzlib-ng/test/minigzip.c b/contrib/libzlib-ng/test/minigzip.c deleted file mode 100644 index 9c71fd1b8d5..00000000000 --- a/contrib/libzlib-ng/test/minigzip.c +++ /dev/null @@ -1,530 +0,0 @@ -/* minigzip.c -- simulate gzip using the zlib compression library - * Copyright (C) 1995-2006, 2010, 2011 Jean-loup Gailly. - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* - * minigzip is a minimal implementation of the gzip utility. This is - * only an example of using zlib and isn't meant to replace the - * full-featured gzip. No attempt is made to deal with file systems - * limiting names to 14 or 8+3 characters, etc... Error checking is - * very limited. So use minigzip only for testing; use gzip for the - * real thing. - */ - -/* @(#) $Id$ */ - -#include "zlib.h" -#include - -#include -#include - -#ifdef USE_MMAP -# include -# include -# include -#endif - -#if defined(WIN32) || defined(__CYGWIN__) -# include -# include -# define SET_BINARY_MODE(file) setmode(fileno(file), O_BINARY) -#else -# define SET_BINARY_MODE(file) -#endif - -#if defined(_MSC_VER) && _MSC_VER < 1900 -# define snprintf _snprintf -#endif - -#if !defined(Z_HAVE_UNISTD_H) && !defined(_LARGEFILE64_SOURCE) -#ifndef WIN32 /* unlink already in stdio.h for WIN32 */ - extern int unlink (const char *); -#endif -#endif - -#ifndef GZ_SUFFIX -# define GZ_SUFFIX ".gz" -#endif -#define SUFFIX_LEN (sizeof(GZ_SUFFIX)-1) - -#define BUFLEN 16384 /* read buffer size */ -#define BUFLENW (BUFLEN * 3) /* write buffer size */ -#define MAX_NAME_LEN 1024 - -#ifndef WITH_GZFILEOP -/* without WITH_GZFILEOP, create simplified gz* functions using deflate and inflate */ - -#if defined(Z_HAVE_UNISTD_H) || defined(Z_LARGE) -# include /* for unlink() */ -#endif - -void *myalloc (void *, unsigned, unsigned); -void myfree (void *, void *); - -void *myalloc(void *q, unsigned n, unsigned m) -{ - (void)q; - return calloc(n, m); -} - -void myfree(void *q, void *p) -{ - (void)q; - free(p); -} - -typedef struct gzFile_s { - FILE *file; - int write; - int err; - const char *msg; - z_stream strm; - unsigned char *buf; -} *gzFile; - -gzFile gzopen (const char *, const char *); -gzFile gzdopen (int, const char *); -gzFile gz_open (const char *, int, const char *); - -gzFile gzopen(const char *path, const char *mode) -{ - return gz_open(path, -1, mode); -} - -gzFile gzdopen(int fd, const char *mode) -{ - return gz_open(NULL, fd, mode); -} - -gzFile gz_open(const char *path, int fd, const char *mode) -{ - gzFile gz; - int ret; - int level = Z_DEFAULT_COMPRESSION; - const char *plevel = mode; - - gz = malloc(sizeof(struct gzFile_s)); - if (gz == NULL) - return NULL; - gz->write = strchr(mode, 'w') != NULL; - gz->strm.zalloc = myalloc; - gz->strm.zfree = myfree; - gz->strm.opaque = Z_NULL; - gz->buf = malloc(gz->write ? BUFLENW : BUFLEN); - - if (gz->buf == NULL) { - free(gz); - return NULL; - } - - while (*plevel) { - if (*plevel >= '0' && *plevel <= '9') { - level = *plevel - '0'; - break; - } - plevel++; - } - if (gz->write) - ret = deflateInit2(&(gz->strm), level, 8, 15 + 16, 8, 0); - else { - gz->strm.next_in = 0; - gz->strm.avail_in = Z_NULL; - ret = inflateInit2(&(gz->strm), 15 + 16); - } - if (ret != Z_OK) { - free(gz); - return NULL; - } - gz->file = path == NULL ? fdopen(fd, gz->write ? "wb" : "rb") : - fopen(path, gz->write ? "wb" : "rb"); - if (gz->file == NULL) { - gz->write ? deflateEnd(&(gz->strm)) : inflateEnd(&(gz->strm)); - free(gz); - return NULL; - } - gz->err = 0; - gz->msg = ""; - return gz; -} - -int gzwrite (gzFile, const void *, unsigned); - -int gzwrite(gzFile gz, const void *buf, unsigned len) -{ - z_stream *strm; - - if (gz == NULL || !gz->write) - return 0; - strm = &(gz->strm); - strm->next_in = (void *)buf; - strm->avail_in = len; - do { - strm->next_out = gz->buf; - strm->avail_out = BUFLENW; - (void)deflate(strm, Z_NO_FLUSH); - fwrite(gz->buf, 1, BUFLENW - strm->avail_out, gz->file); - } while (strm->avail_out == 0); - return len; -} - -int gzread (gzFile, void *, unsigned); - -int gzread(gzFile gz, void *buf, unsigned len) -{ - z_stream *strm; - - if (gz == NULL || gz->write || gz->err) - return 0; - strm = &(gz->strm); - strm->next_out = buf; - strm->avail_out = len; - do { - if (strm->avail_in == 0) - { - strm->next_in = gz->buf; - strm->avail_in = (uint32_t)fread(gz->buf, 1, BUFLEN, gz->file); - } - if (strm->avail_in > 0) - { - int ret = inflate(strm, Z_NO_FLUSH); - if (ret == Z_DATA_ERROR) { - gz->err = ret; - gz->msg = strm->msg; - return 0; - } - else if (ret == Z_STREAM_END) - inflateReset(strm); - } - else - break; - } while (strm->avail_out); - return len - strm->avail_out; -} - -int gzclose (gzFile); - -int gzclose(gzFile gz) -{ - z_stream *strm; - - if (gz == NULL) - return Z_STREAM_ERROR; - strm = &(gz->strm); - if (gz->write) { - strm->next_in = Z_NULL; - strm->avail_in = 0; - do { - strm->next_out = gz->buf; - strm->avail_out = BUFLENW; - (void)deflate(strm, Z_FINISH); - fwrite(gz->buf, 1, BUFLENW - strm->avail_out, gz->file); - } while (strm->avail_out == 0); - deflateEnd(strm); - } - else - inflateEnd(strm); - free(gz->buf); - fclose(gz->file); - free(gz); - return Z_OK; -} - -const char *gzerror (gzFile, int *); - -const char *gzerror(gzFile gz, int *err) -{ - *err = gz->err; - return gz->msg; -} - -#endif - -char *prog; - -void error (const char *msg); -void gz_compress (FILE *in, gzFile out); -#ifdef USE_MMAP -int gz_compress_mmap (FILE *in, gzFile out); -#endif -void gz_uncompress (gzFile in, FILE *out); -void file_compress (char *file, char *mode); -void file_uncompress (char *file); -int main (int argc, char *argv[]); - -/* =========================================================================== - * Display error message and exit - */ -void error(const char *msg) -{ - fprintf(stderr, "%s: %s\n", prog, msg); - exit(1); -} - -/* =========================================================================== - * Compress input to output then close both files. - */ - -void gz_compress(FILE *in, gzFile out) -{ - char buf[BUFLEN]; - int len; - int err; - -#ifdef USE_MMAP - /* Try first compressing with mmap. If mmap fails (minigzip used in a - * pipe), use the normal fread loop. - */ - if (gz_compress_mmap(in, out) == Z_OK) return; -#endif - for (;;) { - len = (int)fread(buf, 1, sizeof(buf), in); - if (ferror(in)) { - perror("fread"); - exit(1); - } - if (len == 0) break; - - if (gzwrite(out, buf, (unsigned)len) != len) error(gzerror(out, &err)); - } - fclose(in); - if (gzclose(out) != Z_OK) error("failed gzclose"); -} - -#ifdef USE_MMAP /* MMAP version, Miguel Albrecht */ - -/* Try compressing the input file at once using mmap. Return Z_OK if - * if success, Z_ERRNO otherwise. - */ -int gz_compress_mmap(FILE *in, gzFile out) -{ - int len; - int err; - int ifd = fileno(in); - caddr_t buf; /* mmap'ed buffer for the entire input file */ - off_t buf_len; /* length of the input file */ - struct stat sb; - - /* Determine the size of the file, needed for mmap: */ - if (fstat(ifd, &sb) < 0) return Z_ERRNO; - buf_len = sb.st_size; - if (buf_len <= 0) return Z_ERRNO; - - /* Now do the actual mmap: */ - buf = mmap((caddr_t) 0, buf_len, PROT_READ, MAP_SHARED, ifd, (off_t)0); - if (buf == (caddr_t)(-1)) return Z_ERRNO; - - /* Compress the whole file at once: */ - len = gzwrite(out, (char *)buf, (unsigned)buf_len); - - if (len != (int)buf_len) error(gzerror(out, &err)); - - munmap(buf, buf_len); - fclose(in); - if (gzclose(out) != Z_OK) error("failed gzclose"); - return Z_OK; -} -#endif /* USE_MMAP */ - -/* =========================================================================== - * Uncompress input to output then close both files. - */ -void gz_uncompress(gzFile in, FILE *out) -{ - char buf[BUFLENW]; - int len; - int err; - - for (;;) { - len = gzread(in, buf, sizeof(buf)); - if (len < 0) error (gzerror(in, &err)); - if (len == 0) break; - - if ((int)fwrite(buf, 1, (unsigned)len, out) != len) { - error("failed fwrite"); - } - } - if (fclose(out)) error("failed fclose"); - - if (gzclose(in) != Z_OK) error("failed gzclose"); -} - - -/* =========================================================================== - * Compress the given file: create a corresponding .gz file and remove the - * original. - */ -void file_compress(char *file, char *mode) -{ - char outfile[MAX_NAME_LEN]; - FILE *in; - gzFile out; - - if (strlen(file) + strlen(GZ_SUFFIX) >= sizeof(outfile)) { - fprintf(stderr, "%s: filename too long\n", prog); - exit(1); - } - - snprintf(outfile, sizeof(outfile), "%s%s", file, GZ_SUFFIX); - - in = fopen(file, "rb"); - if (in == NULL) { - perror(file); - exit(1); - } - out = gzopen(outfile, mode); - if (out == NULL) { - fprintf(stderr, "%s: can't gzopen %s\n", prog, outfile); - exit(1); - } - gz_compress(in, out); - - unlink(file); -} - - -/* =========================================================================== - * Uncompress the given file and remove the original. - */ -void file_uncompress(char *file) -{ - char buf[MAX_NAME_LEN]; - char *infile, *outfile; - FILE *out; - gzFile in; - size_t len = strlen(file); - - if (len + strlen(GZ_SUFFIX) >= sizeof(buf)) { - fprintf(stderr, "%s: filename too long\n", prog); - exit(1); - } - - snprintf(buf, sizeof(buf), "%s", file); - - if (len > SUFFIX_LEN && strcmp(file+len-SUFFIX_LEN, GZ_SUFFIX) == 0) { - infile = file; - outfile = buf; - outfile[len-3] = '\0'; - } else { - outfile = file; - infile = buf; - snprintf(buf + len, sizeof(buf) - len, "%s", GZ_SUFFIX); - } - in = gzopen(infile, "rb"); - if (in == NULL) { - fprintf(stderr, "%s: can't gzopen %s\n", prog, infile); - exit(1); - } - out = fopen(outfile, "wb"); - if (out == NULL) { - perror(file); - exit(1); - } - - gz_uncompress(in, out); - - unlink(infile); -} - - -/* =========================================================================== - * Usage: minigzip [-c] [-d] [-f] [-h] [-r] [-1 to -9] [files...] - * -c : write to standard output - * -d : decompress - * -f : compress with Z_FILTERED - * -h : compress with Z_HUFFMAN_ONLY - * -r : compress with Z_RLE - * -1 to -9 : compression level - */ - -int main(int argc, char *argv[]) -{ - int copyout = 0; - int uncompr = 0; - gzFile file; - char *bname, outmode[20]; - - snprintf(outmode, sizeof(outmode), "%s", "wb6 "); - - prog = argv[0]; - bname = strrchr(argv[0], '/'); - if (bname) - bname++; - else - bname = argv[0]; - argc--, argv++; - - if (!strcmp(bname, "gunzip")) - uncompr = 1; - else if (!strcmp(bname, "zcat")) - copyout = uncompr = 1; - - while (argc > 0) { - if (strcmp(*argv, "-c") == 0) - copyout = 1; - else if (strcmp(*argv, "-d") == 0) - uncompr = 1; - else if (strcmp(*argv, "-f") == 0) - outmode[3] = 'f'; - else if (strcmp(*argv, "-h") == 0) - outmode[3] = 'h'; - else if (strcmp(*argv, "-r") == 0) - outmode[3] = 'R'; - else if ((*argv)[0] == '-' && (*argv)[1] >= '1' && (*argv)[1] <= '9' && - (*argv)[2] == 0) - outmode[2] = (*argv)[1]; - else - break; - argc--, argv++; - } - if (outmode[3] == ' ') - outmode[3] = 0; - if (argc == 0) { - SET_BINARY_MODE(stdin); - SET_BINARY_MODE(stdout); - if (uncompr) { - file = gzdopen(fileno(stdin), "rb"); - if (file == NULL) error("can't gzdopen stdin"); - gz_uncompress(file, stdout); - } else { - file = gzdopen(fileno(stdout), outmode); - if (file == NULL) error("can't gzdopen stdout"); - gz_compress(stdin, file); - } - } else { - if (copyout) { - SET_BINARY_MODE(stdout); - } - do { - if (uncompr) { - if (copyout) { - file = gzopen(*argv, "rb"); - if (file == NULL) - fprintf(stderr, "%s: can't gzopen %s\n", prog, *argv); - else - gz_uncompress(file, stdout); - } else { - file_uncompress(*argv); - } - } else { - if (copyout) { - FILE * in = fopen(*argv, "rb"); - - if (in == NULL) { - perror(*argv); - } else { - file = gzdopen(fileno(stdout), outmode); - if (file == NULL) error("can't gzdopen stdout"); - - gz_compress(in, file); - } - - } else { - file_compress(*argv, outmode); - } - } - } while (argv++, --argc); - } - return 0; -} diff --git a/contrib/libzlib-ng/test/testCVEinputs.sh b/contrib/libzlib-ng/test/testCVEinputs.sh deleted file mode 100755 index 046856e7884..00000000000 --- a/contrib/libzlib-ng/test/testCVEinputs.sh +++ /dev/null @@ -1,22 +0,0 @@ -#!/bin/bash -TESTDIR="$(dirname "$0")" - -CVEs="CVE-2002-0059 CVE-2004-0797 CVE-2005-1849 CVE-2005-2096" - -for CVE in $CVEs; do - fail=0 - for testcase in ${TESTDIR}/${CVE}/*.gz; do - ../minigzip -d < "$testcase" - # we expect that a 1 error code is OK - # for a vulnerable failure we'd expect 134 or similar - if [ $? -ne 1 ]; then - fail=1 - fi - done - if [ $fail -eq 0 ]; then - echo " --- zlib not vulnerable to $CVE ---"; - else - echo " --- zlib VULNERABLE to $CVE ---"; exit 1; - fi -done - diff --git a/contrib/libzlib-ng/treebuild.xml b/contrib/libzlib-ng/treebuild.xml deleted file mode 100644 index 38d29d75efc..00000000000 --- a/contrib/libzlib-ng/treebuild.xml +++ /dev/null @@ -1,116 +0,0 @@ - - - - zip compression library - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/contrib/libzlib-ng/trees.c b/contrib/libzlib-ng/trees.c deleted file mode 100644 index ee756fc5a25..00000000000 --- a/contrib/libzlib-ng/trees.c +++ /dev/null @@ -1,1119 +0,0 @@ -/* trees.c -- output deflated data using Huffman coding - * Copyright (C) 1995-2012 Jean-loup Gailly - * detect_data_type() function provided freely by Cosmin Truta, 2006 - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* - * ALGORITHM - * - * The "deflation" process uses several Huffman trees. The more - * common source values are represented by shorter bit sequences. - * - * Each code tree is stored in a compressed form which is itself - * a Huffman encoding of the lengths of all the code strings (in - * ascending order by source values). The actual code strings are - * reconstructed from the lengths in the inflate process, as described - * in the deflate specification. - * - * REFERENCES - * - * Deutsch, L.P.,"'Deflate' Compressed Data Format Specification". - * Available in ftp.uu.net:/pub/archiving/zip/doc/deflate-1.1.doc - * - * Storer, James A. - * Data Compression: Methods and Theory, pp. 49-50. - * Computer Science Press, 1988. ISBN 0-7167-8156-5. - * - * Sedgewick, R. - * Algorithms, p290. - * Addison-Wesley, 1983. ISBN 0-201-06672-6. - */ - -/* @(#) $Id$ */ - -/* #define GEN_TREES_H */ - -#include "deflate.h" - -#ifdef DEBUG -# include -#endif - -/* =========================================================================== - * Constants - */ - -#define MAX_BL_BITS 7 -/* Bit length codes must not exceed MAX_BL_BITS bits */ - -#define REP_3_6 16 -/* repeat previous bit length 3-6 times (2 bits of repeat count) */ - -#define REPZ_3_10 17 -/* repeat a zero length 3-10 times (3 bits of repeat count) */ - -#define REPZ_11_138 18 -/* repeat a zero length 11-138 times (7 bits of repeat count) */ - -local const int extra_lbits[LENGTH_CODES] /* extra bits for each length code */ - = {0,0,0,0,0,0,0,0,1,1,1,1,2,2,2,2,3,3,3,3,4,4,4,4,5,5,5,5,0}; - -local const int extra_dbits[D_CODES] /* extra bits for each distance code */ - = {0,0,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13}; - -local const int extra_blbits[BL_CODES] /* extra bits for each bit length code */ - = {0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,2,3,7}; - -local const unsigned char bl_order[BL_CODES] - = {16,17,18,0,8,7,9,6,10,5,11,4,12,3,13,2,14,1,15}; -/* The lengths of the bit length codes are sent in order of decreasing - * probability, to avoid transmitting the lengths for unused bit length codes. - */ - -/* =========================================================================== - * Local data. These are initialized only once. - */ - -#define DIST_CODE_LEN 512 /* see definition of array dist_code below */ - -#if defined(GEN_TREES_H) -/* non ANSI compilers may not accept trees.h */ - -ZLIB_INTERNAL ct_data static_ltree[L_CODES+2]; -/* The static literal tree. Since the bit lengths are imposed, there is no - * need for the L_CODES extra codes used during heap construction. However - * The codes 286 and 287 are needed to build a canonical tree (see _tr_init - * below). - */ - -local ct_data static_dtree[D_CODES]; -/* The static distance tree. (Actually a trivial tree since all codes use - * 5 bits.) - */ - -uch _dist_code[DIST_CODE_LEN]; -/* Distance codes. The first 256 values correspond to the distances - * 3 .. 258, the last 256 values correspond to the top 8 bits of - * the 15 bit distances. - */ - -uch _length_code[MAX_MATCH-MIN_MATCH+1]; -/* length code for each normalized match length (0 == MIN_MATCH) */ - -local int base_length[LENGTH_CODES]; -/* First normalized length for each code (0 = MIN_MATCH) */ - -local int base_dist[D_CODES]; -/* First normalized distance for each code (0 = distance of 1) */ - -#else -# include "trees.h" -#endif /* GEN_TREES_H */ - -struct static_tree_desc_s { - const ct_data *static_tree; /* static tree or NULL */ - const int *extra_bits; /* extra bits for each code or NULL */ - int extra_base; /* base index for extra_bits */ - int elems; /* max number of elements in the tree */ - unsigned int max_length; /* max bit length for the codes */ -}; - -local const static_tree_desc static_l_desc = -{static_ltree, extra_lbits, LITERALS+1, L_CODES, MAX_BITS}; - -local const static_tree_desc static_d_desc = -{static_dtree, extra_dbits, 0, D_CODES, MAX_BITS}; - -local const static_tree_desc static_bl_desc = -{(const ct_data *)0, extra_blbits, 0, BL_CODES, MAX_BL_BITS}; - -/* =========================================================================== - * Local (static) routines in this file. - */ - -local void tr_static_init (void); -local void init_block (deflate_state *s); -local void pqdownheap (deflate_state *s, ct_data *tree, int k); -local void gen_bitlen (deflate_state *s, tree_desc *desc); -local void gen_codes (ct_data *tree, int max_code, uint16_t *bl_count); -local void build_tree (deflate_state *s, tree_desc *desc); -local void scan_tree (deflate_state *s, ct_data *tree, int max_code); -local void send_tree (deflate_state *s, ct_data *tree, int max_code); -local int build_bl_tree (deflate_state *s); -local void send_all_trees (deflate_state *s, int lcodes, int dcodes, int blcodes); -local void compress_block (deflate_state *s, const ct_data *ltree, const ct_data *dtree); -local int detect_data_type (deflate_state *s); -local unsigned bi_reverse (unsigned value, int length); -local void bi_flush (deflate_state *s); -local void copy_block (deflate_state *s, char *buf, unsigned len, int header); - -#ifdef GEN_TREES_H -local void gen_trees_header(void); -#endif - -/* =========================================================================== - * Initialize the various 'constant' tables. - */ -local void tr_static_init(void) { -#if defined(GEN_TREES_H) - static int static_init_done = 0; - int n; /* iterates over tree elements */ - int bits; /* bit counter */ - int length; /* length value */ - int code; /* code value */ - int dist; /* distance index */ - uint16_t bl_count[MAX_BITS+1]; - /* number of codes at each bit length for an optimal tree */ - - if (static_init_done) - return; - - /* For some embedded targets, global variables are not initialized: */ -#ifdef NO_INIT_GLOBAL_POINTERS - static_l_desc.static_tree = static_ltree; - static_l_desc.extra_bits = extra_lbits; - static_d_desc.static_tree = static_dtree; - static_d_desc.extra_bits = extra_dbits; - static_bl_desc.extra_bits = extra_blbits; -#endif - - /* Initialize the mapping length (0..255) -> length code (0..28) */ - length = 0; - for (code = 0; code < LENGTH_CODES-1; code++) { - base_length[code] = length; - for (n = 0; n < (1 << extra_lbits[code]); n++) { - _length_code[length++] = (unsigned char)code; - } - } - Assert(length == 256, "tr_static_init: length != 256"); - /* Note that the length 255 (match length 258) can be represented - * in two different ways: code 284 + 5 bits or code 285, so we - * overwrite length_code[255] to use the best encoding: - */ - _length_code[length-1] = (unsigned char)code; - - /* Initialize the mapping dist (0..32K) -> dist code (0..29) */ - dist = 0; - for (code = 0 ; code < 16; code++) { - base_dist[code] = dist; - for (n = 0; n < (1 << extra_dbits[code]); n++) { - _dist_code[dist++] = (unsigned char)code; - } - } - Assert(dist == 256, "tr_static_init: dist != 256"); - dist >>= 7; /* from now on, all distances are divided by 128 */ - for ( ; code < D_CODES; code++) { - base_dist[code] = dist << 7; - for (n = 0; n < (1 << (extra_dbits[code]-7)); n++) { - _dist_code[256 + dist++] = (unsigned char)code; - } - } - Assert(dist == 256, "tr_static_init: 256+dist != 512"); - - /* Construct the codes of the static literal tree */ - for (bits = 0; bits <= MAX_BITS; bits++) - bl_count[bits] = 0; - n = 0; - while (n <= 143) static_ltree[n++].Len = 8, bl_count[8]++; - while (n <= 255) static_ltree[n++].Len = 9, bl_count[9]++; - while (n <= 279) static_ltree[n++].Len = 7, bl_count[7]++; - while (n <= 287) static_ltree[n++].Len = 8, bl_count[8]++; - /* Codes 286 and 287 do not exist, but we must include them in the - * tree construction to get a canonical Huffman tree (longest code - * all ones) - */ - gen_codes((ct_data *)static_ltree, L_CODES+1, bl_count); - - /* The static distance tree is trivial: */ - for (n = 0; n < D_CODES; n++) { - static_dtree[n].Len = 5; - static_dtree[n].Code = bi_reverse((unsigned)n, 5); - } - static_init_done = 1; - -# ifdef GEN_TREES_H - gen_trees_header(); -# endif -#endif /* defined(GEN_TREES_H) */ -} - -/* =========================================================================== - * Genererate the file trees.h describing the static trees. - */ -#ifdef GEN_TREES_H -# ifndef DEBUG -# include -# endif - -# define SEPARATOR(i, last, width) \ - ((i) == (last)? "\n};\n\n" : \ - ((i) % (width) == (width)-1 ? ",\n" : ", ")) - -void gen_trees_header() { - FILE *header = fopen("trees.h", "w"); - int i; - - Assert(header != NULL, "Can't open trees.h"); - fprintf(header, "/* header created automatically with -DGEN_TREES_H */\n\n"); - - fprintf(header, "ZLIB_INTERNAL const ct_data static_ltree[L_CODES+2] = {\n"); - for (i = 0; i < L_CODES+2; i++) { - fprintf(header, "{{%3u},{%3u}}%s", static_ltree[i].Code, static_ltree[i].Len, SEPARATOR(i, L_CODES+1, 5)); - } - - fprintf(header, "local const ct_data static_dtree[D_CODES] = {\n"); - for (i = 0; i < D_CODES; i++) { - fprintf(header, "{{%2u},{%2u}}%s", static_dtree[i].Code, static_dtree[i].Len, SEPARATOR(i, D_CODES-1, 5)); - } - - fprintf(header, "const unsigned char ZLIB_INTERNAL _dist_code[DIST_CODE_LEN] = {\n"); - for (i = 0; i < DIST_CODE_LEN; i++) { - fprintf(header, "%2u%s", _dist_code[i], SEPARATOR(i, DIST_CODE_LEN-1, 20)); - } - - fprintf(header, "const unsigned char ZLIB_INTERNAL _length_code[MAX_MATCH-MIN_MATCH+1]= {\n"); - for (i = 0; i < MAX_MATCH-MIN_MATCH+1; i++) { - fprintf(header, "%2u%s", _length_code[i], SEPARATOR(i, MAX_MATCH-MIN_MATCH, 20)); - } - - fprintf(header, "local const int base_length[LENGTH_CODES] = {\n"); - for (i = 0; i < LENGTH_CODES; i++) { - fprintf(header, "%d%s", base_length[i], SEPARATOR(i, LENGTH_CODES-1, 20)); - } - - fprintf(header, "local const int base_dist[D_CODES] = {\n"); - for (i = 0; i < D_CODES; i++) { - fprintf(header, "%5d%s", base_dist[i], SEPARATOR(i, D_CODES-1, 10)); - } - - fclose(header); -} -#endif /* GEN_TREES_H */ - -/* =========================================================================== - * Initialize the tree data structures for a new zlib stream. - */ -void ZLIB_INTERNAL _tr_init(deflate_state *s) { - tr_static_init(); - - s->l_desc.dyn_tree = s->dyn_ltree; - s->l_desc.stat_desc = &static_l_desc; - - s->d_desc.dyn_tree = s->dyn_dtree; - s->d_desc.stat_desc = &static_d_desc; - - s->bl_desc.dyn_tree = s->bl_tree; - s->bl_desc.stat_desc = &static_bl_desc; - - s->bi_buf = 0; - s->bi_valid = 0; -#ifdef DEBUG - s->compressed_len = 0L; - s->bits_sent = 0L; -#endif - - /* Initialize the first block of the first file: */ - init_block(s); -} - -/* =========================================================================== - * Initialize a new block. - */ -local void init_block(deflate_state *s) { - int n; /* iterates over tree elements */ - - /* Initialize the trees. */ - for (n = 0; n < L_CODES; n++) - s->dyn_ltree[n].Freq = 0; - for (n = 0; n < D_CODES; n++) - s->dyn_dtree[n].Freq = 0; - for (n = 0; n < BL_CODES; n++) - s->bl_tree[n].Freq = 0; - - s->dyn_ltree[END_BLOCK].Freq = 1; - s->opt_len = s->static_len = 0L; - s->last_lit = s->matches = 0; -} - -#define SMALLEST 1 -/* Index within the heap array of least frequent node in the Huffman tree */ - - -/* =========================================================================== - * Remove the smallest element from the heap and recreate the heap with - * one less element. Updates heap and heap_len. - */ -#define pqremove(s, tree, top) \ -{\ - top = s->heap[SMALLEST]; \ - s->heap[SMALLEST] = s->heap[s->heap_len--]; \ - pqdownheap(s, tree, SMALLEST); \ -} - -/* =========================================================================== - * Compares to subtrees, using the tree depth as tie breaker when - * the subtrees have equal frequency. This minimizes the worst case length. - */ -#define smaller(tree, n, m, depth) \ - (tree[n].Freq < tree[m].Freq || \ - (tree[n].Freq == tree[m].Freq && depth[n] <= depth[m])) - -/* =========================================================================== - * Restore the heap property by moving down the tree starting at node k, - * exchanging a node with the smallest of its two sons if necessary, stopping - * when the heap property is re-established (each father smaller than its - * two sons). - */ -local void pqdownheap(deflate_state *s, ct_data *tree, int k) { - /* tree: the tree to restore */ - /* k: node to move down */ - int v = s->heap[k]; - int j = k << 1; /* left son of k */ - while (j <= s->heap_len) { - /* Set j to the smallest of the two sons: */ - if (j < s->heap_len && smaller(tree, s->heap[j+1], s->heap[j], s->depth)) { - j++; - } - /* Exit if v is smaller than both sons */ - if (smaller(tree, v, s->heap[j], s->depth)) - break; - - /* Exchange v with the smallest son */ - s->heap[k] = s->heap[j]; - k = j; - - /* And continue down the tree, setting j to the left son of k */ - j <<= 1; - } - s->heap[k] = v; -} - -/* =========================================================================== - * Compute the optimal bit lengths for a tree and update the total bit length - * for the current block. - * IN assertion: the fields freq and dad are set, heap[heap_max] and - * above are the tree nodes sorted by increasing frequency. - * OUT assertions: the field len is set to the optimal bit length, the - * array bl_count contains the frequencies for each bit length. - * The length opt_len is updated; static_len is also updated if stree is - * not null. - */ -local void gen_bitlen(deflate_state *s, tree_desc *desc) { - /* desc: the tree descriptor */ - ct_data *tree = desc->dyn_tree; - int max_code = desc->max_code; - const ct_data *stree = desc->stat_desc->static_tree; - const int *extra = desc->stat_desc->extra_bits; - int base = desc->stat_desc->extra_base; - unsigned int max_length = desc->stat_desc->max_length; - int h; /* heap index */ - int n, m; /* iterate over the tree elements */ - unsigned int bits; /* bit length */ - int xbits; /* extra bits */ - uint16_t f; /* frequency */ - int overflow = 0; /* number of elements with bit length too large */ - - for (bits = 0; bits <= MAX_BITS; bits++) - s->bl_count[bits] = 0; - - /* In a first pass, compute the optimal bit lengths (which may - * overflow in the case of the bit length tree). - */ - tree[s->heap[s->heap_max]].Len = 0; /* root of the heap */ - - for (h = s->heap_max+1; h < HEAP_SIZE; h++) { - n = s->heap[h]; - bits = tree[tree[n].Dad].Len + 1; - if (bits > max_length) - bits = max_length, overflow++; - tree[n].Len = (uint16_t)bits; - /* We overwrite tree[n].Dad which is no longer needed */ - - if (n > max_code) /* not a leaf node */ - continue; - - s->bl_count[bits]++; - xbits = 0; - if (n >= base) - xbits = extra[n-base]; - f = tree[n].Freq; - s->opt_len += (unsigned long)f * (bits + xbits); - if (stree) - s->static_len += (unsigned long)f * (stree[n].Len + xbits); - } - if (overflow == 0) - return; - - Trace((stderr, "\nbit length overflow\n")); - /* This happens for example on obj2 and pic of the Calgary corpus */ - - /* Find the first bit length which could increase: */ - do { - bits = max_length-1; - while (s->bl_count[bits] == 0) - bits--; - s->bl_count[bits]--; /* move one leaf down the tree */ - s->bl_count[bits+1] += 2; /* move one overflow item as its brother */ - s->bl_count[max_length]--; - /* The brother of the overflow item also moves one step up, - * but this does not affect bl_count[max_length] - */ - overflow -= 2; - } while (overflow > 0); - - /* Now recompute all bit lengths, scanning in increasing frequency. - * h is still equal to HEAP_SIZE. (It is simpler to reconstruct all - * lengths instead of fixing only the wrong ones. This idea is taken - * from 'ar' written by Haruhiko Okumura.) - */ - for (bits = max_length; bits != 0; bits--) { - n = s->bl_count[bits]; - while (n != 0) { - m = s->heap[--h]; - if (m > max_code) - continue; - if (tree[m].Len != bits) { - Trace((stderr, "code %d bits %d->%u\n", m, tree[m].Len, bits)); - s->opt_len += (long)((bits - tree[m].Len) * tree[m].Freq); - tree[m].Len = (uint16_t)bits; - } - n--; - } - } -} - -/* =========================================================================== - * Generate the codes for a given tree and bit counts (which need not be - * optimal). - * IN assertion: the array bl_count contains the bit length statistics for - * the given tree and the field len is set for all tree elements. - * OUT assertion: the field code is set for all tree elements of non - * zero code length. - */ -local void gen_codes(ct_data *tree, int max_code, uint16_t *bl_count) { - /* tree: the tree to decorate */ - /* max_code: largest code with non zero frequency */ - /* bl_count: number of codes at each bit length */ - uint16_t next_code[MAX_BITS+1]; /* next code value for each bit length */ - uint16_t code = 0; /* running code value */ - int bits; /* bit index */ - int n; /* code index */ - - /* The distribution counts are first used to generate the code values - * without bit reversal. - */ - for (bits = 1; bits <= MAX_BITS; bits++) { - next_code[bits] = code = (code + bl_count[bits-1]) << 1; - } - /* Check that the bit counts in bl_count are consistent. The last code - * must be all ones. - */ - Assert(code + bl_count[MAX_BITS]-1 == (1 << MAX_BITS)-1, "inconsistent bit counts"); - Tracev((stderr, "\ngen_codes: max_code %d ", max_code)); - - for (n = 0; n <= max_code; n++) { - int len = tree[n].Len; - if (len == 0) - continue; - /* Now reverse the bits */ - tree[n].Code = bi_reverse(next_code[len]++, len); - - Tracecv(tree != static_ltree, (stderr, "\nn %3d %c l %2d c %4x (%x) ", - n, (isgraph(n) ? n : ' '), len, tree[n].Code, next_code[len]-1)); - } -} - -/* =========================================================================== - * Construct one Huffman tree and assigns the code bit strings and lengths. - * Update the total bit length for the current block. - * IN assertion: the field freq is set for all tree elements. - * OUT assertions: the fields len and code are set to the optimal bit length - * and corresponding code. The length opt_len is updated; static_len is - * also updated if stree is not null. The field max_code is set. - */ -local void build_tree(deflate_state *s, tree_desc *desc) { - /* desc: the tree descriptor */ - ct_data *tree = desc->dyn_tree; - const ct_data *stree = desc->stat_desc->static_tree; - int elems = desc->stat_desc->elems; - int n, m; /* iterate over heap elements */ - int max_code = -1; /* largest code with non zero frequency */ - int node; /* new node being created */ - - /* Construct the initial heap, with least frequent element in - * heap[SMALLEST]. The sons of heap[n] are heap[2*n] and heap[2*n+1]. - * heap[0] is not used. - */ - s->heap_len = 0, s->heap_max = HEAP_SIZE; - - for (n = 0; n < elems; n++) { - if (tree[n].Freq != 0) { - s->heap[++(s->heap_len)] = max_code = n; - s->depth[n] = 0; - } else { - tree[n].Len = 0; - } - } - - /* The pkzip format requires that at least one distance code exists, - * and that at least one bit should be sent even if there is only one - * possible code. So to avoid special checks later on we force at least - * two codes of non zero frequency. - */ - while (s->heap_len < 2) { - node = s->heap[++(s->heap_len)] = (max_code < 2 ? ++max_code : 0); - tree[node].Freq = 1; - s->depth[node] = 0; - s->opt_len--; - if (stree) - s->static_len -= stree[node].Len; - /* node is 0 or 1 so it does not have extra bits */ - } - desc->max_code = max_code; - - /* The elements heap[heap_len/2+1 .. heap_len] are leaves of the tree, - * establish sub-heaps of increasing lengths: - */ - for (n = s->heap_len/2; n >= 1; n--) - pqdownheap(s, tree, n); - - /* Construct the Huffman tree by repeatedly combining the least two - * frequent nodes. - */ - node = elems; /* next internal node of the tree */ - do { - pqremove(s, tree, n); /* n = node of least frequency */ - m = s->heap[SMALLEST]; /* m = node of next least frequency */ - - s->heap[--(s->heap_max)] = n; /* keep the nodes sorted by frequency */ - s->heap[--(s->heap_max)] = m; - - /* Create a new node father of n and m */ - tree[node].Freq = tree[n].Freq + tree[m].Freq; - s->depth[node] = (unsigned char)((s->depth[n] >= s->depth[m] ? - s->depth[n] : s->depth[m]) + 1); - tree[n].Dad = tree[m].Dad = (uint16_t)node; -#ifdef DUMP_BL_TREE - if (tree == s->bl_tree) { - fprintf(stderr, "\nnode %d(%d), sons %d(%d) %d(%d)", - node, tree[node].Freq, n, tree[n].Freq, m, tree[m].Freq); - } -#endif - /* and insert the new node in the heap */ - s->heap[SMALLEST] = node++; - pqdownheap(s, tree, SMALLEST); - } while (s->heap_len >= 2); - - s->heap[--(s->heap_max)] = s->heap[SMALLEST]; - - /* At this point, the fields freq and dad are set. We can now - * generate the bit lengths. - */ - gen_bitlen(s, (tree_desc *)desc); - - /* The field len is now set, we can generate the bit codes */ - gen_codes((ct_data *)tree, max_code, s->bl_count); -} - -/* =========================================================================== - * Scan a literal or distance tree to determine the frequencies of the codes - * in the bit length tree. - */ -local void scan_tree(deflate_state *s, ct_data *tree, int max_code) { - /* tree: the tree to be scanned */ - /* max_code: and its largest code of non zero frequency */ - int n; /* iterates over all tree elements */ - int prevlen = -1; /* last emitted length */ - int curlen; /* length of current code */ - int nextlen = tree[0].Len; /* length of next code */ - int count = 0; /* repeat count of the current code */ - int max_count = 7; /* max repeat count */ - int min_count = 4; /* min repeat count */ - - if (nextlen == 0) - max_count = 138, min_count = 3; - - tree[max_code+1].Len = (uint16_t)0xffff; /* guard */ - - for (n = 0; n <= max_code; n++) { - curlen = nextlen; - nextlen = tree[n+1].Len; - if (++count < max_count && curlen == nextlen) { - continue; - } else if (count < min_count) { - s->bl_tree[curlen].Freq += count; - } else if (curlen != 0) { - if (curlen != prevlen) - s->bl_tree[curlen].Freq++; - s->bl_tree[REP_3_6].Freq++; - } else if (count <= 10) { - s->bl_tree[REPZ_3_10].Freq++; - } else { - s->bl_tree[REPZ_11_138].Freq++; - } - count = 0; - prevlen = curlen; - if (nextlen == 0) { - max_count = 138, min_count = 3; - } else if (curlen == nextlen) { - max_count = 6, min_count = 3; - } else { - max_count = 7, min_count = 4; - } - } -} - -/* =========================================================================== - * Send a literal or distance tree in compressed form, using the codes in - * bl_tree. - */ -local void send_tree(deflate_state *s, ct_data *tree, int max_code) { - /* tree: the tree to be scanned */ - /* max_code and its largest code of non zero frequency */ - int n; /* iterates over all tree elements */ - int prevlen = -1; /* last emitted length */ - int curlen; /* length of current code */ - int nextlen = tree[0].Len; /* length of next code */ - int count = 0; /* repeat count of the current code */ - int max_count = 7; /* max repeat count */ - int min_count = 4; /* min repeat count */ - - /* tree[max_code+1].Len = -1; */ /* guard already set */ - if (nextlen == 0) - max_count = 138, min_count = 3; - - for (n = 0; n <= max_code; n++) { - curlen = nextlen; - nextlen = tree[n+1].Len; - if (++count < max_count && curlen == nextlen) { - continue; - } else if (count < min_count) { - do { - send_code(s, curlen, s->bl_tree); - } while (--count != 0); - - } else if (curlen != 0) { - if (curlen != prevlen) { - send_code(s, curlen, s->bl_tree); - count--; - } - Assert(count >= 3 && count <= 6, " 3_6?"); - send_code(s, REP_3_6, s->bl_tree); - send_bits(s, count-3, 2); - - } else if (count <= 10) { - send_code(s, REPZ_3_10, s->bl_tree); - send_bits(s, count-3, 3); - - } else { - send_code(s, REPZ_11_138, s->bl_tree); - send_bits(s, count-11, 7); - } - count = 0; - prevlen = curlen; - if (nextlen == 0) { - max_count = 138, min_count = 3; - } else if (curlen == nextlen) { - max_count = 6, min_count = 3; - } else { - max_count = 7, min_count = 4; - } - } -} - -/* =========================================================================== - * Construct the Huffman tree for the bit lengths and return the index in - * bl_order of the last bit length code to send. - */ -local int build_bl_tree(deflate_state *s) { - int max_blindex; /* index of last bit length code of non zero freq */ - - /* Determine the bit length frequencies for literal and distance trees */ - scan_tree(s, (ct_data *)s->dyn_ltree, s->l_desc.max_code); - scan_tree(s, (ct_data *)s->dyn_dtree, s->d_desc.max_code); - - /* Build the bit length tree: */ - build_tree(s, (tree_desc *)(&(s->bl_desc))); - /* opt_len now includes the length of the tree representations, except - * the lengths of the bit lengths codes and the 5+5+4 bits for the counts. - */ - - /* Determine the number of bit length codes to send. The pkzip format - * requires that at least 4 bit length codes be sent. (appnote.txt says - * 3 but the actual value used is 4.) - */ - for (max_blindex = BL_CODES-1; max_blindex >= 3; max_blindex--) { - if (s->bl_tree[bl_order[max_blindex]].Len != 0) - break; - } - /* Update opt_len to include the bit length tree and counts */ - s->opt_len += 3*(max_blindex+1) + 5+5+4; - Tracev((stderr, "\ndyn trees: dyn %lu, stat %lu", s->opt_len, s->static_len)); - - return max_blindex; -} - -/* =========================================================================== - * Send the header for a block using dynamic Huffman trees: the counts, the - * lengths of the bit length codes, the literal tree and the distance tree. - * IN assertion: lcodes >= 257, dcodes >= 1, blcodes >= 4. - */ -local void send_all_trees(deflate_state *s, int lcodes, int dcodes, int blcodes) { - int rank; /* index in bl_order */ - - Assert(lcodes >= 257 && dcodes >= 1 && blcodes >= 4, "not enough codes"); - Assert(lcodes <= L_CODES && dcodes <= D_CODES && blcodes <= BL_CODES, "too many codes"); - Tracev((stderr, "\nbl counts: ")); - send_bits(s, lcodes-257, 5); /* not +255 as stated in appnote.txt */ - send_bits(s, dcodes-1, 5); - send_bits(s, blcodes-4, 4); /* not -3 as stated in appnote.txt */ - for (rank = 0; rank < blcodes; rank++) { - Tracev((stderr, "\nbl code %2u ", bl_order[rank])); - send_bits(s, s->bl_tree[bl_order[rank]].Len, 3); - } - Tracev((stderr, "\nbl tree: sent %lu", s->bits_sent)); - - send_tree(s, (ct_data *)s->dyn_ltree, lcodes-1); /* literal tree */ - Tracev((stderr, "\nlit tree: sent %lu", s->bits_sent)); - - send_tree(s, (ct_data *)s->dyn_dtree, dcodes-1); /* distance tree */ - Tracev((stderr, "\ndist tree: sent %lu", s->bits_sent)); -} - -/* =========================================================================== - * Send a stored block - */ -void ZLIB_INTERNAL _tr_stored_block(deflate_state *s, char *buf, unsigned long stored_len, int last) { - /* buf: input block */ - /* stored_len: length of input block */ - /* last: one if this is the last block for a file */ - send_bits(s, (STORED_BLOCK << 1)+last, 3); /* send block type */ -#ifdef DEBUG - s->compressed_len = (s->compressed_len + 3 + 7) & (unsigned long)~7L; - s->compressed_len += (stored_len + 4) << 3; -#endif - copy_block(s, buf, (unsigned)stored_len, 1); /* with header */ -} - -/* =========================================================================== - * Flush the bits in the bit buffer to pending output (leaves at most 7 bits) - */ -void ZLIB_INTERNAL _tr_flush_bits(deflate_state *s) { - bi_flush(s); -} - -/* =========================================================================== - * Send one empty static block to give enough lookahead for inflate. - * This takes 10 bits, of which 7 may remain in the bit buffer. - */ -void ZLIB_INTERNAL _tr_align(deflate_state *s) { - send_bits(s, STATIC_TREES << 1, 3); - send_code(s, END_BLOCK, static_ltree); -#ifdef DEBUG - s->compressed_len += 10L; /* 3 for block type, 7 for EOB */ -#endif - bi_flush(s); -} - -/* =========================================================================== - * Determine the best encoding for the current block: dynamic trees, static - * trees or store, and output the encoded block to the zip file. - */ -void ZLIB_INTERNAL _tr_flush_block(deflate_state *s, char *buf, unsigned long stored_len, int last) { - /* buf: input block, or NULL if too old */ - /* stored_len: length of input block */ - /* last: one if this is the last block for a file */ - unsigned long opt_lenb, static_lenb; /* opt_len and static_len in bytes */ - int max_blindex = 0; /* index of last bit length code of non zero freq */ - - /* Build the Huffman trees unless a stored block is forced */ - if (s->level > 0) { - /* Check if the file is binary or text */ - if (s->strm->data_type == Z_UNKNOWN) - s->strm->data_type = detect_data_type(s); - - /* Construct the literal and distance trees */ - build_tree(s, (tree_desc *)(&(s->l_desc))); - Tracev((stderr, "\nlit data: dyn %lu, stat %lu", s->opt_len, s->static_len)); - - build_tree(s, (tree_desc *)(&(s->d_desc))); - Tracev((stderr, "\ndist data: dyn %lu, stat %lu", s->opt_len, s->static_len)); - /* At this point, opt_len and static_len are the total bit lengths of - * the compressed block data, excluding the tree representations. - */ - - /* Build the bit length tree for the above two trees, and get the index - * in bl_order of the last bit length code to send. - */ - max_blindex = build_bl_tree(s); - - /* Determine the best encoding. Compute the block lengths in bytes. */ - opt_lenb = (s->opt_len+3+7) >> 3; - static_lenb = (s->static_len+3+7) >> 3; - - Tracev((stderr, "\nopt %lu(%lu) stat %lu(%lu) stored %lu lit %u ", - opt_lenb, s->opt_len, static_lenb, s->static_len, stored_len, s->last_lit)); - - if (static_lenb <= opt_lenb) - opt_lenb = static_lenb; - - } else { - Assert(buf != NULL, "lost buf"); - opt_lenb = static_lenb = stored_len + 5; /* force a stored block */ - } - -#ifdef FORCE_STORED - if (buf != NULL) { /* force stored block */ -#else - if (stored_len+4 <= opt_lenb && buf != NULL) { - /* 4: two words for the lengths */ -#endif - /* The test buf != NULL is only necessary if LIT_BUFSIZE > WSIZE. - * Otherwise we can't have processed more than WSIZE input bytes since - * the last block flush, because compression would have been - * successful. If LIT_BUFSIZE <= WSIZE, it is never too late to - * transform a block into a stored block. - */ - _tr_stored_block(s, buf, stored_len, last); - -#ifdef FORCE_STATIC - } else if (static_lenb >= 0) { /* force static trees */ -#else - } else if (s->strategy == Z_FIXED || static_lenb == opt_lenb) { -#endif - send_bits(s, (STATIC_TREES << 1)+last, 3); - compress_block(s, (const ct_data *)static_ltree, (const ct_data *)static_dtree); -#ifdef DEBUG - s->compressed_len += 3 + s->static_len; -#endif - } else { - send_bits(s, (DYN_TREES << 1)+last, 3); - send_all_trees(s, s->l_desc.max_code+1, s->d_desc.max_code+1, max_blindex+1); - compress_block(s, (const ct_data *)s->dyn_ltree, (const ct_data *)s->dyn_dtree); -#ifdef DEBUG - s->compressed_len += 3 + s->opt_len; -#endif - } - Assert(s->compressed_len == s->bits_sent, "bad compressed size"); - /* The above check is made mod 2^32, for files larger than 512 MB - * and unsigned long implemented on 32 bits. - */ - init_block(s); - - if (last) { - bi_windup(s); -#ifdef DEBUG - s->compressed_len += 7; /* align on byte boundary */ -#endif - } - Tracev((stderr, "\ncomprlen %lu(%lu) ", s->compressed_len>>3, s->compressed_len-7*last)); -} - -/* =========================================================================== - * Save the match info and tally the frequency counts. Return true if - * the current block must be flushed. - */ -int ZLIB_INTERNAL _tr_tally(deflate_state *s, unsigned dist, unsigned lc) { - /* dist: distance of matched string */ - /* lc: match length-MIN_MATCH or unmatched char (if dist==0) */ - s->d_buf[s->last_lit] = (uint16_t)dist; - s->l_buf[s->last_lit++] = (unsigned char)lc; - if (dist == 0) { - /* lc is the unmatched char */ - s->dyn_ltree[lc].Freq++; - } else { - s->matches++; - /* Here, lc is the match length - MIN_MATCH */ - dist--; /* dist = match distance - 1 */ - Assert((uint16_t)dist < (uint16_t)MAX_DIST(s) && - (uint16_t)lc <= (uint16_t)(MAX_MATCH-MIN_MATCH) && - (uint16_t)d_code(dist) < (uint16_t)D_CODES, "_tr_tally: bad match"); - - s->dyn_ltree[_length_code[lc]+LITERALS+1].Freq++; - s->dyn_dtree[d_code(dist)].Freq++; - } - -#ifdef TRUNCATE_BLOCK - /* Try to guess if it is profitable to stop the current block here */ - if ((s->last_lit & 0x1fff) == 0 && s->level > 2) { - /* Compute an upper bound for the compressed length */ - unsigned long out_length = (unsigned long)s->last_lit*8L; - unsigned long in_length = (unsigned long)((long)s->strstart - s->block_start); - int dcode; - for (dcode = 0; dcode < D_CODES; dcode++) { - out_length += (unsigned long)s->dyn_dtree[dcode].Freq * (5L+extra_dbits[dcode]); - } - out_length >>= 3; - Tracev((stderr, "\nlast_lit %u, in %ld, out ~%ld(%ld%%) ", - s->last_lit, in_length, out_length, 100L - out_length*100L/in_length)); - if (s->matches < s->last_lit/2 && out_length < in_length/2) - return 1; - } -#endif - return (s->last_lit == s->lit_bufsize-1); - /* We avoid equality with lit_bufsize because of wraparound at 64K - * on 16 bit machines and because stored blocks are restricted to - * 64K-1 bytes. - */ -} - -/* =========================================================================== - * Send the block data compressed using the given Huffman trees - */ -local void compress_block(deflate_state *s, const ct_data *ltree, const ct_data *dtree) { - /* ltree: literal tree */ - /* dtree: distance tree */ - unsigned dist; /* distance of matched string */ - int lc; /* match length or unmatched char (if dist == 0) */ - unsigned lx = 0; /* running index in l_buf */ - unsigned code; /* the code to send */ - int extra; /* number of extra bits to send */ - - if (s->last_lit != 0) { - do { - dist = s->d_buf[lx]; - lc = s->l_buf[lx++]; - if (dist == 0) { - send_code(s, lc, ltree); /* send a literal byte */ - Tracecv(isgraph(lc), (stderr, " '%c' ", lc)); - } else { - /* Here, lc is the match length - MIN_MATCH */ - code = _length_code[lc]; - send_code(s, code+LITERALS+1, ltree); /* send the length code */ - extra = extra_lbits[code]; - if (extra != 0) { - lc -= base_length[code]; - send_bits(s, lc, extra); /* send the extra length bits */ - } - dist--; /* dist is now the match distance - 1 */ - code = d_code(dist); - Assert(code < D_CODES, "bad d_code"); - - send_code(s, code, dtree); /* send the distance code */ - extra = extra_dbits[code]; - if (extra != 0) { - dist -= base_dist[code]; - send_bits(s, dist, extra); /* send the extra distance bits */ - } - } /* literal or match pair ? */ - - /* Check that the overlay between pending_buf and d_buf+l_buf is ok: */ - Assert((unsigned int)(s->pending) < s->lit_bufsize + 2*lx, "pendingBuf overflow"); - } while (lx < s->last_lit); - } - - send_code(s, END_BLOCK, ltree); -} - -/* =========================================================================== - * Check if the data type is TEXT or BINARY, using the following algorithm: - * - TEXT if the two conditions below are satisfied: - * a) There are no non-portable control characters belonging to the - * "black list" (0..6, 14..25, 28..31). - * b) There is at least one printable character belonging to the - * "white list" (9 {TAB}, 10 {LF}, 13 {CR}, 32..255). - * - BINARY otherwise. - * - The following partially-portable control characters form a - * "gray list" that is ignored in this detection algorithm: - * (7 {BEL}, 8 {BS}, 11 {VT}, 12 {FF}, 26 {SUB}, 27 {ESC}). - * IN assertion: the fields Freq of dyn_ltree are set. - */ -local int detect_data_type(deflate_state *s) { - /* black_mask is the bit mask of black-listed bytes - * set bits 0..6, 14..25, and 28..31 - * 0xf3ffc07f = binary 11110011111111111100000001111111 - */ - unsigned long black_mask = 0xf3ffc07fUL; - int n; - - /* Check for non-textual ("black-listed") bytes. */ - for (n = 0; n <= 31; n++, black_mask >>= 1) - if ((black_mask & 1) && (s->dyn_ltree[n].Freq != 0)) - return Z_BINARY; - - /* Check for textual ("white-listed") bytes. */ - if (s->dyn_ltree[9].Freq != 0 || s->dyn_ltree[10].Freq != 0 || s->dyn_ltree[13].Freq != 0) - return Z_TEXT; - for (n = 32; n < LITERALS; n++) - if (s->dyn_ltree[n].Freq != 0) - return Z_TEXT; - - /* There are no "black-listed" or "white-listed" bytes: - * this stream either is empty or has tolerated ("gray-listed") bytes only. - */ - return Z_BINARY; -} - -/* =========================================================================== - * Reverse the first len bits of a code, using straightforward code (a faster - * method would use a table) - * IN assertion: 1 <= len <= 15 - */ -local unsigned bi_reverse(unsigned code, int len) { - /* code: the value to invert */ - /* len: its bit length */ - register unsigned res = 0; - do { - res |= code & 1; - code >>= 1, res <<= 1; - } while (--len > 0); - return res >> 1; -} - -/* =========================================================================== - * Flush the bit buffer, keeping at most 7 bits in it. - */ -local void bi_flush(deflate_state *s) { - if (s->bi_valid == 16) { - put_short(s, s->bi_buf); - s->bi_buf = 0; - s->bi_valid = 0; - } else if (s->bi_valid >= 8) { - put_byte(s, (unsigned char)s->bi_buf); - s->bi_buf >>= 8; - s->bi_valid -= 8; - } -} - -/* =========================================================================== - * Flush the bit buffer and align the output on a byte boundary - */ -ZLIB_INTERNAL void bi_windup(deflate_state *s) { - if (s->bi_valid > 8) { - put_short(s, s->bi_buf); - } else if (s->bi_valid > 0) { - put_byte(s, (unsigned char)s->bi_buf); - } - s->bi_buf = 0; - s->bi_valid = 0; -#ifdef DEBUG - s->bits_sent = (s->bits_sent+7) & ~7; -#endif -} - -/* =========================================================================== - * Copy a stored block, storing first the length and its - * one's complement if requested. - */ -local void copy_block(deflate_state *s, char *buf, unsigned len, int header) { - /* buf: the input data */ - /* len: its length */ - /* header: true if block header must be written */ - bi_windup(s); /* align on byte boundary */ - - if (header) { - put_short(s, (uint16_t)len); - put_short(s, (uint16_t)~len); -#ifdef DEBUG - s->bits_sent += 2*16; -#endif - } -#ifdef DEBUG - s->bits_sent += (unsigned long)len << 3; -#endif - while (len--) { - put_byte(s, *buf++); - } -} - diff --git a/contrib/libzlib-ng/trees.h b/contrib/libzlib-ng/trees.h deleted file mode 100644 index debfbdd8584..00000000000 --- a/contrib/libzlib-ng/trees.h +++ /dev/null @@ -1,132 +0,0 @@ -#ifndef TREES_H_ -#define TREES_H_ - -/* header created automatically with -DGEN_TREES_H */ - -ZLIB_INTERNAL const ct_data static_ltree[L_CODES+2] = { -{{ 12},{ 8}}, {{140},{ 8}}, {{ 76},{ 8}}, {{204},{ 8}}, {{ 44},{ 8}}, -{{172},{ 8}}, {{108},{ 8}}, {{236},{ 8}}, {{ 28},{ 8}}, {{156},{ 8}}, -{{ 92},{ 8}}, {{220},{ 8}}, {{ 60},{ 8}}, {{188},{ 8}}, {{124},{ 8}}, -{{252},{ 8}}, {{ 2},{ 8}}, {{130},{ 8}}, {{ 66},{ 8}}, {{194},{ 8}}, -{{ 34},{ 8}}, {{162},{ 8}}, {{ 98},{ 8}}, {{226},{ 8}}, {{ 18},{ 8}}, -{{146},{ 8}}, {{ 82},{ 8}}, {{210},{ 8}}, {{ 50},{ 8}}, {{178},{ 8}}, -{{114},{ 8}}, {{242},{ 8}}, {{ 10},{ 8}}, {{138},{ 8}}, {{ 74},{ 8}}, -{{202},{ 8}}, {{ 42},{ 8}}, {{170},{ 8}}, {{106},{ 8}}, {{234},{ 8}}, -{{ 26},{ 8}}, {{154},{ 8}}, {{ 90},{ 8}}, {{218},{ 8}}, {{ 58},{ 8}}, -{{186},{ 8}}, {{122},{ 8}}, {{250},{ 8}}, {{ 6},{ 8}}, {{134},{ 8}}, -{{ 70},{ 8}}, {{198},{ 8}}, {{ 38},{ 8}}, {{166},{ 8}}, {{102},{ 8}}, -{{230},{ 8}}, {{ 22},{ 8}}, {{150},{ 8}}, {{ 86},{ 8}}, {{214},{ 8}}, -{{ 54},{ 8}}, {{182},{ 8}}, {{118},{ 8}}, {{246},{ 8}}, {{ 14},{ 8}}, -{{142},{ 8}}, {{ 78},{ 8}}, {{206},{ 8}}, {{ 46},{ 8}}, {{174},{ 8}}, -{{110},{ 8}}, {{238},{ 8}}, {{ 30},{ 8}}, {{158},{ 8}}, {{ 94},{ 8}}, -{{222},{ 8}}, {{ 62},{ 8}}, {{190},{ 8}}, {{126},{ 8}}, {{254},{ 8}}, -{{ 1},{ 8}}, {{129},{ 8}}, {{ 65},{ 8}}, {{193},{ 8}}, {{ 33},{ 8}}, -{{161},{ 8}}, {{ 97},{ 8}}, {{225},{ 8}}, {{ 17},{ 8}}, {{145},{ 8}}, -{{ 81},{ 8}}, {{209},{ 8}}, {{ 49},{ 8}}, {{177},{ 8}}, {{113},{ 8}}, -{{241},{ 8}}, {{ 9},{ 8}}, {{137},{ 8}}, {{ 73},{ 8}}, {{201},{ 8}}, -{{ 41},{ 8}}, {{169},{ 8}}, {{105},{ 8}}, {{233},{ 8}}, {{ 25},{ 8}}, -{{153},{ 8}}, {{ 89},{ 8}}, {{217},{ 8}}, {{ 57},{ 8}}, {{185},{ 8}}, -{{121},{ 8}}, {{249},{ 8}}, {{ 5},{ 8}}, {{133},{ 8}}, {{ 69},{ 8}}, -{{197},{ 8}}, {{ 37},{ 8}}, {{165},{ 8}}, {{101},{ 8}}, {{229},{ 8}}, -{{ 21},{ 8}}, {{149},{ 8}}, {{ 85},{ 8}}, {{213},{ 8}}, {{ 53},{ 8}}, -{{181},{ 8}}, {{117},{ 8}}, {{245},{ 8}}, {{ 13},{ 8}}, {{141},{ 8}}, -{{ 77},{ 8}}, {{205},{ 8}}, {{ 45},{ 8}}, {{173},{ 8}}, {{109},{ 8}}, -{{237},{ 8}}, {{ 29},{ 8}}, {{157},{ 8}}, {{ 93},{ 8}}, {{221},{ 8}}, -{{ 61},{ 8}}, {{189},{ 8}}, {{125},{ 8}}, {{253},{ 8}}, {{ 19},{ 9}}, -{{275},{ 9}}, {{147},{ 9}}, {{403},{ 9}}, {{ 83},{ 9}}, {{339},{ 9}}, -{{211},{ 9}}, {{467},{ 9}}, {{ 51},{ 9}}, {{307},{ 9}}, {{179},{ 9}}, -{{435},{ 9}}, {{115},{ 9}}, {{371},{ 9}}, {{243},{ 9}}, {{499},{ 9}}, -{{ 11},{ 9}}, {{267},{ 9}}, {{139},{ 9}}, {{395},{ 9}}, {{ 75},{ 9}}, -{{331},{ 9}}, {{203},{ 9}}, {{459},{ 9}}, {{ 43},{ 9}}, {{299},{ 9}}, -{{171},{ 9}}, {{427},{ 9}}, {{107},{ 9}}, {{363},{ 9}}, {{235},{ 9}}, -{{491},{ 9}}, {{ 27},{ 9}}, {{283},{ 9}}, {{155},{ 9}}, {{411},{ 9}}, -{{ 91},{ 9}}, {{347},{ 9}}, {{219},{ 9}}, {{475},{ 9}}, {{ 59},{ 9}}, -{{315},{ 9}}, {{187},{ 9}}, {{443},{ 9}}, {{123},{ 9}}, {{379},{ 9}}, -{{251},{ 9}}, {{507},{ 9}}, {{ 7},{ 9}}, {{263},{ 9}}, {{135},{ 9}}, -{{391},{ 9}}, {{ 71},{ 9}}, {{327},{ 9}}, {{199},{ 9}}, {{455},{ 9}}, -{{ 39},{ 9}}, {{295},{ 9}}, {{167},{ 9}}, {{423},{ 9}}, {{103},{ 9}}, -{{359},{ 9}}, {{231},{ 9}}, {{487},{ 9}}, {{ 23},{ 9}}, {{279},{ 9}}, -{{151},{ 9}}, {{407},{ 9}}, {{ 87},{ 9}}, {{343},{ 9}}, {{215},{ 9}}, -{{471},{ 9}}, {{ 55},{ 9}}, {{311},{ 9}}, {{183},{ 9}}, {{439},{ 9}}, -{{119},{ 9}}, {{375},{ 9}}, {{247},{ 9}}, {{503},{ 9}}, {{ 15},{ 9}}, -{{271},{ 9}}, {{143},{ 9}}, {{399},{ 9}}, {{ 79},{ 9}}, {{335},{ 9}}, -{{207},{ 9}}, {{463},{ 9}}, {{ 47},{ 9}}, {{303},{ 9}}, {{175},{ 9}}, -{{431},{ 9}}, {{111},{ 9}}, {{367},{ 9}}, {{239},{ 9}}, {{495},{ 9}}, -{{ 31},{ 9}}, {{287},{ 9}}, {{159},{ 9}}, {{415},{ 9}}, {{ 95},{ 9}}, -{{351},{ 9}}, {{223},{ 9}}, {{479},{ 9}}, {{ 63},{ 9}}, {{319},{ 9}}, -{{191},{ 9}}, {{447},{ 9}}, {{127},{ 9}}, {{383},{ 9}}, {{255},{ 9}}, -{{511},{ 9}}, {{ 0},{ 7}}, {{ 64},{ 7}}, {{ 32},{ 7}}, {{ 96},{ 7}}, -{{ 16},{ 7}}, {{ 80},{ 7}}, {{ 48},{ 7}}, {{112},{ 7}}, {{ 8},{ 7}}, -{{ 72},{ 7}}, {{ 40},{ 7}}, {{104},{ 7}}, {{ 24},{ 7}}, {{ 88},{ 7}}, -{{ 56},{ 7}}, {{120},{ 7}}, {{ 4},{ 7}}, {{ 68},{ 7}}, {{ 36},{ 7}}, -{{100},{ 7}}, {{ 20},{ 7}}, {{ 84},{ 7}}, {{ 52},{ 7}}, {{116},{ 7}}, -{{ 3},{ 8}}, {{131},{ 8}}, {{ 67},{ 8}}, {{195},{ 8}}, {{ 35},{ 8}}, -{{163},{ 8}}, {{ 99},{ 8}}, {{227},{ 8}} -}; - -local const ct_data static_dtree[D_CODES] = { -{{ 0},{ 5}}, {{16},{ 5}}, {{ 8},{ 5}}, {{24},{ 5}}, {{ 4},{ 5}}, -{{20},{ 5}}, {{12},{ 5}}, {{28},{ 5}}, {{ 2},{ 5}}, {{18},{ 5}}, -{{10},{ 5}}, {{26},{ 5}}, {{ 6},{ 5}}, {{22},{ 5}}, {{14},{ 5}}, -{{30},{ 5}}, {{ 1},{ 5}}, {{17},{ 5}}, {{ 9},{ 5}}, {{25},{ 5}}, -{{ 5},{ 5}}, {{21},{ 5}}, {{13},{ 5}}, {{29},{ 5}}, {{ 3},{ 5}}, -{{19},{ 5}}, {{11},{ 5}}, {{27},{ 5}}, {{ 7},{ 5}}, {{23},{ 5}} -}; - -const unsigned char ZLIB_INTERNAL _dist_code[DIST_CODE_LEN] = { - 0, 1, 2, 3, 4, 4, 5, 5, 6, 6, 6, 6, 7, 7, 7, 7, 8, 8, 8, 8, - 8, 8, 8, 8, 9, 9, 9, 9, 9, 9, 9, 9, 10, 10, 10, 10, 10, 10, 10, 10, -10, 10, 10, 10, 10, 10, 10, 10, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, -11, 11, 11, 11, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, -12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 13, 13, 13, 13, -13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, -13, 13, 13, 13, 13, 13, 13, 13, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, -14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, -14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, -14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 15, 15, 15, 15, 15, 15, 15, 15, -15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, -15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, -15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 0, 0, 16, 17, -18, 18, 19, 19, 20, 20, 20, 20, 21, 21, 21, 21, 22, 22, 22, 22, 22, 22, 22, 22, -23, 23, 23, 23, 23, 23, 23, 23, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, -24, 24, 24, 24, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, -26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, -26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 27, 27, 27, 27, 27, 27, 27, 27, -27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, -27, 27, 27, 27, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, -28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, -28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, -28, 28, 28, 28, 28, 28, 28, 28, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, -29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, -29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, -29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29 -}; - -const unsigned char ZLIB_INTERNAL _length_code[MAX_MATCH-MIN_MATCH+1]= { - 0, 1, 2, 3, 4, 5, 6, 7, 8, 8, 9, 9, 10, 10, 11, 11, 12, 12, 12, 12, -13, 13, 13, 13, 14, 14, 14, 14, 15, 15, 15, 15, 16, 16, 16, 16, 16, 16, 16, 16, -17, 17, 17, 17, 17, 17, 17, 17, 18, 18, 18, 18, 18, 18, 18, 18, 19, 19, 19, 19, -19, 19, 19, 19, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, -21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 22, 22, 22, 22, -22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 23, 23, 23, 23, 23, 23, 23, 23, -23, 23, 23, 23, 23, 23, 23, 23, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, -24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, -25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, -25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 26, 26, 26, 26, 26, 26, 26, 26, -26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, -26, 26, 26, 26, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, -27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 28 -}; - -local const int base_length[LENGTH_CODES] = { -0, 1, 2, 3, 4, 5, 6, 7, 8, 10, 12, 14, 16, 20, 24, 28, 32, 40, 48, 56, -64, 80, 96, 112, 128, 160, 192, 224, 0 -}; - -local const int base_dist[D_CODES] = { - 0, 1, 2, 3, 4, 6, 8, 12, 16, 24, - 32, 48, 64, 96, 128, 192, 256, 384, 512, 768, - 1024, 1536, 2048, 3072, 4096, 6144, 8192, 12288, 16384, 24576 -}; - -#endif /* TREES_H_ */ diff --git a/contrib/libzlib-ng/uncompr.c b/contrib/libzlib-ng/uncompr.c deleted file mode 100644 index c2af140db1e..00000000000 --- a/contrib/libzlib-ng/uncompr.c +++ /dev/null @@ -1,75 +0,0 @@ -/* uncompr.c -- decompress a memory buffer - * Copyright (C) 1995-2003, 2010, 2014 Jean-loup Gailly, Mark Adler. - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* @(#) $Id$ */ - -#define ZLIB_INTERNAL -#include "zlib.h" - -/* =========================================================================== - Decompresses the source buffer into the destination buffer. sourceLen is - the byte length of the source buffer. Upon entry, destLen is the total - size of the destination buffer, which must be large enough to hold the - entire uncompressed data. (The size of the uncompressed data must have - been saved previously by the compressor and transmitted to the decompressor - by some mechanism outside the scope of this compression library.) - Upon exit, destLen is the actual size of the compressed buffer. - - uncompress returns Z_OK if success, Z_MEM_ERROR if there was not - enough memory, Z_BUF_ERROR if there was not enough room in the output - buffer, or Z_DATA_ERROR if the input data was corrupted, including if the - input data is an incomplete zlib stream. -*/ -int ZEXPORT uncompress(unsigned char *dest, size_t *destLen, const unsigned char *source, size_t sourceLen) { - z_stream stream; - int err; - const unsigned int max = (unsigned int)0 - 1; - size_t left; - unsigned char buf[1]; /* for detection of incomplete stream when *destLen == 0 */ - - if (*destLen) { - left = *destLen; - *destLen = 0; - } - else { - left = 1; - dest = buf; - } - - stream.next_in = (const unsigned char *)source; - stream.avail_in = 0; - stream.zalloc = (alloc_func)0; - stream.zfree = (free_func)0; - stream.opaque = NULL; - - err = inflateInit(&stream); - if (err != Z_OK) return err; - - stream.next_out = dest; - stream.avail_out = 0; - - do { - if (stream.avail_out == 0) { - stream.avail_out = left > (unsigned long)max ? max : (unsigned int)left; - left -= stream.avail_out; - } - if (stream.avail_in == 0) { - stream.avail_in = sourceLen > (unsigned long)max ? max : (unsigned int)sourceLen; - sourceLen -= stream.avail_in; - } - err = inflate(&stream, Z_NO_FLUSH); - } while (err == Z_OK); - - if (dest != buf) - *destLen = stream.total_out; - else if (stream.total_out && err == Z_BUF_ERROR) - left = 1; - - inflateEnd(&stream); - return err == Z_STREAM_END ? Z_OK : - err == Z_NEED_DICT ? Z_DATA_ERROR : - err == Z_BUF_ERROR && left + stream.avail_out ? Z_DATA_ERROR : - err; -} diff --git a/contrib/libzlib-ng/win32/DLL_FAQ.txt b/contrib/libzlib-ng/win32/DLL_FAQ.txt deleted file mode 100644 index 12c009018c3..00000000000 --- a/contrib/libzlib-ng/win32/DLL_FAQ.txt +++ /dev/null @@ -1,397 +0,0 @@ - - Frequently Asked Questions about ZLIB1.DLL - - -This document describes the design, the rationale, and the usage -of the official DLL build of zlib, named ZLIB1.DLL. If you have -general questions about zlib, you should see the file "FAQ" found -in the zlib distribution, or at the following location: - http://www.gzip.org/zlib/zlib_faq.html - - - 1. What is ZLIB1.DLL, and how can I get it? - - - ZLIB1.DLL is the official build of zlib as a DLL. - (Please remark the character '1' in the name.) - - Pointers to a precompiled ZLIB1.DLL can be found in the zlib - web site at: - http://www.zlib.net/ - - Applications that link to ZLIB1.DLL can rely on the following - specification: - - * The exported symbols are exclusively defined in the source - files "zlib.h" and "zlib.def", found in an official zlib - source distribution. - * The symbols are exported by name, not by ordinal. - * The exported names are undecorated. - * The calling convention of functions is "C" (CDECL). - * The ZLIB1.DLL binary is linked to MSVCRT.DLL. - - The archive in which ZLIB1.DLL is bundled contains compiled - test programs that must run with a valid build of ZLIB1.DLL. - It is recommended to download the prebuilt DLL from the zlib - web site, instead of building it yourself, to avoid potential - incompatibilities that could be introduced by your compiler - and build settings. If you do build the DLL yourself, please - make sure that it complies with all the above requirements, - and it runs with the precompiled test programs, bundled with - the original ZLIB1.DLL distribution. - - If, for any reason, you need to build an incompatible DLL, - please use a different file name. - - - 2. Why did you change the name of the DLL to ZLIB1.DLL? - What happened to the old ZLIB.DLL? - - - The old ZLIB.DLL, built from zlib-1.1.4 or earlier, required - compilation settings that were incompatible to those used by - a static build. The DLL settings were supposed to be enabled - by defining the macro ZLIB_DLL, before including "zlib.h". - Incorrect handling of this macro was silently accepted at - build time, resulting in two major problems: - - * ZLIB_DLL was missing from the old makefile. When building - the DLL, not all people added it to the build options. In - consequence, incompatible incarnations of ZLIB.DLL started - to circulate around the net. - - * When switching from using the static library to using the - DLL, applications had to define the ZLIB_DLL macro and - to recompile all the sources that contained calls to zlib - functions. Failure to do so resulted in creating binaries - that were unable to run with the official ZLIB.DLL build. - - The only possible solution that we could foresee was to make - a binary-incompatible change in the DLL interface, in order to - remove the dependency on the ZLIB_DLL macro, and to release - the new DLL under a different name. - - We chose the name ZLIB1.DLL, where '1' indicates the major - zlib version number. We hope that we will not have to break - the binary compatibility again, at least not as long as the - zlib-1.x series will last. - - There is still a ZLIB_DLL macro, that can trigger a more - efficient build and use of the DLL, but compatibility no - longer dependents on it. - - - 3. Can I build ZLIB.DLL from the new zlib sources, and replace - an old ZLIB.DLL, that was built from zlib-1.1.4 or earlier? - - - In principle, you can do it by assigning calling convention - keywords to the macros ZEXPORT and ZEXPORTVA. In practice, - it depends on what you mean by "an old ZLIB.DLL", because the - old DLL exists in several mutually-incompatible versions. - You have to find out first what kind of calling convention is - being used in your particular ZLIB.DLL build, and to use the - same one in the new build. If you don't know what this is all - about, you might be better off if you would just leave the old - DLL intact. - - - 4. Can I compile my application using the new zlib interface, and - link it to an old ZLIB.DLL, that was built from zlib-1.1.4 or - earlier? - - - The official answer is "no"; the real answer depends again on - what kind of ZLIB.DLL you have. Even if you are lucky, this - course of action is unreliable. - - If you rebuild your application and you intend to use a newer - version of zlib (post- 1.1.4), it is strongly recommended to - link it to the new ZLIB1.DLL. - - - 5. Why are the zlib symbols exported by name, and not by ordinal? - - - Although exporting symbols by ordinal is a little faster, it - is risky. Any single glitch in the maintenance or use of the - DEF file that contains the ordinals can result in incompatible - builds and frustrating crashes. Simply put, the benefits of - exporting symbols by ordinal do not justify the risks. - - Technically, it should be possible to maintain ordinals in - the DEF file, and still export the symbols by name. Ordinals - exist in every DLL, and even if the dynamic linking performed - at the DLL startup is searching for names, ordinals serve as - hints, for a faster name lookup. However, if the DEF file - contains ordinals, the Microsoft linker automatically builds - an implib that will cause the executables linked to it to use - those ordinals, and not the names. It is interesting to - notice that the GNU linker for Win32 does not suffer from this - problem. - - It is possible to avoid the DEF file if the exported symbols - are accompanied by a "__declspec(dllexport)" attribute in the - source files. You can do this in zlib by predefining the - ZLIB_DLL macro. - - - 6. I see that the ZLIB1.DLL functions use the "C" (CDECL) calling - convention. Why not use the STDCALL convention? - STDCALL is the standard convention in Win32, and I need it in - my Visual Basic project! - - (For readability, we use CDECL to refer to the convention - triggered by the "__cdecl" keyword, STDCALL to refer to - the convention triggered by "__stdcall", and FASTCALL to - refer to the convention triggered by "__fastcall".) - - - Most of the native Windows API functions (without varargs) use - indeed the WINAPI convention (which translates to STDCALL in - Win32), but the standard C functions use CDECL. If a user - application is intrinsically tied to the Windows API (e.g. - it calls native Windows API functions such as CreateFile()), - sometimes it makes sense to decorate its own functions with - WINAPI. But if ANSI C or POSIX portability is a goal (e.g. - it calls standard C functions such as fopen()), it is not a - sound decision to request the inclusion of , or to - use non-ANSI constructs, for the sole purpose to make the user - functions STDCALL-able. - - The functionality offered by zlib is not in the category of - "Windows functionality", but is more like "C functionality". - - Technically, STDCALL is not bad; in fact, it is slightly - faster than CDECL, and it works with variable-argument - functions, just like CDECL. It is unfortunate that, in spite - of using STDCALL in the Windows API, it is not the default - convention used by the C compilers that run under Windows. - The roots of the problem reside deep inside the unsafety of - the K&R-style function prototypes, where the argument types - are not specified; but that is another story for another day. - - The remaining fact is that CDECL is the default convention. - Even if an explicit convention is hard-coded into the function - prototypes inside C headers, problems may appear. The - necessity to expose the convention in users' callbacks is one - of these problems. - - The calling convention issues are also important when using - zlib in other programming languages. Some of them, like Ada - (GNAT) and Fortran (GNU G77), have C bindings implemented - initially on Unix, and relying on the C calling convention. - On the other hand, the pre- .NET versions of Microsoft Visual - Basic require STDCALL, while Borland Delphi prefers, although - it does not require, FASTCALL. - - In fairness to all possible uses of zlib outside the C - programming language, we choose the default "C" convention. - Anyone interested in different bindings or conventions is - encouraged to maintain specialized projects. The "contrib/" - directory from the zlib distribution already holds a couple - of foreign bindings, such as Ada, C++, and Delphi. - - - 7. I need a DLL for my Visual Basic project. What can I do? - - - Define the ZLIB_WINAPI macro before including "zlib.h", when - building both the DLL and the user application (except that - you don't need to define anything when using the DLL in Visual - Basic). The ZLIB_WINAPI macro will switch on the WINAPI - (STDCALL) convention. The name of this DLL must be different - than the official ZLIB1.DLL. - - Gilles Vollant has contributed a build named ZLIBWAPI.DLL, - with the ZLIB_WINAPI macro turned on, and with the minizip - functionality built in. For more information, please read - the notes inside "contrib/vstudio/readme.txt", found in the - zlib distribution. - - - 8. I need to use zlib in my Microsoft .NET project. What can I - do? - - - Henrik Ravn has contributed a .NET wrapper around zlib. Look - into contrib/dotzlib/, inside the zlib distribution. - - - 9. If my application uses ZLIB1.DLL, should I link it to - MSVCRT.DLL? Why? - - - It is not required, but it is recommended to link your - application to MSVCRT.DLL, if it uses ZLIB1.DLL. - - The executables (.EXE, .DLL, etc.) that are involved in the - same process and are using the C run-time library (i.e. they - are calling standard C functions), must link to the same - library. There are several libraries in the Win32 system: - CRTDLL.DLL, MSVCRT.DLL, the static C libraries, etc. - Since ZLIB1.DLL is linked to MSVCRT.DLL, the executables that - depend on it should also be linked to MSVCRT.DLL. - - -10. Why are you saying that ZLIB1.DLL and my application should - be linked to the same C run-time (CRT) library? I linked my - application and my DLLs to different C libraries (e.g. my - application to a static library, and my DLLs to MSVCRT.DLL), - and everything works fine. - - - If a user library invokes only pure Win32 API (accessible via - and the related headers), its DLL build will work - in any context. But if this library invokes standard C API, - things get more complicated. - - There is a single Win32 library in a Win32 system. Every - function in this library resides in a single DLL module, that - is safe to call from anywhere. On the other hand, there are - multiple versions of the C library, and each of them has its - own separate internal state. Standalone executables and user - DLLs that call standard C functions must link to a C run-time - (CRT) library, be it static or shared (DLL). Intermixing - occurs when an executable (not necessarily standalone) and a - DLL are linked to different CRTs, and both are running in the - same process. - - Intermixing multiple CRTs is possible, as long as their - internal states are kept intact. The Microsoft Knowledge Base - articles KB94248 "HOWTO: Use the C Run-Time" and KB140584 - "HOWTO: Link with the Correct C Run-Time (CRT) Library" - mention the potential problems raised by intermixing. - - If intermixing works for you, it's because your application - and DLLs are avoiding the corruption of each of the CRTs' - internal states, maybe by careful design, or maybe by fortune. - - Also note that linking ZLIB1.DLL to non-Microsoft CRTs, such - as those provided by Borland, raises similar problems. - - -11. Why are you linking ZLIB1.DLL to MSVCRT.DLL? - - - MSVCRT.DLL exists on every Windows 95 with a new service pack - installed, or with Microsoft Internet Explorer 4 or later, and - on all other Windows 4.x or later (Windows 98, Windows NT 4, - or later). It is freely distributable; if not present in the - system, it can be downloaded from Microsoft or from other - software provider for free. - - The fact that MSVCRT.DLL does not exist on a virgin Windows 95 - is not so problematic. Windows 95 is scarcely found nowadays, - Microsoft ended its support a long time ago, and many recent - applications from various vendors, including Microsoft, do not - even run on it. Furthermore, no serious user should run - Windows 95 without a proper update installed. - - -12. Why are you not linking ZLIB1.DLL to - <> ? - - - We considered and abandoned the following alternatives: - - * Linking ZLIB1.DLL to a static C library (LIBC.LIB, or - LIBCMT.LIB) is not a good option. People are using the DLL - mainly to save disk space. If you are linking your program - to a static C library, you may as well consider linking zlib - in statically, too. - - * Linking ZLIB1.DLL to CRTDLL.DLL looks appealing, because - CRTDLL.DLL is present on every Win32 installation. - Unfortunately, it has a series of problems: it does not - work properly with Microsoft's C++ libraries, it does not - provide support for 64-bit file offsets, (and so on...), - and Microsoft discontinued its support a long time ago. - - * Linking ZLIB1.DLL to MSVCR70.DLL or MSVCR71.DLL, supplied - with the Microsoft .NET platform, and Visual C++ 7.0/7.1, - raises problems related to the status of ZLIB1.DLL as a - system component. According to the Microsoft Knowledge Base - article KB326922 "INFO: Redistribution of the Shared C - Runtime Component in Visual C++ .NET", MSVCR70.DLL and - MSVCR71.DLL are not supposed to function as system DLLs, - because they may clash with MSVCRT.DLL. Instead, the - application's installer is supposed to put these DLLs - (if needed) in the application's private directory. - If ZLIB1.DLL depends on a non-system runtime, it cannot - function as a redistributable system component. - - * Linking ZLIB1.DLL to non-Microsoft runtimes, such as - Borland's, or Cygwin's, raises problems related to the - reliable presence of these runtimes on Win32 systems. - It's easier to let the DLL build of zlib up to the people - who distribute these runtimes, and who may proceed as - explained in the answer to Question 14. - - -13. If ZLIB1.DLL cannot be linked to MSVCR70.DLL or MSVCR71.DLL, - how can I build/use ZLIB1.DLL in Microsoft Visual C++ 7.0 - (Visual Studio .NET) or newer? - - - Due to the problems explained in the Microsoft Knowledge Base - article KB326922 (see the previous answer), the C runtime that - comes with the VC7 environment is no longer considered a - system component. That is, it should not be assumed that this - runtime exists, or may be installed in a system directory. - Since ZLIB1.DLL is supposed to be a system component, it may - not depend on a non-system component. - - In order to link ZLIB1.DLL and your application to MSVCRT.DLL - in VC7, you need the library of Visual C++ 6.0 or older. If - you don't have this library at hand, it's probably best not to - use ZLIB1.DLL. - - We are hoping that, in the future, Microsoft will provide a - way to build applications linked to a proper system runtime, - from the Visual C++ environment. Until then, you have a - couple of alternatives, such as linking zlib in statically. - If your application requires dynamic linking, you may proceed - as explained in the answer to Question 14. - - -14. I need to link my own DLL build to a CRT different than - MSVCRT.DLL. What can I do? - - - Feel free to rebuild the DLL from the zlib sources, and link - it the way you want. You should, however, clearly state that - your build is unofficial. You should give it a different file - name, and/or install it in a private directory that can be - accessed by your application only, and is not visible to the - others (i.e. it's neither in the PATH, nor in the SYSTEM or - SYSTEM32 directories). Otherwise, your build may clash with - applications that link to the official build. - - For example, in Cygwin, zlib is linked to the Cygwin runtime - CYGWIN1.DLL, and it is distributed under the name CYGZ.DLL. - - -15. May I include additional pieces of code that I find useful, - link them in ZLIB1.DLL, and export them? - - - No. A legitimate build of ZLIB1.DLL must not include code - that does not originate from the official zlib source code. - But you can make your own private DLL build, under a different - file name, as suggested in the previous answer. - - For example, zlib is a part of the VCL library, distributed - with Borland Delphi and C++ Builder. The DLL build of VCL - is a redistributable file, named VCLxx.DLL. - - -16. May I remove some functionality out of ZLIB1.DLL, by enabling - macros like NO_GZCOMPRESS or NO_GZIP at compile time? - - - No. A legitimate build of ZLIB1.DLL must provide the complete - zlib functionality, as implemented in the official zlib source - code. But you can make your own private DLL build, under a - different file name, as suggested in the previous answer. - - -17. I made my own ZLIB1.DLL build. Can I test it for compliance? - - - We prefer that you download the official DLL from the zlib - web site. If you need something peculiar from this DLL, you - can send your suggestion to the zlib mailing list. - - However, in case you do rebuild the DLL yourself, you can run - it with the test programs found in the DLL distribution. - Running these test programs is not a guarantee of compliance, - but a failure can imply a detected problem. - -** - -This document is written and maintained by -Cosmin Truta diff --git a/contrib/libzlib-ng/win32/Makefile.msc b/contrib/libzlib-ng/win32/Makefile.msc deleted file mode 100644 index b6375ccb4f8..00000000000 --- a/contrib/libzlib-ng/win32/Makefile.msc +++ /dev/null @@ -1,154 +0,0 @@ -# Makefile for zlib using Microsoft (Visual) C -# zlib is copyright (C) 1995-2006 Jean-loup Gailly and Mark Adler -# -# Usage: -# nmake -f win32/Makefile.msc (standard build) -# nmake -f win32/Makefile.msc LOC=-DFOO (nonstandard build) - -# The toplevel directory of the source tree. -# -TOP = . - -# optional build flags -LOC = - -# variables -STATICLIB = zlib.lib -SHAREDLIB = zlib1.dll -IMPLIB = zdll.lib - -CC = cl -LD = link -AR = lib -RC = rc -CP = copy /y -CFLAGS = -nologo -MD -W3 -O2 -Oy- -Zi -Fd"zlib" $(LOC) -WFLAGS = -D_CRT_SECURE_NO_DEPRECATE -D_CRT_NONSTDC_NO_DEPRECATE -DX86_PCLMULQDQ_CRC -DX86_SSE2_FILL_WINDOW -DX86_CPUID -DX86_SSE4_2_CRC_HASH -DUNALIGNED_OK -DUNROLL_LESS -DX86_QUICK_STRATEGY -LDFLAGS = -nologo -debug -incremental:no -opt:ref -manifest -ARFLAGS = -nologo -RCFLAGS = /dWIN32 /r -DEFFILE = zlib.def -WITH_GZFILEOP = - -OBJS = adler32.obj compress.obj crc32.obj deflate.obj deflate_fast.obj deflate_quick.obj deflate_slow.obj \ - infback.obj inflate.obj inftrees.obj inffast.obj match.obj trees.obj uncompr.obj zutil.obj x86.obj fill_window_sse.obj insert_string_sse.obj crc_folding.obj -!if "$(WITH_GZFILEOP)" != "" -WFLAGS = $(WFLAGS) -DWITH_GZFILEOP -OBJS = $(OBJS) gzclose.obj gzlib.obj gzread.obj gzwrite.obj -DEFFILE = zlibcompat.def -!endif - -# targets -all: $(STATICLIB) $(SHAREDLIB) $(IMPLIB) \ - example.exe minigzip.exe example_d.exe minigzip_d.exe - -$(STATICLIB): $(OBJS) - $(AR) $(ARFLAGS) -out:$@ $(OBJS) - -$(IMPLIB): $(SHAREDLIB) - -$(SHAREDLIB): $(TOP)/win32/zlib.def $(OBJS) zlib1.res - $(LD) $(LDFLAGS) -def:$(TOP)/win32/$(DEFFILE) -dll -implib:$(IMPLIB) \ - -out:$@ -base:0x5A4C0000 $(OBJS) zlib1.res - if exist $@.manifest \ - mt -nologo -manifest $@.manifest -outputresource:$@;2 - -example.exe: example.obj $(STATICLIB) - $(LD) $(LDFLAGS) example.obj $(STATICLIB) - if exist $@.manifest \ - mt -nologo -manifest $@.manifest -outputresource:$@;1 - -minigzip.exe: minigzip.obj $(STATICLIB) - $(LD) $(LDFLAGS) minigzip.obj $(STATICLIB) - if exist $@.manifest \ - mt -nologo -manifest $@.manifest -outputresource:$@;1 - -example_d.exe: example.obj $(IMPLIB) - $(LD) $(LDFLAGS) -out:$@ example.obj $(IMPLIB) - if exist $@.manifest \ - mt -nologo -manifest $@.manifest -outputresource:$@;1 - -minigzip_d.exe: minigzip.obj $(IMPLIB) - $(LD) $(LDFLAGS) -out:$@ minigzip.obj $(IMPLIB) - if exist $@.manifest \ - mt -nologo -manifest $@.manifest -outputresource:$@;1 - -{$(TOP)}.c.obj: - $(CC) -c $(WFLAGS) $(CFLAGS) $< - -{$(TOP)/arch/x86}.c.obj: - $(CC) -c -I$(TOP) $(WFLAGS) $(CFLAGS) $< - -{$(TOP)/test}.c.obj: - $(CC) -c -I$(TOP) $(WFLAGS) $(CFLAGS) $< - -$(TOP)/zconf.h: $(TOP)/zconf.h.in - $(CP) $(TOP)\zconf.h.in $(TOP)\zconf.h - -adler32.obj: $(TOP)/adler32.c $(TOP)/zlib.h $(TOP)/zconf.h - -compress.obj: $(TOP)/compress.c $(TOP)/zlib.h $(TOP)/zconf.h - -crc32.obj: $(TOP)/crc32.c $(TOP)/zlib.h $(TOP)/zconf.h $(TOP)/crc32.h - -deflate.obj: $(TOP)/deflate.c $(TOP)/deflate.h $(TOP)/zutil.h $(TOP)/zlib.h $(TOP)/zconf.h - -gzclose.obj: $(TOP)/gzclose.c $(TOP)/zlib.h $(TOP)/zconf.h $(TOP)/gzguts.h - -gzlib.obj: $(TOP)/gzlib.c $(TOP)/zlib.h $(TOP)/zconf.h $(TOP)/gzguts.h - -gzread.obj: $(TOP)/gzread.c $(TOP)/zlib.h $(TOP)/zconf.h $(TOP)/gzguts.h - -gzwrite.obj: $(TOP)/gzwrite.c $(TOP)/zlib.h $(TOP)/zconf.h $(TOP)/gzguts.h - -infback.obj: $(TOP)/infback.c $(TOP)/zutil.h $(TOP)/zlib.h $(TOP)/zconf.h $(TOP)/inftrees.h $(TOP)/inflate.h \ - $(TOP)/inffast.h $(TOP)/inffixed.h - -inffast.obj: $(TOP)/inffast.c $(TOP)/zutil.h $(TOP)/zlib.h $(TOP)/zconf.h $(TOP)/inftrees.h $(TOP)/inflate.h \ - $(TOP)/inffast.h - -inflate.obj: $(TOP)/inflate.c $(TOP)/zutil.h $(TOP)/zlib.h $(TOP)/zconf.h $(TOP)/inftrees.h $(TOP)/inflate.h \ - $(TOP)/inffast.h $(TOP)/inffixed.h - -inftrees.obj: $(TOP)/inftrees.c $(TOP)/zutil.h $(TOP)/zlib.h $(TOP)/zconf.h $(TOP)/inftrees.h - -match.obj: $(TOP)/match.c $(TOP)/deflate.h - -trees.obj: $(TOP)/trees.c $(TOP)/zutil.h $(TOP)/zlib.h $(TOP)/zconf.h $(TOP)/deflate.h $(TOP)/trees.h - -uncompr.obj: $(TOP)/uncompr.c $(TOP)/zlib.h $(TOP)/zconf.h - -zutil.obj: $(TOP)/zutil.c $(TOP)/zutil.h $(TOP)/zlib.h $(TOP)/zconf.h - -example.obj: $(TOP)/test/example.c $(TOP)/zlib.h $(TOP)/zconf.h - -minigzip.obj: $(TOP)/test/minigzip.c $(TOP)/zlib.h $(TOP)/zconf.h - -zlib1.res: $(TOP)/win32/zlib1.rc - $(RC) $(RCFLAGS) /fo$@ $(TOP)/win32/zlib1.rc - -# testing -test: example.exe minigzip.exe - example - echo hello world | minigzip | minigzip -d - -testdll: example_d.exe minigzip_d.exe - example_d - echo hello world | minigzip_d | minigzip_d -d - - -# cleanup -clean: - -del $(STATICLIB) - -del $(SHAREDLIB) - -del $(IMPLIB) - -del *.obj - -del *.res - -del *.exp - -del *.exe - -del *.pdb - -del *.manifest - -del foo.gz - -distclean: clean - -del zconf.h diff --git a/contrib/libzlib-ng/win32/README-WIN32.txt b/contrib/libzlib-ng/win32/README-WIN32.txt deleted file mode 100644 index 3d77d521e83..00000000000 --- a/contrib/libzlib-ng/win32/README-WIN32.txt +++ /dev/null @@ -1,103 +0,0 @@ -ZLIB DATA COMPRESSION LIBRARY - -zlib 1.2.8 is a general purpose data compression library. All the code is -thread safe. The data format used by the zlib library is described by RFCs -(Request for Comments) 1950 to 1952 in the files -http://www.ietf.org/rfc/rfc1950.txt (zlib format), rfc1951.txt (deflate format) -and rfc1952.txt (gzip format). - -All functions of the compression library are documented in the file zlib.h -(volunteer to write man pages welcome, contact zlib@gzip.org). Two compiled -examples are distributed in this package, example and minigzip. The example_d -and minigzip_d flavors validate that the zlib1.dll file is working correctly. - -Questions about zlib should be sent to . The zlib home page -is http://zlib.net/ . Before reporting a problem, please check this site to -verify that you have the latest version of zlib; otherwise get the latest -version and check whether the problem still exists or not. - -PLEASE read DLL_FAQ.txt, and the the zlib FAQ http://zlib.net/zlib_faq.html -before asking for help. - - -Manifest: - -The package zlib-1.2.8-win32-x86.zip will contain the following files: - - README-WIN32.txt This document - ChangeLog Changes since previous zlib packages - DLL_FAQ.txt Frequently asked questions about zlib1.dll - zlib.3.pdf Documentation of this library in Adobe Acrobat format - - example.exe A statically-bound example (using zlib.lib, not the dll) - example.pdb Symbolic information for debugging example.exe - - example_d.exe A zlib1.dll bound example (using zdll.lib) - example_d.pdb Symbolic information for debugging example_d.exe - - minigzip.exe A statically-bound test program (using zlib.lib, not the dll) - minigzip.pdb Symbolic information for debugging minigzip.exe - - minigzip_d.exe A zlib1.dll bound test program (using zdll.lib) - minigzip_d.pdb Symbolic information for debugging minigzip_d.exe - - zlib.h Install these files into the compilers' INCLUDE path to - zconf.h compile programs which use zlib.lib or zdll.lib - - zdll.lib Install these files into the compilers' LIB path if linking - zdll.exp a compiled program to the zlib1.dll binary - - zlib.lib Install these files into the compilers' LIB path to link zlib - zlib.pdb into compiled programs, without zlib1.dll runtime dependency - (zlib.pdb provides debugging info to the compile time linker) - - zlib1.dll Install this binary shared library into the system PATH, or - the program's runtime directory (where the .exe resides) - zlib1.pdb Install in the same directory as zlib1.dll, in order to debug - an application crash using WinDbg or similar tools. - -All .pdb files above are entirely optional, but are very useful to a developer -attempting to diagnose program misbehavior or a crash. Many additional -important files for developers can be found in the zlib127.zip source package -available from http://zlib.net/ - review that package's README file for details. - - -Acknowledgments: - -The deflate format used by zlib was defined by Phil Katz. The deflate and -zlib specifications were written by L. Peter Deutsch. Thanks to all the -people who reported problems and suggested various improvements in zlib; they -are too numerous to cite here. - - -Copyright notice: - - (C) 1995-2012 Jean-loup Gailly and Mark Adler - - This software is provided 'as-is', without any express or implied - warranty. In no event will the authors be held liable for any damages - arising from the use of this software. - - Permission is granted to anyone to use this software for any purpose, - including commercial applications, and to alter it and redistribute it - freely, subject to the following restrictions: - - 1. The origin of this software must not be misrepresented; you must not - claim that you wrote the original software. If you use this software - in a product, an acknowledgment in the product documentation would be - appreciated but is not required. - 2. Altered source versions must be plainly marked as such, and must not be - misrepresented as being the original software. - 3. This notice may not be removed or altered from any source distribution. - - Jean-loup Gailly Mark Adler - jloup@gzip.org madler@alumni.caltech.edu - -If you use the zlib library in a product, we would appreciate *not* receiving -lengthy legal documents to sign. The sources are provided for free but without -warranty of any kind. The library has been entirely written by Jean-loup -Gailly and Mark Adler; it does not include third-party code. - -If you redistribute modified sources, we would appreciate that you include in -the file ChangeLog history information documenting your changes. Please read -the FAQ for more information on the distribution of modified source versions. diff --git a/contrib/libzlib-ng/win32/VisualC.txt b/contrib/libzlib-ng/win32/VisualC.txt deleted file mode 100644 index 579a5fc9e0f..00000000000 --- a/contrib/libzlib-ng/win32/VisualC.txt +++ /dev/null @@ -1,3 +0,0 @@ - -To build zlib using the Microsoft Visual C++ environment, -use the appropriate project from the projects/ directory. diff --git a/contrib/libzlib-ng/win32/zlib.def b/contrib/libzlib-ng/win32/zlib.def deleted file mode 100644 index 1d0c8947a5b..00000000000 --- a/contrib/libzlib-ng/win32/zlib.def +++ /dev/null @@ -1,55 +0,0 @@ -; zlib data compression library -EXPORTS -; basic functions - zlibVersion - deflate - deflateEnd - inflate - inflateEnd -; advanced functions - deflateSetDictionary - deflateCopy - deflateReset - deflateParams - deflateTune - deflateBound - deflatePending - deflatePrime - deflateSetHeader - inflateSetDictionary - inflateGetDictionary - inflateSync - inflateCopy - inflateReset - inflateReset2 - inflatePrime - inflateMark - inflateGetHeader - inflateBack - inflateBackEnd - zlibCompileFlags -; utility functions - compress - compress2 - compressBound - uncompress -; large file functions - adler32_combine64 - crc32_combine64 -; checksum functions - adler32 - crc32 - adler32_combine - crc32_combine -; various hacks, don't look :) - deflateInit_ - deflateInit2_ - inflateInit_ - inflateInit2_ - inflateBackInit_ - zError - inflateSyncPoint - get_crc_table - inflateUndermine - inflateResetKeep - deflateResetKeep diff --git a/contrib/libzlib-ng/win32/zlib1.rc b/contrib/libzlib-ng/win32/zlib1.rc deleted file mode 100644 index 5c0feed1b44..00000000000 --- a/contrib/libzlib-ng/win32/zlib1.rc +++ /dev/null @@ -1,40 +0,0 @@ -#include -#include "../zlib.h" - -#ifdef GCC_WINDRES -VS_VERSION_INFO VERSIONINFO -#else -VS_VERSION_INFO VERSIONINFO MOVEABLE IMPURE LOADONCALL DISCARDABLE -#endif - FILEVERSION ZLIB_VER_MAJOR,ZLIB_VER_MINOR,ZLIB_VER_REVISION,0 - PRODUCTVERSION ZLIB_VER_MAJOR,ZLIB_VER_MINOR,ZLIB_VER_REVISION,0 - FILEFLAGSMASK VS_FFI_FILEFLAGSMASK -#ifdef _DEBUG - FILEFLAGS 1 -#else - FILEFLAGS 0 -#endif - FILEOS VOS__WINDOWS32 - FILETYPE VFT_DLL - FILESUBTYPE 0 // not used -BEGIN - BLOCK "StringFileInfo" - BEGIN - BLOCK "040904E4" - //language ID = U.S. English, char set = Windows, Multilingual - BEGIN - VALUE "FileDescription", "zlib data compression library\0" - VALUE "FileVersion", ZLIB_VERSION "\0" - VALUE "InternalName", "zlib1.dll\0" - VALUE "LegalCopyright", "(C) 1995-2013 Jean-loup Gailly & Mark Adler\0" - VALUE "OriginalFilename", "zlib1.dll\0" - VALUE "ProductName", "zlib\0" - VALUE "ProductVersion", ZLIB_VERSION "\0" - VALUE "Comments", "For more information visit http://www.zlib.net/\0" - END - END - BLOCK "VarFileInfo" - BEGIN - VALUE "Translation", 0x0409, 1252 - END -END diff --git a/contrib/libzlib-ng/win32/zlibcompat.def b/contrib/libzlib-ng/win32/zlibcompat.def deleted file mode 100644 index face655183a..00000000000 --- a/contrib/libzlib-ng/win32/zlibcompat.def +++ /dev/null @@ -1,86 +0,0 @@ -; zlib data compression library -EXPORTS -; basic functions - zlibVersion - deflate - deflateEnd - inflate - inflateEnd -; advanced functions - deflateSetDictionary - deflateCopy - deflateReset - deflateParams - deflateTune - deflateBound - deflatePending - deflatePrime - deflateSetHeader - inflateSetDictionary - inflateGetDictionary - inflateSync - inflateCopy - inflateReset - inflateReset2 - inflatePrime - inflateMark - inflateGetHeader - inflateBack - inflateBackEnd - zlibCompileFlags -; utility functions - compress - compress2 - compressBound - uncompress - gzopen - gzdopen - gzbuffer - gzsetparams - gzread - gzwrite - gzprintf - gzvprintf - gzputs - gzgets - gzputc - gzgetc - gzungetc - gzflush - gzseek - gzrewind - gztell - gzoffset - gzeof - gzdirect - gzclose - gzclose_r - gzclose_w - gzerror - gzclearerr -; large file functions - gzopen64 - gzseek64 - gztell64 - gzoffset64 - adler32_combine64 - crc32_combine64 -; checksum functions - adler32 - crc32 - adler32_combine - crc32_combine -; various hacks, don't look :) - deflateInit_ - deflateInit2_ - inflateInit_ - inflateInit2_ - inflateBackInit_ - gzgetc_ - zError - inflateSyncPoint - get_crc_table - inflateUndermine - inflateResetKeep - deflateResetKeep - gzopen_w diff --git a/contrib/libzlib-ng/zconf.h.in b/contrib/libzlib-ng/zconf.h.in deleted file mode 100644 index 7cacf1b79ed..00000000000 --- a/contrib/libzlib-ng/zconf.h.in +++ /dev/null @@ -1,176 +0,0 @@ -/* zconf.h -- configuration of the zlib compression library - * Copyright (C) 1995-2013 Jean-loup Gailly. - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* @(#) $Id$ */ - -#ifndef ZCONF_H -#define ZCONF_H - -#if defined(_WINDOWS) && !defined(WINDOWS) -# define WINDOWS -#endif -#if defined(_WIN32) || defined(__WIN32__) -# ifndef WIN32 -# define WIN32 -# endif -#endif - -#ifdef __STDC_VERSION__ -# if __STDC_VERSION__ >= 199901L -# ifndef STDC99 -# define STDC99 -# endif -# endif -#endif - -/* Maximum value for memLevel in deflateInit2 */ -#ifndef MAX_MEM_LEVEL -# define MAX_MEM_LEVEL 9 -#endif - -/* Maximum value for windowBits in deflateInit2 and inflateInit2. - * WARNING: reducing MAX_WBITS makes minigzip unable to extract .gz files - * created by gzip. (Files created by minigzip can still be extracted by - * gzip.) - */ -#ifndef MAX_WBITS -# define MAX_WBITS 15 /* 32K LZ77 window */ -#endif - -/* The memory requirements for deflate are (in bytes): - (1 << (windowBits+2)) + (1 << (memLevel+9)) - that is: 128K for windowBits=15 + 128K for memLevel = 8 (default values) - plus a few kilobytes for small objects. For example, if you want to reduce - the default memory requirements from 256K to 128K, compile with - make CFLAGS="-O -DMAX_WBITS=14 -DMAX_MEM_LEVEL=7" - Of course this will generally degrade compression (there's no free lunch). - - The memory requirements for inflate are (in bytes) 1 << windowBits - that is, 32K for windowBits=15 (default value) plus a few kilobytes - for small objects. -*/ - - /* Type declarations */ - - -#if defined(WINDOWS) || defined(WIN32) - /* If building or using zlib as a DLL, define ZLIB_DLL. - * This is not mandatory, but it offers a little performance increase. - */ -# ifdef ZLIB_DLL -# if defined(WIN32) && (!defined(__BORLANDC__) || (__BORLANDC__ >= 0x500)) -# ifdef ZLIB_INTERNAL -# define ZEXTERN extern __declspec(dllexport) -# else -# define ZEXTERN extern __declspec(dllimport) -# endif -# endif -# endif /* ZLIB_DLL */ - /* If building or using zlib with the WINAPI/WINAPIV calling convention, - * define ZLIB_WINAPI. - * Caution: the standard ZLIB1.DLL is NOT compiled using ZLIB_WINAPI. - */ -# ifdef ZLIB_WINAPI -# include - /* No need for _export, use ZLIB.DEF instead. */ - /* For complete Windows compatibility, use WINAPI, not __stdcall. */ -# define ZEXPORT WINAPI -# define ZEXPORTVA WINAPIV -# endif -#endif - -#ifndef ZEXTERN -# define ZEXTERN extern -#endif -#ifndef ZEXPORT -# define ZEXPORT -#endif -#ifndef ZEXPORTVA -# define ZEXPORTVA -#endif - -/* Fallback for something that includes us. */ -typedef unsigned char Byte; -typedef Byte Bytef; - -typedef unsigned int uInt; /* 16 bits or more */ -typedef unsigned long uLong; /* 32 bits or more */ - -typedef char charf; -typedef int intf; -typedef uInt uIntf; -typedef uLong uLongf; - -typedef void const *voidpc; -typedef void *voidpf; -typedef void *voidp; - -#ifdef HAVE_UNISTD_H /* may be set to #if 1 by ./configure */ -# define Z_HAVE_UNISTD_H -#endif - -#ifdef HAVE_STDARG_H /* may be set to #if 1 by ./configure */ -# define Z_HAVE_STDARG_H -#endif - -#include /* for off_t */ -#include /* for va_list */ - -#ifdef WIN32 -# include /* for wchar_t */ -#endif - -/* a little trick to accommodate both "#define _LARGEFILE64_SOURCE" and - * "#define _LARGEFILE64_SOURCE 1" as requesting 64-bit operations, (even - * though the former does not conform to the LFS document), but considering - * both "#undef _LARGEFILE64_SOURCE" and "#define _LARGEFILE64_SOURCE 0" as - * equivalently requesting no 64-bit operations - */ -#if defined(_LARGEFILE64_SOURCE) && -_LARGEFILE64_SOURCE - -1 == 1 -# undef _LARGEFILE64_SOURCE -#endif - -#if defined(Z_HAVE_UNISTD_H) || defined(_LARGEFILE64_SOURCE) -# include /* for SEEK_*, off_t, and _LFS64_LARGEFILE */ -# ifndef z_off_t -# define z_off_t off_t -# endif -#endif - -#if defined(_LFS64_LARGEFILE) && _LFS64_LARGEFILE-0 -# define Z_LFS64 -#endif - -#if defined(_LARGEFILE64_SOURCE) && defined(Z_LFS64) -# define Z_LARGE64 -#endif - -#if defined(_FILE_OFFSET_BITS) && _FILE_OFFSET_BITS-0 == 64 && defined(Z_LFS64) -# define Z_WANT64 -#endif - -#if !defined(SEEK_SET) && defined(WITH_GZFILEOP) -# define SEEK_SET 0 /* Seek from beginning of file. */ -# define SEEK_CUR 1 /* Seek from current position. */ -# define SEEK_END 2 /* Set file pointer to EOF plus "offset" */ -#endif - -#ifndef z_off_t -# define z_off_t long -#endif - -#if !defined(WIN32) && defined(Z_LARGE64) -# define z_off64_t off64_t -#else -# if defined(__MSYS__) -# define z_off64_t _off64_t -# elif defined(WIN32) && !defined(__GNUC__) -# define z_off64_t __int64 -# else -# define z_off64_t z_off_t -# endif -#endif - -#endif /* ZCONF_H */ diff --git a/contrib/libzlib-ng/zlib.3 b/contrib/libzlib-ng/zlib.3 deleted file mode 100644 index 1cc38059d78..00000000000 --- a/contrib/libzlib-ng/zlib.3 +++ /dev/null @@ -1,169 +0,0 @@ -.TH ZLIB 3 "7 July 2015" -.SH NAME -zlib \- compression/decompression library -.SH SYNOPSIS -[see -.I zlib.h -for full description] -.SH DESCRIPTION -The -.I zlib -library is a general purpose data compression library. -The code is thread safe, assuming that the standard library functions -used are thread safe, such as memory allocation routines. -It provides in-memory compression and decompression functions, -including integrity checks of the uncompressed data. -This version of the library supports only one compression method (deflation) -but other algorithms may be added later -with the same stream interface. -.LP -Compression can be done in a single step if the buffers are large enough -or can be done by repeated calls of the compression function. -In the latter case, -the application must provide more input and/or consume the output -(providing more output space) before each call. -.LP -The library also supports reading and writing files in -.IR gzip (1) -(.gz) format -with an interface similar to that of stdio. -.LP -The library does not install any signal handler. -The decoder checks the consistency of the compressed data, -so the library should never crash even in the case of corrupted input. -.LP -All functions of the compression library are documented in the file -.IR zlib.h . -The distribution source includes examples of use of the library -in the files -.I test/example.c -and -.IR test/minigzip.c, -as well as other examples in the -.IR examples/ -directory. -.LP -Changes to this version are documented in the file -.I ChangeLog -that accompanies the source. -.LP -.I zlib -is available in Java using the java.util.zip package: -.IP -http://java.sun.com/developer/technicalArticles/Programming/compression/ -.LP -A Perl interface to -.IR zlib , -written by Paul Marquess (pmqs@cpan.org), -is available at CPAN (Comprehensive Perl Archive Network) sites, -including: -.IP -http://search.cpan.org/~pmqs/IO-Compress-Zlib/ -.LP -A Python interface to -.IR zlib , -written by A.M. Kuchling (amk@magnet.com), -is available in Python 1.5 and later versions: -.IP -http://docs.python.org/library/zlib.html -.LP -.I zlib -is built into -.IR tcl: -.IP -http://wiki.tcl.tk/4610 -.LP -An experimental package to read and write files in .zip format, -written on top of -.I zlib -by Gilles Vollant (info@winimage.com), -is available at: -.IP -http://www.winimage.com/zLibDll/minizip.html -and also in the -.I contrib/minizip -directory of the main -.I zlib -source distribution. -.SH "SEE ALSO" -The -.I zlib -web site can be found at: -.IP -http://zlib.net/ -.LP -The data format used by the zlib library is described by RFC -(Request for Comments) 1950 to 1952 in the files: -.IP -http://tools.ietf.org/html/rfc1950 (for the zlib header and trailer format) -.br -http://tools.ietf.org/html/rfc1951 (for the deflate compressed data format) -.br -http://tools.ietf.org/html/rfc1952 (for the gzip header and trailer format) -.LP -Mark Nelson wrote an article about -.I zlib -for the Jan. 1997 issue of Dr. Dobb's Journal; -a copy of the article is available at: -.IP -http://marknelson.us/1997/01/01/zlib-engine/ -.SH "REPORTING PROBLEMS" -Before reporting a problem, -please check the -.I zlib -web site to verify that you have the latest version of -.IR zlib ; -otherwise, -obtain the latest version and see if the problem still exists. -Please read the -.I zlib -FAQ at: -.IP -http://zlib.net/zlib_faq.html -.LP -before asking for help. -Send questions and/or comments to zlib@gzip.org, -or (for the Windows DLL version) to Gilles Vollant (info@winimage.com). -.SH AUTHORS AND LICENSE -Version 1.2.8.1 -.LP -Copyright (C) 1995-2015 Jean-loup Gailly and Mark Adler -.LP -This software is provided 'as-is', without any express or implied -warranty. In no event will the authors be held liable for any damages -arising from the use of this software. -.LP -Permission is granted to anyone to use this software for any purpose, -including commercial applications, and to alter it and redistribute it -freely, subject to the following restrictions: -.LP -.nr step 1 1 -.IP \n[step]. 3 -The origin of this software must not be misrepresented; you must not -claim that you wrote the original software. If you use this software -in a product, an acknowledgment in the product documentation would be -appreciated but is not required. -.IP \n+[step]. -Altered source versions must be plainly marked as such, and must not be -misrepresented as being the original software. -.IP \n+[step]. -This notice may not be removed or altered from any source distribution. -.LP -Jean-loup Gailly Mark Adler -.br -jloup@gzip.org madler@alumni.caltech.edu -.LP -The deflate format used by -.I zlib -was defined by Phil Katz. -The deflate and -.I zlib -specifications were written by L. Peter Deutsch. -Thanks to all the people who reported problems and suggested various -improvements in -.IR zlib ; -who are too numerous to cite here. -.LP -UNIX manual page by R. P. C. Rodgers, -U.S. National Library of Medicine (rodgers@nlm.nih.gov). -.\" end of man page diff --git a/contrib/libzlib-ng/zlib.3.pdf b/contrib/libzlib-ng/zlib.3.pdf deleted file mode 100644 index a346b5d7e24834806b0871b209637c728fb36d60..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8734 zcmcgyc|6o>+eadkB}+)9$uc7`yJ0M2-wiSxOO#;-leL+#muyG-WLHRJUy>zz3rU6S zLM3F05=xee>iv!CoX*pE-sgEgpYwTt|G4M+UDtJgukF6C`~H!&)H6^-!Z0A&2X8K1 z1u20MU|*LbAa!-P8O6t);Q_`16f3v^)r&!)f#C*TBnCx~;_6GL;PD_jgGM2FgM!j? z9a`guYXxtu+KypMPQgUjw>eI>FBw(nR$xtcGv6}0B05}bkEZG;*B0dazZDz0GJb*q z!A4dYp#x^RbEPgCO)f4{uHqA7Q9|RI?`wvfO7IYJ-{EL7{oTjN!jt#b_XSMmd)md0 ziw&-decEjxWF9XkHQ2BiO=?Mb=uJojv?wj&~&Qk{-JhWY8BD z8EET!7#8-Hvw79@e(SPCzn1;s+1JeVlOl6Iqe;gPS9Kh53i&Gi_NMMDg-(z3aI3ek zs& zLd0iNOWJSGh?+vE;t;8Vl}$tAVpZ?g3lzwSxt9V5#Ne75Uq8HVUM^l;=#*Ry31vKu zrMdjJ$l6cqnloM4x;lFkA&bMzF~*I3Gc6FqZcleF(Wad%5s&4DlFJ6`Wip4#=Gu-C zJ~`sHI6jiOopMwnbb;sdp0)BJ*l@a>Z#g{isuH$@!Q&PArAue99A zb#S>Ma5J3qLLANv_r6P}5|dY6cCxPI=FOAaAGui*`zItgxcYaRY{w4RBJ*`j+#+0s zz*LtwYLHY*wSuTZw!M7zSQd0nBPN+kUhpJ_Lt1=iM6N>@7XGTNFND05v| zkwY=`>f|087mR*w z))jbhJ~mNpidwMgu8#8on8=VjRjX?B;yJmqP-k~*+GTu~hOkJm?mQc+2eX|M@@g;h z$Z+0pf=Pp+Hcm{CkMM|d;#MRl+gLxnda~Rgvefp&f$8UYGL$}Mnj$;*$I$206SOv+ zw0H}MY?GZcH+N9{j>(0#<2+pUQR(&7NE35eK_`U?L`05Rr0scT*)c7mXz=xH2@?xZ zYGreI4k4$#>3T`8E}}JOfFpWp?YPqpwx@OSn3YZ1>y4I;7@bBi+jhu#mGtfRIwp#a ztZXw5=h}}EAb+jhpLCne^TQ;zObA)4B?ZB?-Prm`BS=!Z^5$W-p@Ixm;mu z*B!D7NK#CEnYGo+aG&>8Pxo}6JckmaHhDLWJ*r`Ue(d)<@*>hN%kyPzbJ_ZCN0F_~ zH_sycW*EG}Hu4Q%#I0yhHm@+qie#(JyHh;6mMo>l8IxR~!`n0@Fk9qgr|Z#h62sp| zTw(V5UQ#o58h)}ms5o?u%H1st!Xs|p>`CGmL)x7E~jF0Niav7W}|LIwU^Wft4|_3dxKmg zv~BI$vNPBXS2Cs~jbwyB#}HbNXxTUoZGOpyu1Q(s4JWE$V&8Q%O*bm`pE7}1tUDX{*aVzc#C@-Dw;ZJ^ioEWbmF-v zo9p=jrz7bo*~Tro`{)8@SmbnuI?VC`!X#3NRIKcmD&=IsKW0rY>P=MVC1G#*<|`BFoSo_~j?AkyKEM^^1=e@4h4!EEyIpNQ z1BX_7{pp_RaV}$7Cob<7kBCedeh;$JicZyeBs^ww=8dc3iHFKl zXx*Z{sO)a#fdKK9uHTBI__FkwcPbBRt(9ku6+aF+q$J`S=x8X$oEXes$@f{M)JU|s zJY(B`?Rutg5$N2Kz-6YG?fuM5y=2$R*Id+gX_Tx-q(UU(=H>P}vYq+eJ)&43>{(Z? zp{y%Og2O89wrcf6O5--z5+eK3PR`$pUf3A%h76(r*pU=Q*S+(+X;s`-r{Gn5VrWrCmJ(FGu3S4 zNc)F7FMtayC?ib64C)v&Q|**_j^E*zep`+lulfY7_gQ}Zxp$?#{mR~9YiwiBrC#TK z#dW6T8DE>LFDvb|H1x{Z!-mG5%bK%uENHlP!o6U2`Of~{%O0g$E_y;r zj}5l?n4ipVTPmD6X!4~f6gtM^T7FINlyBA~e!aF`nScX#*Ui+#URS!uuf7Iz$T)X` zsE3y;R^@sSkB*xOf5N;{?ON&0a&7k9`oSuq8~bz5lqSQfY*51K zx7umWJWtaXuI(~Mo_J(h(_D>dTV>d;IzHV-<$$yTThxoAc+#j~cO`Gz<19cawYV zPH#2cH3!L3F=sw_optps$8O2r1QAEQ-wgR8`}H;-JZDg&?dbo+uw_iS;I|x|PYK1% zx^?gG?wT!Kp}jnHUif&n36i#7>Ea5M*C>)*^3Ab`0;#a;pQXxcF8g_5C2BD~r!71_ zZ3%G+D3E=7ZT5P8r4CGBPO|pvUVH9SCPSoohg3A3@|qw!Znki2RqxCAB3qj@_XyJa z+MuuYCAW8ymCIXS`>^p(Ww=?ZWv({Z&WI%HRh&307bM(b&6YMD8Se zd3=9j+YJG_7{&6+u$C(=qzAj{!{^R9lu}*S?r(rm$soUcd5q~ z&My|>cqO^2AMr;`n|&GOOdU(R9az8Ew_-?VhCo8b7X>&p376(G{WI3B)cy<+RCWw4!RVgfIKFY{7Lip(j|4eYf{=+ zR~I`PpKNr5n-Zpue=8OJ?n{8S9YoiZl&sG5kZ27qX$H;vH{2>xhnmiJS6A86i)k0v zugSw8t3?g){=Aw9F}}+}*U#1b4vJVsZ`%_q3OXIlyI2$h_jelO@rt_~Pz#_h`-)?YrmW4`WSXIkbhqOF-x$wC;S)pK*sN`~sp;VWUI?;j1tL9Tx#+gO_E z=_SfO;&XJf9o#|R)EuB0>KU*gre5VfQm>4W$zG3*bv+mmJU%zbq6z`x*@gaXdqyZz=!H#vD zWBo;ARM4RBG$iJiQy-Fb5d4qFzT>vfXcKzdj<4F9y^|rnBbQ3>99p?sp+JU{J93TW zOv<6vcJ?$G-2FAaw_S`b{fb*#r>t8B9xXAh;E&fYYl%-qzkseE_JX7g#+20aNEf#@ zM>psTeNn_tKG}sOe{c@deCMu$j12+Q7qd8N3$ozN36P<5?7FRx)fe!8xXyz49ex2flk28t`HQ#>XYRk_7H zN_)=kNS)l*(JiMNrQEuFHqiHae&m7s4AhC`m$Jdi>&MpKyl$CY?HipQ-SayKtfN+I zYb!eRLZF{MpF0PKUL~djHSO8w(EHT8Z(Op4T13FkBfosQU1FO9epX#auc0+3wE4gu zR#)h|x4L+Xb+Q}VfZRoeVIl7b;SB$D^>59uu;;VJi{ZK8Pw-7lJ} zgeDfQ54DZhTY{6el9;dkDCY3MK^fEXl!dR|+pa zV|TJPc3;Ot?ujI)g$n+MFWa79?lU`S^J!i@Bv0hRD^hPtJ=EXxY_6`o+Zhm;qEPUK zIG|IN_Yqgt82N_R)-nMzFm{eO<@xqm{^Wxu=vXS-6Q1fbU-rA3Xi>)bq7cJ+Nk?n; zC2~j$J3JfF_&52r?8ZoQMZT=DKqGsY&W`%ObRtm0x>X4a^q)7}R+4BP0u{kuw!^Qc$s z{uq3OX3>~ebf*5KTKiM!aA+!YtpcXp(qh@BQo#(pTw2cEvyn}Hks>zz^dw;q@z11@)nZqxlZx$Gd{ls_#B2T zPL6M_fUo&i&h7<^oxQ3~a8lqEITRyd!oN~GSit^v^18QY1AnliPqht??VF?S$;4BE zvG=Ze=kI!w?9TIcVViS%QbU`~I2g5*3BoVnv=K^7w zdwL5HVc?w76Ml0!4n`%{GO6Hvm6CHW&o5qvmRhtVyfb!Cwr(r^pj(WTTDt$#uDn}`CwwmU z=Y!Om9Rz{2d{=}$9kmrlF52v(@E!VYwt-8#i`4C_6F|X268;X zE4*o`<;_juyA2Lc-qd&V@Lh=K4|$$3AZ}LJtnl&hB91fe14?nW5~F>j@6Cw{hHCg4 zcFPp_!$<2VOXoeaX)*n<%i{4m*aY*G5ym(7Vf=)zxmfsIX`)eInUSd|~mz)!t^ zb+8`=Zb@>dzzGyGm89bv1U`rW`T{Hx1y;tPoIr3ZUj}d=3`PK4G>Q)cjAnHW0GUGf z^{2T~=)g6yCC%5BNMRg=0~gI;xHTn+0m$hGF${?tqDbU75d&X99MH>>2X2<(y1;Kh zBOScah_FaMnibX@-><0uUjS}gfP>)Q04o0qaHD(L0R0`|p8x@S0T+Tdd`JhwO{ru$ z_#lYoVJi^wo3EU}fT{j~?Wmt^{Huuo3yDGU@^$~A>ARZ=6b6Y5ya6L|KVW@_z=HX+ zkN)sC2yS6X03-4EAKqlS|2wCZGHAoRpM1@V$B(y21PIBBz7^aG2ilkrZlwZJVNrgP z_|abik^dWgKjE_a@AR=|{Ek3}M5nN#05_s|1yC4NSCXQIix>5%KV`$s8xaD*iT*AO zR^GBa57_x5ZCHT+y&fH3FYrvg@C_qU8HFOY!L}B3!su!68LShhbH!9rC3xrfg!@a3K{&c55ZQMwMpVj?HkUza* zM=<;GVF!uEVDb3xrvr7(AU~3;Cxroap}13hG@uI&cc5S@Sp#aTL_iSybSWNGqYxT} z7-DYi8sh1SBSSUuAazX=aHyat29f*}-8Q}u(ZLkDtA>=Wm6?GQiy=rg$lK2w$Ww5T zx0erHHAn-xA)*S5S!6gAyurfo)PUL%EWr$4UoTH8Pz^8~45^4jD}dc-ByUQfFU=E- zgkhn8N*dWs)e2Zkz9|HLX+S*~3_n#kJTNd27Kn!V(%j)l91aIZpx`K!BEX?Y5B6b@ zf)st|(!hKhK&Wn`Z1`!K48|wBo|+Qh6dE% zpGsC$A|aJgu1aJ@ii$Eu5s9Rb6%oLrNWx-W+{kEGlq(tcV>-W|VE$!3-~})t8t?-! zQUw-EvcD^ZrU5myGy@xY_|h5Q<_F_o2qh@|yZQeVDj+t0?*$;B-`$`N@Td}iWOfVI z1D1L`3V}o`A~1?56>B6`6@gY|{Zv*(Ak^Xi!U+T)5Y+{~5CUHe{|nKdIDZoLrC9?h zfp@3*y19V^C^R}N5=a;l2Sa_EaYO7ompYskqo2IcC3|mElFfZ`7W)7vtN&i^{4cRWFi{y9P8!kxX$T`Fk;-cpIM$Zt}Ni zcv11+1{-qlKXv_0tTsqMZ3AJ0|FLr5LEwMr`R5ewQ4C!>SU%;jB85(^F*?83Kb+L1Gc;jrxKj5Kw>(sKfuj@x2=4{;pp3{juu@ z(I{>p1Q>+?u@^^?-eT&{PnpUua4=WuS-nE1wb;1MIhd zr6CZkcJr?^v=SEBAOA|jA`yV&{)L9Y{344*D+9g8zwn_EKx_0bG$iJ4Xv)9PlvEJE zU`8vc{DMyji~5CzMWKGtheavf7pY6lhpG=x7eGiW5L7lj7={{t8aJp-)14ps@RkJLx1pm0h!eXNcy3Z;b6 e$Ny&zCsyl8XOL)&jl@tvsvtqKvicSVp#K4G_#7?( diff --git a/contrib/libzlib-ng/zlib.h b/contrib/libzlib-ng/zlib.h deleted file mode 100644 index 3a1cbb2f05b..00000000000 --- a/contrib/libzlib-ng/zlib.h +++ /dev/null @@ -1,1728 +0,0 @@ -#ifndef ZLIB_H_ -#define ZLIB_H_ -/* zlib.h -- interface of the 'zlib-ng' compression library - Forked from and compatible with zlib 1.2.8 - - Copyright (C) 1995-2013 Jean-loup Gailly and Mark Adler - - This software is provided 'as-is', without any express or implied - warranty. In no event will the authors be held liable for any damages - arising from the use of this software. - - Permission is granted to anyone to use this software for any purpose, - including commercial applications, and to alter it and redistribute it - freely, subject to the following restrictions: - - 1. The origin of this software must not be misrepresented; you must not - claim that you wrote the original software. If you use this software - in a product, an acknowledgment in the product documentation would be - appreciated but is not required. - 2. Altered source versions must be plainly marked as such, and must not be - misrepresented as being the original software. - 3. This notice may not be removed or altered from any source distribution. - - Jean-loup Gailly Mark Adler - jloup@gzip.org madler@alumni.caltech.edu - - - The data format used by the zlib library is described by RFCs (Request for - Comments) 1950 to 1952 in the files http://tools.ietf.org/html/rfc1950 - (zlib format), rfc1951 (deflate format) and rfc1952 (gzip format). -*/ - -#include -#include "zconf.h" - -#ifdef __cplusplus -extern "C" { -#endif - -#define ZLIBNG_VERSION "1.9.9" -#define ZLIBNG_VERNUM 0x1990 -#define ZLIBNG_VER_MAJOR 1 -#define ZLIBNG_VER_MINOR 9 -#define ZLIBNG_VER_REVISION 9 -#define ZLIBNG_VER_SUBREVISION 0 - -#define ZLIB_VERSION "1.2.8.zlib-ng" -#define ZLIB_VERNUM 0x128f -#define ZLIB_VER_MAJOR 1 -#define ZLIB_VER_MINOR 2 -#define ZLIB_VER_REVISION 8 -#define ZLIB_VER_SUBREVISION 0 - -/* - The 'zlib' compression library provides in-memory compression and - decompression functions, including integrity checks of the uncompressed data. - This version of the library supports only one compression method (deflation) - but other algorithms will be added later and will have the same stream - interface. - - Compression can be done in a single step if the buffers are large enough, - or can be done by repeated calls of the compression function. In the latter - case, the application must provide more input and/or consume the output - (providing more output space) before each call. - - The compressed data format used by default by the in-memory functions is - the zlib format, which is a zlib wrapper documented in RFC 1950, wrapped - around a deflate stream, which is itself documented in RFC 1951. - - The library also supports reading and writing files in gzip (.gz) format - with an interface similar to that of stdio using the functions that start - with "gz". The gzip format is different from the zlib format. gzip is a - gzip wrapper, documented in RFC 1952, wrapped around a deflate stream. - - This library can optionally read and write gzip streams in memory as well. - - The zlib format was designed to be compact and fast for use in memory - and on communications channels. The gzip format was designed for single- - file compression on file systems, has a larger header than zlib to maintain - directory information, and uses a different, slower check method than zlib. - - The library does not install any signal handler. The decoder checks - the consistency of the compressed data, so the library should never crash - even in case of corrupted input. -*/ - -typedef void *(*alloc_func) (void *opaque, unsigned int items, unsigned int size); -typedef void (*free_func) (void *opaque, void *address); - -struct internal_state; - -typedef struct z_stream_s { - const unsigned char *next_in; /* next input byte */ - uint32_t avail_in; /* number of bytes available at next_in */ - size_t total_in; /* total number of input bytes read so far */ - - unsigned char *next_out; /* next output byte should be put there */ - uint32_t avail_out; /* remaining free space at next_out */ - size_t total_out; /* total number of bytes output so far */ - - const char *msg; /* last error message, NULL if no error */ - struct internal_state *state; /* not visible by applications */ - - alloc_func zalloc; /* used to allocate the internal state */ - free_func zfree; /* used to free the internal state */ - void *opaque; /* private data object passed to zalloc and zfree */ - - int data_type; /* best guess about the data type: binary or text */ - uint32_t adler; /* adler32 value of the uncompressed data */ - unsigned long reserved; /* reserved for future use */ -} z_stream; - -typedef z_stream *z_streamp; // Obsolete type, retained for compatability only - -/* - gzip header information passed to and from zlib routines. See RFC 1952 - for more details on the meanings of these fields. -*/ -typedef struct gz_header_s { - int text; /* true if compressed data believed to be text */ - unsigned long time; /* modification time */ - int xflags; /* extra flags (not used when writing a gzip file) */ - int os; /* operating system */ - unsigned char *extra; /* pointer to extra field or Z_NULL if none */ - unsigned int extra_len; /* extra field length (valid if extra != Z_NULL) */ - unsigned int extra_max; /* space at extra (only when reading header) */ - unsigned char *name; /* pointer to zero-terminated file name or Z_NULL */ - unsigned int name_max; /* space at name (only when reading header) */ - unsigned char *comment; /* pointer to zero-terminated comment or Z_NULL */ - unsigned int comm_max; /* space at comment (only when reading header) */ - int hcrc; /* true if there was or will be a header crc */ - int done; /* true when done reading gzip header (not used when writing a gzip file) */ -} gz_header; - -typedef gz_header *gz_headerp; - -/* - The application must update next_in and avail_in when avail_in has dropped - to zero. It must update next_out and avail_out when avail_out has dropped - to zero. The application must initialize zalloc, zfree and opaque before - calling the init function. All other fields are set by the compression - library and must not be updated by the application. - - The opaque value provided by the application will be passed as the first - parameter for calls of zalloc and zfree. This can be useful for custom - memory management. The compression library attaches no meaning to the - opaque value. - - zalloc must return Z_NULL if there is not enough memory for the object. - If zlib is used in a multi-threaded application, zalloc and zfree must be - thread safe. - - The fields total_in and total_out can be used for statistics or progress - reports. After compression, total_in holds the total size of the - uncompressed data and may be saved for use in the decompressor (particularly - if the decompressor wants to decompress everything in a single step). -*/ - - /* constants */ - -#define Z_NO_FLUSH 0 -#define Z_PARTIAL_FLUSH 1 -#define Z_SYNC_FLUSH 2 -#define Z_FULL_FLUSH 3 -#define Z_FINISH 4 -#define Z_BLOCK 5 -#define Z_TREES 6 -/* Allowed flush values; see deflate() and inflate() below for details */ - -#define Z_OK 0 -#define Z_STREAM_END 1 -#define Z_NEED_DICT 2 -#define Z_ERRNO (-1) -#define Z_STREAM_ERROR (-2) -#define Z_DATA_ERROR (-3) -#define Z_MEM_ERROR (-4) -#define Z_BUF_ERROR (-5) -#define Z_VERSION_ERROR (-6) -/* Return codes for the compression/decompression functions. Negative values - * are errors, positive values are used for special but normal events. - */ - -#define Z_NO_COMPRESSION 0 -#define Z_BEST_SPEED 1 -#define Z_BEST_COMPRESSION 9 -#define Z_DEFAULT_COMPRESSION (-1) -/* compression levels */ - -#define Z_FILTERED 1 -#define Z_HUFFMAN_ONLY 2 -#define Z_RLE 3 -#define Z_FIXED 4 -#define Z_DEFAULT_STRATEGY 0 -/* compression strategy; see deflateInit2() below for details */ - -#define Z_BINARY 0 -#define Z_TEXT 1 -#define Z_ASCII Z_TEXT /* for compatibility with 1.2.2 and earlier */ -#define Z_UNKNOWN 2 -/* Possible values of the data_type field (though see inflate()) */ - -#define Z_DEFLATED 8 -/* The deflate compression method (the only one supported in this version) */ - -#define Z_NULL 0 /* for initializing zalloc, zfree, opaque */ - -#define zlib_version zlibVersion() -/* for compatibility with versions < 1.0.2 */ - - - /* basic functions */ - -ZEXTERN const char * ZEXPORT zlibVersion(void); -/* The application can compare zlibVersion and ZLIB_VERSION for consistency. - If the first character differs, the library code actually used is not - compatible with the zlib.h header file used by the application. This check - is automatically made by deflateInit and inflateInit. - */ - -/* -ZEXTERN int ZEXPORT deflateInit (z_stream *strm, int level); - - Initializes the internal stream state for compression. The fields - zalloc, zfree and opaque must be initialized before by the caller. If - zalloc and zfree are set to Z_NULL, deflateInit updates them to use default - allocation functions. - - The compression level must be Z_DEFAULT_COMPRESSION, or between 0 and 9: - 1 gives best speed, 9 gives best compression, 0 gives no compression at all - (the input data is simply copied a block at a time). Z_DEFAULT_COMPRESSION - requests a default compromise between speed and compression (currently - equivalent to level 6). - - deflateInit returns Z_OK if success, Z_MEM_ERROR if there was not enough - memory, Z_STREAM_ERROR if level is not a valid compression level, or - Z_VERSION_ERROR if the zlib library version (zlib_version) is incompatible - with the version assumed by the caller (ZLIB_VERSION). msg is set to null - if there is no error message. deflateInit does not perform any compression: - this will be done by deflate(). -*/ - - -ZEXTERN int ZEXPORT deflate(z_stream *strm, int flush); -/* - deflate compresses as much data as possible, and stops when the input - buffer becomes empty or the output buffer becomes full. It may introduce - some output latency (reading input without producing any output) except when - forced to flush. - - The detailed semantics are as follows. deflate performs one or both of the - following actions: - - - Compress more input starting at next_in and update next_in and avail_in - accordingly. If not all input can be processed (because there is not - enough room in the output buffer), next_in and avail_in are updated and - processing will resume at this point for the next call of deflate(). - - - Provide more output starting at next_out and update next_out and avail_out - accordingly. This action is forced if the parameter flush is non zero. - Forcing flush frequently degrades the compression ratio, so this parameter - should be set only when necessary (in interactive applications). Some - output may be provided even if flush is not set. - - Before the call of deflate(), the application should ensure that at least - one of the actions is possible, by providing more input and/or consuming more - output, and updating avail_in or avail_out accordingly; avail_out should - never be zero before the call. The application can consume the compressed - output when it wants, for example when the output buffer is full (avail_out - == 0), or after each call of deflate(). If deflate returns Z_OK and with - zero avail_out, it must be called again after making room in the output - buffer because there might be more output pending. - - Normally the parameter flush is set to Z_NO_FLUSH, which allows deflate to - decide how much data to accumulate before producing output, in order to - maximize compression. - - If the parameter flush is set to Z_SYNC_FLUSH, all pending output is - flushed to the output buffer and the output is aligned on a byte boundary, so - that the decompressor can get all input data available so far. (In - particular avail_in is zero after the call if enough output space has been - provided before the call.) Flushing may degrade compression for some - compression algorithms and so it should be used only when necessary. This - completes the current deflate block and follows it with an empty stored block - that is three bits plus filler bits to the next byte, followed by four bytes - (00 00 ff ff). - - If flush is set to Z_PARTIAL_FLUSH, all pending output is flushed to the - output buffer, but the output is not aligned to a byte boundary. All of the - input data so far will be available to the decompressor, as for Z_SYNC_FLUSH. - This completes the current deflate block and follows it with an empty fixed - codes block that is 10 bits long. This assures that enough bytes are output - in order for the decompressor to finish the block before the empty fixed code - block. - - If flush is set to Z_BLOCK, a deflate block is completed and emitted, as - for Z_SYNC_FLUSH, but the output is not aligned on a byte boundary, and up to - seven bits of the current block are held to be written as the next byte after - the next deflate block is completed. In this case, the decompressor may not - be provided enough bits at this point in order to complete decompression of - the data provided so far to the compressor. It may need to wait for the next - block to be emitted. This is for advanced applications that need to control - the emission of deflate blocks. - - If flush is set to Z_FULL_FLUSH, all output is flushed as with - Z_SYNC_FLUSH, and the compression state is reset so that decompression can - restart from this point if previous compressed data has been damaged or if - random access is desired. Using Z_FULL_FLUSH too often can seriously degrade - compression. - - If deflate returns with avail_out == 0, this function must be called again - with the same value of the flush parameter and more output space (updated - avail_out), until the flush is complete (deflate returns with non-zero - avail_out). In the case of a Z_FULL_FLUSH or Z_SYNC_FLUSH, make sure that - avail_out is greater than six to avoid repeated flush markers due to - avail_out == 0 on return. - - If the parameter flush is set to Z_FINISH, pending input is processed, - pending output is flushed and deflate returns with Z_STREAM_END if there was - enough output space; if deflate returns with Z_OK, this function must be - called again with Z_FINISH and more output space (updated avail_out) but no - more input data, until it returns with Z_STREAM_END or an error. After - deflate has returned Z_STREAM_END, the only possible operations on the stream - are deflateReset or deflateEnd. - - Z_FINISH can be used immediately after deflateInit if all the compression - is to be done in a single step. In this case, avail_out must be at least the - value returned by deflateBound (see below). Then deflate is guaranteed to - return Z_STREAM_END. If not enough output space is provided, deflate will - not return Z_STREAM_END, and it must be called again as described above. - - deflate() sets strm->adler to the adler32 checksum of all input read - so far (that is, total_in bytes). - - deflate() may update strm->data_type if it can make a good guess about - the input data type (Z_BINARY or Z_TEXT). In doubt, the data is considered - binary. This field is only for information purposes and does not affect the - compression algorithm in any manner. - - deflate() returns Z_OK if some progress has been made (more input - processed or more output produced), Z_STREAM_END if all input has been - consumed and all output has been produced (only when flush is set to - Z_FINISH), Z_STREAM_ERROR if the stream state was inconsistent (for example - if next_in or next_out was Z_NULL), Z_BUF_ERROR if no progress is possible - (for example avail_in or avail_out was zero). Note that Z_BUF_ERROR is not - fatal, and deflate() can be called again with more input and more output - space to continue compressing. -*/ - - -ZEXTERN int ZEXPORT deflateEnd(z_stream *strm); -/* - All dynamically allocated data structures for this stream are freed. - This function discards any unprocessed input and does not flush any pending - output. - - deflateEnd returns Z_OK if success, Z_STREAM_ERROR if the - stream state was inconsistent, Z_DATA_ERROR if the stream was freed - prematurely (some input or output was discarded). In the error case, msg - may be set but then points to a static string (which must not be - deallocated). -*/ - - -/* -ZEXTERN int ZEXPORT inflateInit (z_stream *strm); - - Initializes the internal stream state for decompression. The fields - next_in, avail_in, zalloc, zfree and opaque must be initialized before by - the caller. If next_in is not Z_NULL and avail_in is large enough (the - exact value depends on the compression method), inflateInit determines the - compression method from the zlib header and allocates all data structures - accordingly; otherwise the allocation will be deferred to the first call of - inflate. If zalloc and zfree are set to Z_NULL, inflateInit updates them to - use default allocation functions. - - inflateInit returns Z_OK if success, Z_MEM_ERROR if there was not enough - memory, Z_VERSION_ERROR if the zlib library version is incompatible with the - version assumed by the caller, or Z_STREAM_ERROR if the parameters are - invalid, such as a null pointer to the structure. msg is set to null if - there is no error message. inflateInit does not perform any decompression - apart from possibly reading the zlib header if present: actual decompression - will be done by inflate(). (So next_in and avail_in may be modified, but - next_out and avail_out are unused and unchanged.) The current implementation - of inflateInit() does not process any header information -- that is deferred - until inflate() is called. -*/ - - -ZEXTERN int ZEXPORT inflate(z_stream *strm, int flush); -/* - inflate decompresses as much data as possible, and stops when the input - buffer becomes empty or the output buffer becomes full. It may introduce - some output latency (reading input without producing any output) except when - forced to flush. - - The detailed semantics are as follows. inflate performs one or both of the - following actions: - - - Decompress more input starting at next_in and update next_in and avail_in - accordingly. If not all input can be processed (because there is not - enough room in the output buffer), next_in is updated and processing will - resume at this point for the next call of inflate(). - - - Provide more output starting at next_out and update next_out and avail_out - accordingly. inflate() provides as much output as possible, until there is - no more input data or no more space in the output buffer (see below about - the flush parameter). - - Before the call of inflate(), the application should ensure that at least - one of the actions is possible, by providing more input and/or consuming more - output, and updating the next_* and avail_* values accordingly. The - application can consume the uncompressed output when it wants, for example - when the output buffer is full (avail_out == 0), or after each call of - inflate(). If inflate returns Z_OK and with zero avail_out, it must be - called again after making room in the output buffer because there might be - more output pending. - - The flush parameter of inflate() can be Z_NO_FLUSH, Z_SYNC_FLUSH, Z_FINISH, - Z_BLOCK, or Z_TREES. Z_SYNC_FLUSH requests that inflate() flush as much - output as possible to the output buffer. Z_BLOCK requests that inflate() - stop if and when it gets to the next deflate block boundary. When decoding - the zlib or gzip format, this will cause inflate() to return immediately - after the header and before the first block. When doing a raw inflate, - inflate() will go ahead and process the first block, and will return when it - gets to the end of that block, or when it runs out of data. - - The Z_BLOCK option assists in appending to or combining deflate streams. - Also to assist in this, on return inflate() will set strm->data_type to the - number of unused bits in the last byte taken from strm->next_in, plus 64 if - inflate() is currently decoding the last block in the deflate stream, plus - 128 if inflate() returned immediately after decoding an end-of-block code or - decoding the complete header up to just before the first byte of the deflate - stream. The end-of-block will not be indicated until all of the uncompressed - data from that block has been written to strm->next_out. The number of - unused bits may in general be greater than seven, except when bit 7 of - data_type is set, in which case the number of unused bits will be less than - eight. data_type is set as noted here every time inflate() returns for all - flush options, and so can be used to determine the amount of currently - consumed input in bits. - - The Z_TREES option behaves as Z_BLOCK does, but it also returns when the - end of each deflate block header is reached, before any actual data in that - block is decoded. This allows the caller to determine the length of the - deflate block header for later use in random access within a deflate block. - 256 is added to the value of strm->data_type when inflate() returns - immediately after reaching the end of the deflate block header. - - inflate() should normally be called until it returns Z_STREAM_END or an - error. However if all decompression is to be performed in a single step (a - single call of inflate), the parameter flush should be set to Z_FINISH. In - this case all pending input is processed and all pending output is flushed; - avail_out must be large enough to hold all of the uncompressed data for the - operation to complete. (The size of the uncompressed data may have been - saved by the compressor for this purpose.) The use of Z_FINISH is not - required to perform an inflation in one step. However it may be used to - inform inflate that a faster approach can be used for the single inflate() - call. Z_FINISH also informs inflate to not maintain a sliding window if the - stream completes, which reduces inflate's memory footprint. If the stream - does not complete, either because not all of the stream is provided or not - enough output space is provided, then a sliding window will be allocated and - inflate() can be called again to continue the operation as if Z_NO_FLUSH had - been used. - - In this implementation, inflate() always flushes as much output as - possible to the output buffer, and always uses the faster approach on the - first call. So the effects of the flush parameter in this implementation are - on the return value of inflate() as noted below, when inflate() returns early - when Z_BLOCK or Z_TREES is used, and when inflate() avoids the allocation of - memory for a sliding window when Z_FINISH is used. - - If a preset dictionary is needed after this call (see inflateSetDictionary - below), inflate sets strm->adler to the Adler-32 checksum of the dictionary - chosen by the compressor and returns Z_NEED_DICT; otherwise it sets - strm->adler to the Adler-32 checksum of all output produced so far (that is, - total_out bytes) and returns Z_OK, Z_STREAM_END or an error code as described - below. At the end of the stream, inflate() checks that its computed adler32 - checksum is equal to that saved by the compressor and returns Z_STREAM_END - only if the checksum is correct. - - inflate() can decompress and check either zlib-wrapped or gzip-wrapped - deflate data. The header type is detected automatically, if requested when - initializing with inflateInit2(). Any information contained in the gzip - header is not retained, so applications that need that information should - instead use raw inflate, see inflateInit2() below, or inflateBack() and - perform their own processing of the gzip header and trailer. When processing - gzip-wrapped deflate data, strm->adler32 is set to the CRC-32 of the output - producted so far. The CRC-32 is checked against the gzip trailer. - - inflate() returns Z_OK if some progress has been made (more input processed - or more output produced), Z_STREAM_END if the end of the compressed data has - been reached and all uncompressed output has been produced, Z_NEED_DICT if a - preset dictionary is needed at this point, Z_DATA_ERROR if the input data was - corrupted (input stream not conforming to the zlib format or incorrect check - value), Z_STREAM_ERROR if the stream structure was inconsistent (for example - next_in or next_out was Z_NULL), Z_MEM_ERROR if there was not enough memory, - Z_BUF_ERROR if no progress is possible or if there was not enough room in the - output buffer when Z_FINISH is used. Note that Z_BUF_ERROR is not fatal, and - inflate() can be called again with more input and more output space to - continue decompressing. If Z_DATA_ERROR is returned, the application may - then call inflateSync() to look for a good compression block if a partial - recovery of the data is desired. -*/ - - -ZEXTERN int ZEXPORT inflateEnd(z_stream *strm); -/* - All dynamically allocated data structures for this stream are freed. - This function discards any unprocessed input and does not flush any pending - output. - - inflateEnd returns Z_OK if success, Z_STREAM_ERROR if the stream state - was inconsistent. In the error case, msg may be set but then points to a - static string (which must not be deallocated). -*/ - - - /* Advanced functions */ - -/* - The following functions are needed only in some special applications. -*/ - -/* -ZEXTERN int ZEXPORT deflateInit2 (z_stream *strm, - int level, - int method, - int windowBits, - int memLevel, - int strategy); - - This is another version of deflateInit with more compression options. The - fields next_in, zalloc, zfree and opaque must be initialized before by the - caller. - - The method parameter is the compression method. It must be Z_DEFLATED in - this version of the library. - - The windowBits parameter is the base two logarithm of the window size - (the size of the history buffer). It should be in the range 8..15 for this - version of the library. Larger values of this parameter result in better - compression at the expense of memory usage. The default value is 15 if - deflateInit is used instead. - - For the current implementation of deflate(), a windowBits value of 8 (a - window size of 256 bytes) is not supported. As a result, a request for 8 - will result in 9 (a 512-byte window). In that case, providing 8 to - inflateInit2() will result in an error when the zlib header with 9 is - checked against the initialization of inflate(). The remedy is to not use 8 - with deflateInit2() with this initialization, or at least in that case use 9 - with inflateInit2(). - - windowBits can also be -8..-15 for raw deflate. In this case, -windowBits - determines the window size. deflate() will then generate raw deflate data - with no zlib header or trailer, and will not compute an adler32 check value. - - windowBits can also be greater than 15 for optional gzip encoding. Add - 16 to windowBits to write a simple gzip header and trailer around the - compressed data instead of a zlib wrapper. The gzip header will have no - file name, no extra data, no comment, no modification time (set to zero), no - header crc, and the operating system will be set to 255 (unknown). If a - gzip stream is being written, strm->adler is a crc32 instead of an adler32. - - The memLevel parameter specifies how much memory should be allocated - for the internal compression state. memLevel=1 uses minimum memory but is - slow and reduces compression ratio; memLevel=9 uses maximum memory for - optimal speed. The default value is 8. See zconf.h for total memory usage - as a function of windowBits and memLevel. - - The strategy parameter is used to tune the compression algorithm. Use the - value Z_DEFAULT_STRATEGY for normal data, Z_FILTERED for data produced by a - filter (or predictor), Z_HUFFMAN_ONLY to force Huffman encoding only (no - string match), or Z_RLE to limit match distances to one (run-length - encoding). Filtered data consists mostly of small values with a somewhat - random distribution. In this case, the compression algorithm is tuned to - compress them better. The effect of Z_FILTERED is to force more Huffman - coding and less string matching; it is somewhat intermediate between - Z_DEFAULT_STRATEGY and Z_HUFFMAN_ONLY. Z_RLE is designed to be almost as - fast as Z_HUFFMAN_ONLY, but give better compression for PNG image data. The - strategy parameter only affects the compression ratio but not the - correctness of the compressed output even if it is not set appropriately. - Z_FIXED prevents the use of dynamic Huffman codes, allowing for a simpler - decoder for special applications. - - deflateInit2 returns Z_OK if success, Z_MEM_ERROR if there was not enough - memory, Z_STREAM_ERROR if any parameter is invalid (such as an invalid - method), or Z_VERSION_ERROR if the zlib library version (zlib_version) is - incompatible with the version assumed by the caller (ZLIB_VERSION). msg is - set to null if there is no error message. deflateInit2 does not perform any - compression: this will be done by deflate(). -*/ - -ZEXTERN int ZEXPORT deflateSetDictionary(z_stream *strm, - const unsigned char *dictionary, - unsigned int dictLength); -/* - Initializes the compression dictionary from the given byte sequence - without producing any compressed output. When using the zlib format, this - function must be called immediately after deflateInit, deflateInit2 or - deflateReset, and before any call of deflate. When doing raw deflate, this - function must be called either before any call of deflate, or immediately - after the completion of a deflate block, i.e. after all input has been - consumed and all output has been delivered when using any of the flush - options Z_BLOCK, Z_PARTIAL_FLUSH, Z_SYNC_FLUSH, or Z_FULL_FLUSH. The - compressor and decompressor must use exactly the same dictionary (see - inflateSetDictionary). - - The dictionary should consist of strings (byte sequences) that are likely - to be encountered later in the data to be compressed, with the most commonly - used strings preferably put towards the end of the dictionary. Using a - dictionary is most useful when the data to be compressed is short and can be - predicted with good accuracy; the data can then be compressed better than - with the default empty dictionary. - - Depending on the size of the compression data structures selected by - deflateInit or deflateInit2, a part of the dictionary may in effect be - discarded, for example if the dictionary is larger than the window size - provided in deflateInit or deflateInit2. Thus the strings most likely to be - useful should be put at the end of the dictionary, not at the front. In - addition, the current implementation of deflate will use at most the window - size minus 262 bytes of the provided dictionary. - - Upon return of this function, strm->adler is set to the adler32 value - of the dictionary; the decompressor may later use this value to determine - which dictionary has been used by the compressor. (The adler32 value - applies to the whole dictionary even if only a subset of the dictionary is - actually used by the compressor.) If a raw deflate was requested, then the - adler32 value is not computed and strm->adler is not set. - - deflateSetDictionary returns Z_OK if success, or Z_STREAM_ERROR if a - parameter is invalid (e.g. dictionary being Z_NULL) or the stream state is - inconsistent (for example if deflate has already been called for this stream - or if not at a block boundary for raw deflate). deflateSetDictionary does - not perform any compression: this will be done by deflate(). -*/ - -ZEXTERN int ZEXPORT deflateCopy(z_stream *dest, z_stream *source); -/* - Sets the destination stream as a complete copy of the source stream. - - This function can be useful when several compression strategies will be - tried, for example when there are several ways of pre-processing the input - data with a filter. The streams that will be discarded should then be freed - by calling deflateEnd. Note that deflateCopy duplicates the internal - compression state which can be quite large, so this strategy is slow and can - consume lots of memory. - - deflateCopy returns Z_OK if success, Z_MEM_ERROR if there was not - enough memory, Z_STREAM_ERROR if the source stream state was inconsistent - (such as zalloc being Z_NULL). msg is left unchanged in both source and - destination. -*/ - -ZEXTERN int ZEXPORT deflateReset(z_stream *strm); -/* - This function is equivalent to deflateEnd followed by deflateInit, but - does not free and reallocate the internal compression state. The stream - will leave the compression level and any other attributes that may have been - set unchanged. - - deflateReset returns Z_OK if success, or Z_STREAM_ERROR if the source - stream state was inconsistent (such as zalloc or state being Z_NULL). -*/ - -ZEXTERN int ZEXPORT deflateParams(z_stream *strm, int level, int strategy); -/* - Dynamically update the compression level and compression strategy. The - interpretation of level and strategy is as in deflateInit2(). This can be - used to switch between compression and straight copy of the input data, or - to switch to a different kind of input data requiring a different strategy. - If the compression approach (which is a function of the level) or the - strategy is changed, then the input available so far is compressed with the - old level and strategy using deflate(strm, Z_BLOCK). There are three - approaches for the compression levels 0, 1..3, and 4..9 respectively. The - new level and strategy will take effect at the next call of deflate(). - - If a deflate(strm, Z_BLOCK) is performed by deflateParams(), and it does - not have enough output space to complete, then the parameter change will - take effect at an undetermined location in the uncompressed data provided so - far. In order to assure a change in the parameters at a specific location - in the uncompressed data, the deflate stream should first be flushed with - Z_BLOCK or another flush parameter, and deflate() called until - strm.avail_out is not zero, before the call of deflateParams(). Then no - more input data should be provided before the deflateParams() call. If this - is done, the old level and strategy will be applied to the data compressed - before deflateParams(), and the new level and strategy will be applied to - the the data compressed after deflateParams(). - - deflateParams returns Z_OK if success, Z_STREAM_ERROR if the source stream - state was inconsistent or if a parameter was invalid, or Z_BUF_ERROR if - there was not enough output space to complete the compression before the - parameters were changed. Note that in the case of a Z_BUF_ERROR, the - parameters are changed nevertheless, and will take effect at an undetermined - location in the previously supplied uncompressed data. Compression may - proceed after a Z_BUF_ERROR. -*/ - -ZEXTERN int ZEXPORT deflateTune(z_stream *strm, int good_length, int max_lazy, int nice_length, int max_chain); -/* - Fine tune deflate's internal compression parameters. This should only be - used by someone who understands the algorithm used by zlib's deflate for - searching for the best matching string, and even then only by the most - fanatic optimizer trying to squeeze out the last compressed bit for their - specific input data. Read the deflate.c source code for the meaning of the - max_lazy, good_length, nice_length, and max_chain parameters. - - deflateTune() can be called after deflateInit() or deflateInit2(), and - returns Z_OK on success, or Z_STREAM_ERROR for an invalid deflate stream. - */ - -ZEXTERN unsigned long ZEXPORT deflateBound(z_stream *strm, unsigned long sourceLen); -/* - deflateBound() returns an upper bound on the compressed size after - deflation of sourceLen bytes. It must be called after deflateInit() or - deflateInit2(), and after deflateSetHeader(), if used. This would be used - to allocate an output buffer for deflation in a single pass, and so would be - called before deflate(). If that first deflate() call is provided the - sourceLen input bytes, an output buffer allocated to the size returned by - deflateBound(), and the flush value Z_FINISH, then deflate() is guaranteed - to return Z_STREAM_END. Note that it is possible for the compressed size to - be larger than the value returned by deflateBound() if flush options other - than Z_FINISH or Z_NO_FLUSH are used. -*/ - -ZEXTERN int ZEXPORT deflatePending(z_stream *strm, uint32_t *pending, int *bits); -/* - deflatePending() returns the number of bytes and bits of output that have - been generated, but not yet provided in the available output. The bytes not - provided would be due to the available output space having being consumed. - The number of bits of output not provided are between 0 and 7, where they - await more bits to join them in order to fill out a full byte. If pending - or bits are Z_NULL, then those values are not set. - - deflatePending returns Z_OK if success, or Z_STREAM_ERROR if the source - stream state was inconsistent. - */ - -ZEXTERN int ZEXPORT deflatePrime(z_stream *strm, int bits, int value); -/* - deflatePrime() inserts bits in the deflate output stream. The intent - is that this function is used to start off the deflate output with the bits - leftover from a previous deflate stream when appending to it. As such, this - function can only be used for raw deflate, and must be used before the first - deflate() call after a deflateInit2() or deflateReset(). bits must be less - than or equal to 16, and that many of the least significant bits of value - will be inserted in the output. - - deflatePrime returns Z_OK if success, Z_BUF_ERROR if there was not enough - room in the internal buffer to insert the bits, or Z_STREAM_ERROR if the - source stream state was inconsistent. -*/ - -ZEXTERN int ZEXPORT deflateSetHeader(z_stream *strm, gz_headerp head); -/* - deflateSetHeader() provides gzip header information for when a gzip - stream is requested by deflateInit2(). deflateSetHeader() may be called - after deflateInit2() or deflateReset() and before the first call of - deflate(). The text, time, os, extra field, name, and comment information - in the provided gz_header structure are written to the gzip header (xflag is - ignored -- the extra flags are set according to the compression level). The - caller must assure that, if not Z_NULL, name and comment are terminated with - a zero byte, and that if extra is not Z_NULL, that extra_len bytes are - available there. If hcrc is true, a gzip header crc is included. Note that - the current versions of the command-line version of gzip (up through version - 1.3.x) do not support header crc's, and will report that it is a "multi-part - gzip file" and give up. - - If deflateSetHeader is not used, the default gzip header has text false, - the time set to zero, and os set to 255, with no extra, name, or comment - fields. The gzip header is returned to the default state by deflateReset(). - - deflateSetHeader returns Z_OK if success, or Z_STREAM_ERROR if the source - stream state was inconsistent. -*/ - -/* -ZEXTERN int ZEXPORT inflateInit2(z_stream *strm, int windowBits); - - This is another version of inflateInit with an extra parameter. The - fields next_in, avail_in, zalloc, zfree and opaque must be initialized - before by the caller. - - The windowBits parameter is the base two logarithm of the maximum window - size (the size of the history buffer). It should be in the range 8..15 for - this version of the library. The default value is 15 if inflateInit is used - instead. windowBits must be greater than or equal to the windowBits value - provided to deflateInit2() while compressing, or it must be equal to 15 if - deflateInit2() was not used. If a compressed stream with a larger window - size is given as input, inflate() will return with the error code - Z_DATA_ERROR instead of trying to allocate a larger window. - - windowBits can also be zero to request that inflate use the window size in - the zlib header of the compressed stream. - - windowBits can also be -8..-15 for raw inflate. In this case, -windowBits - determines the window size. inflate() will then process raw deflate data, - not looking for a zlib or gzip header, not generating a check value, and not - looking for any check values for comparison at the end of the stream. This - is for use with other formats that use the deflate compressed data format - such as zip. Those formats provide their own check values. If a custom - format is developed using the raw deflate format for compressed data, it is - recommended that a check value such as an adler32 or a crc32 be applied to - the uncompressed data as is done in the zlib, gzip, and zip formats. For - most applications, the zlib format should be used as is. Note that comments - above on the use in deflateInit2() applies to the magnitude of windowBits. - - windowBits can also be greater than 15 for optional gzip decoding. Add - 32 to windowBits to enable zlib and gzip decoding with automatic header - detection, or add 16 to decode only the gzip format (the zlib format will - return a Z_DATA_ERROR). If a gzip stream is being decoded, strm->adler is a - crc32 instead of an adler32. - - inflateInit2 returns Z_OK if success, Z_MEM_ERROR if there was not enough - memory, Z_VERSION_ERROR if the zlib library version is incompatible with the - version assumed by the caller, or Z_STREAM_ERROR if the parameters are - invalid, such as a null pointer to the structure. msg is set to null if - there is no error message. inflateInit2 does not perform any decompression - apart from possibly reading the zlib header if present: actual decompression - will be done by inflate(). (So next_in and avail_in may be modified, but - next_out and avail_out are unused and unchanged.) The current implementation - of inflateInit2() does not process any header information -- that is - deferred until inflate() is called. -*/ - -ZEXTERN int ZEXPORT inflateSetDictionary(z_stream *strm, const unsigned char *dictionary, unsigned int dictLength); -/* - Initializes the decompression dictionary from the given uncompressed byte - sequence. This function must be called immediately after a call of inflate, - if that call returned Z_NEED_DICT. The dictionary chosen by the compressor - can be determined from the adler32 value returned by that call of inflate. - The compressor and decompressor must use exactly the same dictionary (see - deflateSetDictionary). For raw inflate, this function can be called at any - time to set the dictionary. If the provided dictionary is smaller than the - window and there is already data in the window, then the provided dictionary - will amend what's there. The application must insure that the dictionary - that was used for compression is provided. - - inflateSetDictionary returns Z_OK if success, Z_STREAM_ERROR if a - parameter is invalid (e.g. dictionary being Z_NULL) or the stream state is - inconsistent, Z_DATA_ERROR if the given dictionary doesn't match the - expected one (incorrect adler32 value). inflateSetDictionary does not - perform any decompression: this will be done by subsequent calls of - inflate(). -*/ - -ZEXTERN int ZEXPORT inflateGetDictionary(z_stream *strm, unsigned char *dictionary, unsigned int *dictLength); -/* - Returns the sliding dictionary being maintained by inflate. dictLength is - set to the number of bytes in the dictionary, and that many bytes are copied - to dictionary. dictionary must have enough space, where 32768 bytes is - always enough. If inflateGetDictionary() is called with dictionary equal to - Z_NULL, then only the dictionary length is returned, and nothing is copied. - Similary, if dictLength is Z_NULL, then it is not set. - - inflateGetDictionary returns Z_OK on success, or Z_STREAM_ERROR if the - stream state is inconsistent. -*/ - -ZEXTERN int ZEXPORT inflateSync(z_stream *strm); -/* - Skips invalid compressed data until a possible full flush point (see above - for the description of deflate with Z_FULL_FLUSH) can be found, or until all - available input is skipped. No output is provided. - - inflateSync searches for a 00 00 FF FF pattern in the compressed data. - All full flush points have this pattern, but not all occurrences of this - pattern are full flush points. - - inflateSync returns Z_OK if a possible full flush point has been found, - Z_BUF_ERROR if no more input was provided, Z_DATA_ERROR if no flush point - has been found, or Z_STREAM_ERROR if the stream structure was inconsistent. - In the success case, the application may save the current current value of - total_in which indicates where valid compressed data was found. In the - error case, the application may repeatedly call inflateSync, providing more - input each time, until success or end of the input data. -*/ - -ZEXTERN int ZEXPORT inflateCopy(z_stream *dest, z_stream *source); -/* - Sets the destination stream as a complete copy of the source stream. - - This function can be useful when randomly accessing a large stream. The - first pass through the stream can periodically record the inflate state, - allowing restarting inflate at those points when randomly accessing the - stream. - - inflateCopy returns Z_OK if success, Z_MEM_ERROR if there was not - enough memory, Z_STREAM_ERROR if the source stream state was inconsistent - (such as zalloc being Z_NULL). msg is left unchanged in both source and - destination. -*/ - -ZEXTERN int ZEXPORT inflateReset(z_stream *strm); -/* - This function is equivalent to inflateEnd followed by inflateInit, - but does not free and reallocate all the internal decompression state. The - stream will keep attributes that may have been set by inflateInit2. - - inflateReset returns Z_OK if success, or Z_STREAM_ERROR if the source - stream state was inconsistent (such as zalloc or state being Z_NULL). -*/ - -ZEXTERN int ZEXPORT inflateReset2(z_stream *strm, int windowBits); -/* - This function is the same as inflateReset, but it also permits changing - the wrap and window size requests. The windowBits parameter is interpreted - the same as it is for inflateInit2. - - inflateReset2 returns Z_OK if success, or Z_STREAM_ERROR if the source - stream state was inconsistent (such as zalloc or state being Z_NULL), or if - the windowBits parameter is invalid. -*/ - -ZEXTERN int ZEXPORT inflatePrime(z_stream *strm, int bits, int value); -/* - This function inserts bits in the inflate input stream. The intent is - that this function is used to start inflating at a bit position in the - middle of a byte. The provided bits will be used before any bytes are used - from next_in. This function should only be used with raw inflate, and - should be used before the first inflate() call after inflateInit2() or - inflateReset(). bits must be less than or equal to 16, and that many of the - least significant bits of value will be inserted in the input. - - If bits is negative, then the input stream bit buffer is emptied. Then - inflatePrime() can be called again to put bits in the buffer. This is used - to clear out bits leftover after feeding inflate a block description prior - to feeding inflate codes. - - inflatePrime returns Z_OK if success, or Z_STREAM_ERROR if the source - stream state was inconsistent. -*/ - -ZEXTERN long ZEXPORT inflateMark(z_stream *strm); -/* - This function returns two values, one in the lower 16 bits of the return - value, and the other in the remaining upper bits, obtained by shifting the - return value down 16 bits. If the upper value is -1 and the lower value is - zero, then inflate() is currently decoding information outside of a block. - If the upper value is -1 and the lower value is non-zero, then inflate is in - the middle of a stored block, with the lower value equaling the number of - bytes from the input remaining to copy. If the upper value is not -1, then - it is the number of bits back from the current bit position in the input of - the code (literal or length/distance pair) currently being processed. In - that case the lower value is the number of bytes already emitted for that - code. - - A code is being processed if inflate is waiting for more input to complete - decoding of the code, or if it has completed decoding but is waiting for - more output space to write the literal or match data. - - inflateMark() is used to mark locations in the input data for random - access, which may be at bit positions, and to note those cases where the - output of a code may span boundaries of random access blocks. The current - location in the input stream can be determined from avail_in and data_type - as noted in the description for the Z_BLOCK flush parameter for inflate. - - inflateMark returns the value noted above or -65536 if the provided - source stream state was inconsistent. -*/ - -ZEXTERN int ZEXPORT inflateGetHeader(z_stream *strm, gz_headerp head); -/* - inflateGetHeader() requests that gzip header information be stored in the - provided gz_header structure. inflateGetHeader() may be called after - inflateInit2() or inflateReset(), and before the first call of inflate(). - As inflate() processes the gzip stream, head->done is zero until the header - is completed, at which time head->done is set to one. If a zlib stream is - being decoded, then head->done is set to -1 to indicate that there will be - no gzip header information forthcoming. Note that Z_BLOCK or Z_TREES can be - used to force inflate() to return immediately after header processing is - complete and before any actual data is decompressed. - - The text, time, xflags, and os fields are filled in with the gzip header - contents. hcrc is set to true if there is a header CRC. (The header CRC - was valid if done is set to one.) If extra is not Z_NULL, then extra_max - contains the maximum number of bytes to write to extra. Once done is true, - extra_len contains the actual extra field length, and extra contains the - extra field, or that field truncated if extra_max is less than extra_len. - If name is not Z_NULL, then up to name_max characters are written there, - terminated with a zero unless the length is greater than name_max. If - comment is not Z_NULL, then up to comm_max characters are written there, - terminated with a zero unless the length is greater than comm_max. When any - of extra, name, or comment are not Z_NULL and the respective field is not - present in the header, then that field is set to Z_NULL to signal its - absence. This allows the use of deflateSetHeader() with the returned - structure to duplicate the header. However if those fields are set to - allocated memory, then the application will need to save those pointers - elsewhere so that they can be eventually freed. - - If inflateGetHeader is not used, then the header information is simply - discarded. The header is always checked for validity, including the header - CRC if present. inflateReset() will reset the process to discard the header - information. The application would need to call inflateGetHeader() again to - retrieve the header from the next gzip stream. - - inflateGetHeader returns Z_OK if success, or Z_STREAM_ERROR if the source - stream state was inconsistent. -*/ - -/* -ZEXTERN int ZEXPORT inflateBackInit (z_stream *strm, int windowBits, unsigned char *window); - - Initialize the internal stream state for decompression using inflateBack() - calls. The fields zalloc, zfree and opaque in strm must be initialized - before the call. If zalloc and zfree are Z_NULL, then the default library- - derived memory allocation routines are used. windowBits is the base two - logarithm of the window size, in the range 8..15. window is a caller - supplied buffer of that size. Except for special applications where it is - assured that deflate was used with small window sizes, windowBits must be 15 - and a 32K byte window must be supplied to be able to decompress general - deflate streams. - - See inflateBack() for the usage of these routines. - - inflateBackInit will return Z_OK on success, Z_STREAM_ERROR if any of - the parameters are invalid, Z_MEM_ERROR if the internal state could not be - allocated, or Z_VERSION_ERROR if the version of the library does not match - the version of the header file. -*/ - -typedef uint32_t (*in_func) (void *, const unsigned char * *); -typedef int (*out_func) (void *, unsigned char *, uint32_t); - -ZEXTERN int ZEXPORT inflateBack(z_stream *strm, in_func in, void *in_desc, out_func out, void *out_desc); -/* - inflateBack() does a raw inflate with a single call using a call-back - interface for input and output. This is potentially more efficient than - inflate() for file i/o applications, in that it avoids copying between the - output and the sliding window by simply making the window itself the output - buffer. inflate() can be faster on modern CPUs when used with large - buffers. inflateBack() trusts the application to not change the output - buffer passed by the output function, at least until inflateBack() returns. - - inflateBackInit() must be called first to allocate the internal state - and to initialize the state with the user-provided window buffer. - inflateBack() may then be used multiple times to inflate a complete, raw - deflate stream with each call. inflateBackEnd() is then called to free the - allocated state. - - A raw deflate stream is one with no zlib or gzip header or trailer. - This routine would normally be used in a utility that reads zip or gzip - files and writes out uncompressed files. The utility would decode the - header and process the trailer on its own, hence this routine expects only - the raw deflate stream to decompress. This is different from the normal - behavior of inflate(), which expects either a zlib or gzip header and - trailer around the deflate stream. - - inflateBack() uses two subroutines supplied by the caller that are then - called by inflateBack() for input and output. inflateBack() calls those - routines until it reads a complete deflate stream and writes out all of the - uncompressed data, or until it encounters an error. The function's - parameters and return types are defined above in the in_func and out_func - typedefs. inflateBack() will call in(in_desc, &buf) which should return the - number of bytes of provided input, and a pointer to that input in buf. If - there is no input available, in() must return zero--buf is ignored in that - case--and inflateBack() will return a buffer error. inflateBack() will call - out(out_desc, buf, len) to write the uncompressed data buf[0..len-1]. out() - should return zero on success, or non-zero on failure. If out() returns - non-zero, inflateBack() will return with an error. Neither in() nor out() - are permitted to change the contents of the window provided to - inflateBackInit(), which is also the buffer that out() uses to write from. - The length written by out() will be at most the window size. Any non-zero - amount of input may be provided by in(). - - For convenience, inflateBack() can be provided input on the first call by - setting strm->next_in and strm->avail_in. If that input is exhausted, then - in() will be called. Therefore strm->next_in must be initialized before - calling inflateBack(). If strm->next_in is Z_NULL, then in() will be called - immediately for input. If strm->next_in is not Z_NULL, then strm->avail_in - must also be initialized, and then if strm->avail_in is not zero, input will - initially be taken from strm->next_in[0 .. strm->avail_in - 1]. - - The in_desc and out_desc parameters of inflateBack() is passed as the - first parameter of in() and out() respectively when they are called. These - descriptors can be optionally used to pass any information that the caller- - supplied in() and out() functions need to do their job. - - On return, inflateBack() will set strm->next_in and strm->avail_in to - pass back any unused input that was provided by the last in() call. The - return values of inflateBack() can be Z_STREAM_END on success, Z_BUF_ERROR - if in() or out() returned an error, Z_DATA_ERROR if there was a format error - in the deflate stream (in which case strm->msg is set to indicate the nature - of the error), or Z_STREAM_ERROR if the stream was not properly initialized. - In the case of Z_BUF_ERROR, an input or output error can be distinguished - using strm->next_in which will be Z_NULL only if in() returned an error. If - strm->next_in is not Z_NULL, then the Z_BUF_ERROR was due to out() returning - non-zero. (in() will always be called before out(), so strm->next_in is - assured to be defined if out() returns non-zero.) Note that inflateBack() - cannot return Z_OK. -*/ - -ZEXTERN int ZEXPORT inflateBackEnd(z_stream *strm); -/* - All memory allocated by inflateBackInit() is freed. - - inflateBackEnd() returns Z_OK on success, or Z_STREAM_ERROR if the stream - state was inconsistent. -*/ - -ZEXTERN unsigned long ZEXPORT zlibCompileFlags(void); -/* Return flags indicating compile-time options. - - Type sizes, two bits each, 00 = 16 bits, 01 = 32, 10 = 64, 11 = other: - 1.0: size of unsigned int - 3.2: size of unsigned long - 5.4: size of void * (pointer) - 7.6: size of z_off_t - - Compiler, assembler, and debug options: - 8: DEBUG - 9: ASMV or ASMINF -- use ASM code - 10: ZLIB_WINAPI -- exported functions use the WINAPI calling convention - 11: 0 (reserved) - - One-time table building (smaller code, but not thread-safe if true): - 12: BUILDFIXED -- build static block decoding tables when needed - 13: DYNAMIC_CRC_TABLE -- build CRC calculation tables when needed - 14,15: 0 (reserved) - - Library content (indicates missing functionality): - 16: NO_GZCOMPRESS -- gz* functions cannot compress (to avoid linking - deflate code when not needed) - 17: NO_GZIP -- deflate can't write gzip streams, and inflate can't detect - and decode gzip streams (to avoid linking crc code) - 18-19: 0 (reserved) - - Operation variations (changes in library functionality): - 20: PKZIP_BUG_WORKAROUND -- slightly more permissive inflate - 21: FASTEST -- deflate algorithm with only one, lowest compression level - 22,23: 0 (reserved) - - The sprintf variant used by gzprintf (zero is best): - 24: 0 = vs*, 1 = s* -- 1 means limited to 20 arguments after the format - 25: 0 = *nprintf, 1 = *printf -- 1 means gzprintf() not secure! - 26: 0 = returns value, 1 = void -- 1 means inferred string length returned - - Remainder: - 27-31: 0 (reserved) - */ - - - /* utility functions */ - -/* - The following utility functions are implemented on top of the basic - stream-oriented functions. To simplify the interface, some default options - are assumed (compression level and memory usage, standard memory allocation - functions). The source code of these utility functions can be modified if - you need special options. -*/ - -ZEXTERN int ZEXPORT compress(unsigned char *dest, size_t *destLen, const unsigned char *source, size_t sourceLen); -/* - Compresses the source buffer into the destination buffer. sourceLen is - the byte length of the source buffer. Upon entry, destLen is the total size - of the destination buffer, which must be at least the value returned by - compressBound(sourceLen). Upon exit, destLen is the actual size of the - compressed buffer. compress() is equivalent to compress2() with a level - parameter of Z_DEFAULT_COMPRESSION. - - compress returns Z_OK if success, Z_MEM_ERROR if there was not - enough memory, Z_BUF_ERROR if there was not enough room in the output - buffer. -*/ - -ZEXTERN int ZEXPORT compress2(unsigned char *dest, size_t *destLen, const unsigned char *source, - size_t sourceLen, int level); -/* - Compresses the source buffer into the destination buffer. The level - parameter has the same meaning as in deflateInit. sourceLen is the byte - length of the source buffer. Upon entry, destLen is the total size of the - destination buffer, which must be at least the value returned by - compressBound(sourceLen). Upon exit, destLen is the actual size of the - compressed buffer. - - compress2 returns Z_OK if success, Z_MEM_ERROR if there was not enough - memory, Z_BUF_ERROR if there was not enough room in the output buffer, - Z_STREAM_ERROR if the level parameter is invalid. -*/ - -ZEXTERN size_t ZEXPORT compressBound(size_t sourceLen); -/* - compressBound() returns an upper bound on the compressed size after - compress() or compress2() on sourceLen bytes. It would be used before a - compress() or compress2() call to allocate the destination buffer. -*/ - -ZEXTERN int ZEXPORT uncompress(unsigned char *dest, size_t *destLen, const unsigned char *source, size_t sourceLen); -/* - Decompresses the source buffer into the destination buffer. sourceLen is - the byte length of the source buffer. Upon entry, destLen is the total size - of the destination buffer, which must be large enough to hold the entire - uncompressed data. (The size of the uncompressed data must have been saved - previously by the compressor and transmitted to the decompressor by some - mechanism outside the scope of this compression library.) Upon exit, destLen - is the actual size of the uncompressed buffer. - - uncompress returns Z_OK if success, Z_MEM_ERROR if there was not - enough memory, Z_BUF_ERROR if there was not enough room in the output - buffer, or Z_DATA_ERROR if the input data was corrupted or incomplete. In - the case where there is not enough room, uncompress() will fill the output - buffer with the uncompressed data up to that point. -*/ - -#ifdef WITH_GZFILEOP - /* gzip file access functions */ - -/* - This library supports reading and writing files in gzip (.gz) format with - an interface similar to that of stdio, using the functions that start with - "gz". The gzip format is different from the zlib format. gzip is a gzip - wrapper, documented in RFC 1952, wrapped around a deflate stream. -*/ - -typedef struct gzFile_s *gzFile; /* semi-opaque gzip file descriptor */ - -/* -ZEXTERN gzFile ZEXPORT gzopen(const char *path, const char *mode); - - Opens a gzip (.gz) file for reading or writing. The mode parameter is as - in fopen ("rb" or "wb") but can also include a compression level ("wb9") or - a strategy: 'f' for filtered data as in "wb6f", 'h' for Huffman-only - compression as in "wb1h", 'R' for run-length encoding as in "wb1R", or 'F' - for fixed code compression as in "wb9F". (See the description of - deflateInit2 for more information about the strategy parameter.) 'T' will - request transparent writing or appending with no compression and not using - the gzip format. - - "a" can be used instead of "w" to request that the gzip stream that will - be written be appended to the file. "+" will result in an error, since - reading and writing to the same gzip file is not supported. The addition of - "x" when writing will create the file exclusively, which fails if the file - already exists. On systems that support it, the addition of "e" when - reading or writing will set the flag to close the file on an execve() call. - - These functions, as well as gzip, will read and decode a sequence of gzip - streams in a file. The append function of gzopen() can be used to create - such a file. (Also see gzflush() for another way to do this.) When - appending, gzopen does not test whether the file begins with a gzip stream, - nor does it look for the end of the gzip streams to begin appending. gzopen - will simply append a gzip stream to the existing file. - - gzopen can be used to read a file which is not in gzip format; in this - case gzread will directly read from the file without decompression. When - reading, this will be detected automatically by looking for the magic two- - byte gzip header. - - gzopen returns NULL if the file could not be opened, if there was - insufficient memory to allocate the gzFile state, or if an invalid mode was - specified (an 'r', 'w', or 'a' was not provided, or '+' was provided). - errno can be checked to determine if the reason gzopen failed was that the - file could not be opened. -*/ - -ZEXTERN gzFile ZEXPORT gzdopen(int fd, const char *mode); -/* - gzdopen associates a gzFile with the file descriptor fd. File descriptors - are obtained from calls like open, dup, creat, pipe or fileno (if the file - has been previously opened with fopen). The mode parameter is as in gzopen. - - The next call of gzclose on the returned gzFile will also close the file - descriptor fd, just like fclose(fdopen(fd, mode)) closes the file descriptor - fd. If you want to keep fd open, use fd = dup(fd_keep); gz = gzdopen(fd, - mode);. The duplicated descriptor should be saved to avoid a leak, since - gzdopen does not close fd if it fails. If you are using fileno() to get the - file descriptor from a FILE *, then you will have to use dup() to avoid - double-close()ing the file descriptor. Both gzclose() and fclose() will - close the associated file descriptor, so they need to have different file - descriptors. - - gzdopen returns NULL if there was insufficient memory to allocate the - gzFile state, if an invalid mode was specified (an 'r', 'w', or 'a' was not - provided, or '+' was provided), or if fd is -1. The file descriptor is not - used until the next gz* read, write, seek, or close operation, so gzdopen - will not detect if fd is invalid (unless fd is -1). -*/ - -ZEXTERN int ZEXPORT gzbuffer(gzFile file, unsigned size); -/* - Set the internal buffer size used by this library's functions. The - default buffer size is 8192 bytes. This function must be called after - gzopen() or gzdopen(), and before any other calls that read or write the - file. The buffer memory allocation is always deferred to the first read or - write. Three times that size in buffer space is allocated. A larger buffer - size of, for example, 64K or 128K bytes will noticeably increase the speed - of decompression (reading). - - The new buffer size also affects the maximum length for gzprintf(). - - gzbuffer() returns 0 on success, or -1 on failure, such as being called - too late. -*/ - -ZEXTERN int ZEXPORT gzsetparams(gzFile file, int level, int strategy); -/* - Dynamically update the compression level or strategy. See the description - of deflateInit2 for the meaning of these parameters. - - gzsetparams returns Z_OK if success, or Z_STREAM_ERROR if the file was not - opened for writing. -*/ - -ZEXTERN int ZEXPORT gzread(gzFile file, void *buf, unsigned len); -/* - Reads the given number of uncompressed bytes from the compressed file. If - the input file is not in gzip format, gzread copies the given number of - bytes into the buffer directly from the file. - - After reaching the end of a gzip stream in the input, gzread will continue - to read, looking for another gzip stream. Any number of gzip streams may be - concatenated in the input file, and will all be decompressed by gzread(). - If something other than a gzip stream is encountered after a gzip stream, - that remaining trailing garbage is ignored (and no error is returned). - - gzread can be used to read a gzip file that is being concurrently written. - Upon reaching the end of the input, gzread will return with the available - data. If the error code returned by gzerror is Z_OK or Z_BUF_ERROR, then - gzclearerr can be used to clear the end of file indicator in order to permit - gzread to be tried again. Z_OK indicates that a gzip stream was completed - on the last gzread. Z_BUF_ERROR indicates that the input file ended in the - middle of a gzip stream. Note that gzread does not return -1 in the event - of an incomplete gzip stream. This error is deferred until gzclose(), which - will return Z_BUF_ERROR if the last gzread ended in the middle of a gzip - stream. Alternatively, gzerror can be used before gzclose to detect this - case. - - gzread returns the number of uncompressed bytes actually read, less than - len for end of file, or -1 for error. -*/ - -ZEXTERN int ZEXPORT gzwrite(gzFile file, void const *buf, unsigned len); -/* - Writes the given number of uncompressed bytes into the compressed file. - gzwrite returns the number of uncompressed bytes written or 0 in case of - error. -*/ - -ZEXTERN int ZEXPORTVA gzprintf(gzFile file, const char *format, ...); -/* - Converts, formats, and writes the arguments to the compressed file under - control of the format string, as in fprintf. gzprintf returns the number of - uncompressed bytes actually written, or 0 in case of error. The number of - uncompressed bytes written is limited to 8191, or one less than the buffer - size given to gzbuffer(). The caller should assure that this limit is not - exceeded. If it is exceeded, then gzprintf() will return an error (0) with - nothing written. In this case, there may also be a buffer overflow with - unpredictable consequences, which is possible only if zlib was compiled with - the insecure functions sprintf() or vsprintf() because the secure snprintf() - or vsnprintf() functions were not available. This can be determined using - zlibCompileFlags(). -*/ - -ZEXTERN int ZEXPORT gzputs(gzFile file, const char *s); -/* - Writes the given null-terminated string to the compressed file, excluding - the terminating null character. - - gzputs returns the number of characters written, or -1 in case of error. -*/ - -ZEXTERN char * ZEXPORT gzgets(gzFile file, char *buf, int len); -/* - Reads bytes from the compressed file until len-1 characters are read, or a - newline character is read and transferred to buf, or an end-of-file - condition is encountered. If any characters are read or if len == 1, the - string is terminated with a null character. If no characters are read due - to an end-of-file or len < 1, then the buffer is left untouched. - - gzgets returns buf which is a null-terminated string, or it returns NULL - for end-of-file or in case of error. If there was an error, the contents at - buf are indeterminate. -*/ - -ZEXTERN int ZEXPORT gzputc(gzFile file, int c); -/* - Writes c, converted to an unsigned char, into the compressed file. gzputc - returns the value that was written, or -1 in case of error. -*/ - -ZEXTERN int ZEXPORT gzgetc(gzFile file); -/* - Reads one byte from the compressed file. gzgetc returns this byte or -1 - in case of end of file or error. This is implemented as a macro for speed. - As such, it does not do all of the checking the other functions do. I.e. - it does not check to see if file is NULL, nor whether the structure file - points to has been clobbered or not. -*/ - -ZEXTERN int ZEXPORT gzungetc(int c, gzFile file); -/* - Push one character back onto the stream to be read as the first character - on the next read. At least one character of push-back is allowed. - gzungetc() returns the character pushed, or -1 on failure. gzungetc() will - fail if c is -1, and may fail if a character has been pushed but not read - yet. If gzungetc is used immediately after gzopen or gzdopen, at least the - output buffer size of pushed characters is allowed. (See gzbuffer above.) - The pushed character will be discarded if the stream is repositioned with - gzseek() or gzrewind(). -*/ - -ZEXTERN int ZEXPORT gzflush(gzFile file, int flush); -/* - Flushes all pending output into the compressed file. The parameter flush - is as in the deflate() function. The return value is the zlib error number - (see function gzerror below). gzflush is only permitted when writing. - - If the flush parameter is Z_FINISH, the remaining data is written and the - gzip stream is completed in the output. If gzwrite() is called again, a new - gzip stream will be started in the output. gzread() is able to read such - concatented gzip streams. - - gzflush should be called only when strictly necessary because it will - degrade compression if called too often. -*/ - -/* -ZEXTERN z_off_t ZEXPORT gzseek (gzFile file, z_off_t offset, int whence); - - Sets the starting position for the next gzread or gzwrite on the given - compressed file. The offset represents a number of bytes in the - uncompressed data stream. The whence parameter is defined as in lseek(2); - the value SEEK_END is not supported. - - If the file is opened for reading, this function is emulated but can be - extremely slow. If the file is opened for writing, only forward seeks are - supported; gzseek then compresses a sequence of zeroes up to the new - starting position. - - gzseek returns the resulting offset location as measured in bytes from - the beginning of the uncompressed stream, or -1 in case of error, in - particular if the file is opened for writing and the new starting position - would be before the current position. -*/ - -ZEXTERN int ZEXPORT gzrewind(gzFile file); -/* - Rewinds the given file. This function is supported only for reading. - - gzrewind(file) is equivalent to (int)gzseek(file, 0L, SEEK_SET) -*/ - -/* -ZEXTERN z_off_t ZEXPORT gztell(gzFile file); - - Returns the starting position for the next gzread or gzwrite on the given - compressed file. This position represents a number of bytes in the - uncompressed data stream, and is zero when starting, even if appending or - reading a gzip stream from the middle of a file using gzdopen(). - - gztell(file) is equivalent to gzseek(file, 0L, SEEK_CUR) -*/ - -/* -ZEXTERN z_off_t ZEXPORT gzoffset(gzFile file); - - Returns the current offset in the file being read or written. This offset - includes the count of bytes that precede the gzip stream, for example when - appending or when using gzdopen() for reading. When reading, the offset - does not include as yet unused buffered input. This information can be used - for a progress indicator. On error, gzoffset() returns -1. -*/ - -ZEXTERN int ZEXPORT gzeof(gzFile file); -/* - Returns true (1) if the end-of-file indicator has been set while reading, - false (0) otherwise. Note that the end-of-file indicator is set only if the - read tried to go past the end of the input, but came up short. Therefore, - just like feof(), gzeof() may return false even if there is no more data to - read, in the event that the last read request was for the exact number of - bytes remaining in the input file. This will happen if the input file size - is an exact multiple of the buffer size. - - If gzeof() returns true, then the read functions will return no more data, - unless the end-of-file indicator is reset by gzclearerr() and the input file - has grown since the previous end of file was detected. -*/ - -ZEXTERN int ZEXPORT gzdirect(gzFile file); -/* - Returns true (1) if file is being copied directly while reading, or false - (0) if file is a gzip stream being decompressed. - - If the input file is empty, gzdirect() will return true, since the input - does not contain a gzip stream. - - If gzdirect() is used immediately after gzopen() or gzdopen() it will - cause buffers to be allocated to allow reading the file to determine if it - is a gzip file. Therefore if gzbuffer() is used, it should be called before - gzdirect(). - - When writing, gzdirect() returns true (1) if transparent writing was - requested ("wT" for the gzopen() mode), or false (0) otherwise. (Note: - gzdirect() is not needed when writing. Transparent writing must be - explicitly requested, so the application already knows the answer. When - linking statically, using gzdirect() will include all of the zlib code for - gzip file reading and decompression, which may not be desired.) -*/ - -ZEXTERN int ZEXPORT gzclose(gzFile file); -/* - Flushes all pending output if necessary, closes the compressed file and - deallocates the (de)compression state. Note that once file is closed, you - cannot call gzerror with file, since its structures have been deallocated. - gzclose must not be called more than once on the same file, just as free - must not be called more than once on the same allocation. - - gzclose will return Z_STREAM_ERROR if file is not valid, Z_ERRNO on a - file operation error, Z_MEM_ERROR if out of memory, Z_BUF_ERROR if the - last read ended in the middle of a gzip stream, or Z_OK on success. -*/ - -ZEXTERN int ZEXPORT gzclose_r(gzFile file); -ZEXTERN int ZEXPORT gzclose_w(gzFile file); -/* - Same as gzclose(), but gzclose_r() is only for use when reading, and - gzclose_w() is only for use when writing or appending. The advantage to - using these instead of gzclose() is that they avoid linking in zlib - compression or decompression code that is not used when only reading or only - writing respectively. If gzclose() is used, then both compression and - decompression code will be included the application when linking to a static - zlib library. -*/ - -ZEXTERN const char * ZEXPORT gzerror(gzFile file, int *errnum); -/* - Returns the error message for the last error which occurred on the given - compressed file. errnum is set to zlib error number. If an error occurred - in the file system and not in the compression library, errnum is set to - Z_ERRNO and the application may consult errno to get the exact error code. - - The application must not modify the returned string. Future calls to - this function may invalidate the previously returned string. If file is - closed, then the string previously returned by gzerror will no longer be - available. - - gzerror() should be used to distinguish errors from end-of-file for those - functions above that do not distinguish those cases in their return values. -*/ - -ZEXTERN void ZEXPORT gzclearerr(gzFile file); -/* - Clears the error and end-of-file flags for file. This is analogous to the - clearerr() function in stdio. This is useful for continuing to read a gzip - file that is being written concurrently. -*/ - -#endif /* WITH_GZFILEOP */ - - /* checksum functions */ - -/* - These functions are not related to compression but are exported - anyway because they might be useful in applications using the compression - library. -*/ - -ZEXTERN uint32_t ZEXPORT adler32(uint32_t adler, const unsigned char *buf, uint32_t len); -/* - Update a running Adler-32 checksum with the bytes buf[0..len-1] and - return the updated checksum. If buf is Z_NULL, this function returns the - required initial value for the checksum. - - An Adler-32 checksum is almost as reliable as a CRC32 but can be computed - much faster. - - Usage example: - - uint32_t adler = adler32(0L, Z_NULL, 0); - - while (read_buffer(buffer, length) != EOF) { - adler = adler32(adler, buffer, length); - } - if (adler != original_adler) error(); -*/ - -/* -ZEXTERN uint32_t ZEXPORT adler32_combine(uint32_t adler1, uint32_t adler2, z_off_t len2); - - Combine two Adler-32 checksums into one. For two sequences of bytes, seq1 - and seq2 with lengths len1 and len2, Adler-32 checksums were calculated for - each, adler1 and adler2. adler32_combine() returns the Adler-32 checksum of - seq1 and seq2 concatenated, requiring only adler1, adler2, and len2. Note - that the z_off_t type (like off_t) is a signed integer. If len2 is - negative, the result has no meaning or utility. -*/ - -ZEXTERN uint32_t ZEXPORT crc32(uint32_t crc, const unsigned char *buf, z_off64_t len); -/* - Update a running CRC-32 with the bytes buf[0..len-1] and return the - updated CRC-32. If buf is Z_NULL, this function returns the required - initial value for the crc. Pre- and post-conditioning (one's complement) is - performed within this function so it shouldn't be done by the application. - - Usage example: - - uint32_t crc = crc32(0L, Z_NULL, 0); - - while (read_buffer(buffer, length) != EOF) { - crc = crc32(crc, buffer, length); - } - if (crc != original_crc) error(); -*/ - -/* -ZEXTERN uint32_t ZEXPORT crc32_combine(uint32_t crc1, uint32_t crc2, z_off64_t len2); - - Combine two CRC-32 check values into one. For two sequences of bytes, - seq1 and seq2 with lengths len1 and len2, CRC-32 check values were - calculated for each, crc1 and crc2. crc32_combine() returns the CRC-32 - check value of seq1 and seq2 concatenated, requiring only crc1, crc2, and - len2. -*/ - - - /* various hacks, don't look :) */ - -/* deflateInit and inflateInit are macros to allow checking the zlib version - * and the compiler's view of z_stream: - */ -ZEXTERN int ZEXPORT deflateInit_(z_stream *strm, int level, const char *version, int stream_size); -ZEXTERN int ZEXPORT inflateInit_(z_stream *strm, const char *version, int stream_size); -ZEXTERN int ZEXPORT deflateInit2_(z_stream *strm, int level, int method, int windowBits, int memLevel, - int strategy, const char *version, int stream_size); -ZEXTERN int ZEXPORT inflateInit2_(z_stream *strm, int windowBits, const char *version, int stream_size); -ZEXTERN int ZEXPORT inflateBackInit_(z_stream *strm, int windowBits, unsigned char *window, - const char *version, int stream_size); -#define deflateInit(strm, level) deflateInit_((strm), (level), ZLIB_VERSION, (int)sizeof(z_stream)) -#define inflateInit(strm) inflateInit_((strm), ZLIB_VERSION, (int)sizeof(z_stream)) -#define deflateInit2(strm, level, method, windowBits, memLevel, strategy) \ - deflateInit2_((strm), (level), (method), (windowBits), (memLevel), \ - (strategy), ZLIB_VERSION, (int)sizeof(z_stream)) -#define inflateInit2(strm, windowBits) inflateInit2_((strm), (windowBits), ZLIB_VERSION, (int)sizeof(z_stream)) -#define inflateBackInit(strm, windowBits, window) \ - inflateBackInit_((strm), (windowBits), (window), ZLIB_VERSION, (int)sizeof(z_stream)) - -#ifdef WITH_GZFILEOP - -/* gzgetc() macro and its supporting function and exposed data structure. Note - * that the real internal state is much larger than the exposed structure. - * This abbreviated structure exposes just enough for the gzgetc() macro. The - * user should not mess with these exposed elements, since their names or - * behavior could change in the future, perhaps even capriciously. They can - * only be used by the gzgetc() macro. You have been warned. - */ -struct gzFile_s { - unsigned have; - unsigned char *next; - z_off64_t pos; -}; -ZEXTERN int ZEXPORT gzgetc_(gzFile file); /* backward compatibility */ -# define gzgetc(g) ((g)->have ? ((g)->have--, (g)->pos++, *((g)->next)++) : gzgetc(g)) - -/* provide 64-bit offset functions if _LARGEFILE64_SOURCE defined, and/or - * change the regular functions to 64 bits if _FILE_OFFSET_BITS is 64 (if - * both are true, the application gets the *64 functions, and the regular - * functions are changed to 64 bits) -- in case these are set on systems - * without large file support, _LFS64_LARGEFILE must also be true - */ -#ifdef Z_LARGE64 - ZEXTERN gzFile ZEXPORT gzopen64(const char *, const char *); - ZEXTERN z_off64_t ZEXPORT gzseek64(gzFile, z_off64_t, int); - ZEXTERN z_off64_t ZEXPORT gztell64(gzFile); - ZEXTERN z_off64_t ZEXPORT gzoffset64(gzFile); -#endif - -#if !defined(ZLIB_INTERNAL) && defined(Z_WANT64) -# define gzopen gzopen64 -# define gzseek gzseek64 -# define gztell gztell64 -# define gzoffset gzoffset64 -# ifndef Z_LARGE64 - ZEXTERN gzFile ZEXPORT gzopen64(const char *, const char *); - ZEXTERN z_off_t ZEXPORT gzseek64(gzFile, z_off_t, int); - ZEXTERN z_off_t ZEXPORT gztell64(gzFile); - ZEXTERN z_off_t ZEXPORT gzoffset64(gzFile); -# endif -#else - ZEXTERN gzFile ZEXPORT gzopen(const char *, const char *); - ZEXTERN z_off_t ZEXPORT gzseek(gzFile, z_off_t, int); - ZEXTERN z_off_t ZEXPORT gztell(gzFile); - ZEXTERN z_off_t ZEXPORT gzoffset(gzFile); -#endif -#endif /* WITH_GZFILEOP */ - - -/* provide 64-bit offset functions if _LARGEFILE64_SOURCE defined, and/or - * change the regular functions to 64 bits if _FILE_OFFSET_BITS is 64 (if - * both are true, the application gets the *64 functions, and the regular - * functions are changed to 64 bits) -- in case these are set on systems - * without large file support, _LFS64_LARGEFILE must also be true - */ -#ifdef Z_LARGE64 - ZEXTERN uint32_t ZEXPORT adler32_combine64(uint32_t, uint32_t, z_off64_t); - ZEXTERN uint32_t ZEXPORT crc32_combine64(uint32_t, uint32_t, z_off64_t); -#endif - -#if !defined(ZLIB_INTERNAL) && defined(Z_WANT64) -# define adler32_combine adler32_combine64 -# define crc32_combine crc32_combine64 -# ifndef Z_LARGE64 - ZEXTERN uint32_t ZEXPORT adler32_combine64(uint32_t, uint32_t, z_off_t); - ZEXTERN uint32_t ZEXPORT crc32_combine64(uint32_t, uint32_t, z_off_t); -# endif -#else - ZEXTERN uint32_t ZEXPORT adler32_combine(uint32_t, uint32_t, z_off_t); - ZEXTERN uint32_t ZEXPORT crc32_combine(uint32_t, uint32_t, z_off_t); -#endif - - -/* undocumented functions */ -ZEXTERN const char * ZEXPORT zError (int); -ZEXTERN int ZEXPORT inflateSyncPoint (z_stream *); -ZEXTERN const uint32_t * ZEXPORT get_crc_table (void); -ZEXTERN int ZEXPORT inflateUndermine (z_stream *, int); -ZEXTERN int ZEXPORT inflateResetKeep (z_stream *); -ZEXTERN int ZEXPORT deflateResetKeep (z_stream *); - -#ifdef WITH_GZFILEOP -# if (defined(WIN32) || defined(__CYGWIN__) || defined(__MINGW__)) - ZEXTERN gzFile ZEXPORT gzopen_w(const wchar_t *path, const char *mode); -# endif -ZEXTERN int ZEXPORTVA gzvprintf(gzFile file, const char *format, va_list va); -#endif - -#ifdef __cplusplus -} -#endif - -#endif /* ZLIB_H_ */ diff --git a/contrib/libzlib-ng/zlib.map b/contrib/libzlib-ng/zlib.map deleted file mode 100644 index 55c6647eb46..00000000000 --- a/contrib/libzlib-ng/zlib.map +++ /dev/null @@ -1,83 +0,0 @@ -ZLIB_1.2.0 { - global: - compressBound; - deflateBound; - inflateBack; - inflateBackEnd; - inflateBackInit_; - inflateCopy; - local: - deflate_copyright; - inflate_copyright; - inflate_fast; - inflate_table; - zcalloc; - zcfree; - z_errmsg; - gz_error; - gz_intmax; - _*; -}; - -ZLIB_1.2.0.2 { - gzclearerr; - gzungetc; - zlibCompileFlags; -} ZLIB_1.2.0; - -ZLIB_1.2.0.8 { - deflatePrime; -} ZLIB_1.2.0.2; - -ZLIB_1.2.2 { - adler32_combine; - crc32_combine; - deflateSetHeader; - inflateGetHeader; -} ZLIB_1.2.0.8; - -ZLIB_1.2.2.3 { - deflateTune; - gzdirect; -} ZLIB_1.2.2; - -ZLIB_1.2.2.4 { - inflatePrime; -} ZLIB_1.2.2.3; - -ZLIB_1.2.3.3 { - adler32_combine64; - crc32_combine64; - gzopen64; - gzseek64; - gztell64; - inflateUndermine; -} ZLIB_1.2.2.4; - -ZLIB_1.2.3.4 { - inflateReset2; - inflateMark; -} ZLIB_1.2.3.3; - -ZLIB_1.2.3.5 { - gzbuffer; - gzoffset; - gzoffset64; - gzclose_r; - gzclose_w; -} ZLIB_1.2.3.4; - -ZLIB_1.2.5.1 { - deflatePending; -} ZLIB_1.2.3.5; - -ZLIB_1.2.5.2 { - deflateResetKeep; - gzgetc_; - inflateResetKeep; -} ZLIB_1.2.5.1; - -ZLIB_1.2.7.1 { - inflateGetDictionary; - gzvprintf; -} ZLIB_1.2.5.2; diff --git a/contrib/libzlib-ng/zlib.pc.cmakein b/contrib/libzlib-ng/zlib.pc.cmakein deleted file mode 100644 index a5e642938c6..00000000000 --- a/contrib/libzlib-ng/zlib.pc.cmakein +++ /dev/null @@ -1,13 +0,0 @@ -prefix=@CMAKE_INSTALL_PREFIX@ -exec_prefix=@CMAKE_INSTALL_PREFIX@ -libdir=@INSTALL_LIB_DIR@ -sharedlibdir=@INSTALL_LIB_DIR@ -includedir=@INSTALL_INC_DIR@ - -Name: zlib -Description: zlib compression library -Version: @VERSION@ - -Requires: -Libs: -L${libdir} -L${sharedlibdir} -lz -Cflags: -I${includedir} diff --git a/contrib/libzlib-ng/zlib.pc.in b/contrib/libzlib-ng/zlib.pc.in deleted file mode 100644 index 7e5acf9c77e..00000000000 --- a/contrib/libzlib-ng/zlib.pc.in +++ /dev/null @@ -1,13 +0,0 @@ -prefix=@prefix@ -exec_prefix=@exec_prefix@ -libdir=@libdir@ -sharedlibdir=@sharedlibdir@ -includedir=@includedir@ - -Name: zlib -Description: zlib compression library -Version: @VERSION@ - -Requires: -Libs: -L${libdir} -L${sharedlibdir} -lz -Cflags: -I${includedir} diff --git a/contrib/libzlib-ng/zutil.c b/contrib/libzlib-ng/zutil.c deleted file mode 100644 index e46277b8786..00000000000 --- a/contrib/libzlib-ng/zutil.c +++ /dev/null @@ -1,124 +0,0 @@ -/* zutil.c -- target dependent utility functions for the compression library - * Copyright (C) 1995-2005, 2010, 2011, 2012 Jean-loup Gailly. - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* @(#) $Id$ */ - -#include "zutil.h" -#ifdef WITH_GZFILEOP -# include "gzguts.h" -#endif - -const char * const z_errmsg[10] = { -"need dictionary", /* Z_NEED_DICT 2 */ -"stream end", /* Z_STREAM_END 1 */ -"", /* Z_OK 0 */ -"file error", /* Z_ERRNO (-1) */ -"stream error", /* Z_STREAM_ERROR (-2) */ -"data error", /* Z_DATA_ERROR (-3) */ -"insufficient memory", /* Z_MEM_ERROR (-4) */ -"buffer error", /* Z_BUF_ERROR (-5) */ -"incompatible version",/* Z_VERSION_ERROR (-6) */ -""}; - -const char zlibng_string[] = - " zlib-ng 1.9.9 forked from zlib 1.2.8 "; - -const char * ZEXPORT zlibVersion(void) -{ - return ZLIB_VERSION; -} - -unsigned long ZEXPORT zlibCompileFlags(void) -{ - unsigned long flags; - - flags = 0; - switch ((int)(sizeof(unsigned int))) { - case 2: break; - case 4: flags += 1; break; - case 8: flags += 2; break; - default: flags += 3; - } - switch ((int)(sizeof(unsigned long))) { - case 2: break; - case 4: flags += 1 << 2; break; - case 8: flags += 2 << 2; break; - default: flags += 3 << 2; - } - switch ((int)(sizeof(void *))) { - case 2: break; - case 4: flags += 1 << 4; break; - case 8: flags += 2 << 4; break; - default: flags += 3 << 4; - } - switch ((int)(sizeof(z_off_t))) { - case 2: break; - case 4: flags += 1 << 6; break; - case 8: flags += 2 << 6; break; - default: flags += 3 << 6; - } -#ifdef DEBUG - flags += 1 << 8; -#endif -#ifdef ZLIB_WINAPI - flags += 1 << 10; -#endif -#ifdef BUILDFIXED - flags += 1 << 12; -#endif -#ifdef DYNAMIC_CRC_TABLE - flags += 1 << 13; -#endif -#ifdef NO_GZCOMPRESS - flags += 1L << 16; -#endif -#ifdef NO_GZIP - flags += 1L << 17; -#endif -#ifdef PKZIP_BUG_WORKAROUND - flags += 1L << 20; -#endif - return flags; -} - -#ifdef DEBUG - -# ifndef verbose -# define verbose 0 -# endif -int ZLIB_INTERNAL z_verbose = verbose; - -void ZLIB_INTERNAL z_error (m) - char *m; -{ - fprintf(stderr, "%s\n", m); - exit(1); -} -#endif - -/* exported to allow conversion of error code to string for compress() and - * uncompress() - */ -const char * ZEXPORT zError(int err) -{ - return ERR_MSG(err); -} - -#ifndef MY_ZCALLOC /* Any system without a special alloc function */ - -void ZLIB_INTERNAL *zcalloc (void *opaque, unsigned items, unsigned size) -{ - (void)opaque; - return sizeof(unsigned int) > 2 ? (void *)malloc(items * size) : - (void *)calloc(items, size); -} - -void ZLIB_INTERNAL zcfree (void *opaque, void *ptr) -{ - (void)opaque; - free(ptr); -} - -#endif /* MY_ZCALLOC */ diff --git a/contrib/libzlib-ng/zutil.h b/contrib/libzlib-ng/zutil.h deleted file mode 100644 index b6e858c9cd2..00000000000 --- a/contrib/libzlib-ng/zutil.h +++ /dev/null @@ -1,185 +0,0 @@ -#ifndef ZUTIL_H_ -#define ZUTIL_H_ -/* zutil.h -- internal interface and configuration of the compression library - * Copyright (C) 1995-2013 Jean-loup Gailly. - * For conditions of distribution and use, see copyright notice in zlib.h - */ - -/* WARNING: this file should *not* be used by applications. It is - part of the implementation of the compression library and is - subject to change. Applications should only use zlib.h. - */ - -/* @(#) $Id$ */ - -#if defined(HAVE_INTERNAL) -# define ZLIB_INTERNAL __attribute__((visibility ("internal"))) -#elif defined(HAVE_HIDDEN) -# define ZLIB_INTERNAL __attribute__((visibility ("hidden"))) -#else -# define ZLIB_INTERNAL -#endif - -#include -#include -#include -#include -#include "zlib.h" - -#ifndef local -# define local static -#endif -/* compile with -Dlocal if your debugger can't find static symbols */ - -typedef unsigned char uch; /* Included for compatibility with external code only */ -typedef uint16_t ush; /* Included for compatibility with external code only */ -typedef unsigned long ulg; - -extern const char * const z_errmsg[10]; /* indexed by 2-zlib_error */ -/* (size given to avoid silly warnings with Visual C++) */ - -#define ERR_MSG(err) z_errmsg[Z_NEED_DICT-(err)] - -#define ERR_RETURN(strm, err) return (strm->msg = ERR_MSG(err), (err)) -/* To be used only when the state is known to be valid */ - - /* common constants */ - -#ifndef DEF_WBITS -# define DEF_WBITS MAX_WBITS -#endif -/* default windowBits for decompression. MAX_WBITS is for compression only */ - -#if MAX_MEM_LEVEL >= 8 -# define DEF_MEM_LEVEL 8 -#else -# define DEF_MEM_LEVEL MAX_MEM_LEVEL -#endif -/* default memLevel */ - -#define STORED_BLOCK 0 -#define STATIC_TREES 1 -#define DYN_TREES 2 -/* The three kinds of block type */ - -#define MIN_MATCH 3 -#define MAX_MATCH 258 -/* The minimum and maximum match lengths */ - -#define PRESET_DICT 0x20 /* preset dictionary flag in zlib header */ - - /* target dependencies */ - -#ifdef WIN32 -# ifndef __CYGWIN__ /* Cygwin is Unix, not Win32 */ -# define OS_CODE 0x0b -# endif -#endif - -#if (defined(_MSC_VER) && (_MSC_VER > 600)) -# define fdopen(fd, type) _fdopen(fd, type) -#endif - -/* provide prototypes for these when building zlib without LFS */ -#if !defined(WIN32) && !defined(__MSYS__) && (!defined(_LARGEFILE64_SOURCE) || _LFS64_LARGEFILE-0 == 0) - ZEXTERN uint32_t ZEXPORT adler32_combine64(uint32_t, uint32_t, z_off_t); - ZEXTERN uint32_t ZEXPORT crc32_combine64(uint32_t, uint32_t, z_off_t); -#endif - -/* MS Visual Studio does not allow inline in C, only C++. - But it provides __inline instead, so use that. */ -#if defined(_MSC_VER) && !defined(inline) -# define inline __inline -#endif - - /* common defaults */ - -#ifndef OS_CODE -# define OS_CODE 0x03 /* assume Unix */ -#endif - -#ifndef F_OPEN -# define F_OPEN(name, mode) fopen((name), (mode)) -#endif - - /* functions */ - -/* Diagnostic functions */ -#ifdef DEBUG -# include - extern int ZLIB_INTERNAL z_verbose; - extern void ZLIB_INTERNAL z_error(char *m); -# define Assert(cond, msg) {if(!(cond)) z_error(msg);} -# define Trace(x) {if (z_verbose >= 0) fprintf x;} -# define Tracev(x) {if (z_verbose > 0) fprintf x;} -# define Tracevv(x) {if (z_verbose > 1) fprintf x;} -# define Tracec(c, x) {if (z_verbose > 0 && (c)) fprintf x;} -# define Tracecv(c, x) {if (z_verbose > 1 && (c)) fprintf x;} -#else -# define Assert(cond, msg) -# define Trace(x) -# define Tracev(x) -# define Tracevv(x) -# define Tracec(c, x) -# define Tracecv(c, x) -#endif - -void ZLIB_INTERNAL *zcalloc(void *opaque, unsigned items, unsigned size); -void ZLIB_INTERNAL zcfree(void *opaque, void *ptr); - -#define ZALLOC(strm, items, size) (*((strm)->zalloc))((strm)->opaque, (items), (size)) -#define ZFREE(strm, addr) (*((strm)->zfree))((strm)->opaque, (void *)(addr)) -#define TRY_FREE(s, p) {if (p) ZFREE(s, p);} - -/* Reverse the bytes in a 32-bit value. Use compiler intrinsics when - possible to take advantage of hardware implementations. */ -#if defined(WIN32) && (_MSC_VER >= 1300) -# pragma intrinsic(_byteswap_ulong) -# define ZSWAP32(q) _byteswap_ulong(q) - -#elif defined(__Clang__) || (defined(__GNUC__) && \ - (__GNUC__ > 4 || (__GNUC__ == 4 && __GNUC_MINOR__ >= 2))) -# define ZSWAP32(q) __builtin_bswap32(q) - -#elif defined(__GNUC__) && (__GNUC__ >= 2) && defined(__linux__) -# include -# define ZSWAP32(q) bswap_32(q) - -#elif defined(__FreeBSD__) || defined(__OpenBSD__) || defined(__NetBSD__) || defined(__DragonFly__) -# include -# define ZSWAP32(q) bswap32(q) - -#elif defined(__INTEL_COMPILER) -# define ZSWAP32(q) _bswap(q) - -#else -# define ZSWAP32(q) ((((q) >> 24) & 0xff) + (((q) >> 8) & 0xff00) + \ - (((q) & 0xff00) << 8) + (((q) & 0xff) << 24)) -#endif /* ZSWAP32 */ - -/* Only enable likely/unlikely if the compiler is known to support it */ -#if (defined(__GNUC__) && (__GNUC__ >= 3)) || defined(__INTEL_COMPILER) || defined(__Clang__) -# ifndef likely -# define likely(x) __builtin_expect(!!(x), 1) -# endif -# ifndef unlikely -# define unlikely(x) __builtin_expect(!!(x), 0) -# endif -#else -# ifndef likely -# define likely(x) x -# endif -# ifndef unlikely -# define unlikely(x) x -# endif -#endif /* (un)likely */ - -#if defined(_MSC_VER) -#define ALIGNED_(x) __declspec(align(x)) -#else -#if defined(__GNUC__) -#define ALIGNED_(x) __attribute__ ((aligned(x))) -#endif -#endif - -#endif /* ZUTIL_H_ */ From b972516bfc2f8433ba6e00f572f278ca14158e34 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 22 Nov 2017 15:15:17 +0300 Subject: [PATCH 009/509] Zlib: remove unused test targets --- cmake/find_zlib.cmake | 1 + contrib/CMakeLists.txt | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/cmake/find_zlib.cmake b/cmake/find_zlib.cmake index 93e62497c25..3b41280f1c5 100644 --- a/cmake/find_zlib.cmake +++ b/cmake/find_zlib.cmake @@ -7,6 +7,7 @@ endif () if (NOT ZLIB_FOUND) set (USE_INTERNAL_ZLIB_LIBRARY 1) set (ZLIB_COMPAT 1) # for zlib-ng, also enables WITH_GZFILEOP + set (WITH_NATIVE_INSTRUCTIONS ${ARCHNATIVE}) set (ZLIB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libzlib-ng" "${ClickHouse_BINARY_DIR}/contrib/libzlib-ng") # generated zconf.h set (ZLIB_INCLUDE_DIRS ${ZLIB_INCLUDE_DIR}) # for poco set (ZLIB_FOUND 1) # for poco diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 137eea988bc..814e1c54c08 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -50,6 +50,10 @@ if (USE_INTERNAL_ZLIB_LIBRARY) # We should use same defines when including zlib.h as used when zlib compiled target_compile_definitions (zlib PUBLIC ZLIB_COMPAT WITH_GZFILEOP) target_compile_definitions (zlibstatic PUBLIC ZLIB_COMPAT WITH_GZFILEOP) + set_target_properties(example PROPERTIES EXCLUDE_FROM_ALL 1) + set_target_properties(example64 PROPERTIES EXCLUDE_FROM_ALL 1) + set_target_properties(minigzip PROPERTIES EXCLUDE_FROM_ALL 1) + set_target_properties(minigzip64 PROPERTIES EXCLUDE_FROM_ALL 1) endif () if (USE_INTERNAL_CCTZ_LIBRARY) From 5d21519d2762895ce02571962ff2ca25e742b590 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 22 Nov 2017 17:03:09 +0300 Subject: [PATCH 010/509] Freebsd: zlib: dont use optimizations (broken asm) --- cmake/find_zlib.cmake | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cmake/find_zlib.cmake b/cmake/find_zlib.cmake index 3b41280f1c5..d38ef13a5e9 100644 --- a/cmake/find_zlib.cmake +++ b/cmake/find_zlib.cmake @@ -8,6 +8,9 @@ if (NOT ZLIB_FOUND) set (USE_INTERNAL_ZLIB_LIBRARY 1) set (ZLIB_COMPAT 1) # for zlib-ng, also enables WITH_GZFILEOP set (WITH_NATIVE_INSTRUCTIONS ${ARCHNATIVE}) + if (CMAKE_SYSTEM MATCHES "FreeBSD") + set (WITH_OPTIM 0 CACHE INTERNAL "") # Bug in assembler + endif () set (ZLIB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libzlib-ng" "${ClickHouse_BINARY_DIR}/contrib/libzlib-ng") # generated zconf.h set (ZLIB_INCLUDE_DIRS ${ZLIB_INCLUDE_DIR}) # for poco set (ZLIB_FOUND 1) # for poco From 2ea3e6c67fe468a6b69263a0a15cdbee3d4f432e Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 22 Nov 2017 22:15:17 +0300 Subject: [PATCH 011/509] Fix include path contrib/zlib-ng --- cmake/find_poco.cmake | 4 ++-- cmake/find_zlib.cmake | 2 +- utils/check_include.sh | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/cmake/find_poco.cmake b/cmake/find_poco.cmake index c63cbc3da13..19b0cf3ffca 100644 --- a/cmake/find_poco.cmake +++ b/cmake/find_poco.cmake @@ -62,8 +62,8 @@ else () if (USE_STATIC_LIBRARIES AND USE_INTERNAL_ZLIB_LIBRARY) list (APPEND Poco_INCLUDE_DIRS - "${ClickHouse_SOURCE_DIR}/contrib/libzlib-ng/" - "${ClickHouse_BINARY_DIR}/contrib/libzlib-ng/" + "${ClickHouse_SOURCE_DIR}/contrib/zlib-ng/" + "${ClickHouse_BINARY_DIR}/contrib/zlib-ng/" ) endif () diff --git a/cmake/find_zlib.cmake b/cmake/find_zlib.cmake index d38ef13a5e9..db26c0ad2de 100644 --- a/cmake/find_zlib.cmake +++ b/cmake/find_zlib.cmake @@ -11,7 +11,7 @@ if (NOT ZLIB_FOUND) if (CMAKE_SYSTEM MATCHES "FreeBSD") set (WITH_OPTIM 0 CACHE INTERNAL "") # Bug in assembler endif () - set (ZLIB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libzlib-ng" "${ClickHouse_BINARY_DIR}/contrib/libzlib-ng") # generated zconf.h + set (ZLIB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/zlib-ng" "${ClickHouse_BINARY_DIR}/contrib/zlib-ng") # generated zconf.h set (ZLIB_INCLUDE_DIRS ${ZLIB_INCLUDE_DIR}) # for poco set (ZLIB_FOUND 1) # for poco if (USE_STATIC_LIBRARIES) diff --git a/utils/check_include.sh b/utils/check_include.sh index 22429d16a40..80c39ddff0c 100755 --- a/utils/check_include.sh +++ b/utils/check_include.sh @@ -15,8 +15,8 @@ inc="-I. \ -I./contrib/zookeeper/src/c/include \ -I./contrib/zookeeper/src/c/generated \ -I./contrib/libtcmalloc/include \ --I./build/contrib/libzlib-ng \ --I./contrib/libzlib-ng \ +-I./build/contrib/zlib-ng \ +-I./contrib/zlib-ng \ -I./contrib/poco/MongoDB/include \ -I./contrib/poco/XML/include \ -I./contrib/poco/Crypto/include \ From c0dab866d9be60d88e5c3f860e76bfe4da6faef3 Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 17 Nov 2017 16:14:13 +0300 Subject: [PATCH 012/509] Update build_debian.sh --- utils/build/build_debian.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/utils/build/build_debian.sh b/utils/build/build_debian.sh index 6124c1b482d..6d9594049b4 100755 --- a/utils/build/build_debian.sh +++ b/utils/build/build_debian.sh @@ -6,9 +6,9 @@ # curl https://raw.githubusercontent.com/yandex/ClickHouse/master/utils/build/build_debian.sh | sh # install compiler and libs -sudo apt install -y git bash cmake gcc-6 g++-6 libicu-dev libreadline-dev libmysqlclient-dev unixodbc-dev libltdl-dev libssl-dev +sudo apt install -y git bash cmake gcc-7 g++-7 libicu-dev libreadline-dev libmysqlclient-dev unixodbc-dev libltdl-dev libssl-dev # for -DUNBUNDLED=1 mode: -#sudo apt install -y libboost-dev zlib1g-dev liblz4-dev libdouble-conversion-dev libzstd-dev libre2-dev libzookeeper-mt-dev libsparsehash-dev # libpoco-dev +#sudo apt install -y libboost-dev zlib1g-dev liblz4-dev libdouble-conversion-dev libzstd-dev libre2-dev libzookeeper-mt-dev libsparsehash-dev librdkafka-dev libpoco-dev libsparsehash-dev libgoogle-perftools-dev libunwind-dev libzstd-dev # install testing only stuff if you want: sudo apt install -y python python-lxml python-termcolor curl perl @@ -19,7 +19,7 @@ git clone --recursive https://github.com/yandex/ClickHouse.git # Build! mkdir -p ClickHouse/build cd ClickHouse/build -cmake .. -DCMAKE_CXX_COMPILER=`which g++-6` -DCMAKE_C_COMPILER=`which gcc-6` +cmake .. -DCMAKE_CXX_COMPILER=`which g++-7` -DCMAKE_C_COMPILER=`which gcc-7` make -j $(nproc || sysctl -n hw.ncpu || echo 2) cd .. From 33faccbfd325908fb70d9445be43a18d04136ab3 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Wed, 29 Nov 2017 15:37:55 +0300 Subject: [PATCH 013/509] Update compiler version. [#CLICKHOUSE-2] --- release | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/release b/release index 255f973eb91..ed622de6487 100755 --- a/release +++ b/release @@ -7,8 +7,8 @@ cd $CURDIR source "./release_lib.sh" -DEB_CC=gcc-6 -DEB_CXX=g++-6 +DEB_CC=gcc-7 +DEB_CXX=g++-7 CONTROL=debian/control DEBUILD_NOSIGN_OPTIONS="-us -uc" From 9b7f75940a852273e46d1e4559a87bb9e9756668 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Thu, 30 Nov 2017 16:34:07 +0300 Subject: [PATCH 014/509] Added missing include dirs for dynamic compilation. [#CLICKHOUSE-2] --- dbms/src/Interpreters/Compiler.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/Compiler.cpp b/dbms/src/Interpreters/Compiler.cpp index fde9e717690..e791ac7457a 100644 --- a/dbms/src/Interpreters/Compiler.cpp +++ b/dbms/src/Interpreters/Compiler.cpp @@ -215,7 +215,7 @@ void Compiler::compile( std::stringstream command; - /// Slightly unconvenient. + /// Slightly inconvenient. command << "LD_LIBRARY_PATH=" PATH_SHARE "/clickhouse/bin/" " " INTERNAL_COMPILER_EXECUTABLE @@ -224,6 +224,7 @@ void Compiler::compile( #if INTERNAL_COMPILER_CUSTOM_ROOT " -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/local/include/" " -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/include/" + " -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/local/include/c++/*/" " -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/include/c++/*/" " -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/include/x86_64-linux-gnu/" " -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/include/x86_64-linux-gnu/c++/*/" From 902c9374b62bcbdd2fcb787665312643b6145a29 Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 30 Nov 2017 20:09:33 +0300 Subject: [PATCH 015/509] zlib fix flags in headers --- contrib/CMakeLists.txt | 5 +++ dbms/src/IO/ZlibDeflatingWriteBuffer.cpp | 2 +- dbms/src/IO/ZlibInflatingReadBuffer.cpp | 2 +- .../0_stateless/00302_http_compression.sh | 36 ++++++++++--------- 4 files changed, 27 insertions(+), 18 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 814e1c54c08..3f46ccfaaca 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -50,6 +50,11 @@ if (USE_INTERNAL_ZLIB_LIBRARY) # We should use same defines when including zlib.h as used when zlib compiled target_compile_definitions (zlib PUBLIC ZLIB_COMPAT WITH_GZFILEOP) target_compile_definitions (zlibstatic PUBLIC ZLIB_COMPAT WITH_GZFILEOP) + if(CMAKE_SYSTEM_PROCESSOR MATCHES "x86_64" OR CMAKE_SYSTEM_PROCESSOR MATCHES "AMD64") + target_compile_definitions (zlib PUBLIC X86_64) + target_compile_definitions (zlibstatic PUBLIC X86_64) + endif () + set_target_properties(example PROPERTIES EXCLUDE_FROM_ALL 1) set_target_properties(example64 PROPERTIES EXCLUDE_FROM_ALL 1) set_target_properties(minigzip PROPERTIES EXCLUDE_FROM_ALL 1) diff --git a/dbms/src/IO/ZlibDeflatingWriteBuffer.cpp b/dbms/src/IO/ZlibDeflatingWriteBuffer.cpp index c2165c382a9..61450e19db4 100644 --- a/dbms/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/dbms/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -34,7 +34,7 @@ ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer( #pragma GCC diagnostic pop if (rc != Z_OK) - throw Exception(std::string("deflateInit2 failed: ") + zError(rc), ErrorCodes::ZLIB_DEFLATE_FAILED); + throw Exception(std::string("deflateInit2 failed: ") + zError(rc) + "; zlib version: " + ZLIB_VERSION, ErrorCodes::ZLIB_DEFLATE_FAILED); } ZlibDeflatingWriteBuffer::~ZlibDeflatingWriteBuffer() diff --git a/dbms/src/IO/ZlibInflatingReadBuffer.cpp b/dbms/src/IO/ZlibInflatingReadBuffer.cpp index 869108515d3..8100cf88de7 100644 --- a/dbms/src/IO/ZlibInflatingReadBuffer.cpp +++ b/dbms/src/IO/ZlibInflatingReadBuffer.cpp @@ -34,7 +34,7 @@ ZlibInflatingReadBuffer::ZlibInflatingReadBuffer( #pragma GCC diagnostic pop if (rc != Z_OK) - throw Exception(std::string("inflateInit2 failed: ") + zError(rc), ErrorCodes::ZLIB_INFLATE_FAILED); + throw Exception(std::string("inflateInit2 failed: ") + zError(rc) + "; zlib version: " + ZLIB_VERSION, ErrorCodes::ZLIB_INFLATE_FAILED); } ZlibInflatingReadBuffer::~ZlibInflatingReadBuffer() diff --git a/dbms/tests/queries/0_stateless/00302_http_compression.sh b/dbms/tests/queries/0_stateless/00302_http_compression.sh index f71b8975fd0..013f3988d0e 100755 --- a/dbms/tests/queries/0_stateless/00302_http_compression.sh +++ b/dbms/tests/queries/0_stateless/00302_http_compression.sh @@ -1,24 +1,28 @@ #!/usr/bin/env bash -curl -sS 'http://localhost:8123/?enable_http_compression=1' -d 'SELECT number FROM system.numbers LIMIT 10'; -curl -sS 'http://localhost:8123/?enable_http_compression=0' -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10'; -curl -sS 'http://localhost:8123/?enable_http_compression=1' -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d; -curl -sS 'http://localhost:8123/?enable_http_compression=1' -H 'Accept-Encoding: gzip, deflate' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d; -curl -sS 'http://localhost:8123/?enable_http_compression=1' -H 'Accept-Encoding: zip, eflate' -d 'SELECT number FROM system.numbers LIMIT 10'; +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh -curl -vsS 'http://localhost:8123/?enable_http_compression=1' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding'; -curl -vsS 'http://localhost:8123/?enable_http_compression=1' -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding'; -curl -vsS 'http://localhost:8123/?enable_http_compression=1' -H 'Accept-Encoding: deflate' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding'; -curl -vsS 'http://localhost:8123/?enable_http_compression=1' -H 'Accept-Encoding: gzip, deflate' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding'; -curl -vsS 'http://localhost:8123/?enable_http_compression=1' -H 'Accept-Encoding: zip, eflate' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding'; -echo "SELECT 1" | curl -sS --data-binary @- 'http://localhost:8123/'; -echo "SELECT 1" | gzip -c | curl -sS --data-binary @- -H 'Content-Encoding: gzip' 'http://localhost:8123/'; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?enable_http_compression=1" -d 'SELECT number FROM system.numbers LIMIT 10'; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?enable_http_compression=0" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10'; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: gzip, deflate' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: zip, eflate' -d 'SELECT number FROM system.numbers LIMIT 10'; -echo "'Hello, world'" | curl -sS --data-binary @- 'http://localhost:8123/?query=SELECT'; -echo "'Hello, world'" | gzip -c | curl -sS --data-binary @- -H 'Content-Encoding: gzip' 'http://localhost:8123/?query=SELECT'; +${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?enable_http_compression=1" -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding'; +${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding'; +${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: deflate' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding'; +${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: gzip, deflate' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding'; + -vsS '${CLICKHOUSE_URL}?enable_http_compression=1' -H 'Accept-Encoding: zip, eflate' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding'; -curl -sS 'http://localhost:8123/?enable_http_compression=1' -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 0' | wc -c; +echo "SELECT 1" | ${CLICKHOUSE_CURL} -sS --data-binary @- ${CLICKHOUSE_URL}; +echo "SELECT 1" | gzip -c | ${CLICKHOUSE_CURL} -sS --data-binary @- -H 'Content-Encoding: gzip' ${CLICKHOUSE_URL}; + +echo "'Hello, world'" | ${CLICKHOUSE_CURL} -sS --data-binary @- "${CLICKHOUSE_URL}?query=SELECT"; +echo "'Hello, world'" | gzip -c | ${CLICKHOUSE_CURL} -sS --data-binary @- -H 'Content-Encoding: gzip' "${CLICKHOUSE_URL}?query=SELECT"; + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 0' | wc -c; # POST multiple concatenated gzip streams. -(echo -n "SELECT 'Part1" | gzip -c; echo " Part2'" | gzip -c) | curl -sS -H 'Content-Encoding: gzip' 'http://localhost:8123/?' --data-binary @- +(echo -n "SELECT 'Part1" | gzip -c; echo " Part2'" | gzip -c) | ${CLICKHOUSE_CURL} -sS -H 'Content-Encoding: gzip' "${CLICKHOUSE_URL}?" --data-binary @- From 89e633d0649c94b160f8b6afe94f67f072b19d4a Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 30 Nov 2017 20:21:03 +0300 Subject: [PATCH 016/509] Fix compile (missing boost libs) --- .../boost/archive/archive_exception.hpp | 100 + .../boost/archive/basic_archive.hpp | 304 +++ .../boost/archive/basic_binary_iarchive.hpp | 204 ++ .../boost/archive/basic_binary_iprimitive.hpp | 198 ++ .../boost/archive/basic_binary_oarchive.hpp | 185 ++ .../boost/archive/basic_binary_oprimitive.hpp | 188 ++ .../archive/basic_streambuf_locale_saver.hpp | 108 ++ .../boost/archive/basic_text_iarchive.hpp | 96 + .../boost/archive/basic_text_iprimitive.hpp | 142 ++ .../boost/archive/basic_text_oarchive.hpp | 119 ++ .../boost/archive/basic_text_oprimitive.hpp | 209 ++ .../boost/archive/basic_xml_archive.hpp | 67 + .../boost/archive/basic_xml_iarchive.hpp | 119 ++ .../boost/archive/basic_xml_oarchive.hpp | 138 ++ .../boost/archive/binary_iarchive.hpp | 64 + .../boost/archive/binary_iarchive_impl.hpp | 105 + .../boost/archive/binary_oarchive.hpp | 64 + .../boost/archive/binary_oarchive_impl.hpp | 106 + .../boost/archive/binary_wiarchive.hpp | 56 + .../boost/archive/binary_woarchive.hpp | 59 + .../boost/archive/codecvt_null.hpp | 109 ++ .../boost/archive/detail/abi_prefix.hpp | 16 + .../boost/archive/detail/abi_suffix.hpp | 15 + .../archive/detail/archive_serializer_map.hpp | 54 + .../archive/detail/auto_link_archive.hpp | 48 + .../archive/detail/auto_link_warchive.hpp | 47 + .../boost/archive/detail/basic_iarchive.hpp | 105 + .../archive/detail/basic_iserializer.hpp | 91 + .../boost/archive/detail/basic_oarchive.hpp | 94 + .../archive/detail/basic_oserializer.hpp | 89 + .../detail/basic_pointer_iserializer.hpp | 70 + .../detail/basic_pointer_oserializer.hpp | 68 + .../boost/archive/detail/basic_serializer.hpp | 77 + .../archive/detail/basic_serializer_map.hpp | 69 + .../boost/archive/detail/check.hpp | 169 ++ .../boost/archive/detail/common_iarchive.hpp | 88 + .../boost/archive/detail/common_oarchive.hpp | 88 + .../boost/archive/detail/decl.hpp | 57 + .../archive/detail/helper_collection.hpp | 99 + .../archive/detail/interface_iarchive.hpp | 85 + .../archive/detail/interface_oarchive.hpp | 87 + .../boost/archive/detail/iserializer.hpp | 631 ++++++ .../boost/archive/detail/oserializer.hpp | 540 ++++++ .../detail/polymorphic_iarchive_route.hpp | 218 +++ .../detail/polymorphic_oarchive_route.hpp | 209 ++ .../boost/archive/detail/register_archive.hpp | 91 + .../archive/detail/utf8_codecvt_facet.hpp | 39 + .../boost_1_65_0/boost/archive/dinkumware.hpp | 224 +++ .../archive/impl/archive_serializer_map.ipp | 75 + .../archive/impl/basic_binary_iarchive.ipp | 134 ++ .../archive/impl/basic_binary_iprimitive.ipp | 171 ++ .../archive/impl/basic_binary_oarchive.ipp | 42 + .../archive/impl/basic_binary_oprimitive.ipp | 126 ++ .../archive/impl/basic_text_iarchive.ipp | 76 + .../archive/impl/basic_text_iprimitive.ipp | 137 ++ .../archive/impl/basic_text_oarchive.ipp | 62 + .../archive/impl/basic_text_oprimitive.ipp | 115 ++ .../boost/archive/impl/basic_xml_grammar.hpp | 173 ++ .../boost/archive/impl/basic_xml_iarchive.ipp | 115 ++ .../boost/archive/impl/basic_xml_oarchive.ipp | 272 +++ .../boost/archive/impl/text_iarchive_impl.ipp | 128 ++ .../boost/archive/impl/text_oarchive_impl.ipp | 122 ++ .../archive/impl/text_wiarchive_impl.ipp | 118 ++ .../archive/impl/text_woarchive_impl.ipp | 85 + .../boost/archive/impl/xml_iarchive_impl.ipp | 199 ++ .../boost/archive/impl/xml_oarchive_impl.ipp | 142 ++ .../boost/archive/impl/xml_wiarchive_impl.ipp | 189 ++ .../boost/archive/impl/xml_woarchive_impl.ipp | 171 ++ .../archive/iterators/base64_exception.hpp | 68 + .../archive/iterators/base64_from_binary.hpp | 109 ++ .../archive/iterators/binary_from_base64.hpp | 118 ++ .../boost/archive/iterators/dataflow.hpp | 102 + .../archive/iterators/dataflow_exception.hpp | 80 + .../boost/archive/iterators/escape.hpp | 115 ++ .../archive/iterators/insert_linebreaks.hpp | 99 + .../archive/iterators/istream_iterator.hpp | 92 + .../boost/archive/iterators/mb_from_wchar.hpp | 139 ++ .../archive/iterators/ostream_iterator.hpp | 83 + .../archive/iterators/remove_whitespace.hpp | 167 ++ .../archive/iterators/transform_width.hpp | 177 ++ .../boost/archive/iterators/unescape.hpp | 89 + .../boost/archive/iterators/wchar_from_mb.hpp | 194 ++ .../boost/archive/iterators/xml_escape.hpp | 121 ++ .../boost/archive/iterators/xml_unescape.hpp | 125 ++ .../iterators/xml_unescape_exception.hpp | 49 + .../archive/polymorphic_binary_iarchive.hpp | 54 + .../archive/polymorphic_binary_oarchive.hpp | 43 + .../boost/archive/polymorphic_iarchive.hpp | 168 ++ .../boost/archive/polymorphic_oarchive.hpp | 154 ++ .../archive/polymorphic_text_iarchive.hpp | 54 + .../archive/polymorphic_text_oarchive.hpp | 39 + .../archive/polymorphic_text_wiarchive.hpp | 59 + .../archive/polymorphic_text_woarchive.hpp | 44 + .../archive/polymorphic_xml_iarchive.hpp | 54 + .../archive/polymorphic_xml_oarchive.hpp | 39 + .../archive/polymorphic_xml_wiarchive.hpp | 50 + .../archive/polymorphic_xml_woarchive.hpp | 44 + .../boost/archive/text_iarchive.hpp | 132 ++ .../boost/archive/text_oarchive.hpp | 121 ++ .../boost/archive/text_wiarchive.hpp | 137 ++ .../boost/archive/text_woarchive.hpp | 155 ++ .../boost_1_65_0/boost/archive/tmpdir.hpp | 50 + .../boost_1_65_0/boost/archive/wcslen.hpp | 58 + .../boost/archive/xml_archive_exception.hpp | 57 + .../boost/archive/xml_iarchive.hpp | 142 ++ .../boost/archive/xml_oarchive.hpp | 137 ++ .../boost/archive/xml_wiarchive.hpp | 149 ++ .../boost/archive/xml_woarchive.hpp | 134 ++ .../boost_1_65_0/boost/foreach_fwd.hpp | 51 + .../boost/multi_index/composite_key.hpp | 1513 +++++++++++++++ .../multi_index/detail/access_specifier.hpp | 54 + .../boost/multi_index/detail/adl_swap.hpp | 44 + .../detail/archive_constructed.hpp | 83 + .../boost/multi_index/detail/auto_space.hpp | 91 + .../boost/multi_index/detail/base_type.hpp | 74 + .../detail/bidir_node_iterator.hpp | 114 ++ .../boost/multi_index/detail/bucket_array.hpp | 243 +++ .../multi_index/detail/cons_stdtuple.hpp | 93 + .../boost/multi_index/detail/converter.hpp | 52 + .../boost/multi_index/detail/copy_map.hpp | 142 ++ .../detail/do_not_copy_elements_tag.hpp | 34 + .../detail/duplicates_iterator.hpp | 120 ++ .../boost/multi_index/detail/has_tag.hpp | 42 + .../multi_index/detail/hash_index_args.hpp | 105 + .../detail/hash_index_iterator.hpp | 166 ++ .../multi_index/detail/hash_index_node.hpp | 778 ++++++++ .../multi_index/detail/header_holder.hpp | 50 + .../detail/ignore_wstrict_aliasing.hpp | 18 + .../boost/multi_index/detail/index_base.hpp | 293 +++ .../boost/multi_index/detail/index_loader.hpp | 139 ++ .../multi_index/detail/index_matcher.hpp | 249 +++ .../multi_index/detail/index_node_base.hpp | 135 ++ .../boost/multi_index/detail/index_saver.hpp | 135 ++ .../multi_index/detail/invariant_assert.hpp | 21 + .../multi_index/detail/is_index_list.hpp | 40 + .../multi_index/detail/is_transparent.hpp | 135 ++ .../boost/multi_index/detail/iter_adaptor.hpp | 321 +++ .../multi_index/detail/modify_key_adaptor.hpp | 49 + .../multi_index/detail/no_duplicate_tags.hpp | 97 + .../boost/multi_index/detail/node_type.hpp | 66 + .../multi_index/detail/ord_index_args.hpp | 83 + .../multi_index/detail/ord_index_impl.hpp | 1567 +++++++++++++++ .../multi_index/detail/ord_index_impl_fwd.hpp | 128 ++ .../multi_index/detail/ord_index_node.hpp | 658 +++++++ .../multi_index/detail/ord_index_ops.hpp | 266 +++ .../boost/multi_index/detail/promotes_arg.hpp | 83 + .../boost/multi_index/detail/raw_ptr.hpp | 52 + .../detail/restore_wstrict_aliasing.hpp | 11 + .../multi_index/detail/rnd_index_loader.hpp | 173 ++ .../multi_index/detail/rnd_index_node.hpp | 273 +++ .../multi_index/detail/rnd_index_ops.hpp | 203 ++ .../detail/rnd_index_ptr_array.hpp | 144 ++ .../multi_index/detail/rnd_node_iterator.hpp | 140 ++ .../multi_index/detail/rnk_index_ops.hpp | 300 +++ .../boost/multi_index/detail/safe_mode.hpp | 588 ++++++ .../boost/multi_index/detail/scope_guard.hpp | 453 +++++ .../multi_index/detail/seq_index_node.hpp | 217 +++ .../multi_index/detail/seq_index_ops.hpp | 203 ++ .../detail/serialization_version.hpp | 73 + .../boost/multi_index/detail/uintptr_type.hpp | 76 + .../boost/multi_index/detail/unbounded.hpp | 66 + .../multi_index/detail/value_compare.hpp | 56 + .../multi_index/detail/vartempl_support.hpp | 247 +++ .../boost/multi_index/global_fun.hpp | 185 ++ .../boost/multi_index/hashed_index.hpp | 1725 +++++++++++++++++ .../boost/multi_index/hashed_index_fwd.hpp | 74 + .../boost/multi_index/identity.hpp | 145 ++ .../boost/multi_index/identity_fwd.hpp | 26 + .../boost/multi_index/indexed_by.hpp | 68 + .../boost/multi_index/key_extractors.hpp | 22 + .../boost/multi_index/mem_fun.hpp | 205 ++ .../boost_1_65_0/boost/multi_index/member.hpp | 262 +++ .../boost/multi_index/ordered_index.hpp | 114 ++ .../boost/multi_index/ordered_index_fwd.hpp | 35 + .../boost/multi_index/random_access_index.hpp | 1167 +++++++++++ .../multi_index/random_access_index_fwd.hpp | 91 + .../boost/multi_index/ranked_index.hpp | 382 ++++ .../boost/multi_index/ranked_index_fwd.hpp | 35 + .../boost/multi_index/safe_mode_errors.hpp | 48 + .../boost/multi_index/sequenced_index.hpp | 1062 ++++++++++ .../boost/multi_index/sequenced_index_fwd.hpp | 91 + .../boost_1_65_0/boost/multi_index/tag.hpp | 88 + .../boost/multi_index_container.hpp | 1362 +++++++++++++ .../boost/multi_index_container_fwd.hpp | 121 ++ .../boost/serialization/access.hpp | 145 ++ .../archive_input_unordered_map.hpp | 85 + .../archive_input_unordered_set.hpp | 72 + .../boost/serialization/array.hpp | 48 + .../serialization/array_optimization.hpp | 37 + .../boost/serialization/array_wrapper.hpp | 121 ++ .../boost/serialization/assume_abstract.hpp | 60 + .../boost/serialization/base_object.hpp | 100 + .../boost/serialization/binary_object.hpp | 79 + .../boost/serialization/bitset.hpp | 75 + .../boost/serialization/boost_array.hpp | 33 + .../serialization/boost_unordered_map.hpp | 154 ++ .../serialization/boost_unordered_set.hpp | 150 ++ .../serialization/collection_size_type.hpp | 62 + .../boost/serialization/collection_traits.hpp | 79 + .../serialization/collections_load_imp.hpp | 106 + .../serialization/collections_save_imp.hpp | 82 + .../boost/serialization/complex.hpp | 81 + .../boost/serialization/config.hpp | 74 + .../boost/serialization/deque.hpp | 80 + .../detail/is_default_constructible.hpp | 54 + .../serialization/detail/shared_count_132.hpp | 551 ++++++ .../serialization/detail/shared_ptr_132.hpp | 443 +++++ .../detail/shared_ptr_nmt_132.hpp | 182 ++ .../detail/stack_constructor.hpp | 66 + .../boost/serialization/ephemeral.hpp | 72 + .../boost/serialization/export.hpp | 225 +++ .../serialization/extended_type_info.hpp | 116 ++ .../extended_type_info_no_rtti.hpp | 182 ++ .../extended_type_info_typeid.hpp | 167 ++ .../boost/serialization/factory.hpp | 102 + .../boost/serialization/force_include.hpp | 55 + .../boost/serialization/forward_list.hpp | 124 ++ .../hash_collections_load_imp.hpp | 77 + .../hash_collections_save_imp.hpp | 97 + .../boost/serialization/hash_map.hpp | 232 +++ .../boost/serialization/hash_set.hpp | 222 +++ .../serialization/is_bitwise_serializable.hpp | 46 + .../boost/serialization/item_version_type.hpp | 68 + .../boost/serialization/level.hpp | 116 ++ .../boost/serialization/level_enum.hpp | 55 + .../boost_1_65_0/boost/serialization/list.hpp | 85 + .../boost_1_65_0/boost/serialization/map.hpp | 139 ++ .../boost_1_65_0/boost/serialization/nvp.hpp | 123 ++ .../boost/serialization/optional.hpp | 107 + .../boost/serialization/priority_queue.hpp | 76 + .../boost/serialization/queue.hpp | 76 + .../boost/serialization/scoped_ptr.hpp | 58 + .../boost/serialization/serialization.hpp | 154 ++ .../boost_1_65_0/boost/serialization/set.hpp | 137 ++ .../boost/serialization/shared_ptr.hpp | 281 +++ .../boost/serialization/shared_ptr_132.hpp | 222 +++ .../boost/serialization/shared_ptr_helper.hpp | 209 ++ .../boost/serialization/singleton.hpp | 166 ++ .../boost/serialization/slist.hpp | 145 ++ .../boost/serialization/smart_cast.hpp | 275 +++ .../boost/serialization/split_free.hpp | 93 + .../boost/serialization/split_member.hpp | 86 + .../boost/serialization/stack.hpp | 76 + .../boost/serialization/state_saver.hpp | 96 + .../boost/serialization/static_warning.hpp | 103 + .../boost/serialization/string.hpp | 30 + .../boost/serialization/strong_typedef.hpp | 50 + .../boost/serialization/throw_exception.hpp | 44 + .../boost/serialization/tracking.hpp | 118 ++ .../boost/serialization/tracking_enum.hpp | 41 + .../boost/serialization/traits.hpp | 65 + .../type_info_implementation.hpp | 73 + .../boost/serialization/unique_ptr.hpp | 68 + .../unordered_collections_load_imp.hpp | 73 + .../unordered_collections_save_imp.hpp | 86 + .../boost/serialization/unordered_map.hpp | 160 ++ .../boost/serialization/unordered_set.hpp | 162 ++ .../boost/serialization/utility.hpp | 56 + .../boost/serialization/valarray.hpp | 86 + .../boost/serialization/variant.hpp | 158 ++ .../boost/serialization/vector.hpp | 233 +++ .../boost/serialization/vector_135.hpp | 26 + .../boost/serialization/version.hpp | 107 + .../boost/serialization/void_cast.hpp | 298 +++ .../boost/serialization/void_cast_fwd.hpp | 37 + .../boost/serialization/weak_ptr.hpp | 99 + .../boost/serialization/wrapper.hpp | 60 + 267 files changed, 41588 insertions(+) create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/archive_exception.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_archive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iprimitive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oprimitive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_streambuf_locale_saver.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_text_iarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_text_iprimitive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_text_oarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_text_oprimitive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_xml_archive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_xml_iarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_xml_oarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive_impl.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive_impl.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_wiarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_woarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/codecvt_null.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/abi_prefix.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/abi_suffix.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/archive_serializer_map.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_archive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_warchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iserializer.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oserializer.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_iserializer.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_oserializer.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer_map.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/check.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/common_iarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/common_oarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/decl.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/helper_collection.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/interface_iarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/interface_oarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/iserializer.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/oserializer.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_iarchive_route.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_oarchive_route.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/register_archive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/utf8_codecvt_facet.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/dinkumware.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/archive_serializer_map.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iarchive.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iprimitive.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oarchive.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oprimitive.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iarchive.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iprimitive.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oarchive.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oprimitive.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_grammar.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_iarchive.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_oarchive.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/text_iarchive_impl.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/text_oarchive_impl.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/text_wiarchive_impl.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/text_woarchive_impl.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/xml_iarchive_impl.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/xml_oarchive_impl.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/xml_wiarchive_impl.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/xml_woarchive_impl.ipp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_exception.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_from_binary.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/binary_from_base64.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow_exception.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/escape.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/insert_linebreaks.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/istream_iterator.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/mb_from_wchar.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/ostream_iterator.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/remove_whitespace.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/transform_width.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/unescape.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/wchar_from_mb.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_escape.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape_exception.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_iarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_oarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_iarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_oarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_iarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_oarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_wiarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_woarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_iarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_oarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_wiarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_woarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/text_iarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/text_oarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/text_wiarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/text_woarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/tmpdir.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/wcslen.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_archive_exception.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_iarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_oarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_wiarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_woarchive.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/foreach_fwd.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/composite_key.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/access_specifier.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/adl_swap.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/archive_constructed.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/auto_space.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/base_type.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/bidir_node_iterator.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/bucket_array.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/cons_stdtuple.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/converter.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/copy_map.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/do_not_copy_elements_tag.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/duplicates_iterator.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/has_tag.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_args.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_iterator.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_node.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/header_holder.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ignore_wstrict_aliasing.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_base.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_loader.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_matcher.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_node_base.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_saver.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/invariant_assert.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_index_list.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_transparent.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/iter_adaptor.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/modify_key_adaptor.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/no_duplicate_tags.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/node_type.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_args.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl_fwd.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_node.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_ops.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/promotes_arg.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/raw_ptr.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/restore_wstrict_aliasing.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_loader.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_node.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ops.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ptr_array.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_node_iterator.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnk_index_ops.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/safe_mode.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/scope_guard.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_node.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_ops.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/serialization_version.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/uintptr_type.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/unbounded.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/value_compare.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/vartempl_support.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/global_fun.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index_fwd.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/identity.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/identity_fwd.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/indexed_by.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/key_extractors.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/mem_fun.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/member.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index_fwd.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index_fwd.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index_fwd.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/safe_mode_errors.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index_fwd.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/tag.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index_container.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index_container_fwd.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/access.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_map.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_set.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/array.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/array_optimization.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/array_wrapper.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/assume_abstract.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/base_object.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/binary_object.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/bitset.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/boost_array.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_map.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_set.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/collection_size_type.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/collection_traits.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/collections_load_imp.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/collections_save_imp.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/complex.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/config.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/deque.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/is_default_constructible.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_count_132.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_132.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_nmt_132.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/stack_constructor.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/ephemeral.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/export.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_no_rtti.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_typeid.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/factory.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/force_include.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/forward_list.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_load_imp.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_save_imp.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/hash_map.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/hash_set.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/is_bitwise_serializable.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/item_version_type.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/level.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/level_enum.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/list.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/map.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/nvp.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/optional.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/priority_queue.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/queue.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/scoped_ptr.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/serialization.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/set.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_132.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_helper.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/singleton.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/slist.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/smart_cast.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/split_free.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/split_member.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/stack.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/state_saver.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/static_warning.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/string.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/strong_typedef.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/throw_exception.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/tracking.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/tracking_enum.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/traits.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/type_info_implementation.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unique_ptr.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_load_imp.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_save_imp.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unordered_map.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unordered_set.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/utility.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/valarray.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/variant.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/vector.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/vector_135.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/version.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/void_cast.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/void_cast_fwd.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/weak_ptr.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/wrapper.hpp diff --git a/contrib/libboost/boost_1_65_0/boost/archive/archive_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/archive_exception.hpp new file mode 100644 index 00000000000..fabcdb5fa71 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/archive_exception.hpp @@ -0,0 +1,100 @@ +#ifndef BOOST_ARCHIVE_ARCHIVE_EXCEPTION_HPP +#define BOOST_ARCHIVE_ARCHIVE_EXCEPTION_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// archive/archive_exception.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include + +#include +#include + +// note: the only reason this is in here is that windows header +// includes #define exception_code _exception_code (arrrgghhhh!). +// the most expedient way to address this is be sure that this +// header is always included whenever this header file is included. +#if defined(BOOST_WINDOWS) +#include +#endif + +#include // must be the last header + +namespace boost { +namespace archive { + +////////////////////////////////////////////////////////////////////// +// exceptions thrown by archives +// +class BOOST_SYMBOL_VISIBLE archive_exception : + public virtual std::exception +{ +private: + char m_buffer[128]; +protected: + BOOST_ARCHIVE_DECL unsigned int + append(unsigned int l, const char * a); + BOOST_ARCHIVE_DECL + archive_exception() BOOST_NOEXCEPT; +public: + typedef enum { + no_exception, // initialized without code + other_exception, // any excepton not listed below + unregistered_class, // attempt to serialize a pointer of + // an unregistered class + invalid_signature, // first line of archive does not contain + // expected string + unsupported_version,// archive created with library version + // subsequent to this one + pointer_conflict, // an attempt has been made to directly + // serialize an object which has + // already been serialized through a pointer. + // Were this permitted, the archive load would result + // in the creation of an extra copy of the obect. + incompatible_native_format, // attempt to read native binary format + // on incompatible platform + array_size_too_short,// array being loaded doesn't fit in array allocated + input_stream_error, // error on input stream + invalid_class_name, // class name greater than the maximum permitted. + // most likely a corrupted archive or an attempt + // to insert virus via buffer overrun method. + unregistered_cast, // base - derived relationship not registered with + // void_cast_register + unsupported_class_version, // type saved with a version # greater than the + // one used by the program. This indicates that the program + // needs to be rebuilt. + multiple_code_instantiation, // code for implementing serialization for some + // type has been instantiated in more than one module. + output_stream_error // error on input stream + } exception_code; + exception_code code; + + BOOST_ARCHIVE_DECL archive_exception( + exception_code c, + const char * e1 = NULL, + const char * e2 = NULL + ) BOOST_NOEXCEPT; + BOOST_ARCHIVE_DECL archive_exception(archive_exception const &) BOOST_NOEXCEPT ; + virtual BOOST_ARCHIVE_DECL ~archive_exception() BOOST_NOEXCEPT_OR_NOTHROW ; + virtual BOOST_ARCHIVE_DECL const char * what() const BOOST_NOEXCEPT_OR_NOTHROW ; +}; + +}// namespace archive +}// namespace boost + +#include // pops abi_suffix.hpp pragmas + +#endif //BOOST_ARCHIVE_ARCHIVE_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_archive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_archive.hpp new file mode 100644 index 00000000000..ce7ac99a6dd --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/basic_archive.hpp @@ -0,0 +1,304 @@ +#ifndef BOOST_ARCHIVE_BASIC_ARCHIVE_HPP +#define BOOST_ARCHIVE_BASIC_ARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_archive.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. +#include // count +#include +#include +#include // size_t +#include +#include + +#include +#include // must be the last header + +namespace boost { +namespace archive { + +#if defined(_MSC_VER) +#pragma warning( push ) +#pragma warning( disable : 4244 4267 ) +#endif + +/* NOTE : Warning : Warning : Warning : Warning : Warning + * Don't ever changes this. If you do, they previously created + * binary archives won't be readable !!! + */ +class library_version_type { +private: + typedef uint_least16_t base_type; + base_type t; +public: + library_version_type(): t(0) {}; + explicit library_version_type(const unsigned int & t_) : t(t_){ + BOOST_ASSERT(t_ <= boost::integer_traits::const_max); + } + library_version_type(const library_version_type & t_) : + t(t_.t) + {} + library_version_type & operator=(const library_version_type & rhs){ + t = rhs.t; + return *this; + } + // used for text output + operator base_type () const { + return t; + } + // used for text input + operator base_type & (){ + return t; + } + bool operator==(const library_version_type & rhs) const { + return t == rhs.t; + } + bool operator<(const library_version_type & rhs) const { + return t < rhs.t; + } +}; + +BOOST_ARCHIVE_DECL library_version_type +BOOST_ARCHIVE_VERSION(); + +class version_type { +private: + typedef uint_least32_t base_type; + base_type t; +public: + // should be private - but MPI fails if it's not!!! + version_type(): t(0) {}; + explicit version_type(const unsigned int & t_) : t(t_){ + BOOST_ASSERT(t_ <= boost::integer_traits::const_max); + } + version_type(const version_type & t_) : + t(t_.t) + {} + version_type & operator=(const version_type & rhs){ + t = rhs.t; + return *this; + } + // used for text output + operator base_type () const { + return t; + } + // used for text intput + operator base_type & (){ + return t; + } + bool operator==(const version_type & rhs) const { + return t == rhs.t; + } + bool operator<(const version_type & rhs) const { + return t < rhs.t; + } +}; + +class class_id_type { +private: + typedef int_least16_t base_type; + base_type t; +public: + // should be private - but then can't use BOOST_STRONG_TYPE below + class_id_type() : t(0) {}; + explicit class_id_type(const int t_) : t(t_){ + BOOST_ASSERT(t_ <= boost::integer_traits::const_max); + } + explicit class_id_type(const std::size_t t_) : t(t_){ + // BOOST_ASSERT(t_ <= boost::integer_traits::const_max); + } + class_id_type(const class_id_type & t_) : + t(t_.t) + {} + class_id_type & operator=(const class_id_type & rhs){ + t = rhs.t; + return *this; + } + + // used for text output + operator int () const { + return t; + } + // used for text input + operator int_least16_t &() { + return t; + } + bool operator==(const class_id_type & rhs) const { + return t == rhs.t; + } + bool operator<(const class_id_type & rhs) const { + return t < rhs.t; + } +}; + +#define NULL_POINTER_TAG boost::archive::class_id_type(-1) + +class object_id_type { +private: + typedef uint_least32_t base_type; + base_type t; +public: + object_id_type(): t(0) {}; + // note: presumes that size_t >= unsigned int. + explicit object_id_type(const std::size_t & t_) : t(t_){ + BOOST_ASSERT(t_ <= boost::integer_traits::const_max); + } + object_id_type(const object_id_type & t_) : + t(t_.t) + {} + object_id_type & operator=(const object_id_type & rhs){ + t = rhs.t; + return *this; + } + // used for text output + operator uint_least32_t () const { + return t; + } + // used for text input + operator uint_least32_t & () { + return t; + } + bool operator==(const object_id_type & rhs) const { + return t == rhs.t; + } + bool operator<(const object_id_type & rhs) const { + return t < rhs.t; + } +}; + +#if defined(_MSC_VER) +#pragma warning( pop ) +#endif + +struct tracking_type { + bool t; + explicit tracking_type(const bool t_ = false) + : t(t_) + {}; + tracking_type(const tracking_type & t_) + : t(t_.t) + {} + operator bool () const { + return t; + }; + operator bool & () { + return t; + }; + tracking_type & operator=(const bool t_){ + t = t_; + return *this; + } + bool operator==(const tracking_type & rhs) const { + return t == rhs.t; + } + bool operator==(const bool & rhs) const { + return t == rhs; + } + tracking_type & operator=(const tracking_type & rhs){ + t = rhs.t; + return *this; + } +}; + +struct class_name_type : + private boost::noncopyable +{ + char *t; + operator const char * & () const { + return const_cast(t); + } + operator char * () { + return t; + } + std::size_t size() const { + return std::strlen(t); + } + explicit class_name_type(const char *key_) + : t(const_cast(key_)){} + explicit class_name_type(char *key_) + : t(key_){} + class_name_type & operator=(const class_name_type & rhs){ + t = rhs.t; + return *this; + } +}; + +enum archive_flags { + no_header = 1, // suppress archive header info + no_codecvt = 2, // suppress alteration of codecvt facet + no_xml_tag_checking = 4, // suppress checking of xml tags + no_tracking = 8, // suppress ALL tracking + flags_last = 8 +}; + +BOOST_ARCHIVE_DECL const char * +BOOST_ARCHIVE_SIGNATURE(); + +/* NOTE : Warning : Warning : Warning : Warning : Warning + * If any of these are changed to different sized types, + * binary_iarchive won't be able to read older archives + * unless you rev the library version and include conditional + * code based on the library version. There is nothing + * inherently wrong in doing this - but you have to be super + * careful because it's easy to get wrong and start breaking + * old archives !!! + */ + +#define BOOST_ARCHIVE_STRONG_TYPEDEF(T, D) \ + class D : public T { \ + public: \ + explicit D(const T tt) : T(tt){} \ + }; \ +/**/ + +BOOST_ARCHIVE_STRONG_TYPEDEF(class_id_type, class_id_reference_type) +BOOST_ARCHIVE_STRONG_TYPEDEF(class_id_type, class_id_optional_type) +BOOST_ARCHIVE_STRONG_TYPEDEF(object_id_type, object_reference_type) + +}// namespace archive +}// namespace boost + +#include // pops abi_suffix.hpp pragmas + +#include + +// set implementation level to primitive for all types +// used internally by the serialization library + +BOOST_CLASS_IMPLEMENTATION(boost::archive::library_version_type, primitive_type) +BOOST_CLASS_IMPLEMENTATION(boost::archive::version_type, primitive_type) +BOOST_CLASS_IMPLEMENTATION(boost::archive::class_id_type, primitive_type) +BOOST_CLASS_IMPLEMENTATION(boost::archive::class_id_reference_type, primitive_type) +BOOST_CLASS_IMPLEMENTATION(boost::archive::class_id_optional_type, primitive_type) +BOOST_CLASS_IMPLEMENTATION(boost::archive::class_name_type, primitive_type) +BOOST_CLASS_IMPLEMENTATION(boost::archive::object_id_type, primitive_type) +BOOST_CLASS_IMPLEMENTATION(boost::archive::object_reference_type, primitive_type) +BOOST_CLASS_IMPLEMENTATION(boost::archive::tracking_type, primitive_type) + +#include + +// set types used internally by the serialization library +// to be bitwise serializable + +BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::library_version_type) +BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::version_type) +BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::class_id_type) +BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::class_id_reference_type) +BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::class_id_optional_type) +BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::class_name_type) +BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::object_id_type) +BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::object_reference_type) +BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::tracking_type) + +#endif //BOOST_ARCHIVE_BASIC_ARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iarchive.hpp new file mode 100644 index 00000000000..c0cc655c997 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iarchive.hpp @@ -0,0 +1,204 @@ +#ifndef BOOST_ARCHIVE_BASIC_BINARY_IARCHIVE_HPP +#define BOOST_ARCHIVE_BASIC_BINARY_IARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_binary_iarchive.hpp +// +// archives stored as native binary - this should be the fastest way +// to archive the state of a group of obects. It makes no attempt to +// convert to any canonical form. + +// IN GENERAL, ARCHIVES CREATED WITH THIS CLASS WILL NOT BE READABLE +// ON PLATFORM APART FROM THE ONE THEY ARE CREATED ON + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include + +#include +#include +#include +#include +#include +#include + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +#include // must be the last header + +namespace boost { +namespace archive { + +namespace detail { + template class interface_iarchive; +} // namespace detail + +///////////////////////////////////////////////////////////////////////// +// class basic_binary_iarchive - read serialized objects from a input binary stream +template +class BOOST_SYMBOL_VISIBLE basic_binary_iarchive : + public detail::common_iarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) + // for some inexplicable reason insertion of "class" generates compile erro + // on msvc 7.1 + friend detail::interface_iarchive; + #else + friend class detail::interface_iarchive; + #endif +#endif + // intermediate level to support override of operators + // fot templates in the absence of partial function + // template ordering. If we get here pass to base class + // note extra nonsense to sneak it pass the borland compiers + typedef detail::common_iarchive detail_common_iarchive; + template + void load_override(T & t){ + this->detail_common_iarchive::load_override(t); + } + + // include these to trap a change in binary format which + // isn't specifically handled + // upto 32K classes + BOOST_STATIC_ASSERT(sizeof(class_id_type) == sizeof(int_least16_t)); + BOOST_STATIC_ASSERT(sizeof(class_id_reference_type) == sizeof(int_least16_t)); + // upto 2G objects + BOOST_STATIC_ASSERT(sizeof(object_id_type) == sizeof(uint_least32_t)); + BOOST_STATIC_ASSERT(sizeof(object_reference_type) == sizeof(uint_least32_t)); + + // binary files don't include the optional information + void load_override(class_id_optional_type & /* t */){} + + void load_override(tracking_type & t, int /*version*/){ + library_version_type lvt = this->get_library_version(); + if(boost::archive::library_version_type(6) < lvt){ + int_least8_t x=0; + * this->This() >> x; + t = boost::archive::tracking_type(x); + } + else{ + bool x=0; + * this->This() >> x; + t = boost::archive::tracking_type(x); + } + } + void load_override(class_id_type & t){ + library_version_type lvt = this->get_library_version(); + if(boost::archive::library_version_type(7) < lvt){ + this->detail_common_iarchive::load_override(t); + } + else + if(boost::archive::library_version_type(6) < lvt){ + int_least16_t x=0; + * this->This() >> x; + t = boost::archive::class_id_type(x); + } + else{ + int x=0; + * this->This() >> x; + t = boost::archive::class_id_type(x); + } + } + void load_override(class_id_reference_type & t){ + load_override(static_cast(t)); + } + + void load_override(version_type & t){ + library_version_type lvt = this->get_library_version(); + if(boost::archive::library_version_type(7) < lvt){ + this->detail_common_iarchive::load_override(t); + } + else + if(boost::archive::library_version_type(6) < lvt){ + uint_least8_t x=0; + * this->This() >> x; + t = boost::archive::version_type(x); + } + else + if(boost::archive::library_version_type(5) < lvt){ + uint_least16_t x=0; + * this->This() >> x; + t = boost::archive::version_type(x); + } + else + if(boost::archive::library_version_type(2) < lvt){ + // upto 255 versions + unsigned char x=0; + * this->This() >> x; + t = version_type(x); + } + else{ + unsigned int x=0; + * this->This() >> x; + t = boost::archive::version_type(x); + } + } + + void load_override(boost::serialization::item_version_type & t){ + library_version_type lvt = this->get_library_version(); +// if(boost::archive::library_version_type(7) < lvt){ + if(boost::archive::library_version_type(6) < lvt){ + this->detail_common_iarchive::load_override(t); + } + else + if(boost::archive::library_version_type(6) < lvt){ + uint_least16_t x=0; + * this->This() >> x; + t = boost::serialization::item_version_type(x); + } + else{ + unsigned int x=0; + * this->This() >> x; + t = boost::serialization::item_version_type(x); + } + } + + void load_override(serialization::collection_size_type & t){ + if(boost::archive::library_version_type(5) < this->get_library_version()){ + this->detail_common_iarchive::load_override(t); + } + else{ + unsigned int x=0; + * this->This() >> x; + t = serialization::collection_size_type(x); + } + } + + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + load_override(class_name_type & t); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + init(); + + basic_binary_iarchive(unsigned int flags) : + detail::common_iarchive(flags) + {} +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_BASIC_BINARY_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iprimitive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iprimitive.hpp new file mode 100644 index 00000000000..665d3e81e1f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iprimitive.hpp @@ -0,0 +1,198 @@ +#ifndef BOOST_ARCHIVE_BINARY_IPRIMITIVE_HPP +#define BOOST_ARCHIVE_BINARY_IPRIMITIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +#if defined(_MSC_VER) +#pragma warning( disable : 4800 ) +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_binary_iprimitive.hpp +// +// archives stored as native binary - this should be the fastest way +// to archive the state of a group of obects. It makes no attempt to +// convert to any canonical form. + +// IN GENERAL, ARCHIVES CREATED WITH THIS CLASS WILL NOT BE READABLE +// ON PLATFORM APART FROM THE ONE THEY ARE CREATED ON + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include +#include // std::memcpy +#include // std::size_t +#include // basic_streambuf +#include + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::memcpy; + using ::size_t; +} // namespace std +#endif + +#include +#include +#include +#include + +//#include +#include +#include + +#include +#include +#include +#include +#include // must be the last header + +namespace boost { +namespace archive { + +///////////////////////////////////////////////////////////////////////////// +// class binary_iarchive - read serialized objects from a input binary stream +template +class BOOST_SYMBOL_VISIBLE basic_binary_iprimitive { +#ifndef BOOST_NO_MEMBER_TEMPLATE_FRIENDS + friend class load_access; +protected: +#else +public: +#endif + std::basic_streambuf & m_sb; + // return a pointer to the most derived class + Archive * This(){ + return static_cast(this); + } + + #ifndef BOOST_NO_STD_LOCALE + // note order! - if you change this, libstd++ will fail! + // a) create new locale with new codecvt facet + // b) save current locale + // c) change locale to new one + // d) use stream buffer + // e) change locale back to original + // f) destroy new codecvt facet + boost::archive::codecvt_null codecvt_null_facet; + basic_streambuf_locale_saver locale_saver; + std::locale archive_locale; + #endif + + // main template for serilization of primitive types + template + void load(T & t){ + load_binary(& t, sizeof(T)); + } + + ///////////////////////////////////////////////////////// + // fundamental types that need special treatment + + // trap usage of invalid uninitialized boolean + void load(bool & t){ + load_binary(& t, sizeof(t)); + int i = t; + BOOST_ASSERT(0 == i || 1 == i); + (void)i; // warning suppression for release builds. + } + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + load(std::string &s); + #ifndef BOOST_NO_STD_WSTRING + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + load(std::wstring &ws); + #endif + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + load(char * t); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + load(wchar_t * t); + + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + init(); + BOOST_ARCHIVE_OR_WARCHIVE_DECL + basic_binary_iprimitive( + std::basic_streambuf & sb, + bool no_codecvt + ); + BOOST_ARCHIVE_OR_WARCHIVE_DECL + ~basic_binary_iprimitive(); +public: + // we provide an optimized load for all fundamental types + // typedef serialization::is_bitwise_serializable + // use_array_optimization; + struct use_array_optimization { + template + #if defined(BOOST_NO_DEPENDENT_NESTED_DERIVATIONS) + struct apply { + typedef typename boost::serialization::is_bitwise_serializable< T >::type type; + }; + #else + struct apply : public boost::serialization::is_bitwise_serializable< T > {}; + #endif + }; + + // the optimized load_array dispatches to load_binary + template + void load_array(serialization::array_wrapper& a, unsigned int) + { + load_binary(a.address(),a.count()*sizeof(ValueType)); + } + + void + load_binary(void *address, std::size_t count); +}; + +template +inline void +basic_binary_iprimitive::load_binary( + void *address, + std::size_t count +){ + // note: an optimizer should eliminate the following for char files + BOOST_ASSERT( + static_cast(count / sizeof(Elem)) + <= boost::integer_traits::const_max + ); + std::streamsize s = static_cast(count / sizeof(Elem)); + std::streamsize scount = m_sb.sgetn( + static_cast(address), + s + ); + if(scount != s) + boost::serialization::throw_exception( + archive_exception(archive_exception::input_stream_error) + ); + // note: an optimizer should eliminate the following for char files + BOOST_ASSERT(count % sizeof(Elem) <= boost::integer_traits::const_max); + s = static_cast(count % sizeof(Elem)); + if(0 < s){ +// if(is.fail()) +// boost::serialization::throw_exception( +// archive_exception(archive_exception::stream_error) +// ); + Elem t; + scount = m_sb.sgetn(& t, 1); + if(scount != 1) + boost::serialization::throw_exception( + archive_exception(archive_exception::input_stream_error) + ); + std::memcpy(static_cast(address) + (count - s), &t, static_cast(s)); + } +} + +} // namespace archive +} // namespace boost + +#include // pop pragmas + +#endif // BOOST_ARCHIVE_BINARY_IPRIMITIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oarchive.hpp new file mode 100644 index 00000000000..f05f2f86d55 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oarchive.hpp @@ -0,0 +1,185 @@ +#ifndef BOOST_ARCHIVE_BASIC_BINARY_OARCHIVE_HPP +#define BOOST_ARCHIVE_BASIC_BINARY_OARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_binary_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// archives stored as native binary - this should be the fastest way +// to archive the state of a group of obects. It makes no attempt to +// convert to any canonical form. + +// IN GENERAL, ARCHIVES CREATED WITH THIS CLASS WILL NOT BE READABLE +// ON PLATFORM APART FROM THE ONE THEY ARE CREATE ON + +#include +#include +#include + +#include +#include + +#include +#include +#include +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_oarchive; +} // namespace detail + +////////////////////////////////////////////////////////////////////// +// class basic_binary_oarchive - write serialized objects to a binary output stream +// note: this archive has no pretensions to portability. Archive format +// may vary across machine architectures and compilers. About the only +// guarentee is that an archive created with this code will be readable +// by a program built with the same tools for the same machne. This class +// does have the virtue of buiding the smalles archive in the minimum amount +// of time. So under some circumstances it may be he right choice. +template +class BOOST_SYMBOL_VISIBLE basic_binary_oarchive : + public detail::common_oarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) + // for some inexplicable reason insertion of "class" generates compile erro + // on msvc 7.1 + friend detail::interface_oarchive; + #else + friend class detail::interface_oarchive; + #endif +#endif + // any datatype not specifed below will be handled by base class + typedef detail::common_oarchive detail_common_oarchive; + template + void save_override(const T & t){ + this->detail_common_oarchive::save_override(t); + } + + // include these to trap a change in binary format which + // isn't specifically handled + BOOST_STATIC_ASSERT(sizeof(tracking_type) == sizeof(bool)); + // upto 32K classes + BOOST_STATIC_ASSERT(sizeof(class_id_type) == sizeof(int_least16_t)); + BOOST_STATIC_ASSERT(sizeof(class_id_reference_type) == sizeof(int_least16_t)); + // upto 2G objects + BOOST_STATIC_ASSERT(sizeof(object_id_type) == sizeof(uint_least32_t)); + BOOST_STATIC_ASSERT(sizeof(object_reference_type) == sizeof(uint_least32_t)); + + // binary files don't include the optional information + void save_override(const class_id_optional_type & /* t */){} + + // enable this if we decide to support generation of previous versions + #if 0 + void save_override(const boost::archive::version_type & t){ + library_version_type lvt = this->get_library_version(); + if(boost::archive::library_version_type(7) < lvt){ + this->detail_common_oarchive::save_override(t); + } + else + if(boost::archive::library_version_type(6) < lvt){ + const boost::uint_least16_t x = t; + * this->This() << x; + } + else{ + const unsigned int x = t; + * this->This() << x; + } + } + void save_override(const boost::serialization::item_version_type & t){ + library_version_type lvt = this->get_library_version(); + if(boost::archive::library_version_type(7) < lvt){ + this->detail_common_oarchive::save_override(t); + } + else + if(boost::archive::library_version_type(6) < lvt){ + const boost::uint_least16_t x = t; + * this->This() << x; + } + else{ + const unsigned int x = t; + * this->This() << x; + } + } + + void save_override(class_id_type & t){ + library_version_type lvt = this->get_library_version(); + if(boost::archive::library_version_type(7) < lvt){ + this->detail_common_oarchive::save_override(t); + } + else + if(boost::archive::library_version_type(6) < lvt){ + const boost::int_least16_t x = t; + * this->This() << x; + } + else{ + const int x = t; + * this->This() << x; + } + } + void save_override(class_id_reference_type & t){ + save_override(static_cast(t)); + } + + #endif + + // explicitly convert to char * to avoid compile ambiguities + void save_override(const class_name_type & t){ + const std::string s(t); + * this->This() << s; + } + + #if 0 + void save_override(const serialization::collection_size_type & t){ + if (get_library_version() < boost::archive::library_version_type(6)){ + unsigned int x=0; + * this->This() >> x; + t = serialization::collection_size_type(x); + } + else{ + * this->This() >> t; + } + } + #endif + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + init(); + + basic_binary_oarchive(unsigned int flags) : + detail::common_oarchive(flags) + {} +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_BASIC_BINARY_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oprimitive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oprimitive.hpp new file mode 100644 index 00000000000..6dc770c60e8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oprimitive.hpp @@ -0,0 +1,188 @@ +#ifndef BOOST_ARCHIVE_BASIC_BINARY_OPRIMITIVE_HPP +#define BOOST_ARCHIVE_BASIC_BINARY_OPRIMITIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_binary_oprimitive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// archives stored as native binary - this should be the fastest way +// to archive the state of a group of obects. It makes no attempt to +// convert to any canonical form. + +// IN GENERAL, ARCHIVES CREATED WITH THIS CLASS WILL NOT BE READABLE +// ON PLATFORM APART FROM THE ONE THEY ARE CREATE ON + +#include +#include +#include +#include // basic_streambuf +#include +#include // size_t + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include +#include +#include +#include +#include + +//#include +#include +#include + +#include +#include +#include +#include +#include // must be the last header + +namespace boost { +namespace archive { + +///////////////////////////////////////////////////////////////////////// +// class basic_binary_oprimitive - binary output of prmitives + +template +class BOOST_SYMBOL_VISIBLE basic_binary_oprimitive { +#ifndef BOOST_NO_MEMBER_TEMPLATE_FRIENDS + friend class save_access; +protected: +#else +public: +#endif + std::basic_streambuf & m_sb; + // return a pointer to the most derived class + Archive * This(){ + return static_cast(this); + } + #ifndef BOOST_NO_STD_LOCALE + // note order! - if you change this, libstd++ will fail! + // a) create new locale with new codecvt facet + // b) save current locale + // c) change locale to new one + // d) use stream buffer + // e) change locale back to original + // f) destroy new codecvt facet + boost::archive::codecvt_null codecvt_null_facet; + basic_streambuf_locale_saver locale_saver; + std::locale archive_locale; + #endif + // default saving of primitives. + template + void save(const T & t) + { + save_binary(& t, sizeof(T)); + } + + ///////////////////////////////////////////////////////// + // fundamental types that need special treatment + + // trap usage of invalid uninitialized boolean which would + // otherwise crash on load. + void save(const bool t){ + BOOST_ASSERT(0 == static_cast(t) || 1 == static_cast(t)); + save_binary(& t, sizeof(t)); + } + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save(const std::string &s); + #ifndef BOOST_NO_STD_WSTRING + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save(const std::wstring &ws); + #endif + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save(const char * t); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save(const wchar_t * t); + + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + init(); + + BOOST_ARCHIVE_OR_WARCHIVE_DECL + basic_binary_oprimitive( + std::basic_streambuf & sb, + bool no_codecvt + ); + BOOST_ARCHIVE_OR_WARCHIVE_DECL + ~basic_binary_oprimitive(); +public: + + // we provide an optimized save for all fundamental types + // typedef serialization::is_bitwise_serializable + // use_array_optimization; + // workaround without using mpl lambdas + struct use_array_optimization { + template + #if defined(BOOST_NO_DEPENDENT_NESTED_DERIVATIONS) + struct apply { + typedef typename boost::serialization::is_bitwise_serializable< T >::type type; + }; + #else + struct apply : public boost::serialization::is_bitwise_serializable< T > {}; + #endif + }; + + // the optimized save_array dispatches to save_binary + template + void save_array(boost::serialization::array_wrapper const& a, unsigned int) + { + save_binary(a.address(),a.count()*sizeof(ValueType)); + } + + void save_binary(const void *address, std::size_t count); +}; + +template +inline void +basic_binary_oprimitive::save_binary( + const void *address, + std::size_t count +){ + // BOOST_ASSERT(count <= std::size_t(boost::integer_traits::const_max)); + // note: if the following assertions fail + // a likely cause is that the output stream is set to "text" + // mode where by cr characters recieve special treatment. + // be sure that the output stream is opened with ios::binary + //if(os.fail()) + // boost::serialization::throw_exception( + // archive_exception(archive_exception::output_stream_error) + // ); + // figure number of elements to output - round up + count = ( count + sizeof(Elem) - 1) / sizeof(Elem); + std::streamsize scount = m_sb.sputn( + static_cast(address), + static_cast(count) + ); + if(count != static_cast(scount)) + boost::serialization::throw_exception( + archive_exception(archive_exception::output_stream_error) + ); + //os.write( + // static_cast(address), + // count + //); + //BOOST_ASSERT(os.good()); +} + +} //namespace boost +} //namespace archive + +#include // pop pragmas + +#endif // BOOST_ARCHIVE_BASIC_BINARY_OPRIMITIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_streambuf_locale_saver.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_streambuf_locale_saver.hpp new file mode 100644 index 00000000000..5cd4b36f081 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/basic_streambuf_locale_saver.hpp @@ -0,0 +1,108 @@ +#ifndef BOOST_ARCHIVE_BASIC_STREAMBUF_LOCALE_SAVER_HPP +#define BOOST_ARCHIVE_BASIC_STREAMBUF_LOCALE_SAVER_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_streambuf_locale_saver.hpp + +// (C) Copyright 2005 Robert Ramey - http://www.rrsd.com + +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// note derived from boost/io/ios_state.hpp +// Copyright 2002, 2005 Daryle Walker. Use, modification, and distribution +// are subject to the Boost Software License, Version 1.0. (See accompanying +// file LICENSE_1_0.txt or a copy at .) + +// See for the library's home page. + +#ifndef BOOST_NO_STD_LOCALE + +#include // for std::locale +#include +#include // for std::basic_streambuf + +#include +#include + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost{ +namespace archive{ + +template < typename Ch, class Tr > +class basic_streambuf_locale_saver : + private boost::noncopyable +{ +public: + explicit basic_streambuf_locale_saver(std::basic_streambuf &s) : + m_streambuf(s), + m_locale(s.getloc()) + {} + ~basic_streambuf_locale_saver(){ + m_streambuf.pubsync(); + m_streambuf.pubimbue(m_locale); + } +private: + std::basic_streambuf & m_streambuf; + std::locale const m_locale; +}; + +template < typename Ch, class Tr > +class basic_istream_locale_saver : + private boost::noncopyable +{ +public: + explicit basic_istream_locale_saver(std::basic_istream &s) : + m_istream(s), + m_locale(s.getloc()) + {} + ~basic_istream_locale_saver(){ + // libstdc++ crashes without this + m_istream.sync(); + m_istream.imbue(m_locale); + } +private: + std::basic_istream & m_istream; + std::locale const m_locale; +}; + +template < typename Ch, class Tr > +class basic_ostream_locale_saver : + private boost::noncopyable +{ +public: + explicit basic_ostream_locale_saver(std::basic_ostream &s) : + m_ostream(s), + m_locale(s.getloc()) + {} + ~basic_ostream_locale_saver(){ + m_ostream.flush(); + m_ostream.imbue(m_locale); + } +private: + std::basic_ostream & m_ostream; + std::locale const m_locale; +}; + + +} // archive +} // boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_NO_STD_LOCALE +#endif // BOOST_ARCHIVE_BASIC_STREAMBUF_LOCALE_SAVER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iarchive.hpp new file mode 100644 index 00000000000..48a646cc1f7 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iarchive.hpp @@ -0,0 +1,96 @@ +#ifndef BOOST_ARCHIVE_BASIC_TEXT_IARCHIVE_HPP +#define BOOST_ARCHIVE_BASIC_TEXT_IARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_text_iarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// archives stored as text - note these ar templated on the basic +// stream templates to accommodate wide (and other?) kind of characters +// +// note the fact that on libraries without wide characters, ostream is +// is not a specialization of basic_ostream which in fact is not defined +// in such cases. So we can't use basic_istream but rather +// use two template parameters + +#include +#include + +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_iarchive; +} // namespace detail + +///////////////////////////////////////////////////////////////////////// +// class basic_text_iarchive - read serialized objects from a input text stream +template +class BOOST_SYMBOL_VISIBLE basic_text_iarchive : + public detail::common_iarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) + // for some inexplicable reason insertion of "class" generates compile erro + // on msvc 7.1 + friend detail::interface_iarchive; + #else + friend class detail::interface_iarchive; + #endif +#endif + // intermediate level to support override of operators + // fot templates in the absence of partial function + // template ordering + typedef detail::common_iarchive detail_common_iarchive; + template + void load_override(T & t){ + this->detail_common_iarchive::load_override(t); + } + // text file don't include the optional information + void load_override(class_id_optional_type & /*t*/){} + + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + load_override(class_name_type & t); + + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + init(void); + + basic_text_iarchive(unsigned int flags) : + detail::common_iarchive(flags) + {} + ~basic_text_iarchive(){} +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_BASIC_TEXT_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iprimitive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iprimitive.hpp new file mode 100644 index 00000000000..bf936b55546 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iprimitive.hpp @@ -0,0 +1,142 @@ +#ifndef BOOST_ARCHIVE_BASIC_TEXT_IPRIMITIVE_HPP +#define BOOST_ARCHIVE_BASIC_TEXT_IPRIMITIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_text_iprimitive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// archives stored as text - note these are templated on the basic +// stream templates to accommodate wide (and other?) kind of characters +// +// Note the fact that on libraries without wide characters, ostream is +// not a specialization of basic_ostream which in fact is not defined +// in such cases. So we can't use basic_ostream but rather +// use two template parameters + +#include +#include // size_t + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; + #if ! defined(BOOST_DINKUMWARE_STDLIB) && ! defined(__SGI_STL_PORT) + using ::locale; + #endif +} // namespace std +#endif + +#include +#include + +#include +#if BOOST_WORKAROUND(BOOST_DINKUMWARE_STDLIB, == 1) +#include +#endif +#include +#include +#include +#include +#include // must be the last header + +namespace boost { +namespace archive { + +///////////////////////////////////////////////////////////////////////// +// class basic_text_iarchive - load serialized objects from a input text stream +#if defined(_MSC_VER) +#pragma warning( push ) +#pragma warning( disable : 4244 4267 ) +#endif + +template +class BOOST_SYMBOL_VISIBLE basic_text_iprimitive { +protected: + IStream &is; + io::ios_flags_saver flags_saver; + io::ios_precision_saver precision_saver; + + #ifndef BOOST_NO_STD_LOCALE + // note order! - if you change this, libstd++ will fail! + // a) create new locale with new codecvt facet + // b) save current locale + // c) change locale to new one + // d) use stream buffer + // e) change locale back to original + // f) destroy new codecvt facet + boost::archive::codecvt_null codecvt_null_facet; + std::locale archive_locale; + basic_istream_locale_saver< + typename IStream::char_type, + typename IStream::traits_type + > locale_saver; + #endif + + template + void load(T & t) + { + if(is >> t) + return; + boost::serialization::throw_exception( + archive_exception(archive_exception::input_stream_error) + ); + } + + void load(char & t) + { + short int i; + load(i); + t = i; + } + void load(signed char & t) + { + short int i; + load(i); + t = i; + } + void load(unsigned char & t) + { + unsigned short int i; + load(i); + t = i; + } + + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + void load(wchar_t & t) + { + BOOST_STATIC_ASSERT(sizeof(wchar_t) <= sizeof(int)); + int i; + load(i); + t = i; + } + #endif + BOOST_ARCHIVE_OR_WARCHIVE_DECL + basic_text_iprimitive(IStream &is, bool no_codecvt); + BOOST_ARCHIVE_OR_WARCHIVE_DECL + ~basic_text_iprimitive(); +public: + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + load_binary(void *address, std::size_t count); +}; + +#if defined(_MSC_VER) +#pragma warning( pop ) +#endif + +} // namespace archive +} // namespace boost + +#include // pop pragmas + +#endif // BOOST_ARCHIVE_BASIC_TEXT_IPRIMITIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oarchive.hpp new file mode 100644 index 00000000000..6f7f8fb167d --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oarchive.hpp @@ -0,0 +1,119 @@ +#ifndef BOOST_ARCHIVE_BASIC_TEXT_OARCHIVE_HPP +#define BOOST_ARCHIVE_BASIC_TEXT_OARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_text_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// archives stored as text - note these ar templated on the basic +// stream templates to accommodate wide (and other?) kind of characters +// +// note the fact that on libraries without wide characters, ostream is +// is not a specialization of basic_ostream which in fact is not defined +// in such cases. So we can't use basic_ostream but rather +// use two template parameters + +#include +#include +#include +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_oarchive; +} // namespace detail + +///////////////////////////////////////////////////////////////////////// +// class basic_text_oarchive +template +class BOOST_SYMBOL_VISIBLE basic_text_oarchive : + public detail::common_oarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) + // for some inexplicable reason insertion of "class" generates compile erro + // on msvc 7.1 + friend detail::interface_oarchive; + #else + friend class detail::interface_oarchive; + #endif +#endif + + enum { + none, + eol, + space + } delimiter; + + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + newtoken(); + + void newline(){ + delimiter = eol; + } + + // default processing - kick back to base class. Note the + // extra stuff to get it passed borland compilers + typedef detail::common_oarchive detail_common_oarchive; + template + void save_override(T & t){ + this->detail_common_oarchive::save_override(t); + } + + // start new objects on a new line + void save_override(const object_id_type & t){ + this->This()->newline(); + this->detail_common_oarchive::save_override(t); + } + + // text file don't include the optional information + void save_override(const class_id_optional_type & /* t */){} + + void save_override(const class_name_type & t){ + const std::string s(t); + * this->This() << s; + } + + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + init(); + + basic_text_oarchive(unsigned int flags) : + detail::common_oarchive(flags), + delimiter(none) + {} + ~basic_text_oarchive(){} +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_BASIC_TEXT_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oprimitive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oprimitive.hpp new file mode 100644 index 00000000000..45f09358ece --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oprimitive.hpp @@ -0,0 +1,209 @@ +#ifndef BOOST_ARCHIVE_BASIC_TEXT_OPRIMITIVE_HPP +#define BOOST_ARCHIVE_BASIC_TEXT_OPRIMITIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_text_oprimitive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// archives stored as text - note these ar templated on the basic +// stream templates to accommodate wide (and other?) kind of characters +// +// note the fact that on libraries without wide characters, ostream is +// is not a specialization of basic_ostream which in fact is not defined +// in such cases. So we can't use basic_ostream but rather +// use two template parameters + +#include +#include +#include // size_t + +#include +#include +#include + +#include +#if BOOST_WORKAROUND(BOOST_DINKUMWARE_STDLIB, == 1) +#include +#endif + +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; + #if ! defined(BOOST_DINKUMWARE_STDLIB) && ! defined(__SGI_STL_PORT) + using ::locale; + #endif +} // namespace std +#endif + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include // must be the last header + +namespace boost { +namespace archive { + +///////////////////////////////////////////////////////////////////////// +// class basic_text_oprimitive - output of prmitives to stream +template +class BOOST_SYMBOL_VISIBLE basic_text_oprimitive +{ +protected: + OStream &os; + io::ios_flags_saver flags_saver; + io::ios_precision_saver precision_saver; + + #ifndef BOOST_NO_STD_LOCALE + // note order! - if you change this, libstd++ will fail! + // a) create new locale with new codecvt facet + // b) save current locale + // c) change locale to new one + // d) use stream buffer + // e) change locale back to original + // f) destroy new codecvt facet + boost::archive::codecvt_null codecvt_null_facet; + std::locale archive_locale; + basic_ostream_locale_saver< + typename OStream::char_type, + typename OStream::traits_type + > locale_saver; + #endif + + ///////////////////////////////////////////////////////// + // fundamental types that need special treatment + void save(const bool t){ + // trap usage of invalid uninitialized boolean which would + // otherwise crash on load. + BOOST_ASSERT(0 == static_cast(t) || 1 == static_cast(t)); + if(os.fail()) + boost::serialization::throw_exception( + archive_exception(archive_exception::output_stream_error) + ); + os << t; + } + void save(const signed char t) + { + save(static_cast(t)); + } + void save(const unsigned char t) + { + save(static_cast(t)); + } + void save(const char t) + { + save(static_cast(t)); + } + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + void save(const wchar_t t) + { + BOOST_STATIC_ASSERT(sizeof(wchar_t) <= sizeof(int)); + save(static_cast(t)); + } + #endif + + ///////////////////////////////////////////////////////// + // saving of any types not listed above + + template + void save_impl(const T &t, boost::mpl::bool_ &){ + if(os.fail()) + boost::serialization::throw_exception( + archive_exception(archive_exception::output_stream_error) + ); + os << t; + } + + ///////////////////////////////////////////////////////// + // floating point types need even more special treatment + // the following determines whether the type T is some sort + // of floating point type. Note that we then assume that + // the stream << operator is defined on that type - if not + // we'll get a compile time error. This is meant to automatically + // support synthesized types which support floating point + // operations. Also it should handle compiler dependent types + // such long double. Due to John Maddock. + + template + struct is_float { + typedef typename mpl::bool_< + boost::is_floating_point::value + || (std::numeric_limits::is_specialized + && !std::numeric_limits::is_integer + && !std::numeric_limits::is_exact + && std::numeric_limits::max_exponent) + >::type type; + }; + + template + void save_impl(const T &t, boost::mpl::bool_ &){ + // must be a user mistake - can't serialize un-initialized data + if(os.fail()) + boost::serialization::throw_exception( + archive_exception(archive_exception::output_stream_error) + ); + // The formulae for the number of decimla digits required is given in + // http://www2.open-std.org/JTC1/SC22/WG21/docs/papers/2005/n1822.pdf + // which is derived from Kahan's paper: + // www.eecs.berkeley.edu/~wkahan/ieee754status/ieee754.ps + // const unsigned int digits = (std::numeric_limits::digits * 3010) / 10000; + // note: I've commented out the above because I didn't get good results. e.g. + // in one case I got a difference of 19 units. + #ifndef BOOST_NO_CXX11_NUMERIC_LIMITS + const unsigned int digits = std::numeric_limits::max_digits10; + #else + const unsigned int digits = std::numeric_limits::digits10 + 2; + #endif + os << std::setprecision(digits) << std::scientific << t; + } + + template + void save(const T & t){ + typename is_float::type tf; + save_impl(t, tf); + } + + BOOST_ARCHIVE_OR_WARCHIVE_DECL + basic_text_oprimitive(OStream & os, bool no_codecvt); + BOOST_ARCHIVE_OR_WARCHIVE_DECL + ~basic_text_oprimitive(); +public: + // unformatted append of one character + void put(typename OStream::char_type c){ + if(os.fail()) + boost::serialization::throw_exception( + archive_exception(archive_exception::output_stream_error) + ); + os.put(c); + } + // unformatted append of null terminated string + void put(const char * s){ + while('\0' != *s) + os.put(*s++); + } + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save_binary(const void *address, std::size_t count); +}; + +} //namespace boost +} //namespace archive + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_BASIC_TEXT_OPRIMITIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_archive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_archive.hpp new file mode 100644 index 00000000000..bef368b973b --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_archive.hpp @@ -0,0 +1,67 @@ +#ifndef BOOST_ARCHIVE_BASIC_XML_TEXT_ARCHIVE_HPP +#define BOOST_ARCHIVE_BASIC_XML_TEXT_ARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_xml_archive.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include // must be the last header + +namespace boost { +namespace archive { + +// constant strings used in xml i/o + +extern +BOOST_ARCHIVE_DECL const char * +BOOST_ARCHIVE_XML_OBJECT_ID(); + +extern +BOOST_ARCHIVE_DECL const char * +BOOST_ARCHIVE_XML_OBJECT_REFERENCE(); + +extern +BOOST_ARCHIVE_DECL const char * +BOOST_ARCHIVE_XML_CLASS_ID(); + +extern +BOOST_ARCHIVE_DECL const char * +BOOST_ARCHIVE_XML_CLASS_ID_REFERENCE(); + +extern +BOOST_ARCHIVE_DECL const char * +BOOST_ARCHIVE_XML_CLASS_NAME(); + +extern +BOOST_ARCHIVE_DECL const char * +BOOST_ARCHIVE_XML_TRACKING(); + +extern +BOOST_ARCHIVE_DECL const char * +BOOST_ARCHIVE_XML_VERSION(); + +extern +BOOST_ARCHIVE_DECL const char * +BOOST_ARCHIVE_XML_SIGNATURE(); + +}// namespace archive +}// namespace boost + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_BASIC_XML_TEXT_ARCHIVE_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_iarchive.hpp new file mode 100644 index 00000000000..e9f7482f744 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_iarchive.hpp @@ -0,0 +1,119 @@ +#ifndef BOOST_ARCHIVE_BASIC_XML_IARCHIVE_HPP +#define BOOST_ARCHIVE_BASIC_XML_IARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_xml_iarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include + +#include +#include +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_iarchive; +} // namespace detail + +///////////////////////////////////////////////////////////////////////// +// class basic_xml_iarchive - read serialized objects from a input text stream +template +class BOOST_SYMBOL_VISIBLE basic_xml_iarchive : + public detail::common_iarchive +{ + unsigned int depth; +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + friend class detail::interface_iarchive; +#endif + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + load_start(const char *name); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + load_end(const char *name); + + // Anything not an attribute and not a name-value pair is an + // should be trapped here. + template + void load_override(T & t) + { + // If your program fails to compile here, its most likely due to + // not specifying an nvp wrapper around the variable to + // be serialized. + BOOST_MPL_ASSERT((serialization::is_wrapper< T >)); + this->detail_common_iarchive::load_override(t); + } + + // Anything not an attribute - see below - should be a name value + // pair and be processed here + typedef detail::common_iarchive detail_common_iarchive; + template + void load_override( + const boost::serialization::nvp< T > & t + ){ + this->This()->load_start(t.name()); + this->detail_common_iarchive::load_override(t.value()); + this->This()->load_end(t.name()); + } + + // specific overrides for attributes - handle as + // primitives. These are not name-value pairs + // so they have to be intercepted here and passed on to load. + // although the class_id is included in the xml text file in order + // to make the file self describing, it isn't used when loading + // an xml archive. So we can skip it here. Note: we MUST override + // it otherwise it will be loaded as a normal primitive w/o tag and + // leaving the archive in an undetermined state + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + load_override(class_id_type & t); + void load_override(class_id_optional_type & /* t */){} + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + load_override(object_id_type & t); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + load_override(version_type & t); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + load_override(tracking_type & t); + // class_name_type can't be handled here as it depends upon the + // char type used by the stream. So require the derived implementation + // handle this. + // void load_override(class_name_type & t); + + BOOST_ARCHIVE_OR_WARCHIVE_DECL + basic_xml_iarchive(unsigned int flags); + BOOST_ARCHIVE_OR_WARCHIVE_DECL + ~basic_xml_iarchive(); +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_BASIC_XML_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_oarchive.hpp new file mode 100644 index 00000000000..107fca4ec65 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_oarchive.hpp @@ -0,0 +1,138 @@ +#ifndef BOOST_ARCHIVE_BASIC_XML_OARCHIVE_HPP +#define BOOST_ARCHIVE_BASIC_XML_OARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_xml_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include + +#include +#include +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_oarchive; +} // namespace detail + +////////////////////////////////////////////////////////////////////// +// class basic_xml_oarchive - write serialized objects to a xml output stream +template +class BOOST_SYMBOL_VISIBLE basic_xml_oarchive : + public detail::common_oarchive +{ + // special stuff for xml output + unsigned int depth; + bool pending_preamble; +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + friend class detail::interface_oarchive; +#endif + bool indent_next; + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + indent(); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + init(); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + windup(); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + write_attribute( + const char *attribute_name, + int t, + const char *conjunction = "=\"" + ); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + write_attribute( + const char *attribute_name, + const char *key + ); + // helpers used below + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save_start(const char *name); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save_end(const char *name); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + end_preamble(); + + // Anything not an attribute and not a name-value pair is an + // error and should be trapped here. + template + void save_override(T & t) + { + // If your program fails to compile here, its most likely due to + // not specifying an nvp wrapper around the variable to + // be serialized. + BOOST_MPL_ASSERT((serialization::is_wrapper< T >)); + this->detail_common_oarchive::save_override(t); + } + + // special treatment for name-value pairs. + typedef detail::common_oarchive detail_common_oarchive; + template + void save_override( + const ::boost::serialization::nvp< T > & t + ){ + this->This()->save_start(t.name()); + this->detail_common_oarchive::save_override(t.const_value()); + this->This()->save_end(t.name()); + } + + // specific overrides for attributes - not name value pairs so we + // want to trap them before the above "fall through" + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save_override(const class_id_type & t); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save_override(const class_id_optional_type & t); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save_override(const class_id_reference_type & t); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save_override(const object_id_type & t); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save_override(const object_reference_type & t); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save_override(const version_type & t); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save_override(const class_name_type & t); + BOOST_ARCHIVE_OR_WARCHIVE_DECL void + save_override(const tracking_type & t); + + BOOST_ARCHIVE_OR_WARCHIVE_DECL + basic_xml_oarchive(unsigned int flags); + BOOST_ARCHIVE_OR_WARCHIVE_DECL + ~basic_xml_oarchive(); +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_BASIC_XML_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive.hpp new file mode 100644 index 00000000000..785ce7610b1 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive.hpp @@ -0,0 +1,64 @@ +#ifndef BOOST_ARCHIVE_BINARY_IARCHIVE_HPP +#define BOOST_ARCHIVE_BINARY_IARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// binary_iarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +// do not derive from this class. If you want to extend this functionality +// via inhertance, derived from binary_iarchive_impl instead. This will +// preserve correct static polymorphism. +class BOOST_SYMBOL_VISIBLE binary_iarchive : + public binary_iarchive_impl< + boost::archive::binary_iarchive, + std::istream::char_type, + std::istream::traits_type + >{ +public: + binary_iarchive(std::istream & is, unsigned int flags = 0) : + binary_iarchive_impl< + binary_iarchive, std::istream::char_type, std::istream::traits_type + >(is, flags) + {} + binary_iarchive(std::streambuf & bsb, unsigned int flags = 0) : + binary_iarchive_impl< + binary_iarchive, std::istream::char_type, std::istream::traits_type + >(bsb, flags) + {} +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::binary_iarchive) +BOOST_SERIALIZATION_USE_ARRAY_OPTIMIZATION(boost::archive::binary_iarchive) + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_ARCHIVE_BINARY_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive_impl.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive_impl.hpp new file mode 100644 index 00000000000..b4747c98ece --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive_impl.hpp @@ -0,0 +1,105 @@ +#ifndef BOOST_ARCHIVE_BINARY_IARCHIVE_IMPL_HPP +#define BOOST_ARCHIVE_BINARY_IARCHIVE_IMPL_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// binary_iarchive_impl.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_iarchive; +} // namespace detail + +template +class BOOST_SYMBOL_VISIBLE binary_iarchive_impl : + public basic_binary_iprimitive, + public basic_binary_iarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) + // for some inexplicable reason insertion of "class" generates compile erro + // on msvc 7.1 + friend detail::interface_iarchive; + friend basic_binary_iarchive; + friend load_access; + #else + friend class detail::interface_iarchive; + friend class basic_binary_iarchive; + friend class load_access; + #endif +#endif + template + void load_override(T & t){ + this->basic_binary_iarchive::load_override(t); + } + void init(unsigned int flags){ + if(0 != (flags & no_header)){ + return; + } + #if ! defined(__MWERKS__) + this->basic_binary_iarchive::init(); + this->basic_binary_iprimitive::init(); + #else + basic_binary_iarchive::init(); + basic_binary_iprimitive::init(); + #endif + } + binary_iarchive_impl( + std::basic_streambuf & bsb, + unsigned int flags + ) : + basic_binary_iprimitive( + bsb, + 0 != (flags & no_codecvt) + ), + basic_binary_iarchive(flags) + { + init(flags); + } + binary_iarchive_impl( + std::basic_istream & is, + unsigned int flags + ) : + basic_binary_iprimitive( + * is.rdbuf(), + 0 != (flags & no_codecvt) + ), + basic_binary_iarchive(flags) + { + init(flags); + } +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_ARCHIVE_BINARY_IARCHIVE_IMPL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive.hpp new file mode 100644 index 00000000000..e8313fd7c95 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive.hpp @@ -0,0 +1,64 @@ +#ifndef BOOST_ARCHIVE_BINARY_OARCHIVE_HPP +#define BOOST_ARCHIVE_BINARY_OARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// binary_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include +#include + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +// do not derive from this class. If you want to extend this functionality +// via inhertance, derived from binary_oarchive_impl instead. This will +// preserve correct static polymorphism. +class BOOST_SYMBOL_VISIBLE binary_oarchive : + public binary_oarchive_impl< + binary_oarchive, std::ostream::char_type, std::ostream::traits_type + > +{ +public: + binary_oarchive(std::ostream & os, unsigned int flags = 0) : + binary_oarchive_impl< + binary_oarchive, std::ostream::char_type, std::ostream::traits_type + >(os, flags) + {} + binary_oarchive(std::streambuf & bsb, unsigned int flags = 0) : + binary_oarchive_impl< + binary_oarchive, std::ostream::char_type, std::ostream::traits_type + >(bsb, flags) + {} +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::binary_oarchive) +BOOST_SERIALIZATION_USE_ARRAY_OPTIMIZATION(boost::archive::binary_oarchive) + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_ARCHIVE_BINARY_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive_impl.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive_impl.hpp new file mode 100644 index 00000000000..6b4d018a564 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive_impl.hpp @@ -0,0 +1,106 @@ +#ifndef BOOST_ARCHIVE_BINARY_OARCHIVE_IMPL_HPP +#define BOOST_ARCHIVE_BINARY_OARCHIVE_IMPL_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// binary_oarchive_impl.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include +#include + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_oarchive; +} // namespace detail + +template +class BOOST_SYMBOL_VISIBLE binary_oarchive_impl : + public basic_binary_oprimitive, + public basic_binary_oarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) + // for some inexplicable reason insertion of "class" generates compile erro + // on msvc 7.1 + friend detail::interface_oarchive; + friend basic_binary_oarchive; + friend save_access; + #else + friend class detail::interface_oarchive; + friend class basic_binary_oarchive; + friend class save_access; + #endif +#endif + template + void save_override(T & t){ + this->basic_binary_oarchive::save_override(t); + } + void init(unsigned int flags) { + if(0 != (flags & no_header)){ + return; + } + #if ! defined(__MWERKS__) + this->basic_binary_oarchive::init(); + this->basic_binary_oprimitive::init(); + #else + basic_binary_oarchive::init(); + basic_binary_oprimitive::init(); + #endif + } + binary_oarchive_impl( + std::basic_streambuf & bsb, + unsigned int flags + ) : + basic_binary_oprimitive( + bsb, + 0 != (flags & no_codecvt) + ), + basic_binary_oarchive(flags) + { + init(flags); + } + binary_oarchive_impl( + std::basic_ostream & os, + unsigned int flags + ) : + basic_binary_oprimitive( + * os.rdbuf(), + 0 != (flags & no_codecvt) + ), + basic_binary_oarchive(flags) + { + init(flags); + } +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_ARCHIVE_BINARY_OARCHIVE_IMPL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_wiarchive.hpp new file mode 100644 index 00000000000..775d8f82726 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/binary_wiarchive.hpp @@ -0,0 +1,56 @@ +#ifndef BOOST_ARCHIVE_BINARY_WIARCHIVE_HPP +#define BOOST_ARCHIVE_BINARY_WIARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// binary_wiarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifdef BOOST_NO_STD_WSTREAMBUF +#error "wide char i/o not supported on this platform" +#else + +#include // wistream +#include +#include + +namespace boost { +namespace archive { + +class binary_wiarchive : + public binary_iarchive_impl< + binary_wiarchive, std::wistream::char_type, std::wistream::traits_type + > +{ +public: + binary_wiarchive(std::wistream & is, unsigned int flags = 0) : + binary_iarchive_impl< + binary_wiarchive, std::wistream::char_type, std::wistream::traits_type + >(is, flags) + {} + binary_wiarchive(std::wstreambuf & bsb, unsigned int flags = 0) : + binary_iarchive_impl< + binary_wiarchive, std::wistream::char_type, std::wistream::traits_type + >(bsb, flags) + {} +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::binary_wiarchive) + +#endif // BOOST_NO_STD_WSTREAMBUF +#endif // BOOST_ARCHIVE_BINARY_WIARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_woarchive.hpp new file mode 100644 index 00000000000..a8817d6f8b4 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/binary_woarchive.hpp @@ -0,0 +1,59 @@ +#ifndef BOOST_ARCHIVE_BINARY_WOARCHIVE_HPP +#define BOOST_ARCHIVE_BINARY_WOARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// binary_woarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifdef BOOST_NO_STD_WSTREAMBUF +#error "wide char i/o not supported on this platform" +#else + +#include +#include +#include + +namespace boost { +namespace archive { + +// do not derive from this class. If you want to extend this functionality +// via inhertance, derived from binary_oarchive_impl instead. This will +// preserve correct static polymorphism. +class binary_woarchive : + public binary_oarchive_impl< + binary_woarchive, std::wostream::char_type, std::wostream::traits_type + > +{ +public: + binary_woarchive(std::wostream & os, unsigned int flags = 0) : + binary_oarchive_impl< + binary_woarchive, std::wostream::char_type, std::wostream::traits_type + >(os, flags) + {} + binary_woarchive(std::wstreambuf & bsb, unsigned int flags = 0) : + binary_oarchive_impl< + binary_woarchive, std::wostream::char_type, std::wostream::traits_type + >(bsb, flags) + {} +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::binary_woarchive) + +#endif // BOOST_NO_STD_WSTREAMBUF +#endif // BOOST_ARCHIVE_BINARY_WOARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/codecvt_null.hpp b/contrib/libboost/boost_1_65_0/boost/archive/codecvt_null.hpp new file mode 100644 index 00000000000..7bce2b9b329 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/codecvt_null.hpp @@ -0,0 +1,109 @@ +#ifndef BOOST_ARCHIVE_CODECVT_NULL_HPP +#define BOOST_ARCHIVE_CODECVT_NULL_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// codecvt_null.hpp: + +// (C) Copyright 2004 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include // NULL, size_t +#ifndef BOOST_NO_CWCHAR +#include // for mbstate_t +#endif +#include +#include +#include +#include // must be the last header + +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std { +// For STLport on WinCE, BOOST_NO_STDC_NAMESPACE can get defined if STLport is putting symbols in its own namespace. +// In the case of codecvt, however, this does not mean that codecvt is in the global namespace (it will be in STLport's namespace) +# if !defined(__SGI_STL_PORT) && !defined(_STLPORT_VERSION) + using ::codecvt; +# endif + using ::mbstate_t; + using ::size_t; +} // namespace +#endif + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +template +class codecvt_null; + +template<> +class codecvt_null : public std::codecvt +{ + virtual bool do_always_noconv() const throw() { + return true; + } +public: + explicit codecvt_null(std::size_t no_locale_manage = 0) : + std::codecvt(no_locale_manage) + {} + virtual ~codecvt_null(){}; +}; + +template<> +class BOOST_SYMBOL_VISIBLE codecvt_null : public std::codecvt +{ + virtual BOOST_WARCHIVE_DECL BOOST_DLLEXPORT std::codecvt_base::result + do_out( + std::mbstate_t & state, + const wchar_t * first1, + const wchar_t * last1, + const wchar_t * & next1, + char * first2, + char * last2, + char * & next2 + ) const BOOST_USED; + virtual BOOST_WARCHIVE_DECL BOOST_DLLEXPORT std::codecvt_base::result + do_in( + std::mbstate_t & state, + const char * first1, + const char * last1, + const char * & next1, + wchar_t * first2, + wchar_t * last2, + wchar_t * & next2 + ) const BOOST_USED; + virtual int do_encoding( ) const throw( ){ + return sizeof(wchar_t) / sizeof(char); + } + virtual int do_max_length( ) const throw( ){ + return do_encoding(); + } +public: + BOOST_DLLEXPORT explicit codecvt_null(std::size_t no_locale_manage = 0) : + std::codecvt(no_locale_manage) + {} + virtual ~codecvt_null(){}; +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +# pragma warning(pop) +#endif +#include // pop pragmas + +#endif //BOOST_ARCHIVE_CODECVT_NULL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_prefix.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_prefix.hpp new file mode 100644 index 00000000000..debf79e9f0b --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_prefix.hpp @@ -0,0 +1,16 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// abi_prefix.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // must be the last header +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4251 4231 4660 4275) +#endif + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_suffix.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_suffix.hpp new file mode 100644 index 00000000000..4e054d66214 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_suffix.hpp @@ -0,0 +1,15 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// abi_suffix.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif +#include // pops abi_suffix.hpp pragmas + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/archive_serializer_map.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/archive_serializer_map.hpp new file mode 100644 index 00000000000..5432bfc73e7 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/archive_serializer_map.hpp @@ -0,0 +1,54 @@ +#ifndef BOOST_ARCHIVE_SERIALIZER_MAP_HPP +#define BOOST_ARCHIVE_SERIALIZER_MAP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// archive_serializer_map.hpp: extenstion of type_info required for +// serialization. + +// (C) Copyright 2009 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// note: this is nothing more than the thinest of wrappers around +// basic_serializer_map so we can have a one map / archive type. + +#include +#include +#include // must be the last header + +namespace boost { + +namespace serialization { + class extended_type_info; +} // namespace serialization + +namespace archive { +namespace detail { + +class basic_serializer; + +template +class BOOST_SYMBOL_VISIBLE archive_serializer_map { +public: + static BOOST_ARCHIVE_OR_WARCHIVE_DECL bool insert(const basic_serializer * bs); + static BOOST_ARCHIVE_OR_WARCHIVE_DECL void erase(const basic_serializer * bs); + static BOOST_ARCHIVE_OR_WARCHIVE_DECL const basic_serializer * find( + const boost::serialization::extended_type_info & type_ + ); +}; + +} // namespace detail +} // namespace archive +} // namespace boost + +#include // must be the last header + +#endif //BOOST_ARCHIVE_SERIALIZER_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_archive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_archive.hpp new file mode 100644 index 00000000000..79b0e490d65 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_archive.hpp @@ -0,0 +1,48 @@ +#ifndef BOOST_ARCHIVE_DETAIL_AUTO_LINK_ARCHIVE_HPP +#define BOOST_ARCHIVE_DETAIL_AUTO_LINK_ARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// auto_link_archive.hpp +// +// (c) Copyright Robert Ramey 2004 +// Use, modification, and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See library home page at http://www.boost.org/libs/serialization + +//----------------------------------------------------------------------------// + +// This header implements separate compilation features as described in +// http://www.boost.org/more/separate_compilation.html + +// enable automatic library variant selection ------------------------------// + +#include + +#if !defined(BOOST_ALL_NO_LIB) && !defined(BOOST_SERIALIZATION_NO_LIB) \ +&& !defined(BOOST_ARCHIVE_SOURCE) && !defined(BOOST_WARCHIVE_SOURCE) \ +&& !defined(BOOST_SERIALIZATION_SOURCE) + + // Set the name of our library, this will get undef'ed by auto_link.hpp + // once it's done with it: + // + #define BOOST_LIB_NAME boost_serialization + // + // If we're importing code from a dll, then tell auto_link.hpp about it: + // + #if defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) + # define BOOST_DYN_LINK + #endif + // + // And include the header that does the work: + // + #include +#endif // auto-linking disabled + +#endif // BOOST_ARCHIVE_DETAIL_AUTO_LINK_ARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_warchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_warchive.hpp new file mode 100644 index 00000000000..683d191c20d --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_warchive.hpp @@ -0,0 +1,47 @@ +#ifndef BOOST_ARCHIVE_DETAIL_AUTO_LINK_WARCHIVE_HPP +#define BOOST_ARCHIVE_DETAIL_AUTO_LINK_WARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// auto_link_warchive.hpp +// +// (c) Copyright Robert Ramey 2004 +// Use, modification, and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See library home page at http://www.boost.org/libs/serialization + +//----------------------------------------------------------------------------// + +// This header implements separate compilation features as described in +// http://www.boost.org/more/separate_compilation.html + +// enable automatic library variant selection ------------------------------// + +#include + +#if !defined(BOOST_WARCHIVE_SOURCE) \ +&& !defined(BOOST_ALL_NO_LIB) && !defined(BOOST_SERIALIZATION_NO_LIB) + +// Set the name of our library, this will get undef'ed by auto_link.hpp +// once it's done with it: +// +#define BOOST_LIB_NAME boost_wserialization +// +// If we're importing code from a dll, then tell auto_link.hpp about it: +// +#if defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) +# define BOOST_DYN_LINK +#endif +// +// And include the header that does the work: +// +#include +#endif // auto-linking disabled + +#endif // ARCHIVE_DETAIL_AUTO_LINK_ARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iarchive.hpp new file mode 100644 index 00000000000..1f5a8bf63bf --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iarchive.hpp @@ -0,0 +1,105 @@ +#ifndef BOOST_ARCHIVE_DETAIL_BASIC_IARCHIVE_HPP +#define BOOST_ARCHIVE_DETAIL_BASIC_IARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_iarchive.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// can't use this - much as I'd like to as borland doesn't support it + +#include +#include +#include + +#include +#include +#include +#include +#include // must be the last header + +namespace boost { +namespace serialization { + class extended_type_info; +} // namespace serialization + +namespace archive { +namespace detail { + +class basic_iarchive_impl; +class basic_iserializer; +class basic_pointer_iserializer; + +////////////////////////////////////////////////////////////////////// +// class basic_iarchive - read serialized objects from a input stream +class BOOST_SYMBOL_VISIBLE basic_iarchive : + private boost::noncopyable, + public boost::archive::detail::helper_collection +{ + friend class basic_iarchive_impl; + // hide implementation of this class to minimize header conclusion + boost::scoped_ptr pimpl; + + virtual void vload(version_type &t) = 0; + virtual void vload(object_id_type &t) = 0; + virtual void vload(class_id_type &t) = 0; + virtual void vload(class_id_optional_type &t) = 0; + virtual void vload(class_name_type &t) = 0; + virtual void vload(tracking_type &t) = 0; +protected: + BOOST_ARCHIVE_DECL basic_iarchive(unsigned int flags); + boost::archive::detail::helper_collection & + get_helper_collection(){ + return *this; + } +public: + // some msvc versions require that the following function be public + // otherwise it should really protected. + virtual BOOST_ARCHIVE_DECL ~basic_iarchive(); + // note: NOT part of the public API. + BOOST_ARCHIVE_DECL void next_object_pointer(void *t); + BOOST_ARCHIVE_DECL void register_basic_serializer( + const basic_iserializer & bis + ); + BOOST_ARCHIVE_DECL void load_object( + void *t, + const basic_iserializer & bis + ); + BOOST_ARCHIVE_DECL const basic_pointer_iserializer * + load_pointer( + void * & t, + const basic_pointer_iserializer * bpis_ptr, + const basic_pointer_iserializer * (*finder)( + const boost::serialization::extended_type_info & eti + ) + ); + // real public API starts here + BOOST_ARCHIVE_DECL void + set_library_version(library_version_type archive_library_version); + BOOST_ARCHIVE_DECL library_version_type + get_library_version() const; + BOOST_ARCHIVE_DECL unsigned int + get_flags() const; + BOOST_ARCHIVE_DECL void + reset_object_address(const void * new_address, const void * old_address); + BOOST_ARCHIVE_DECL void + delete_created_pointers(); +}; + +} // namespace detail +} // namespace archive +} // namespace boost + +#include // pops abi_suffix.hpp pragmas + +#endif //BOOST_ARCHIVE_DETAIL_BASIC_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iserializer.hpp new file mode 100644 index 00000000000..0d66674c349 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iserializer.hpp @@ -0,0 +1,91 @@ +#ifndef BOOST_ARCHIVE_DETAIL_BASIC_ISERIALIZER_HPP +#define BOOST_ARCHIVE_DETAIL_BASIC_ISERIALIZER_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_iserializer.hpp: extenstion of type_info required for serialization. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // NULL +#include + +#include +#include +#include +#include +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace serialization { + class extended_type_info; +} // namespace serialization + +// forward declarations +namespace archive { +namespace detail { + +class basic_iarchive; +class basic_pointer_iserializer; + +class BOOST_SYMBOL_VISIBLE basic_iserializer : + public basic_serializer +{ +private: + basic_pointer_iserializer *m_bpis; +protected: + explicit BOOST_ARCHIVE_DECL basic_iserializer( + const boost::serialization::extended_type_info & type + ); + virtual BOOST_ARCHIVE_DECL ~basic_iserializer(); +public: + bool serialized_as_pointer() const { + return m_bpis != NULL; + } + void set_bpis(basic_pointer_iserializer *bpis){ + m_bpis = bpis; + } + const basic_pointer_iserializer * get_bpis_ptr() const { + return m_bpis; + } + virtual void load_object_data( + basic_iarchive & ar, + void *x, + const unsigned int file_version + ) const = 0; + // returns true if class_info should be saved + virtual bool class_info() const = 0 ; + // returns true if objects should be tracked + virtual bool tracking(const unsigned int) const = 0 ; + // returns class version + virtual version_type version() const = 0 ; + // returns true if this class is polymorphic + virtual bool is_polymorphic() const = 0; + virtual void destroy(/*const*/ void *address) const = 0 ; +}; + +} // namespae detail +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_DETAIL_BASIC_ISERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oarchive.hpp new file mode 100644 index 00000000000..c379108d584 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oarchive.hpp @@ -0,0 +1,94 @@ +#ifndef BOOST_ARCHIVE_BASIC_OARCHIVE_HPP +#define BOOST_ARCHIVE_BASIC_OARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_oarchive.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // NULL +#include +#include +#include + +#include +#include +#include +#include // must be the last header + +namespace boost { +namespace serialization { + class extended_type_info; +} // namespace serialization + +namespace archive { +namespace detail { + +class basic_oarchive_impl; +class basic_oserializer; +class basic_pointer_oserializer; + +////////////////////////////////////////////////////////////////////// +// class basic_oarchive - write serialized objects to an output stream +class BOOST_SYMBOL_VISIBLE basic_oarchive : + private boost::noncopyable, + public boost::archive::detail::helper_collection +{ + friend class basic_oarchive_impl; + // hide implementation of this class to minimize header conclusion + boost::scoped_ptr pimpl; + + // overload these to bracket object attributes. Used to implement + // xml archives + virtual void vsave(const version_type t) = 0; + virtual void vsave(const object_id_type t) = 0; + virtual void vsave(const object_reference_type t) = 0; + virtual void vsave(const class_id_type t) = 0; + virtual void vsave(const class_id_optional_type t) = 0; + virtual void vsave(const class_id_reference_type t) = 0; + virtual void vsave(const class_name_type & t) = 0; + virtual void vsave(const tracking_type t) = 0; +protected: + BOOST_ARCHIVE_DECL basic_oarchive(unsigned int flags = 0); + BOOST_ARCHIVE_DECL boost::archive::detail::helper_collection & + get_helper_collection(); + virtual BOOST_ARCHIVE_DECL ~basic_oarchive(); +public: + // note: NOT part of the public interface + BOOST_ARCHIVE_DECL void register_basic_serializer( + const basic_oserializer & bos + ); + BOOST_ARCHIVE_DECL void save_object( + const void *x, + const basic_oserializer & bos + ); + BOOST_ARCHIVE_DECL void save_pointer( + const void * t, + const basic_pointer_oserializer * bpos_ptr + ); + void save_null_pointer(){ + vsave(NULL_POINTER_TAG); + } + // real public interface starts here + BOOST_ARCHIVE_DECL void end_preamble(); // default implementation does nothing + BOOST_ARCHIVE_DECL library_version_type get_library_version() const; + BOOST_ARCHIVE_DECL unsigned int get_flags() const; +}; + +} // namespace detail +} // namespace archive +} // namespace boost + +#include // pops abi_suffix.hpp pragmas + +#endif //BOOST_ARCHIVE_BASIC_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oserializer.hpp new file mode 100644 index 00000000000..94247e90056 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oserializer.hpp @@ -0,0 +1,89 @@ +#ifndef BOOST_SERIALIZATION_BASIC_OSERIALIZER_HPP +#define BOOST_SERIALIZATION_BASIC_OSERIALIZER_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_oserializer.hpp: extenstion of type_info required for serialization. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // NULL +#include +#include + +#include +#include +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace serialization { + class extended_type_info; +} // namespace serialization + +// forward declarations +namespace archive { +namespace detail { + +class basic_oarchive; +class basic_pointer_oserializer; + +class BOOST_SYMBOL_VISIBLE basic_oserializer : + public basic_serializer +{ +private: + basic_pointer_oserializer *m_bpos; +protected: + explicit BOOST_ARCHIVE_DECL basic_oserializer( + const boost::serialization::extended_type_info & type_ + ); + virtual BOOST_ARCHIVE_DECL ~basic_oserializer(); +public: + bool serialized_as_pointer() const { + return m_bpos != NULL; + } + void set_bpos(basic_pointer_oserializer *bpos){ + m_bpos = bpos; + } + const basic_pointer_oserializer * get_bpos() const { + return m_bpos; + } + virtual void save_object_data( + basic_oarchive & ar, const void * x + ) const = 0; + // returns true if class_info should be saved + virtual bool class_info() const = 0; + // returns true if objects should be tracked + virtual bool tracking(const unsigned int flags) const = 0; + // returns class version + virtual version_type version() const = 0; + // returns true if this class is polymorphic + virtual bool is_polymorphic() const = 0; +}; + +} // namespace detail +} // namespace serialization +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_SERIALIZATION_BASIC_OSERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_iserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_iserializer.hpp new file mode 100644 index 00000000000..1fc4b14d6e9 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_iserializer.hpp @@ -0,0 +1,70 @@ +#ifndef BOOST_ARCHIVE_BASIC_POINTER_ISERIALIZER_HPP +#define BOOST_ARCHIVE_BASIC_POINTER_ISERIALIZER_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_pointer_oserializer.hpp: extenstion of type_info required for +// serialization. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. +#include +#include +#include +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace serialization { + class extended_type_info; +} // namespace serialization + +// forward declarations +namespace archive { +namespace detail { + +class basic_iarchive; +class basic_iserializer; + +class BOOST_SYMBOL_VISIBLE basic_pointer_iserializer + : public basic_serializer { +protected: + explicit BOOST_ARCHIVE_DECL basic_pointer_iserializer( + const boost::serialization::extended_type_info & type_ + ); + virtual BOOST_ARCHIVE_DECL ~basic_pointer_iserializer(); +public: + virtual void * heap_allocation() const = 0; + virtual const basic_iserializer & get_basic_serializer() const = 0; + virtual void load_object_ptr( + basic_iarchive & ar, + void * x, + const unsigned int file_version + ) const = 0; +}; + +} // namespace detail +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_BASIC_POINTER_ISERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_oserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_oserializer.hpp new file mode 100644 index 00000000000..1a5d9549eab --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_oserializer.hpp @@ -0,0 +1,68 @@ +#ifndef BOOST_ARCHIVE_BASIC_POINTER_OSERIALIZER_HPP +#define BOOST_ARCHIVE_BASIC_POINTER_OSERIALIZER_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_pointer_oserializer.hpp: extenstion of type_info required for +// serialization. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. +#include +#include +#include +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace serialization { + class extended_type_info; +} // namespace serialization + +namespace archive { +namespace detail { + +class basic_oarchive; +class basic_oserializer; + +class BOOST_SYMBOL_VISIBLE basic_pointer_oserializer : + public basic_serializer +{ +protected: + explicit BOOST_ARCHIVE_DECL basic_pointer_oserializer( + const boost::serialization::extended_type_info & type_ + ); +public: + virtual BOOST_ARCHIVE_DECL ~basic_pointer_oserializer(); + virtual const basic_oserializer & get_basic_serializer() const = 0; + virtual void save_object_ptr( + basic_oarchive & ar, + const void * x + ) const = 0; +}; + +} // namespace detail +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_BASIC_POINTER_OSERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer.hpp new file mode 100644 index 00000000000..f9c4203f862 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer.hpp @@ -0,0 +1,77 @@ +#ifndef BOOST_ARCHIVE_BASIC_SERIALIZER_HPP +#define BOOST_ARCHIVE_BASIC_SERIALIZER_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_serializer.hpp: extenstion of type_info required for serialization. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include // NULL + +#include +#include +#include + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { +namespace detail { + +class basic_serializer : + private boost::noncopyable +{ + const boost::serialization::extended_type_info * m_eti; +protected: + explicit basic_serializer( + const boost::serialization::extended_type_info & eti + ) : + m_eti(& eti) + {} +public: + inline bool + operator<(const basic_serializer & rhs) const { + // can't compare address since there can be multiple eti records + // for the same type in different execution modules (that is, DLLS) + // leave this here as a reminder not to do this! + // return & lhs.get_eti() < & rhs.get_eti(); + return get_eti() < rhs.get_eti(); + } + const char * get_debug_info() const { + return m_eti->get_debug_info(); + } + const boost::serialization::extended_type_info & get_eti() const { + return * m_eti; + } +}; + +class basic_serializer_arg : public basic_serializer { +public: + basic_serializer_arg(const serialization::extended_type_info & eti) : + basic_serializer(eti) + {} +}; + +} // namespace detail +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_ARCHIVE_BASIC_SERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer_map.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer_map.hpp new file mode 100644 index 00000000000..79341803367 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer_map.hpp @@ -0,0 +1,69 @@ +#ifndef BOOST_SERIALIZER_MAP_HPP +#define BOOST_SERIALIZER_MAP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_serializer_map.hpp: extenstion of type_info required for serialization. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include +#include + +#include // must be the last header + +namespace boost { +namespace serialization { + class extended_type_info; +} + +namespace archive { +namespace detail { + +class basic_serializer; + +class BOOST_SYMBOL_VISIBLE +basic_serializer_map : public + boost::noncopyable +{ + struct type_info_pointer_compare + { + bool operator()( + const basic_serializer * lhs, const basic_serializer * rhs + ) const ; + }; + typedef std::set< + const basic_serializer *, + type_info_pointer_compare + > map_type; + map_type m_map; +public: + BOOST_ARCHIVE_DECL bool insert(const basic_serializer * bs); + BOOST_ARCHIVE_DECL void erase(const basic_serializer * bs); + BOOST_ARCHIVE_DECL const basic_serializer * find( + const boost::serialization::extended_type_info & type_ + ) const; +private: + // cw 8.3 requires this + basic_serializer_map& operator=(basic_serializer_map const&); +}; + +} // namespace detail +} // namespace archive +} // namespace boost + +#include // must be the last header + +#endif // BOOST_SERIALIZER_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/check.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/check.hpp new file mode 100644 index 00000000000..10034e7d101 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/check.hpp @@ -0,0 +1,169 @@ +#ifndef BOOST_ARCHIVE_DETAIL_CHECK_HPP +#define BOOST_ARCHIVE_DETAIL_CHECK_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#pragma inline_depth(511) +#pragma inline_recursion(on) +#endif + +#if defined(__MWERKS__) +#pragma inline_depth(511) +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// check.hpp: interface for serialization system. + +// (C) Copyright 2009 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace boost { +namespace archive { +namespace detail { + +// checks for objects + +template +inline void check_object_level(){ + typedef + typename mpl::greater_equal< + serialization::implementation_level< T >, + mpl::int_ + >::type typex; + + // trap attempts to serialize objects marked + // not_serializable + BOOST_STATIC_ASSERT(typex::value); +} + +template +inline void check_object_versioning(){ + typedef + typename mpl::or_< + typename mpl::greater< + serialization::implementation_level< T >, + mpl::int_ + >, + typename mpl::equal_to< + serialization::version< T >, + mpl::int_<0> + > + > typex; + // trap attempts to serialize with objects that don't + // save class information in the archive with versioning. + BOOST_STATIC_ASSERT(typex::value); +} + +template +inline void check_object_tracking(){ + // presume it has already been determined that + // T is not a const + BOOST_STATIC_ASSERT(! boost::is_const< T >::value); + typedef typename mpl::equal_to< + serialization::tracking_level< T >, + mpl::int_ + >::type typex; + // saving an non-const object of a type not marked "track_never) + + // may be an indicator of an error usage of the + // serialization library and should be double checked. + // See documentation on object tracking. Also, see the + // "rationale" section of the documenation + // for motivation for this checking. + + BOOST_STATIC_WARNING(typex::value); +} + +// checks for pointers + +template +inline void check_pointer_level(){ + // we should only invoke this once we KNOW that T + // has been used as a pointer!! + typedef + typename mpl::or_< + typename mpl::greater< + serialization::implementation_level< T >, + mpl::int_ + >, + typename mpl::not_< + typename mpl::equal_to< + serialization::tracking_level< T >, + mpl::int_ + > + > + > typex; + // Address the following when serializing to a pointer: + + // a) This type doesn't save class information in the + // archive. That is, the serialization trait implementation + // level <= object_serializable. + // b) Tracking for this type is set to "track selectively" + + // in this case, indication that an object is tracked is + // not stored in the archive itself - see level == object_serializable + // but rather the existence of the operation ar >> T * is used to + // infer that an object of this type should be tracked. So, if + // you save via a pointer but don't load via a pointer the operation + // will fail on load without given any valid reason for the failure. + + // So if your program traps here, consider changing the + // tracking or implementation level traits - or not + // serializing via a pointer. + BOOST_STATIC_WARNING(typex::value); +} + +template +void inline check_pointer_tracking(){ + typedef typename mpl::greater< + serialization::tracking_level< T >, + mpl::int_ + >::type typex; + // serializing an object of a type marked "track_never" through a pointer + // could result in creating more objects than were saved! + BOOST_STATIC_WARNING(typex::value); +} + +template +inline void check_const_loading(){ + typedef + typename mpl::or_< + typename boost::serialization::is_wrapper< T >, + typename mpl::not_< + typename boost::is_const< T > + > + >::type typex; + // cannot load data into a "const" object unless it's a + // wrapper around some other non-const object. + BOOST_STATIC_ASSERT(typex::value); +} + +} // detail +} // archive +} // boost + +#endif // BOOST_ARCHIVE_DETAIL_CHECK_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/common_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/common_iarchive.hpp new file mode 100644 index 00000000000..82304f1e5ac --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/common_iarchive.hpp @@ -0,0 +1,88 @@ +#ifndef BOOST_ARCHIVE_DETAIL_COMMON_IARCHIVE_HPP +#define BOOST_ARCHIVE_DETAIL_COMMON_IARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// common_iarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include +#include + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { +namespace detail { + +class extended_type_info; + +// note: referred to as Curiously Recurring Template Patter (CRTP) +template +class BOOST_SYMBOL_VISIBLE common_iarchive : + public basic_iarchive, + public interface_iarchive +{ + friend class interface_iarchive; +private: + virtual void vload(version_type & t){ + * this->This() >> t; + } + virtual void vload(object_id_type & t){ + * this->This() >> t; + } + virtual void vload(class_id_type & t){ + * this->This() >> t; + } + virtual void vload(class_id_optional_type & t){ + * this->This() >> t; + } + virtual void vload(tracking_type & t){ + * this->This() >> t; + } + virtual void vload(class_name_type &s){ + * this->This() >> s; + } +protected: + // default processing - invoke serialization library + template + void load_override(T & t){ + archive::load(* this->This(), t); + } + // default implementations of functions which emit start/end tags for + // archive types that require them. + void load_start(const char * /*name*/){} + void load_end(const char * /*name*/){} + // default archive initialization + common_iarchive(unsigned int flags = 0) : + basic_iarchive(flags), + interface_iarchive() + {} +}; + +} // namespace detail +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_ARCHIVE_DETAIL_COMMON_IARCHIVE_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/common_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/common_oarchive.hpp new file mode 100644 index 00000000000..ee42bbe5976 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/common_oarchive.hpp @@ -0,0 +1,88 @@ +#ifndef BOOST_ARCHIVE_DETAIL_COMMON_OARCHIVE_HPP +#define BOOST_ARCHIVE_DETAIL_COMMON_OARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// common_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { +namespace detail { + +// note: referred to as Curiously Recurring Template Patter (CRTP) +template + +class BOOST_SYMBOL_VISIBLE common_oarchive : + public basic_oarchive, + public interface_oarchive +{ + friend class interface_oarchive; +private: + virtual void vsave(const version_type t){ + * this->This() << t; + } + virtual void vsave(const object_id_type t){ + * this->This() << t; + } + virtual void vsave(const object_reference_type t){ + * this->This() << t; + } + virtual void vsave(const class_id_type t){ + * this->This() << t; + } + virtual void vsave(const class_id_reference_type t){ + * this->This() << t; + } + virtual void vsave(const class_id_optional_type t){ + * this->This() << t; + } + virtual void vsave(const class_name_type & t){ + * this->This() << t; + } + virtual void vsave(const tracking_type t){ + * this->This() << t; + } +protected: + // default processing - invoke serialization library + template + void save_override(T & t){ + archive::save(* this->This(), t); + } + void save_start(const char * /*name*/){} + void save_end(const char * /*name*/){} + common_oarchive(unsigned int flags = 0) : + basic_oarchive(flags), + interface_oarchive() + {} +}; + +} // namespace detail +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_ARCHIVE_DETAIL_COMMON_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/decl.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/decl.hpp new file mode 100644 index 00000000000..4f731cded37 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/decl.hpp @@ -0,0 +1,57 @@ +#ifndef BOOST_ARCHIVE_DETAIL_DECL_HPP +#define BOOST_ARCHIVE_DETAIL_DECL_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2///////// 3/////////4/////////5/////////6/////////7/////////8 +// decl.hpp +// +// (c) Copyright Robert Ramey 2004 +// Use, modification, and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See library home page at http://www.boost.org/libs/serialization + +//----------------------------------------------------------------------------// + +// This header implements separate compilation features as described in +// http://www.boost.org/more/separate_compilation.html + +#include + +#if (defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK)) + #if defined(BOOST_ARCHIVE_SOURCE) + #define BOOST_ARCHIVE_DECL BOOST_SYMBOL_EXPORT + #else + #define BOOST_ARCHIVE_DECL BOOST_SYMBOL_IMPORT + #endif + + #if defined(BOOST_WARCHIVE_SOURCE) + #define BOOST_WARCHIVE_DECL BOOST_SYMBOL_EXPORT + #else + #define BOOST_WARCHIVE_DECL BOOST_SYMBOL_IMPORT + #endif + + #if defined(BOOST_WARCHIVE_SOURCE) || defined(BOOST_ARCHIVE_SOURCE) + #define BOOST_ARCHIVE_OR_WARCHIVE_DECL BOOST_SYMBOL_EXPORT + #else + #define BOOST_ARCHIVE_OR_WARCHIVE_DECL BOOST_SYMBOL_IMPORT + #endif + +#endif + +#if ! defined(BOOST_ARCHIVE_DECL) + #define BOOST_ARCHIVE_DECL +#endif +#if ! defined(BOOST_WARCHIVE_DECL) + #define BOOST_WARCHIVE_DECL +#endif +#if ! defined(BOOST_ARCHIVE_OR_WARCHIVE_DECL) + #define BOOST_ARCHIVE_OR_WARCHIVE_DECL +#endif + +#endif // BOOST_ARCHIVE_DETAIL_DECL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/helper_collection.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/helper_collection.hpp new file mode 100644 index 00000000000..edb4125e308 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/helper_collection.hpp @@ -0,0 +1,99 @@ +#ifndef BOOST_ARCHIVE_DETAIL_HELPER_COLLECTION_HPP +#define BOOST_ARCHIVE_DETAIL_HELPER_COLLECTION_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) && (_MSC_VER >= 1020) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// helper_collection.hpp: archive support for run-time helpers + +// (C) Copyright 2002-2008 Robert Ramey and Joaquin M Lopez Munoz +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // NULL +#include +#include +#include +#include + +#include + +#include +#include + +namespace boost { + +namespace archive { +namespace detail { + +class helper_collection +{ + helper_collection(const helper_collection&); // non-copyable + helper_collection& operator = (const helper_collection&); // non-copyable + + // note: we dont' actually "share" the function object pointer + // we only use shared_ptr to make sure that it get's deleted + + typedef std::pair< + const void *, + boost::shared_ptr + > helper_value_type; + template + boost::shared_ptr make_helper_ptr(){ + // use boost::shared_ptr rather than std::shared_ptr to maintain + // c++03 compatibility + return boost::make_shared(); + } + + typedef std::vector collection; + collection m_collection; + + struct predicate { + BOOST_DELETED_FUNCTION(predicate & operator=(const predicate & rhs)) + public: + const void * const m_ti; + bool operator()(helper_value_type const &rhs) const { + return m_ti == rhs.first; + } + predicate(const void * ti) : + m_ti(ti) + {} + }; +protected: + helper_collection(){} + ~helper_collection(){} +public: + template + Helper& find_helper(void * const id = 0) { + collection::const_iterator it = + std::find_if( + m_collection.begin(), + m_collection.end(), + predicate(id) + ); + + void * rval = 0; + if(it == m_collection.end()){ + m_collection.push_back( + std::make_pair(id, make_helper_ptr()) + ); + rval = m_collection.back().second.get(); + } + else{ + rval = it->second.get(); + } + return *static_cast(rval); + } +}; + +} // namespace detail +} // namespace serialization +} // namespace boost + +#endif // BOOST_ARCHIVE_DETAIL_HELPER_COLLECTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_iarchive.hpp new file mode 100644 index 00000000000..4a99e28b59f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_iarchive.hpp @@ -0,0 +1,85 @@ +#ifndef BOOST_ARCHIVE_DETAIL_INTERFACE_IARCHIVE_HPP +#define BOOST_ARCHIVE_DETAIL_INTERFACE_IARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// interface_iarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. +#include // NULL +#include +#include +#include +#include +#include +#include +#include // must be the last header + +namespace boost { +namespace archive { +namespace detail { + +class basic_pointer_iserializer; + +template +class interface_iarchive +{ +protected: + interface_iarchive(){}; +public: + ///////////////////////////////////////////////////////// + // archive public interface + typedef mpl::bool_ is_loading; + typedef mpl::bool_ is_saving; + + // return a pointer to the most derived class + Archive * This(){ + return static_cast(this); + } + + template + const basic_pointer_iserializer * + register_type(T * = NULL){ + const basic_pointer_iserializer & bpis = + boost::serialization::singleton< + pointer_iserializer + >::get_const_instance(); + this->This()->register_basic_serializer(bpis.get_basic_serializer()); + return & bpis; + } + template + Helper & + get_helper(void * const id = 0){ + helper_collection & hc = this->This()->get_helper_collection(); + return hc.template find_helper(id); + } + + template + Archive & operator>>(T & t){ + this->This()->load_override(t); + return * this->This(); + } + + // the & operator + template + Archive & operator&(T & t){ + return *(this->This()) >> t; + } +}; + +} // namespace detail +} // namespace archive +} // namespace boost + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_DETAIL_INTERFACE_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_oarchive.hpp new file mode 100644 index 00000000000..359463ed9d8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_oarchive.hpp @@ -0,0 +1,87 @@ +#ifndef BOOST_ARCHIVE_DETAIL_INTERFACE_OARCHIVE_HPP +#define BOOST_ARCHIVE_DETAIL_INTERFACE_OARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// interface_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. +#include // NULL +#include +#include + +#include +#include +#include // must be the last header + +#include + +namespace boost { +namespace archive { +namespace detail { + +class basic_pointer_oserializer; + +template +class interface_oarchive +{ +protected: + interface_oarchive(){}; +public: + ///////////////////////////////////////////////////////// + // archive public interface + typedef mpl::bool_ is_loading; + typedef mpl::bool_ is_saving; + + // return a pointer to the most derived class + Archive * This(){ + return static_cast(this); + } + + template + const basic_pointer_oserializer * + register_type(const T * = NULL){ + const basic_pointer_oserializer & bpos = + boost::serialization::singleton< + pointer_oserializer + >::get_const_instance(); + this->This()->register_basic_serializer(bpos.get_basic_serializer()); + return & bpos; + } + + template + Helper & + get_helper(void * const id = 0){ + helper_collection & hc = this->This()->get_helper_collection(); + return hc.template find_helper(id); + } + + template + Archive & operator<<(const T & t){ + this->This()->save_override(t); + return * this->This(); + } + + // the & operator + template + Archive & operator&(const T & t){ + return * this ->This() << t; + } +}; + +} // namespace detail +} // namespace archive +} // namespace boost + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_DETAIL_INTERFACE_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/iserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/iserializer.hpp new file mode 100644 index 00000000000..193e98a82e4 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/iserializer.hpp @@ -0,0 +1,631 @@ +#ifndef BOOST_ARCHIVE_DETAIL_ISERIALIZER_HPP +#define BOOST_ARCHIVE_DETAIL_ISERIALIZER_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#pragma inline_depth(511) +#pragma inline_recursion(on) +#endif + +#if defined(__MWERKS__) +#pragma inline_depth(511) +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// iserializer.hpp: interface for serialization system. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // for placement new +#include // size_t, NULL + +#include +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include + +#include +#include +#include +#include +#include + +#ifndef BOOST_SERIALIZATION_DEFAULT_TYPE_INFO + #include +#endif +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include + +#ifndef BOOST_MSVC + #define DONT_USE_HAS_NEW_OPERATOR ( \ + BOOST_WORKAROUND(__IBMCPP__, < 1210) \ + || defined(__SUNPRO_CC) && (__SUNPRO_CC < 0x590) \ + ) +#else + #define DONT_USE_HAS_NEW_OPERATOR 0 +#endif + +#if ! DONT_USE_HAS_NEW_OPERATOR +#include +#endif + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +// the following is need only for dynamic cast of polymorphic pointers +#include +#include +#include +#include +#include +#include + +namespace boost { + +namespace serialization { + class extended_type_info; +} // namespace serialization + +namespace archive { + +// an accessor to permit friend access to archives. Needed because +// some compilers don't handle friend templates completely +class load_access { +public: + template + static void load_primitive(Archive &ar, T &t){ + ar.load(t); + } +}; + +namespace detail { + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +template +class iserializer : public basic_iserializer +{ +private: + virtual void destroy(/*const*/ void *address) const { + boost::serialization::access::destroy(static_cast(address)); + } +protected: + // protected constructor since it's always created by singleton + explicit iserializer() : + basic_iserializer( + boost::serialization::singleton< + typename + boost::serialization::type_info_implementation< T >::type + >::get_const_instance() + ) + {} +public: + virtual BOOST_DLLEXPORT void load_object_data( + basic_iarchive & ar, + void *x, + const unsigned int file_version + ) const BOOST_USED; + virtual bool class_info() const { + return boost::serialization::implementation_level< T >::value + >= boost::serialization::object_class_info; + } + virtual bool tracking(const unsigned int /* flags */) const { + return boost::serialization::tracking_level< T >::value + == boost::serialization::track_always + || ( boost::serialization::tracking_level< T >::value + == boost::serialization::track_selectively + && serialized_as_pointer()); + } + virtual version_type version() const { + return version_type(::boost::serialization::version< T >::value); + } + virtual bool is_polymorphic() const { + return boost::is_polymorphic< T >::value; + } + virtual ~iserializer(){}; +}; + +#ifdef BOOST_MSVC +# pragma warning(pop) +#endif + +template +BOOST_DLLEXPORT void iserializer::load_object_data( + basic_iarchive & ar, + void *x, + const unsigned int file_version +) const { + // note: we now comment this out. Before we permited archive + // version # to be very large. Now we don't. To permit + // readers of these old archives, we have to suppress this + // code. Perhaps in the future we might re-enable it but + // permit its suppression with a runtime switch. + #if 0 + // trap case where the program cannot handle the current version + if(file_version > static_cast(version())) + boost::serialization::throw_exception( + archive::archive_exception( + boost::archive::archive_exception::unsupported_class_version, + get_debug_info() + ) + ); + #endif + // make sure call is routed through the higest interface that might + // be specialized by the user. + boost::serialization::serialize_adl( + boost::serialization::smart_cast_reference(ar), + * static_cast(x), + file_version + ); +} + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +// the purpose of this code is to allocate memory for an object +// without requiring the constructor to be called. Presumably +// the allocated object will be subsequently initialized with +// "placement new". +// note: we have the boost type trait has_new_operator but we +// have no corresponding has_delete_operator. So we presume +// that the former being true would imply that the a delete +// operator is also defined for the class T. + +template +struct heap_allocation { + // boost::has_new_operator< T > doesn't work on these compilers + #if DONT_USE_HAS_NEW_OPERATOR + // This doesn't handle operator new overload for class T + static T * invoke_new(){ + return static_cast(operator new(sizeof(T))); + } + static void invoke_delete(T *t){ + (operator delete(t)); + } + #else + // note: we presume that a true value for has_new_operator + // implies the existence of a class specific delete operator as well + // as a class specific new operator. + struct has_new_operator { + static T * invoke_new() { + return static_cast((T::operator new)(sizeof(T))); + } + static void invoke_delete(T * t) { + // if compilation fails here, the likely cause that the class + // T has a class specific new operator but no class specific + // delete operator which matches the following signature. + // note that this solution addresses the issue that two + // possible signatures. But it doesn't address the possibility + // that the class might have class specific new with NO + // class specific delete at all. Patches (compatible with + // C++03) welcome! + delete t; + } + }; + struct doesnt_have_new_operator { + static T* invoke_new() { + return static_cast(operator new(sizeof(T))); + } + static void invoke_delete(T * t) { + // Note: I'm reliance upon automatic conversion from T * to void * here + delete t; + } + }; + static T * invoke_new() { + typedef typename + mpl::eval_if< + boost::has_new_operator< T >, + mpl::identity, + mpl::identity + >::type typex; + return typex::invoke_new(); + } + static void invoke_delete(T *t) { + typedef typename + mpl::eval_if< + boost::has_new_operator< T >, + mpl::identity, + mpl::identity + >::type typex; + typex::invoke_delete(t); + } + #endif + explicit heap_allocation(){ + m_p = invoke_new(); + } + ~heap_allocation(){ + if (0 != m_p) + invoke_delete(m_p); + } + T* get() const { + return m_p; + } + + T* release() { + T* p = m_p; + m_p = 0; + return p; + } +private: + T* m_p; +}; + +template +class pointer_iserializer : + public basic_pointer_iserializer +{ +private: + virtual void * heap_allocation() const { + detail::heap_allocation h; + T * t = h.get(); + h.release(); + return t; + } + virtual const basic_iserializer & get_basic_serializer() const { + return boost::serialization::singleton< + iserializer + >::get_const_instance(); + } + BOOST_DLLEXPORT virtual void load_object_ptr( + basic_iarchive & ar, + void * x, + const unsigned int file_version + ) const BOOST_USED; +protected: + // this should alway be a singleton so make the constructor protected + pointer_iserializer(); + ~pointer_iserializer(); +}; + +#ifdef BOOST_MSVC +# pragma warning(pop) +#endif + +// note: BOOST_DLLEXPORT is so that code for polymorphic class +// serialized only through base class won't get optimized out +template +BOOST_DLLEXPORT void pointer_iserializer::load_object_ptr( + basic_iarchive & ar, + void * t, + const unsigned int file_version +) const +{ + Archive & ar_impl = + boost::serialization::smart_cast_reference(ar); + + // note that the above will throw std::bad_alloc if the allocation + // fails so we don't have to address this contingency here. + + // catch exception during load_construct_data so that we don't + // automatically delete the t which is most likely not fully + // constructed + BOOST_TRY { + // this addresses an obscure situation that occurs when + // load_constructor de-serializes something through a pointer. + ar.next_object_pointer(t); + boost::serialization::load_construct_data_adl( + ar_impl, + static_cast(t), + file_version + ); + } + BOOST_CATCH(...){ + // if we get here the load_construct failed. The heap_allocation + // will be automatically deleted so we don't have to do anything + // special here. + BOOST_RETHROW; + } + BOOST_CATCH_END + + ar_impl >> boost::serialization::make_nvp(NULL, * static_cast(t)); +} + +template +pointer_iserializer::pointer_iserializer() : + basic_pointer_iserializer( + boost::serialization::singleton< + typename + boost::serialization::type_info_implementation< T >::type + >::get_const_instance() + ) +{ + boost::serialization::singleton< + iserializer + >::get_mutable_instance().set_bpis(this); + archive_serializer_map::insert(this); +} + +template +pointer_iserializer::~pointer_iserializer(){ + archive_serializer_map::erase(this); +} + +template +struct load_non_pointer_type { + // note this bounces the call right back to the archive + // with no runtime overhead + struct load_primitive { + template + static void invoke(Archive & ar, T & t){ + load_access::load_primitive(ar, t); + } + }; + // note this bounces the call right back to the archive + // with no runtime overhead + struct load_only { + template + static void invoke(Archive & ar, const T & t){ + // short cut to user's serializer + // make sure call is routed through the higest interface that might + // be specialized by the user. + boost::serialization::serialize_adl( + ar, + const_cast(t), + boost::serialization::version< T >::value + ); + } + }; + + // note this save class information including version + // and serialization level to the archive + struct load_standard { + template + static void invoke(Archive &ar, const T & t){ + void * x = & const_cast(t); + ar.load_object( + x, + boost::serialization::singleton< + iserializer + >::get_const_instance() + ); + } + }; + + struct load_conditional { + template + static void invoke(Archive &ar, T &t){ + //if(0 == (ar.get_flags() & no_tracking)) + load_standard::invoke(ar, t); + //else + // load_only::invoke(ar, t); + } + }; + + template + static void invoke(Archive & ar, T &t){ + typedef typename mpl::eval_if< + // if its primitive + mpl::equal_to< + boost::serialization::implementation_level< T >, + mpl::int_ + >, + mpl::identity, + // else + typename mpl::eval_if< + // class info / version + mpl::greater_equal< + boost::serialization::implementation_level< T >, + mpl::int_ + >, + // do standard load + mpl::identity, + // else + typename mpl::eval_if< + // no tracking + mpl::equal_to< + boost::serialization::tracking_level< T >, + mpl::int_ + >, + // do a fast load + mpl::identity, + // else + // do a fast load only tracking is turned off + mpl::identity + > > >::type typex; + check_object_versioning< T >(); + check_object_level< T >(); + typex::invoke(ar, t); + } +}; + +template +struct load_pointer_type { + struct abstract + { + template + static const basic_pointer_iserializer * register_type(Archive & /* ar */){ + // it has? to be polymorphic + BOOST_STATIC_ASSERT(boost::is_polymorphic< T >::value); + return static_cast(NULL); + } + }; + + struct non_abstract + { + template + static const basic_pointer_iserializer * register_type(Archive & ar){ + return ar.register_type(static_cast(NULL)); + } + }; + + template + static const basic_pointer_iserializer * register_type(Archive &ar, const T & /*t*/){ + // there should never be any need to load an abstract polymorphic + // class pointer. Inhibiting code generation for this + // permits abstract base classes to be used - note: exception + // virtual serialize functions used for plug-ins + typedef typename + mpl::eval_if< + boost::serialization::is_abstract, + boost::mpl::identity, + boost::mpl::identity + >::type typex; + return typex::template register_type< T >(ar); + } + + template + static T * pointer_tweak( + const boost::serialization::extended_type_info & eti, + void const * const t, + const T & + ) { + // tweak the pointer back to the base class + void * upcast = const_cast( + boost::serialization::void_upcast( + eti, + boost::serialization::singleton< + typename + boost::serialization::type_info_implementation< T >::type + >::get_const_instance(), + t + ) + ); + if(NULL == upcast) + boost::serialization::throw_exception( + archive_exception(archive_exception::unregistered_class) + ); + return static_cast(upcast); + } + + template + static void check_load(T & /* t */){ + check_pointer_level< T >(); + check_pointer_tracking< T >(); + } + + static const basic_pointer_iserializer * + find(const boost::serialization::extended_type_info & type){ + return static_cast( + archive_serializer_map::find(type) + ); + } + + template + static void invoke(Archive & ar, Tptr & t){ + check_load(*t); + const basic_pointer_iserializer * bpis_ptr = register_type(ar, *t); + const basic_pointer_iserializer * newbpis_ptr = ar.load_pointer( + // note major hack here !!! + // I tried every way to convert Tptr &t (where Tptr might + // include const) to void * &. This is the only way + // I could make it work. RR + (void * & )t, + bpis_ptr, + find + ); + // if the pointer isn't that of the base class + if(newbpis_ptr != bpis_ptr){ + t = pointer_tweak(newbpis_ptr->get_eti(), t, *t); + } + } +}; + +template +struct load_enum_type { + template + static void invoke(Archive &ar, T &t){ + // convert integers to correct enum to load + int i; + ar >> boost::serialization::make_nvp(NULL, i); + t = static_cast< T >(i); + } +}; + +template +struct load_array_type { + template + static void invoke(Archive &ar, T &t){ + typedef typename remove_extent< T >::type value_type; + + // convert integers to correct enum to load + // determine number of elements in the array. Consider the + // fact that some machines will align elements on boundries + // other than characters. + std::size_t current_count = sizeof(t) / ( + static_cast(static_cast(&t[1])) + - static_cast(static_cast(&t[0])) + ); + boost::serialization::collection_size_type count; + ar >> BOOST_SERIALIZATION_NVP(count); + if(static_cast(count) > current_count) + boost::serialization::throw_exception( + archive::archive_exception( + boost::archive::archive_exception::array_size_too_short + ) + ); + // explict template arguments to pass intel C++ compiler + ar >> serialization::make_array< + value_type, + boost::serialization::collection_size_type + >( + static_cast(&t[0]), + count + ); + } +}; + +} // detail + +template +inline void load(Archive & ar, T &t){ + // if this assertion trips. It means we're trying to load a + // const object with a compiler that doesn't have correct + // function template ordering. On other compilers, this is + // handled below. + detail::check_const_loading< T >(); + typedef + typename mpl::eval_if, + mpl::identity > + ,//else + typename mpl::eval_if, + mpl::identity > + ,//else + typename mpl::eval_if, + mpl::identity > + ,//else + mpl::identity > + > + > + >::type typex; + typex::invoke(ar, t); +} + +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_DETAIL_ISERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/oserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/oserializer.hpp new file mode 100644 index 00000000000..c120ec55073 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/oserializer.hpp @@ -0,0 +1,540 @@ +#ifndef BOOST_ARCHIVE_OSERIALIZER_HPP +#define BOOST_ARCHIVE_OSERIALIZER_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#pragma inline_depth(511) +#pragma inline_recursion(on) +#endif + +#if defined(__MWERKS__) +#pragma inline_depth(511) +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// oserializer.hpp: interface for serialization system. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include // NULL + +#include +#include +#include + +#include +#include +#include +#include +#include + +#ifndef BOOST_SERIALIZATION_DEFAULT_TYPE_INFO + #include +#endif +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include + +namespace boost { + +namespace serialization { + class extended_type_info; +} // namespace serialization + +namespace archive { + +// an accessor to permit friend access to archives. Needed because +// some compilers don't handle friend templates completely +class save_access { +public: + template + static void end_preamble(Archive & ar){ + ar.end_preamble(); + } + template + static void save_primitive(Archive & ar, const T & t){ + ar.end_preamble(); + ar.save(t); + } +}; + +namespace detail { + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +template +class oserializer : public basic_oserializer +{ +private: + // private constructor to inhibit any existence other than the + // static one +public: + explicit BOOST_DLLEXPORT oserializer() : + basic_oserializer( + boost::serialization::singleton< + typename + boost::serialization::type_info_implementation< T >::type + >::get_const_instance() + ) + {} + virtual BOOST_DLLEXPORT void save_object_data( + basic_oarchive & ar, + const void *x + ) const BOOST_USED; + virtual bool class_info() const { + return boost::serialization::implementation_level< T >::value + >= boost::serialization::object_class_info; + } + virtual bool tracking(const unsigned int /* flags */) const { + return boost::serialization::tracking_level< T >::value == boost::serialization::track_always + || (boost::serialization::tracking_level< T >::value == boost::serialization::track_selectively + && serialized_as_pointer()); + } + virtual version_type version() const { + return version_type(::boost::serialization::version< T >::value); + } + virtual bool is_polymorphic() const { + return boost::is_polymorphic< T >::value; + } + virtual ~oserializer(){} +}; + +#ifdef BOOST_MSVC +# pragma warning(pop) +#endif + +template +BOOST_DLLEXPORT void oserializer::save_object_data( + basic_oarchive & ar, + const void *x +) const { + // make sure call is routed through the highest interface that might + // be specialized by the user. + BOOST_STATIC_ASSERT(boost::is_const< T >::value == false); + boost::serialization::serialize_adl( + boost::serialization::smart_cast_reference(ar), + * static_cast(const_cast(x)), + version() + ); +} + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +template +class pointer_oserializer : + public basic_pointer_oserializer +{ +private: + const basic_oserializer & + get_basic_serializer() const { + return boost::serialization::singleton< + oserializer + >::get_const_instance(); + } + virtual BOOST_DLLEXPORT void save_object_ptr( + basic_oarchive & ar, + const void * x + ) const BOOST_USED; +public: + pointer_oserializer(); + ~pointer_oserializer(); +}; + +#ifdef BOOST_MSVC +# pragma warning(pop) +#endif + +template +BOOST_DLLEXPORT void pointer_oserializer::save_object_ptr( + basic_oarchive & ar, + const void * x +) const { + BOOST_ASSERT(NULL != x); + // make sure call is routed through the highest interface that might + // be specialized by the user. + T * t = static_cast(const_cast(x)); + const unsigned int file_version = boost::serialization::version< T >::value; + Archive & ar_impl + = boost::serialization::smart_cast_reference(ar); + boost::serialization::save_construct_data_adl( + ar_impl, + t, + file_version + ); + ar_impl << boost::serialization::make_nvp(NULL, * t); +} + +template +pointer_oserializer::pointer_oserializer() : + basic_pointer_oserializer( + boost::serialization::singleton< + typename + boost::serialization::type_info_implementation< T >::type + >::get_const_instance() + ) +{ + // make sure appropriate member function is instantiated + boost::serialization::singleton< + oserializer + >::get_mutable_instance().set_bpos(this); + archive_serializer_map::insert(this); +} + +template +pointer_oserializer::~pointer_oserializer(){ + archive_serializer_map::erase(this); +} + +template +struct save_non_pointer_type { + // note this bounces the call right back to the archive + // with no runtime overhead + struct save_primitive { + template + static void invoke(Archive & ar, const T & t){ + save_access::save_primitive(ar, t); + } + }; + // same as above but passes through serialization + struct save_only { + template + static void invoke(Archive & ar, const T & t){ + // make sure call is routed through the highest interface that might + // be specialized by the user. + boost::serialization::serialize_adl( + ar, + const_cast(t), + ::boost::serialization::version< T >::value + ); + } + }; + // adds class information to the archive. This includes + // serialization level and class version + struct save_standard { + template + static void invoke(Archive &ar, const T & t){ + ar.save_object( + & t, + boost::serialization::singleton< + oserializer + >::get_const_instance() + ); + } + }; + + // adds class information to the archive. This includes + // serialization level and class version + struct save_conditional { + template + static void invoke(Archive &ar, const T &t){ + //if(0 == (ar.get_flags() & no_tracking)) + save_standard::invoke(ar, t); + //else + // save_only::invoke(ar, t); + } + }; + + + template + static void invoke(Archive & ar, const T & t){ + typedef + typename mpl::eval_if< + // if its primitive + mpl::equal_to< + boost::serialization::implementation_level< T >, + mpl::int_ + >, + mpl::identity, + // else + typename mpl::eval_if< + // class info / version + mpl::greater_equal< + boost::serialization::implementation_level< T >, + mpl::int_ + >, + // do standard save + mpl::identity, + // else + typename mpl::eval_if< + // no tracking + mpl::equal_to< + boost::serialization::tracking_level< T >, + mpl::int_ + >, + // do a fast save + mpl::identity, + // else + // do a fast save only tracking is turned off + mpl::identity + > > >::type typex; + check_object_versioning< T >(); + typex::invoke(ar, t); + } + template + static void invoke(Archive & ar, T & t){ + check_object_level< T >(); + check_object_tracking< T >(); + invoke(ar, const_cast(t)); + } +}; + +template +struct save_pointer_type { + struct abstract + { + template + static const basic_pointer_oserializer * register_type(Archive & /* ar */){ + // it has? to be polymorphic + BOOST_STATIC_ASSERT(boost::is_polymorphic< T >::value); + return NULL; + } + }; + + struct non_abstract + { + template + static const basic_pointer_oserializer * register_type(Archive & ar){ + return ar.register_type(static_cast(NULL)); + } + }; + + template + static const basic_pointer_oserializer * register_type(Archive &ar, T & /*t*/){ + // there should never be any need to save an abstract polymorphic + // class pointer. Inhibiting code generation for this + // permits abstract base classes to be used - note: exception + // virtual serialize functions used for plug-ins + typedef + typename mpl::eval_if< + boost::serialization::is_abstract< T >, + mpl::identity, + mpl::identity + >::type typex; + return typex::template register_type< T >(ar); + } + + struct non_polymorphic + { + template + static void save( + Archive &ar, + T & t + ){ + const basic_pointer_oserializer & bpos = + boost::serialization::singleton< + pointer_oserializer + >::get_const_instance(); + // save the requested pointer type + ar.save_pointer(& t, & bpos); + } + }; + + struct polymorphic + { + template + static void save( + Archive &ar, + T & t + ){ + typename + boost::serialization::type_info_implementation< T >::type const + & i = boost::serialization::singleton< + typename + boost::serialization::type_info_implementation< T >::type + >::get_const_instance(); + + boost::serialization::extended_type_info const * const this_type = & i; + + // retrieve the true type of the object pointed to + // if this assertion fails its an error in this library + BOOST_ASSERT(NULL != this_type); + + const boost::serialization::extended_type_info * true_type = + i.get_derived_extended_type_info(t); + + // note:if this exception is thrown, be sure that derived pointer + // is either registered or exported. + if(NULL == true_type){ + boost::serialization::throw_exception( + archive_exception( + archive_exception::unregistered_class, + "derived class not registered or exported" + ) + ); + } + + // if its not a pointer to a more derived type + const void *vp = static_cast(&t); + if(*this_type == *true_type){ + const basic_pointer_oserializer * bpos = register_type(ar, t); + ar.save_pointer(vp, bpos); + return; + } + // convert pointer to more derived type. if this is thrown + // it means that the base/derived relationship hasn't be registered + vp = serialization::void_downcast( + *true_type, + *this_type, + static_cast(&t) + ); + if(NULL == vp){ + boost::serialization::throw_exception( + archive_exception( + archive_exception::unregistered_cast, + true_type->get_debug_info(), + this_type->get_debug_info() + ) + ); + } + + // since true_type is valid, and this only gets made if the + // pointer oserializer object has been created, this should never + // fail + const basic_pointer_oserializer * bpos + = static_cast( + boost::serialization::singleton< + archive_serializer_map + >::get_const_instance().find(*true_type) + ); + BOOST_ASSERT(NULL != bpos); + if(NULL == bpos) + boost::serialization::throw_exception( + archive_exception( + archive_exception::unregistered_class, + "derived class not registered or exported" + ) + ); + ar.save_pointer(vp, bpos); + } + }; + + template + static void save( + Archive & ar, + const T & t + ){ + check_pointer_level< T >(); + check_pointer_tracking< T >(); + typedef typename mpl::eval_if< + is_polymorphic< T >, + mpl::identity, + mpl::identity + >::type type; + type::save(ar, const_cast(t)); + } + + template + static void invoke(Archive &ar, const TPtr t){ + register_type(ar, * t); + if(NULL == t){ + basic_oarchive & boa + = boost::serialization::smart_cast_reference(ar); + boa.save_null_pointer(); + save_access::end_preamble(ar); + return; + } + save(ar, * t); + } +}; + +template +struct save_enum_type +{ + template + static void invoke(Archive &ar, const T &t){ + // convert enum to integers on save + const int i = static_cast(t); + ar << boost::serialization::make_nvp(NULL, i); + } +}; + +template +struct save_array_type +{ + template + static void invoke(Archive &ar, const T &t){ + typedef typename boost::remove_extent< T >::type value_type; + + save_access::end_preamble(ar); + // consider alignment + std::size_t c = sizeof(t) / ( + static_cast(static_cast(&t[1])) + - static_cast(static_cast(&t[0])) + ); + boost::serialization::collection_size_type count(c); + ar << BOOST_SERIALIZATION_NVP(count); + // explict template arguments to pass intel C++ compiler + ar << serialization::make_array< + const value_type, + boost::serialization::collection_size_type + >( + static_cast(&t[0]), + count + ); + } +}; + +} // detail + +template +inline void save(Archive & ar, /*const*/ T &t){ + typedef + typename mpl::eval_if, + mpl::identity >, + //else + typename mpl::eval_if, + mpl::identity >, + //else + typename mpl::eval_if, + mpl::identity >, + //else + mpl::identity > + > + > + >::type typex; + typex::invoke(ar, t); +} + +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_OSERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_iarchive_route.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_iarchive_route.hpp new file mode 100644 index 00000000000..105685ebbd8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_iarchive_route.hpp @@ -0,0 +1,218 @@ +#ifndef BOOST_ARCHIVE_DETAIL_POLYMORPHIC_IARCHIVE_ROUTE_HPP +#define BOOST_ARCHIVE_DETAIL_POLYMORPHIC_IARCHIVE_ROUTE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_iarchive_route.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include +#include +#include +#include // must be the last header + +namespace boost { +namespace serialization { + class extended_type_info; +} // namespace serialization +namespace archive { +namespace detail{ + +class basic_iserializer; +class basic_pointer_iserializer; + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +template +class polymorphic_iarchive_route : + public polymorphic_iarchive, + // note: gcc dynamic cross cast fails if the the derivation below is + // not public. I think this is a mistake. + public /*protected*/ ArchiveImplementation +{ +private: + // these are used by the serialization library. + virtual void load_object( + void *t, + const basic_iserializer & bis + ){ + ArchiveImplementation::load_object(t, bis); + } + virtual const basic_pointer_iserializer * load_pointer( + void * & t, + const basic_pointer_iserializer * bpis_ptr, + const basic_pointer_iserializer * (*finder)( + const boost::serialization::extended_type_info & type + ) + ){ + return ArchiveImplementation::load_pointer(t, bpis_ptr, finder); + } + virtual void set_library_version(library_version_type archive_library_version){ + ArchiveImplementation::set_library_version(archive_library_version); + } + virtual library_version_type get_library_version() const{ + return ArchiveImplementation::get_library_version(); + } + virtual unsigned int get_flags() const { + return ArchiveImplementation::get_flags(); + } + virtual void delete_created_pointers(){ + ArchiveImplementation::delete_created_pointers(); + } + virtual void reset_object_address( + const void * new_address, + const void * old_address + ){ + ArchiveImplementation::reset_object_address(new_address, old_address); + } + virtual void load_binary(void * t, std::size_t size){ + ArchiveImplementation::load_binary(t, size); + } + // primitive types the only ones permitted by polymorphic archives + virtual void load(bool & t){ + ArchiveImplementation::load(t); + } + virtual void load(char & t){ + ArchiveImplementation::load(t); + } + virtual void load(signed char & t){ + ArchiveImplementation::load(t); + } + virtual void load(unsigned char & t){ + ArchiveImplementation::load(t); + } + #ifndef BOOST_NO_CWCHAR + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + virtual void load(wchar_t & t){ + ArchiveImplementation::load(t); + } + #endif + #endif + virtual void load(short & t){ + ArchiveImplementation::load(t); + } + virtual void load(unsigned short & t){ + ArchiveImplementation::load(t); + } + virtual void load(int & t){ + ArchiveImplementation::load(t); + } + virtual void load(unsigned int & t){ + ArchiveImplementation::load(t); + } + virtual void load(long & t){ + ArchiveImplementation::load(t); + } + virtual void load(unsigned long & t){ + ArchiveImplementation::load(t); + } + #if defined(BOOST_HAS_LONG_LONG) + virtual void load(boost::long_long_type & t){ + ArchiveImplementation::load(t); + } + virtual void load(boost::ulong_long_type & t){ + ArchiveImplementation::load(t); + } + #elif defined(BOOST_HAS_MS_INT64) + virtual void load(__int64 & t){ + ArchiveImplementation::load(t); + } + virtual void load(unsigned __int64 & t){ + ArchiveImplementation::load(t); + } + #endif + virtual void load(float & t){ + ArchiveImplementation::load(t); + } + virtual void load(double & t){ + ArchiveImplementation::load(t); + } + virtual void load(std::string & t){ + ArchiveImplementation::load(t); + } + #ifndef BOOST_NO_STD_WSTRING + virtual void load(std::wstring & t){ + ArchiveImplementation::load(t); + } + #endif + // used for xml and other tagged formats default does nothing + virtual void load_start(const char * name){ + ArchiveImplementation::load_start(name); + } + virtual void load_end(const char * name){ + ArchiveImplementation::load_end(name); + } + virtual void register_basic_serializer(const basic_iserializer & bis){ + ArchiveImplementation::register_basic_serializer(bis); + } + virtual helper_collection & + get_helper_collection(){ + return ArchiveImplementation::get_helper_collection(); + } +public: + // this can't be inheriteded because they appear in mulitple + // parents + typedef mpl::bool_ is_loading; + typedef mpl::bool_ is_saving; + // the >> operator + template + polymorphic_iarchive & operator>>(T & t){ + return polymorphic_iarchive::operator>>(t); + } + // the & operator + template + polymorphic_iarchive & operator&(T & t){ + return polymorphic_iarchive::operator&(t); + } + // register type function + template + const basic_pointer_iserializer * + register_type(T * t = NULL){ + return ArchiveImplementation::register_type(t); + } + // all current archives take a stream as constructor argument + template + polymorphic_iarchive_route( + std::basic_istream<_Elem, _Tr> & is, + unsigned int flags = 0 + ) : + ArchiveImplementation(is, flags) + {} + virtual ~polymorphic_iarchive_route(){}; +}; + +} // namespace detail +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_DETAIL_POLYMORPHIC_IARCHIVE_DISPATCH_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_oarchive_route.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_oarchive_route.hpp new file mode 100644 index 00000000000..b23fd6bf39d --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_oarchive_route.hpp @@ -0,0 +1,209 @@ +#ifndef BOOST_ARCHIVE_DETAIL_POLYMORPHIC_OARCHIVE_ROUTE_HPP +#define BOOST_ARCHIVE_DETAIL_POLYMORPHIC_OARCHIVE_ROUTE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_oarchive_route.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include // size_t + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include +#include +#include +#include // must be the last header + +namespace boost { +namespace serialization { + class extended_type_info; +} // namespace serialization +namespace archive { +namespace detail{ + +class basic_oserializer; +class basic_pointer_oserializer; + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +template +class polymorphic_oarchive_route : + public polymorphic_oarchive, + // note: gcc dynamic cross cast fails if the the derivation below is + // not public. I think this is a mistake. + public /*protected*/ ArchiveImplementation +{ +private: + // these are used by the serialization library. + virtual void save_object( + const void *x, + const detail::basic_oserializer & bos + ){ + ArchiveImplementation::save_object(x, bos); + } + virtual void save_pointer( + const void * t, + const detail::basic_pointer_oserializer * bpos_ptr + ){ + ArchiveImplementation::save_pointer(t, bpos_ptr); + } + virtual void save_null_pointer(){ + ArchiveImplementation::save_null_pointer(); + } + // primitive types the only ones permitted by polymorphic archives + virtual void save(const bool t){ + ArchiveImplementation::save(t); + } + virtual void save(const char t){ + ArchiveImplementation::save(t); + } + virtual void save(const signed char t){ + ArchiveImplementation::save(t); + } + virtual void save(const unsigned char t){ + ArchiveImplementation::save(t); + } + #ifndef BOOST_NO_CWCHAR + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + virtual void save(const wchar_t t){ + ArchiveImplementation::save(t); + } + #endif + #endif + virtual void save(const short t){ + ArchiveImplementation::save(t); + } + virtual void save(const unsigned short t){ + ArchiveImplementation::save(t); + } + virtual void save(const int t){ + ArchiveImplementation::save(t); + } + virtual void save(const unsigned int t){ + ArchiveImplementation::save(t); + } + virtual void save(const long t){ + ArchiveImplementation::save(t); + } + virtual void save(const unsigned long t){ + ArchiveImplementation::save(t); + } + #if defined(BOOST_HAS_LONG_LONG) + virtual void save(const boost::long_long_type t){ + ArchiveImplementation::save(t); + } + virtual void save(const boost::ulong_long_type t){ + ArchiveImplementation::save(t); + } + #elif defined(BOOST_HAS_MS_INT64) + virtual void save(const boost::int64_t t){ + ArchiveImplementation::save(t); + } + virtual void save(const boost::uint64_t t){ + ArchiveImplementation::save(t); + } + #endif + virtual void save(const float t){ + ArchiveImplementation::save(t); + } + virtual void save(const double t){ + ArchiveImplementation::save(t); + } + virtual void save(const std::string & t){ + ArchiveImplementation::save(t); + } + #ifndef BOOST_NO_STD_WSTRING + virtual void save(const std::wstring & t){ + ArchiveImplementation::save(t); + } + #endif + virtual library_version_type get_library_version() const{ + return ArchiveImplementation::get_library_version(); + } + virtual unsigned int get_flags() const { + return ArchiveImplementation::get_flags(); + } + virtual void save_binary(const void * t, std::size_t size){ + ArchiveImplementation::save_binary(t, size); + } + // used for xml and other tagged formats default does nothing + virtual void save_start(const char * name){ + ArchiveImplementation::save_start(name); + } + virtual void save_end(const char * name){ + ArchiveImplementation::save_end(name); + } + virtual void end_preamble(){ + ArchiveImplementation::end_preamble(); + } + virtual void register_basic_serializer(const detail::basic_oserializer & bos){ + ArchiveImplementation::register_basic_serializer(bos); + } + virtual helper_collection & + get_helper_collection(){ + return ArchiveImplementation::get_helper_collection(); + } +public: + // this can't be inheriteded because they appear in mulitple + // parents + typedef mpl::bool_ is_loading; + typedef mpl::bool_ is_saving; + // the << operator + template + polymorphic_oarchive & operator<<(T & t){ + return polymorphic_oarchive::operator<<(t); + } + // the & operator + template + polymorphic_oarchive & operator&(T & t){ + return polymorphic_oarchive::operator&(t); + } + // register type function + template + const basic_pointer_oserializer * + register_type(T * t = NULL){ + return ArchiveImplementation::register_type(t); + } + // all current archives take a stream as constructor argument + template + polymorphic_oarchive_route( + std::basic_ostream<_Elem, _Tr> & os, + unsigned int flags = 0 + ) : + ArchiveImplementation(os, flags) + {} + virtual ~polymorphic_oarchive_route(){}; +}; + +} // namespace detail +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_DETAIL_POLYMORPHIC_OARCHIVE_DISPATCH_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/register_archive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/register_archive.hpp new file mode 100644 index 00000000000..5ffecc702ce --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/register_archive.hpp @@ -0,0 +1,91 @@ +// Copyright David Abrahams 2006. Distributed under the Boost +// Software License, Version 1.0. (See accompanying +// file LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +#ifndef BOOST_ARCHIVE_DETAIL_REGISTER_ARCHIVE_DWA2006521_HPP +# define BOOST_ARCHIVE_DETAIL_REGISTER_ARCHIVE_DWA2006521_HPP + +namespace boost { namespace archive { namespace detail { + +// No instantiate_ptr_serialization overloads generated by +// BOOST_SERIALIZATION_REGISTER_ARCHIVE that lexically follow the call +// will be seen *unless* they are in an associated namespace of one of +// the arguments, so we pass one of these along to make sure this +// namespace is considered. See temp.dep.candidate (14.6.4.2) in the +// standard. +struct adl_tag {}; + +template +struct ptr_serialization_support; + +// We could've just used ptr_serialization_support, above, but using +// it with only a forward declaration causes vc6/7 to complain about a +// missing instantiate member, even if it has one. This is just a +// friendly layer of indirection. +template +struct _ptr_serialization_support + : ptr_serialization_support +{ + typedef int type; +}; + +#if defined(__SUNPRO_CC) && (__SUNPRO_CC < 0x5130) + +template +struct counter : counter {}; +template<> +struct counter<0> {}; + +template +void instantiate_ptr_serialization(Serializable* s, int, adl_tag) { + instantiate_ptr_serialization(s, counter<20>()); +} + +template +struct get_counter { + static const int value = sizeof(adjust_counter(counter<20>())); + typedef counter type; + typedef counter prior; + typedef char (&next)[value+1]; +}; + +char adjust_counter(counter<0>); +template +void instantiate_ptr_serialization(Serializable*, counter<0>) {} + +#define BOOST_SERIALIZATION_REGISTER_ARCHIVE(Archive) \ +namespace boost { namespace archive { namespace detail { \ + get_counter::next adjust_counter(get_counter::type);\ + template \ + void instantiate_ptr_serialization(Serializable* s, \ + get_counter::type) { \ + ptr_serialization_support x; \ + instantiate_ptr_serialization(s, get_counter::prior()); \ + }\ +}}} + + +#else + +// This function gets called, but its only purpose is to participate +// in overload resolution with the functions declared by +// BOOST_SERIALIZATION_REGISTER_ARCHIVE, below. +template +void instantiate_ptr_serialization(Serializable*, int, adl_tag ) {} + +// The function declaration generated by this macro never actually +// gets called, but its return type gets instantiated, and that's +// enough to cause registration of serialization functions between +// Archive and any exported Serializable type. See also: +// boost/serialization/export.hpp +# define BOOST_SERIALIZATION_REGISTER_ARCHIVE(Archive) \ +namespace boost { namespace archive { namespace detail { \ + \ +template \ +typename _ptr_serialization_support::type \ +instantiate_ptr_serialization( Serializable*, Archive*, adl_tag ); \ + \ +}}} +#endif +}}} // namespace boost::archive::detail + +#endif // BOOST_ARCHIVE_DETAIL_INSTANTIATE_SERIALIZE_DWA2006521_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/utf8_codecvt_facet.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/utf8_codecvt_facet.hpp new file mode 100644 index 00000000000..a40104abea6 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/detail/utf8_codecvt_facet.hpp @@ -0,0 +1,39 @@ +// Copyright (c) 2001 Ronald Garcia, Indiana University (garcia@osl.iu.edu) +// Andrew Lumsdaine, Indiana University (lums@osl.iu.edu). +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +#ifndef BOOST_ARCHIVE_DETAIL_UTF8_CODECVT_FACET_HPP +#define BOOST_ARCHIVE_DETAIL_UTF8_CODECVT_FACET_HPP + +#include + +#ifdef BOOST_NO_STD_WSTREAMBUF +#error "wide char i/o not supported on this platform" +#endif + +// std::codecvt_utf8 doesn't seem to work for any versions of msvc + +#if defined(_MSC_VER) || defined(BOOST_NO_CXX11_HDR_CODECVT) + // use boost's utf8 codecvt facet + #include + #define BOOST_UTF8_BEGIN_NAMESPACE \ + namespace boost { namespace archive { namespace detail { + #define BOOST_UTF8_DECL BOOST_ARCHIVE_DECL + #define BOOST_UTF8_END_NAMESPACE }}} + + #include + + #undef BOOST_UTF8_END_NAMESPACE + #undef BOOST_UTF8_DECL + #undef BOOST_UTF8_BEGIN_NAMESPACE +#else + // use the standard vendor supplied facet + #include + namespace boost { namespace archive { namespace detail { + typedef std::codecvt_utf8 utf8_codecvt_facet; + } } } +#endif + +#endif // BOOST_ARCHIVE_DETAIL_UTF8_CODECVT_FACET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/dinkumware.hpp b/contrib/libboost/boost_1_65_0/boost/archive/dinkumware.hpp new file mode 100644 index 00000000000..90ba6271cdd --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/dinkumware.hpp @@ -0,0 +1,224 @@ +#ifndef BOOST_ARCHIVE_DINKUMWARE_HPP +#define BOOST_ARCHIVE_DINKUMWARE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// dinkumware.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// this file adds a couple of things that are missing from the dinkumware +// implementation of the standard library. + +#include +#include + +#include +#include + +namespace std { + +// define i/o operators for 64 bit integers +template +basic_ostream & +operator<<(basic_ostream & os, boost::uint64_t t){ + // octal rendering of 64 bit number would be 22 octets + eos + CharType d[23]; + unsigned int radix; + + if(os.flags() & (int)std::ios_base::hex) + radix = 16; + else + if(os.flags() & (int)std::ios_base::oct) + radix = 8; + else + //if(s.flags() & (int)std::ios_base::dec) + radix = 10; + unsigned int i = 0; + do{ + unsigned int j = t % radix; + d[i++] = j + ((j < 10) ? '0' : ('a' - 10)); + t /= radix; + } + while(t > 0); + d[i--] = '\0'; + + // reverse digits + unsigned int j = 0; + while(j < i){ + CharType k = d[i]; + d[i] = d[j]; + d[j] = k; + --i;++j; + } + os << d; + return os; + +} + +template +basic_ostream & +operator<<(basic_ostream &os, boost::int64_t t){ + if(0 <= t){ + os << static_cast(t); + } + else{ + os.put('-'); + os << -t; + } + return os; +} + +template +basic_istream & +operator>>(basic_istream &is, boost::int64_t & t){ + CharType d; + do{ + d = is.get(); + } + while(::isspace(d)); + bool negative = (d == '-'); + if(negative) + d = is.get(); + unsigned int radix; + if(is.flags() & (int)std::ios_base::hex) + radix = 16; + else + if(is.flags() & (int)std::ios_base::oct) + radix = 8; + else + //if(s.flags() & (int)std::ios_base::dec) + radix = 10; + t = 0; + do{ + if('0' <= d && d <= '9') + t = t * radix + (d - '0'); + else + if('a' <= d && d <= 'f') + t = t * radix + (d - 'a' + 10); + else + break; + d = is.get(); + } + while(!is.fail()); + // restore the delimiter + is.putback(d); + is.clear(); + if(negative) + t = -t; + return is; +} + +template +basic_istream & +operator>>(basic_istream &is, boost::uint64_t & t){ + boost::int64_t it; + is >> it; + t = it; + return is; +} + +//#endif + +template<> +class back_insert_iterator > : public + iterator +{ +public: + typedef basic_string container_type; + typedef container_type::reference reference; + + explicit back_insert_iterator(container_type & s) + : container(& s) + {} // construct with container + + back_insert_iterator & operator=( + container_type::const_reference Val_ + ){ // push value into container + //container->push_back(Val_); + *container += Val_; + return (*this); + } + + back_insert_iterator & operator*(){ + return (*this); + } + + back_insert_iterator & operator++(){ + // pretend to preincrement + return (*this); + } + + back_insert_iterator operator++(int){ + // pretend to postincrement + return (*this); + } + +protected: + container_type *container; // pointer to container +}; + +template +inline back_insert_iterator > back_inserter( + basic_string & s +){ + return (std::back_insert_iterator >(s)); +} + +template<> +class back_insert_iterator > : public + iterator +{ +public: + typedef basic_string container_type; + typedef container_type::reference reference; + + explicit back_insert_iterator(container_type & s) + : container(& s) + {} // construct with container + + back_insert_iterator & operator=( + container_type::const_reference Val_ + ){ // push value into container + //container->push_back(Val_); + *container += Val_; + return (*this); + } + + back_insert_iterator & operator*(){ + return (*this); + } + + back_insert_iterator & operator++(){ + // pretend to preincrement + return (*this); + } + + back_insert_iterator operator++(int){ + // pretend to postincrement + return (*this); + } + +protected: + container_type *container; // pointer to container +}; + +template +inline back_insert_iterator > back_inserter( + basic_string & s +){ + return (std::back_insert_iterator >(s)); +} + +} // namespace std + +#endif //BOOST_ARCHIVE_DINKUMWARE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/archive_serializer_map.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/archive_serializer_map.ipp new file mode 100644 index 00000000000..7f163ec4076 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/archive_serializer_map.ipp @@ -0,0 +1,75 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// archive_serializer_map.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +////////////////////////////////////////////////////////////////////// +// implementation of basic_text_iprimitive overrides for the combination +// of template parameters used to implement a text_iprimitive + +#include +#include +#include +#include + +namespace boost { +namespace archive { +namespace detail { + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace extra_detail { // anon + template + class map : public basic_serializer_map + {}; +} + +#ifdef BOOST_MSVC +# pragma warning(pop) +#endif + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL bool +archive_serializer_map::insert(const basic_serializer * bs){ + return boost::serialization::singleton< + extra_detail::map + >::get_mutable_instance().insert(bs); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +archive_serializer_map::erase(const basic_serializer * bs){ + BOOST_ASSERT(! boost::serialization::singleton< + extra_detail::map + >::is_destroyed() + ); + if(boost::serialization::singleton< + extra_detail::map + >::is_destroyed()) + return; + boost::serialization::singleton< + extra_detail::map + >::get_mutable_instance().erase(bs); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL const basic_serializer * +archive_serializer_map::find( + const boost::serialization::extended_type_info & eti +) { + return boost::serialization::singleton< + extra_detail::map + >::get_const_instance().find(eti); +} + +} // namespace detail +} // namespace archive +} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iarchive.ipp new file mode 100644 index 00000000000..d5619ab6cf3 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iarchive.ipp @@ -0,0 +1,134 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_binary_iarchive.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. +#include +#include +#include +#include + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::memcpy; + using ::strlen; + using ::size_t; +} +#endif + +#include +#include + +#include + +namespace boost { +namespace archive { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// implementation of binary_binary_archive +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_binary_iarchive::load_override(class_name_type & t){ + std::string cn; + cn.reserve(BOOST_SERIALIZATION_MAX_KEY_SIZE); + load_override(cn); + if(cn.size() > (BOOST_SERIALIZATION_MAX_KEY_SIZE - 1)) + boost::serialization::throw_exception( + archive_exception(archive_exception::invalid_class_name) + ); + std::memcpy(t, cn.data(), cn.size()); + // borland tweak + t.t[cn.size()] = '\0'; +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_binary_iarchive::init(void){ + // read signature in an archive version independent manner + std::string file_signature; + + #if 0 // commented out since it interfers with derivation + BOOST_TRY { + std::size_t l; + this->This()->load(l); + if(l == std::strlen(BOOST_ARCHIVE_SIGNATURE())) { + // borland de-allocator fixup + #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) + if(NULL != file_signature.data()) + #endif + file_signature.resize(l); + // note breaking a rule here - could be a problem on some platform + if(0 < l) + this->This()->load_binary(&(*file_signature.begin()), l); + } + } + BOOST_CATCH(archive_exception const &) { // catch stream_error archive exceptions + // will cause invalid_signature archive exception to be thrown below + file_signature = ""; + } + BOOST_CATCH_END + #else + // https://svn.boost.org/trac/boost/ticket/7301 + * this->This() >> file_signature; + #endif + + if(file_signature != BOOST_ARCHIVE_SIGNATURE()) + boost::serialization::throw_exception( + archive_exception(archive_exception::invalid_signature) + ); + + // make sure the version of the reading archive library can + // support the format of the archive being read + library_version_type input_library_version; + //* this->This() >> input_library_version; + { + int v = 0; + v = this->This()->m_sb.sbumpc(); + #if defined(BOOST_LITTLE_ENDIAN) + if(v < 6){ + ; + } + else + if(v < 7){ + // version 6 - next byte should be zero + this->This()->m_sb.sbumpc(); + } + else + if(v < 8){ + int x1; + // version 7 = might be followed by zero or some other byte + x1 = this->This()->m_sb.sgetc(); + // it's =a zero, push it back + if(0 == x1) + this->This()->m_sb.sbumpc(); + } + else{ + // version 8+ followed by a zero + this->This()->m_sb.sbumpc(); + } + #elif defined(BOOST_BIG_ENDIAN) + if(v == 0) + v = this->This()->m_sb.sbumpc(); + #endif + input_library_version = static_cast(v); + } + + #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) + this->set_library_version(input_library_version); + #else + detail::basic_iarchive::set_library_version(input_library_version); + #endif + + if(BOOST_ARCHIVE_VERSION() < input_library_version) + boost::serialization::throw_exception( + archive_exception(archive_exception::unsupported_version) + ); +} + +} // namespace archive +} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iprimitive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iprimitive.ipp new file mode 100644 index 00000000000..bbe933ccf63 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iprimitive.ipp @@ -0,0 +1,171 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_binary_iprimitive.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include // size_t, NULL +#include // memcpy + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; + using ::memcpy; +} // namespace std +#endif + +#include +#include +#include +#include + +namespace boost { +namespace archive { + +////////////////////////////////////////////////////////////////////// +// implementation of basic_binary_iprimitive + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_binary_iprimitive::init() +{ + // Detect attempts to pass native binary archives across + // incompatible platforms. This is not fool proof but its + // better than nothing. + unsigned char size; + this->This()->load(size); + if(sizeof(int) != size) + boost::serialization::throw_exception( + archive_exception( + archive_exception::incompatible_native_format, + "size of int" + ) + ); + this->This()->load(size); + if(sizeof(long) != size) + boost::serialization::throw_exception( + archive_exception( + archive_exception::incompatible_native_format, + "size of long" + ) + ); + this->This()->load(size); + if(sizeof(float) != size) + boost::serialization::throw_exception( + archive_exception( + archive_exception::incompatible_native_format, + "size of float" + ) + ); + this->This()->load(size); + if(sizeof(double) != size) + boost::serialization::throw_exception( + archive_exception( + archive_exception::incompatible_native_format, + "size of double" + ) + ); + + // for checking endian + int i; + this->This()->load(i); + if(1 != i) + boost::serialization::throw_exception( + archive_exception( + archive_exception::incompatible_native_format, + "endian setting" + ) + ); +} + +#ifndef BOOST_NO_CWCHAR +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_binary_iprimitive::load(wchar_t * ws) +{ + std::size_t l; // number of wchar_t !!! + this->This()->load(l); + load_binary(ws, l * sizeof(wchar_t) / sizeof(char)); + ws[l] = L'\0'; +} +#endif + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_binary_iprimitive::load(std::string & s) +{ + std::size_t l; + this->This()->load(l); + // borland de-allocator fixup + #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) + if(NULL != s.data()) + #endif + s.resize(l); + // note breaking a rule here - could be a problem on some platform + if(0 < l) + load_binary(&(*s.begin()), l); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_binary_iprimitive::load(char * s) +{ + std::size_t l; + this->This()->load(l); + load_binary(s, l); + s[l] = '\0'; +} + +#ifndef BOOST_NO_STD_WSTRING +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_binary_iprimitive::load(std::wstring & ws) +{ + std::size_t l; + this->This()->load(l); + // borland de-allocator fixup + #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) + if(NULL != ws.data()) + #endif + ws.resize(l); + // note breaking a rule here - is could be a problem on some platform + load_binary(const_cast(ws.data()), l * sizeof(wchar_t) / sizeof(char)); +} +#endif + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL +basic_binary_iprimitive::basic_binary_iprimitive( + std::basic_streambuf & sb, + bool no_codecvt +) : +#ifndef BOOST_NO_STD_LOCALE + m_sb(sb), + codecvt_null_facet(1), + locale_saver(m_sb), + archive_locale(sb.getloc(), & codecvt_null_facet) +{ + if(! no_codecvt){ + m_sb.pubsync(); + m_sb.pubimbue(archive_locale); + } +} +#else + m_sb(sb) +{} +#endif + +// scoped_ptr requires that g be a complete type at time of +// destruction so define destructor here rather than in the header +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL +basic_binary_iprimitive::~basic_binary_iprimitive(){} + +} // namespace archive +} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oarchive.ipp new file mode 100644 index 00000000000..d5a019d32bc --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oarchive.ipp @@ -0,0 +1,42 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_binary_oarchive.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. +#include +#include +#include +#include + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::memcpy; +} +#endif + +#include + +namespace boost { +namespace archive { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// implementation of binary_binary_oarchive + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_binary_oarchive::init(){ + // write signature in an archive version independent manner + const std::string file_signature(BOOST_ARCHIVE_SIGNATURE()); + * this->This() << file_signature; + // write library version + const library_version_type v(BOOST_ARCHIVE_VERSION()); + * this->This() << v; +} + +} // namespace archive +} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oprimitive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oprimitive.ipp new file mode 100644 index 00000000000..7b042173a48 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oprimitive.ipp @@ -0,0 +1,126 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_binary_oprimitive.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include // NULL +#include + +#include + +#if defined(BOOST_NO_STDC_NAMESPACE) && ! defined(__LIBCOMO__) +namespace std{ + using ::strlen; +} // namespace std +#endif + +#ifndef BOOST_NO_CWCHAR +#include +#ifdef BOOST_NO_STDC_NAMESPACE +namespace std{ using ::wcslen; } +#endif +#endif + +#include +#include + +namespace boost { +namespace archive { + +////////////////////////////////////////////////////////////////////// +// implementation of basic_binary_oprimitive + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_binary_oprimitive::init() +{ + // record native sizes of fundamental types + // this is to permit detection of attempts to pass + // native binary archives accross incompatible machines. + // This is not foolproof but its better than nothing. + this->This()->save(static_cast(sizeof(int))); + this->This()->save(static_cast(sizeof(long))); + this->This()->save(static_cast(sizeof(float))); + this->This()->save(static_cast(sizeof(double))); + // for checking endianness + this->This()->save(int(1)); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_binary_oprimitive::save(const char * s) +{ + std::size_t l = std::strlen(s); + this->This()->save(l); + save_binary(s, l); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_binary_oprimitive::save(const std::string &s) +{ + std::size_t l = static_cast(s.size()); + this->This()->save(l); + save_binary(s.data(), l); +} + +#ifndef BOOST_NO_CWCHAR +#ifndef BOOST_NO_INTRINSIC_WCHAR_T +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_binary_oprimitive::save(const wchar_t * ws) +{ + std::size_t l = std::wcslen(ws); + this->This()->save(l); + save_binary(ws, l * sizeof(wchar_t) / sizeof(char)); +} +#endif + +#ifndef BOOST_NO_STD_WSTRING +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_binary_oprimitive::save(const std::wstring &ws) +{ + std::size_t l = ws.size(); + this->This()->save(l); + save_binary(ws.data(), l * sizeof(wchar_t) / sizeof(char)); +} +#endif +#endif // BOOST_NO_CWCHAR + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL +basic_binary_oprimitive::basic_binary_oprimitive( + std::basic_streambuf & sb, + bool no_codecvt +) : +#ifndef BOOST_NO_STD_LOCALE + m_sb(sb), + codecvt_null_facet(1), + locale_saver(m_sb), + archive_locale(sb.getloc(), & codecvt_null_facet) +{ + if(! no_codecvt){ + m_sb.pubsync(); + m_sb.pubimbue(archive_locale); + } +} +#else + m_sb(sb) +{} +#endif + +// scoped_ptr requires that g be a complete type at time of +// destruction so define destructor here rather than in the header +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL +basic_binary_oprimitive::~basic_binary_oprimitive(){} + +} // namespace archive +} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iarchive.ipp new file mode 100644 index 00000000000..9ec8c6588c8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iarchive.ipp @@ -0,0 +1,76 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_text_iarchive.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. +#include +#include +#include + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::memcpy; +} +#endif + +#include +#include +#include + +namespace boost { +namespace archive { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// implementation of text_text_archive + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_text_iarchive::load_override(class_name_type & t){ + std::string cn; + cn.reserve(BOOST_SERIALIZATION_MAX_KEY_SIZE); + load_override(cn); + if(cn.size() > (BOOST_SERIALIZATION_MAX_KEY_SIZE - 1)) + boost::serialization::throw_exception( + archive_exception(archive_exception::invalid_class_name) + ); + std::memcpy(t, cn.data(), cn.size()); + // borland tweak + t.t[cn.size()] = '\0'; +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_text_iarchive::init(void){ + // read signature in an archive version independent manner + std::string file_signature; + * this->This() >> file_signature; + if(file_signature != BOOST_ARCHIVE_SIGNATURE()) + boost::serialization::throw_exception( + archive_exception(archive_exception::invalid_signature) + ); + + // make sure the version of the reading archive library can + // support the format of the archive being read + library_version_type input_library_version; + * this->This() >> input_library_version; + + #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) + this->set_library_version(input_library_version); + #else + detail::basic_iarchive::set_library_version(input_library_version); + #endif + + // extra little .t is to get around borland quirk + if(BOOST_ARCHIVE_VERSION() < input_library_version) + boost::serialization::throw_exception( + archive_exception(archive_exception::unsupported_version) + ); +} + +} // namespace archive +} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iprimitive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iprimitive.ipp new file mode 100644 index 00000000000..4e44728068d --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iprimitive.ipp @@ -0,0 +1,137 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_text_iprimitive.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // size_t, NULL +#include // NULL + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include + +#include + +#include +#include +#include +#include + +namespace boost { +namespace archive { + +namespace detail { + template + static inline bool is_whitespace(CharType c); + + template<> + inline bool is_whitespace(char t){ + return 0 != std::isspace(t); + } + + #ifndef BOOST_NO_CWCHAR + template<> + inline bool is_whitespace(wchar_t t){ + return 0 != std::iswspace(t); + } + #endif +} // detail + +// translate base64 text into binary and copy into buffer +// until buffer is full. +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_text_iprimitive::load_binary( + void *address, + std::size_t count +){ + typedef typename IStream::char_type CharType; + + if(0 == count) + return; + + BOOST_ASSERT( + static_cast((std::numeric_limits::max)()) + > (count + sizeof(CharType) - 1)/sizeof(CharType) + ); + + if(is.fail()) + boost::serialization::throw_exception( + archive_exception(archive_exception::input_stream_error) + ); + // convert from base64 to binary + typedef typename + iterators::transform_width< + iterators::binary_from_base64< + iterators::remove_whitespace< + iterators::istream_iterator + > + ,typename IStream::int_type + > + ,8 + ,6 + ,CharType + > + binary; + + binary i = binary(iterators::istream_iterator(is)); + + char * caddr = static_cast(address); + + // take care that we don't increment anymore than necessary + while(count-- > 0){ + *caddr++ = static_cast(*i++); + } + + // skip over any excess input + for(;;){ + typename IStream::int_type r; + r = is.get(); + if(is.eof()) + break; + if(detail::is_whitespace(static_cast(r))) + break; + } +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL +basic_text_iprimitive::basic_text_iprimitive( + IStream &is_, + bool no_codecvt +) : + is(is_), + flags_saver(is_), + precision_saver(is_), +#ifndef BOOST_NO_STD_LOCALE + codecvt_null_facet(1), + archive_locale(is.getloc(), & codecvt_null_facet), + locale_saver(is) +{ + if(! no_codecvt){ + is_.sync(); + is_.imbue(archive_locale); + } + is_ >> std::noboolalpha; +} +#else +{} +#endif + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL +basic_text_iprimitive::~basic_text_iprimitive(){ +} + +} // namespace archive +} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oarchive.ipp new file mode 100644 index 00000000000..44bc1401fd6 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oarchive.ipp @@ -0,0 +1,62 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_text_oarchive.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. +#include +#include +#include + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::memcpy; +} +#endif + +#include + +namespace boost { +namespace archive { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// implementation of basic_text_oarchive + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_text_oarchive::newtoken() +{ + switch(delimiter){ + default: + BOOST_ASSERT(false); + break; + case eol: + this->This()->put('\n'); + delimiter = space; + break; + case space: + this->This()->put(' '); + break; + case none: + delimiter = space; + break; + } +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_text_oarchive::init(){ + // write signature in an archive version independent manner + const std::string file_signature(BOOST_ARCHIVE_SIGNATURE()); + * this->This() << file_signature; + // write library version + const library_version_type v(BOOST_ARCHIVE_VERSION()); + * this->This() << v; +} + +} // namespace archive +} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oprimitive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oprimitive.ipp new file mode 100644 index 00000000000..6030fd44c57 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oprimitive.ipp @@ -0,0 +1,115 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_text_oprimitive.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // NULL +#include // std::copy +#include // std::uncaught_exception +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include + +#include +#include +#include +#include + +namespace boost { +namespace archive { + +// translate to base64 and copy in to buffer. +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_text_oprimitive::save_binary( + const void *address, + std::size_t count +){ + typedef typename OStream::char_type CharType; + + if(0 == count) + return; + + if(os.fail()) + boost::serialization::throw_exception( + archive_exception(archive_exception::output_stream_error) + ); + + os.put('\n'); + + typedef + boost::archive::iterators::insert_linebreaks< + boost::archive::iterators::base64_from_binary< + boost::archive::iterators::transform_width< + const char *, + 6, + 8 + > + > + ,76 + ,const char // cwpro8 needs this + > + base64_text; + + boost::archive::iterators::ostream_iterator oi(os); + std::copy( + base64_text(static_cast(address)), + base64_text( + static_cast(address) + count + ), + oi + ); + + std::size_t tail = count % 3; + if(tail > 0){ + *oi++ = '='; + if(tail < 2) + *oi = '='; + } +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL +basic_text_oprimitive::basic_text_oprimitive( + OStream & os_, + bool no_codecvt +) : + os(os_), + flags_saver(os_), + precision_saver(os_), +#ifndef BOOST_NO_STD_LOCALE + codecvt_null_facet(1), + archive_locale(os.getloc(), & codecvt_null_facet), + locale_saver(os) +{ + if(! no_codecvt){ + os_.flush(); + os_.imbue(archive_locale); + } + os_ << std::noboolalpha; +} +#else +{} +#endif + + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL +basic_text_oprimitive::~basic_text_oprimitive(){ + if(std::uncaught_exception()) + return; + os << std::endl; +} + +} //namespace boost +} //namespace archive diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_grammar.hpp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_grammar.hpp new file mode 100644 index 00000000000..6d4e4683f6a --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_grammar.hpp @@ -0,0 +1,173 @@ +#ifndef BOOST_ARCHIVE_BASIC_XML_GRAMMAR_HPP +#define BOOST_ARCHIVE_BASIC_XML_GRAMMAR_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_xml_grammar.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// this module is derived from simplexml.cpp - an example shipped as part of +// the spirit parser. This example contains the following notice: +/*============================================================================= + simplexml.cpp + + Spirit V1.3 + URL: http://spirit.sourceforge.net/ + + Copyright (c) 2001, Daniel C. Nuffer + + This software is provided 'as-is', without any express or implied + warranty. In no event will the copyright holder be held liable for + any damages arising from the use of this software. + + Permission is granted to anyone to use this software for any purpose, + including commercial applications, and to alter it and redistribute + it freely, subject to the following restrictions: + + 1. The origin of this software must not be misrepresented; you must + not claim that you wrote the original software. If you use this + software in a product, an acknowledgment in the product documentation + would be appreciated but is not required. + + 2. Altered source versions must be plainly marked as such, and must + not be misrepresented as being the original software. + + 3. This notice may not be removed or altered from any source + distribution. +=============================================================================*/ +#include + +#include +#include + +#include +#include + +#include +#include +#include + +namespace boost { +namespace archive { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// XML grammar parsing + +template +class basic_xml_grammar { +public: + // The following is not necessary according to DR45, but at least + // one compiler (Compaq C++ 6.5 in strict_ansi mode) chokes otherwise. + struct return_values; + friend struct return_values; + +private: + typedef typename std::basic_istream IStream; + typedef typename std::basic_string StringType; + typedef typename boost::spirit::classic::chset chset_t; + typedef typename boost::spirit::classic::chlit chlit_t; + typedef typename boost::spirit::classic::scanner< + typename std::basic_string::iterator + > scanner_t; + typedef typename boost::spirit::classic::rule rule_t; + // Start grammar definition + rule_t + Reference, + Eq, + STag, + ETag, + LetterOrUnderscoreOrColon, + AttValue, + CharRef1, + CharRef2, + CharRef, + AmpRef, + LTRef, + GTRef, + AposRef, + QuoteRef, + CharData, + CharDataChars, + content, + AmpName, + LTName, + GTName, + ClassNameChar, + ClassName, + Name, + XMLDecl, + XMLDeclChars, + DocTypeDecl, + DocTypeDeclChars, + ClassIDAttribute, + ObjectIDAttribute, + ClassNameAttribute, + TrackingAttribute, + VersionAttribute, + UnusedAttribute, + Attribute, + SignatureAttribute, + SerializationWrapper, + NameHead, + NameTail, + AttributeList, + S; + + // XML Character classes + chset_t + BaseChar, + Ideographic, + Char, + Letter, + Digit, + CombiningChar, + Extender, + Sch, + NameChar; + + void init_chset(); + + bool my_parse( + IStream & is, + const rule_t &rule_, + const CharType delimiter = L'>' + ) const ; +public: + struct return_values { + StringType object_name; + StringType contents; + //class_id_type class_id; + int_least16_t class_id; + //object_id_type object_id; + uint_least32_t object_id; + //version_type version; + unsigned int version; + tracking_type tracking_level; + StringType class_name; + return_values() : + version(0), + tracking_level(false) + {} + } rv; + bool parse_start_tag(IStream & is) /*const*/; + bool parse_end_tag(IStream & is) const; + bool parse_string(IStream & is, StringType & s) /*const*/; + void init(IStream & is); + bool windup(IStream & is); + basic_xml_grammar(); +}; + +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_BASIC_XML_GRAMMAR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_iarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_iarchive.ipp new file mode 100644 index 00000000000..625458b9eb5 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_iarchive.ipp @@ -0,0 +1,115 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_xml_iarchive.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include // NULL +#include + +#include +#include +#include +#include + +namespace boost { +namespace archive { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// implementation of xml_text_archive + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_iarchive::load_start(const char *name){ + // if there's no name + if(NULL == name) + return; + bool result = this->This()->gimpl->parse_start_tag(this->This()->get_is()); + if(true != result){ + boost::serialization::throw_exception( + archive_exception(archive_exception::input_stream_error) + ); + } + // don't check start tag at highest level + ++depth; + return; +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_iarchive::load_end(const char *name){ + // if there's no name + if(NULL == name) + return; + bool result = this->This()->gimpl->parse_end_tag(this->This()->get_is()); + if(true != result){ + boost::serialization::throw_exception( + archive_exception(archive_exception::input_stream_error) + ); + } + + // don't check start tag at highest level + if(0 == --depth) + return; + + if(0 == (this->get_flags() & no_xml_tag_checking)){ + // double check that the tag matches what is expected - useful for debug + if(0 != name[this->This()->gimpl->rv.object_name.size()] + || ! std::equal( + this->This()->gimpl->rv.object_name.begin(), + this->This()->gimpl->rv.object_name.end(), + name + ) + ){ + boost::serialization::throw_exception( + xml_archive_exception( + xml_archive_exception::xml_archive_tag_mismatch, + name + ) + ); + } + } +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_iarchive::load_override(object_id_type & t){ + t = object_id_type(this->This()->gimpl->rv.object_id); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_iarchive::load_override(version_type & t){ + t = version_type(this->This()->gimpl->rv.version); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_iarchive::load_override(class_id_type & t){ + t = class_id_type(this->This()->gimpl->rv.class_id); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_iarchive::load_override(tracking_type & t){ + t = this->This()->gimpl->rv.tracking_level; +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL +basic_xml_iarchive::basic_xml_iarchive(unsigned int flags) : + detail::common_iarchive(flags), + depth(0) +{} +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL +basic_xml_iarchive::~basic_xml_iarchive(){ +} + +} // namespace archive +} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_oarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_oarchive.ipp new file mode 100644 index 00000000000..3184413f382 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_oarchive.ipp @@ -0,0 +1,272 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// basic_xml_oarchive.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include // NULL +#include +#if defined(BOOST_NO_STDC_NAMESPACE) && ! defined(__LIBCOMO__) +namespace std{ + using ::strlen; +} // namespace std +#endif + +#include +#include +#include +#include + +namespace boost { +namespace archive { + +namespace detail { +template +struct XML_name { + void operator()(CharType t) const{ + const unsigned char lookup_table[] = { + 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, + 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, + 0,0,0,0,0,0,0,0,0,0,0,0,0,1,1,0, // -. + 1,1,1,1,1,1,1,1,1,1,0,0,0,0,0,0, // 0-9 + 0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1, // A- + 1,1,1,1,1,1,1,1,1,1,1,0,0,0,0,1, // -Z _ + 0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1, // a- + 1,1,1,1,1,1,1,1,1,1,1,0,0,0,0,0, // -z + 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, + 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 + }; + if((unsigned)t > 127) + return; + if(0 == lookup_table[(unsigned)t]) + boost::serialization::throw_exception( + xml_archive_exception( + xml_archive_exception::xml_archive_tag_name_error + ) + ); + } +}; + +} // namespace detail + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// implemenations of functions common to both types of xml output + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::write_attribute( + const char *attribute_name, + int t, + const char *conjunction +){ + this->This()->put(' '); + this->This()->put(attribute_name); + this->This()->put(conjunction); + this->This()->save(t); + this->This()->put('"'); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::write_attribute( + const char *attribute_name, + const char *key +){ + this->This()->put(' '); + this->This()->put(attribute_name); + this->This()->put("=\""); + this->This()->save(key); + this->This()->put('"'); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::indent(){ + int i; + for(i = depth; i-- > 0;) + this->This()->put('\t'); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::save_start(const char *name) +{ + if(NULL == name) + return; + + // be sure name has no invalid characters + std::for_each(name, name + std::strlen(name), detail::XML_name()); + + end_preamble(); + if(depth > 0){ + this->This()->put('\n'); + indent(); + } + ++depth; + this->This()->put('<'); + this->This()->save(name); + pending_preamble = true; + indent_next = false; +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::save_end(const char *name) +{ + if(NULL == name) + return; + + // be sure name has no invalid characters + std::for_each(name, name + std::strlen(name), detail::XML_name()); + + end_preamble(); + --depth; + if(indent_next){ + this->This()->put('\n'); + indent(); + } + indent_next = true; + this->This()->put("This()->save(name); + this->This()->put('>'); + if(0 == depth) + this->This()->put('\n'); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::end_preamble(){ + if(pending_preamble){ + this->This()->put('>'); + pending_preamble = false; + } +} +#if 0 +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::save_override(const object_id_type & t) +{ + int i = t.t; // extra .t is for borland + write_attribute(BOOST_ARCHIVE_XML_OBJECT_ID(), i, "=\"_"); +} +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::save_override( + const object_reference_type & t, + int +){ + int i = t.t; // extra .t is for borland + write_attribute(BOOST_ARCHIVE_XML_OBJECT_REFERENCE(), i, "=\"_"); +} +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::save_override(const version_type & t) +{ + int i = t.t; // extra .t is for borland + write_attribute(BOOST_ARCHIVE_XML_VERSION(), i); +} +#endif + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::save_override(const object_id_type & t) +{ + // borland doesn't do conversion of STRONG_TYPEDEFs very well + const unsigned int i = t; + write_attribute(BOOST_ARCHIVE_XML_OBJECT_ID(), i, "=\"_"); +} +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::save_override( + const object_reference_type & t +){ + const unsigned int i = t; + write_attribute(BOOST_ARCHIVE_XML_OBJECT_REFERENCE(), i, "=\"_"); +} +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::save_override(const version_type & t) +{ + const unsigned int i = t; + write_attribute(BOOST_ARCHIVE_XML_VERSION(), i); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::save_override(const class_id_type & t) +{ + write_attribute(BOOST_ARCHIVE_XML_CLASS_ID(), t); +} +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::save_override( + const class_id_reference_type & t +){ + write_attribute(BOOST_ARCHIVE_XML_CLASS_ID_REFERENCE(), t); +} +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::save_override( + const class_id_optional_type & t +){ + write_attribute(BOOST_ARCHIVE_XML_CLASS_ID(), t); +} +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::save_override(const class_name_type & t) +{ + const char * key = t; + if(NULL == key) + return; + write_attribute(BOOST_ARCHIVE_XML_CLASS_NAME(), key); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::save_override(const tracking_type & t) +{ + write_attribute(BOOST_ARCHIVE_XML_TRACKING(), t.t); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::init(){ + // xml header + this->This()->put("\n"); + this->This()->put("\n"); + // xml document wrapper - outer root + this->This()->put("This()->put(">\n"); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL void +basic_xml_oarchive::windup(){ + // xml_trailer + this->This()->put("\n"); +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL +basic_xml_oarchive::basic_xml_oarchive(unsigned int flags) : + detail::common_oarchive(flags), + depth(0), + pending_preamble(false), + indent_next(false) +{ +} + +template +BOOST_ARCHIVE_OR_WARCHIVE_DECL +basic_xml_oarchive::~basic_xml_oarchive(){ +} + +} // namespace archive +} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_iarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_iarchive_impl.ipp new file mode 100644 index 00000000000..ae4e2750ce8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_iarchive_impl.ipp @@ -0,0 +1,128 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// text_iarchive_impl.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +////////////////////////////////////////////////////////////////////// +// implementation of basic_text_iprimitive overrides for the combination +// of template parameters used to implement a text_iprimitive + +#include // size_t, NULL +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include // RogueWave + +#include + +namespace boost { +namespace archive { + +template +BOOST_ARCHIVE_DECL void +text_iarchive_impl::load(char *s) +{ + std::size_t size; + * this->This() >> size; + // skip separating space + is.get(); + // Works on all tested platforms + is.read(s, size); + s[size] = '\0'; +} + +template +BOOST_ARCHIVE_DECL void +text_iarchive_impl::load(std::string &s) +{ + std::size_t size; + * this->This() >> size; + // skip separating space + is.get(); + // borland de-allocator fixup + #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) + if(NULL != s.data()) + #endif + s.resize(size); + if(0 < size) + is.read(&(*s.begin()), size); +} + +#ifndef BOOST_NO_CWCHAR +#ifndef BOOST_NO_INTRINSIC_WCHAR_T +template +BOOST_ARCHIVE_DECL void +text_iarchive_impl::load(wchar_t *ws) +{ + std::size_t size; + * this->This() >> size; + // skip separating space + is.get(); + is.read((char *)ws, size * sizeof(wchar_t)/sizeof(char)); + ws[size] = L'\0'; +} +#endif // BOOST_NO_INTRINSIC_WCHAR_T + +#ifndef BOOST_NO_STD_WSTRING +template +BOOST_ARCHIVE_DECL void +text_iarchive_impl::load(std::wstring &ws) +{ + std::size_t size; + * this->This() >> size; + // borland de-allocator fixup + #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) + if(NULL != ws.data()) + #endif + ws.resize(size); + // skip separating space + is.get(); + is.read((char *)ws.data(), size * sizeof(wchar_t)/sizeof(char)); +} + +#endif // BOOST_NO_STD_WSTRING +#endif // BOOST_NO_CWCHAR + +template +BOOST_ARCHIVE_DECL void +text_iarchive_impl::load_override(class_name_type & t){ + basic_text_iarchive::load_override(t); +} + +template +BOOST_ARCHIVE_DECL void +text_iarchive_impl::init(){ + basic_text_iarchive::init(); +} + +template +BOOST_ARCHIVE_DECL +text_iarchive_impl::text_iarchive_impl( + std::istream & is, + unsigned int flags +) : + basic_text_iprimitive( + is, + 0 != (flags & no_codecvt) + ), + basic_text_iarchive(flags) +{ + if(0 == (flags & no_header)) + #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) + this->init(); + #else + this->basic_text_iarchive::init(); + #endif +} + +} // namespace archive +} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_oarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_oarchive_impl.ipp new file mode 100644 index 00000000000..37d8664a98c --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_oarchive_impl.ipp @@ -0,0 +1,122 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// text_oarchive_impl.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include // size_t + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#ifndef BOOST_NO_CWCHAR +#include +#ifdef BOOST_NO_STDC_NAMESPACE +namespace std{ using ::wcslen; } +#endif +#endif + +#include + +namespace boost { +namespace archive { + +////////////////////////////////////////////////////////////////////// +// implementation of basic_text_oprimitive overrides for the combination +// of template parameters used to create a text_oprimitive + +template +BOOST_ARCHIVE_DECL void +text_oarchive_impl::save(const char * s) +{ + const std::size_t len = std::ostream::traits_type::length(s); + *this->This() << len; + this->This()->newtoken(); + os << s; +} + +template +BOOST_ARCHIVE_DECL void +text_oarchive_impl::save(const std::string &s) +{ + const std::size_t size = s.size(); + *this->This() << size; + this->This()->newtoken(); + os << s; +} + +#ifndef BOOST_NO_CWCHAR +#ifndef BOOST_NO_INTRINSIC_WCHAR_T +template +BOOST_ARCHIVE_DECL void +text_oarchive_impl::save(const wchar_t * ws) +{ + const std::size_t l = std::wcslen(ws); + * this->This() << l; + this->This()->newtoken(); + os.write((const char *)ws, l * sizeof(wchar_t)/sizeof(char)); +} +#endif + +#ifndef BOOST_NO_STD_WSTRING +template +BOOST_ARCHIVE_DECL void +text_oarchive_impl::save(const std::wstring &ws) +{ + const std::size_t l = ws.size(); + * this->This() << l; + this->This()->newtoken(); + os.write((const char *)(ws.data()), l * sizeof(wchar_t)/sizeof(char)); +} +#endif +#endif // BOOST_NO_CWCHAR + +template +BOOST_ARCHIVE_DECL +text_oarchive_impl::text_oarchive_impl( + std::ostream & os, + unsigned int flags +) : + basic_text_oprimitive( + os, + 0 != (flags & no_codecvt) + ), + basic_text_oarchive(flags) +{ + if(0 == (flags & no_header)) + #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) + this->init(); + #else + this->basic_text_oarchive::init(); + #endif +} + +template +BOOST_ARCHIVE_DECL void +text_oarchive_impl::save_binary(const void *address, std::size_t count){ + put('\n'); + this->end_preamble(); + #if ! defined(__MWERKS__) + this->basic_text_oprimitive::save_binary( + #else + this->basic_text_oprimitive::save_binary( + #endif + address, + count + ); + this->delimiter = this->eol; +} + +} // namespace archive +} // namespace boost + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_wiarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_wiarchive_impl.ipp new file mode 100644 index 00000000000..e85625ac326 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_wiarchive_impl.ipp @@ -0,0 +1,118 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// text_text_wiarchive_impl.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // size_t, NULL + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include // fixup for RogueWave + +#ifndef BOOST_NO_STD_WSTREAMBUF +#include + +namespace boost { +namespace archive { + +////////////////////////////////////////////////////////////////////// +// implementation of wiprimtives functions +// +template +BOOST_WARCHIVE_DECL void +text_wiarchive_impl::load(char *s) +{ + std::size_t size; + * this->This() >> size; + // skip separating space + is.get(); + while(size-- > 0){ + *s++ = is.narrow(is.get(), '\0'); + } + *s = '\0'; +} + +template +BOOST_WARCHIVE_DECL void +text_wiarchive_impl::load(std::string &s) +{ + std::size_t size; + * this->This() >> size; + // skip separating space + is.get(); + #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) + if(NULL != s.data()) + #endif + s.resize(0); + s.reserve(size); + while(size-- > 0){ + char x = is.narrow(is.get(), '\0'); + s += x; + } +} + +#ifndef BOOST_NO_INTRINSIC_WCHAR_T +template +BOOST_WARCHIVE_DECL void +text_wiarchive_impl::load(wchar_t *s) +{ + std::size_t size; + * this->This() >> size; + // skip separating space + is.get(); + // Works on all tested platforms + is.read(s, size); + s[size] = L'\0'; +} +#endif + +#ifndef BOOST_NO_STD_WSTRING +template +BOOST_WARCHIVE_DECL void +text_wiarchive_impl::load(std::wstring &ws) +{ + std::size_t size; + * this->This() >> size; + // skip separating space + is.get(); + // borland complains about resize + // borland de-allocator fixup + #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) + if(NULL != ws.data()) + #endif + ws.resize(size); + // note breaking a rule here - is this a problem on some platform + is.read(const_cast(ws.data()), size); +} +#endif + +template +BOOST_WARCHIVE_DECL +text_wiarchive_impl::text_wiarchive_impl( + std::wistream & is, + unsigned int flags +) : + basic_text_iprimitive( + is, + 0 != (flags & no_codecvt) + ), + basic_text_iarchive(flags) +{ + if(0 == (flags & no_header)) + basic_text_iarchive::init(); +} + +} // archive +} // boost + +#endif // BOOST_NO_STD_WSTREAMBUF diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_woarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_woarchive_impl.ipp new file mode 100644 index 00000000000..2b6d427cd3a --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_woarchive_impl.ipp @@ -0,0 +1,85 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// text_woarchive_impl.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifndef BOOST_NO_STD_WSTREAMBUF + +#include +#include // size_t +#if defined(BOOST_NO_STDC_NAMESPACE) && ! defined(__LIBCOMO__) +namespace std{ + using ::strlen; + using ::size_t; +} // namespace std +#endif + +#include + +#include + +namespace boost { +namespace archive { + +////////////////////////////////////////////////////////////////////// +// implementation of woarchive functions +// +template +BOOST_WARCHIVE_DECL void +text_woarchive_impl::save(const char *s) +{ + // note: superfluous local variable fixes borland warning + const std::size_t size = std::strlen(s); + * this->This() << size; + this->This()->newtoken(); + while(*s != '\0') + os.put(os.widen(*s++)); +} + +template +BOOST_WARCHIVE_DECL void +text_woarchive_impl::save(const std::string &s) +{ + const std::size_t size = s.size(); + * this->This() << size; + this->This()->newtoken(); + const char * cptr = s.data(); + for(std::size_t i = size; i-- > 0;) + os.put(os.widen(*cptr++)); +} + +#ifndef BOOST_NO_INTRINSIC_WCHAR_T +template +BOOST_WARCHIVE_DECL void +text_woarchive_impl::save(const wchar_t *ws) +{ + const std::size_t size = std::wostream::traits_type::length(ws); + * this->This() << size; + this->This()->newtoken(); + os.write(ws, size); +} +#endif + +#ifndef BOOST_NO_STD_WSTRING +template +BOOST_WARCHIVE_DECL void +text_woarchive_impl::save(const std::wstring &ws) +{ + const std::size_t size = ws.length(); + * this->This() << size; + this->This()->newtoken(); + os.write(ws.data(), size); +} +#endif + +} // namespace archive +} // namespace boost + +#endif + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_iarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_iarchive_impl.ipp new file mode 100644 index 00000000000..efc32e01632 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_iarchive_impl.ipp @@ -0,0 +1,199 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// xml_iarchive_impl.cpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include // memcpy +#include // NULL +#include + +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::memcpy; +} // namespace std +#endif + +#ifndef BOOST_NO_CWCHAR +#include // mbstate_t and mbrtowc +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::mbstate_t; + using ::mbrtowc; + } // namespace std +#endif +#endif // BOOST_NO_CWCHAR + +#include // RogueWave and Dinkumware +#if BOOST_WORKAROUND(BOOST_DINKUMWARE_STDLIB, == 1) +#include +#endif + +#include + +#include +#include +#include +#include + +#include "basic_xml_grammar.hpp" + +namespace boost { +namespace archive { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// implemenations of functions specific to char archives + +// wide char stuff used by char archives + +#ifndef BOOST_NO_CWCHAR +#ifndef BOOST_NO_STD_WSTRING +template +BOOST_ARCHIVE_DECL void +xml_iarchive_impl::load(std::wstring &ws){ + std::string s; + bool result = gimpl->parse_string(is, s); + if(! result) + boost::serialization::throw_exception( + xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) + ); + + #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) + if(NULL != ws.data()) + #endif + ws.resize(0); + std::mbstate_t mbs = std::mbstate_t(); + const char * start = s.data(); + const char * end = start + s.size(); + while(start < end){ + wchar_t wc; + std::size_t count = std::mbrtowc(&wc, start, end - start, &mbs); + if(count == static_cast(-1)) + boost::serialization::throw_exception( + iterators::dataflow_exception( + iterators::dataflow_exception::invalid_conversion + ) + ); + if(count == static_cast(-2)) + continue; + start += count; + ws += wc; + } +} +#endif // BOOST_NO_STD_WSTRING + +#ifndef BOOST_NO_INTRINSIC_WCHAR_T +template +BOOST_ARCHIVE_DECL void +xml_iarchive_impl::load(wchar_t * ws){ + std::string s; + bool result = gimpl->parse_string(is, s); + if(! result) + boost::serialization::throw_exception( + xml_archive_exception( + xml_archive_exception::xml_archive_parsing_error + ) + ); + + std::mbstate_t mbs = std::mbstate_t(); + const char * start = s.data(); + const char * end = start + s.size(); + while(start < end){ + wchar_t wc; + std::size_t length = std::mbrtowc(&wc, start, end - start, &mbs); + if(static_cast(-1) == length) + boost::serialization::throw_exception( + iterators::dataflow_exception( + iterators::dataflow_exception::invalid_conversion + ) + ); + if(static_cast(-2) == length) + continue; + + start += length; + *ws++ = wc; + } + *ws = L'\0'; +} +#endif // BOOST_NO_INTRINSIC_WCHAR_T + +#endif // BOOST_NO_CWCHAR + +template +BOOST_ARCHIVE_DECL void +xml_iarchive_impl::load(std::string &s){ + bool result = gimpl->parse_string(is, s); + if(! result) + boost::serialization::throw_exception( + xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) + ); +} + +template +BOOST_ARCHIVE_DECL void +xml_iarchive_impl::load(char * s){ + std::string tstring; + bool result = gimpl->parse_string(is, tstring); + if(! result) + boost::serialization::throw_exception( + xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) + ); + std::memcpy(s, tstring.data(), tstring.size()); + s[tstring.size()] = 0; +} + +template +BOOST_ARCHIVE_DECL void +xml_iarchive_impl::load_override(class_name_type & t){ + const std::string & s = gimpl->rv.class_name; + if(s.size() > BOOST_SERIALIZATION_MAX_KEY_SIZE - 1) + boost::serialization::throw_exception( + archive_exception(archive_exception::invalid_class_name) + ); + char * tptr = t; + std::memcpy(tptr, s.data(), s.size()); + tptr[s.size()] = '\0'; +} + +template +BOOST_ARCHIVE_DECL void +xml_iarchive_impl::init(){ + gimpl->init(is); + this->set_library_version( + library_version_type(gimpl->rv.version) + ); +} + +template +BOOST_ARCHIVE_DECL +xml_iarchive_impl::xml_iarchive_impl( + std::istream &is_, + unsigned int flags +) : + basic_text_iprimitive( + is_, + 0 != (flags & no_codecvt) + ), + basic_xml_iarchive(flags), + gimpl(new xml_grammar()) +{ + if(0 == (flags & no_header)) + init(); +} + +template +BOOST_ARCHIVE_DECL +xml_iarchive_impl::~xml_iarchive_impl(){ + if(std::uncaught_exception()) + return; + if(0 == (this->get_flags() & no_header)){ + gimpl->windup(is); + } +} +} // namespace archive +} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_oarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_oarchive_impl.ipp new file mode 100644 index 00000000000..5ebd454e722 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_oarchive_impl.ipp @@ -0,0 +1,142 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// xml_oarchive_impl.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +#include +#include +#include // std::copy +#include +#include + +#include // strlen +#include // msvc 6.0 needs this to suppress warnings +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::strlen; +} // namespace std +#endif + +#include +#include + +#ifndef BOOST_NO_CWCHAR +#include +#include +#endif + +namespace boost { +namespace archive { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// implemenations of functions specific to char archives + +// wide char stuff used by char archives +#ifndef BOOST_NO_CWCHAR +// copy chars to output escaping to xml and translating wide chars to mb chars +template +void save_iterator(std::ostream &os, InputIterator begin, InputIterator end){ + typedef boost::archive::iterators::mb_from_wchar< + boost::archive::iterators::xml_escape + > translator; + std::copy( + translator(begin), + translator(end), + boost::archive::iterators::ostream_iterator(os) + ); +} + +#ifndef BOOST_NO_STD_WSTRING +template +BOOST_ARCHIVE_DECL void +xml_oarchive_impl::save(const std::wstring & ws){ +// at least one library doesn't typedef value_type for strings +// so rather than using string directly make a pointer iterator out of it +// save_iterator(os, ws.data(), ws.data() + std::wcslen(ws.data())); + save_iterator(os, ws.data(), ws.data() + ws.size()); +} +#endif + +#ifndef BOOST_NO_INTRINSIC_WCHAR_T +template +BOOST_ARCHIVE_DECL void +xml_oarchive_impl::save(const wchar_t * ws){ + save_iterator(os, ws, ws + std::wcslen(ws)); +} +#endif + +#endif // BOOST_NO_CWCHAR + +template +BOOST_ARCHIVE_DECL void +xml_oarchive_impl::save(const std::string & s){ +// at least one library doesn't typedef value_type for strings +// so rather than using string directly make a pointer iterator out of it + typedef boost::archive::iterators::xml_escape< + const char * + > xml_escape_translator; + std::copy( + xml_escape_translator(s.data()), + xml_escape_translator(s.data()+ s.size()), + boost::archive::iterators::ostream_iterator(os) + ); +} + +template +BOOST_ARCHIVE_DECL void +xml_oarchive_impl::save(const char * s){ + typedef boost::archive::iterators::xml_escape< + const char * + > xml_escape_translator; + std::copy( + xml_escape_translator(s), + xml_escape_translator(s + std::strlen(s)), + boost::archive::iterators::ostream_iterator(os) + ); +} + +template +BOOST_ARCHIVE_DECL +xml_oarchive_impl::xml_oarchive_impl( + std::ostream & os_, + unsigned int flags +) : + basic_text_oprimitive( + os_, + 0 != (flags & no_codecvt) + ), + basic_xml_oarchive(flags) +{ + if(0 == (flags & no_header)) + this->init(); +} + +template +BOOST_ARCHIVE_DECL void +xml_oarchive_impl::save_binary(const void *address, std::size_t count){ + this->end_preamble(); + #if ! defined(__MWERKS__) + this->basic_text_oprimitive::save_binary( + #else + this->basic_text_oprimitive::save_binary( + #endif + address, + count + ); + this->indent_next = true; +} + +template +BOOST_ARCHIVE_DECL +xml_oarchive_impl::~xml_oarchive_impl(){ + if(std::uncaught_exception()) + return; + if(0 == (this->get_flags() & no_header)) + this->windup(); +} + +} // namespace archive +} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_wiarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_wiarchive_impl.ipp new file mode 100644 index 00000000000..ee66c1263e6 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_wiarchive_impl.ipp @@ -0,0 +1,189 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// xml_wiarchive_impl.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::memcpy; +} //std +#endif + +#include // msvc 6.0 needs this to suppress warnings +#ifndef BOOST_NO_STD_WSTREAMBUF + +#include +#include // std::copy +#include // uncaught exception +#include // Dinkumware and RogueWave +#if BOOST_WORKAROUND(BOOST_DINKUMWARE_STDLIB, == 1) +#include +#endif + +#include +#include +#include + +#include +#include + +#include +#include + +#include + +#include "basic_xml_grammar.hpp" + +namespace boost { +namespace archive { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// implemenations of functions specific to wide char archives + +namespace { // anonymous + +void copy_to_ptr(char * s, const std::wstring & ws){ + std::copy( + iterators::mb_from_wchar( + ws.begin() + ), + iterators::mb_from_wchar( + ws.end() + ), + s + ); + s[ws.size()] = 0; +} + +} // anonymous + +template +BOOST_WARCHIVE_DECL void +xml_wiarchive_impl::load(std::string & s){ + std::wstring ws; + bool result = gimpl->parse_string(is, ws); + if(! result) + boost::serialization::throw_exception( + xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) + ); + #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) + if(NULL != s.data()) + #endif + s.resize(0); + s.reserve(ws.size()); + std::copy( + iterators::mb_from_wchar( + ws.begin() + ), + iterators::mb_from_wchar( + ws.end() + ), + std::back_inserter(s) + ); +} + +#ifndef BOOST_NO_STD_WSTRING +template +BOOST_WARCHIVE_DECL void +xml_wiarchive_impl::load(std::wstring & ws){ + bool result = gimpl->parse_string(is, ws); + if(! result) + boost::serialization::throw_exception( + xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) + ); +} +#endif + +template +BOOST_WARCHIVE_DECL void +xml_wiarchive_impl::load(char * s){ + std::wstring ws; + bool result = gimpl->parse_string(is, ws); + if(! result) + boost::serialization::throw_exception( + xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) + ); + copy_to_ptr(s, ws); +} + +#ifndef BOOST_NO_INTRINSIC_WCHAR_T +template +BOOST_WARCHIVE_DECL void +xml_wiarchive_impl::load(wchar_t * ws){ + std::wstring twstring; + bool result = gimpl->parse_string(is, twstring); + if(! result) + boost::serialization::throw_exception( + xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) + ); + std::memcpy(ws, twstring.c_str(), twstring.size()); + ws[twstring.size()] = L'\0'; +} +#endif + +template +BOOST_WARCHIVE_DECL void +xml_wiarchive_impl::load_override(class_name_type & t){ + const std::wstring & ws = gimpl->rv.class_name; + if(ws.size() > BOOST_SERIALIZATION_MAX_KEY_SIZE - 1) + boost::serialization::throw_exception( + archive_exception(archive_exception::invalid_class_name) + ); + copy_to_ptr(t, ws); +} + +template +BOOST_WARCHIVE_DECL void +xml_wiarchive_impl::init(){ + gimpl->init(is); + this->set_library_version( + library_version_type(gimpl->rv.version) + ); +} + +template +BOOST_WARCHIVE_DECL +xml_wiarchive_impl::xml_wiarchive_impl( + std::wistream &is_, + unsigned int flags +) : + basic_text_iprimitive( + is_, + true // don't change the codecvt - use the one below + ), + basic_xml_iarchive(flags), + gimpl(new xml_wgrammar()) +{ + if(0 == (flags & no_codecvt)){ + std::locale l = std::locale( + is_.getloc(), + new boost::archive::detail::utf8_codecvt_facet + ); + // libstdc++ crashes without this + is_.sync(); + is_.imbue(l); + } + if(0 == (flags & no_header)) + init(); +} + +template +BOOST_WARCHIVE_DECL +xml_wiarchive_impl::~xml_wiarchive_impl(){ + if(std::uncaught_exception()) + return; + if(0 == (this->get_flags() & no_header)){ + gimpl->windup(is); + } +} + +} // namespace archive +} // namespace boost + +#endif // BOOST_NO_STD_WSTREAMBUF diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_woarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_woarchive_impl.ipp new file mode 100644 index 00000000000..01b1a052d51 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_woarchive_impl.ipp @@ -0,0 +1,171 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// xml_woarchive_impl.ipp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +#include +#ifndef BOOST_NO_STD_WSTREAMBUF + +#include +#include +#include // std::copy +#include +#include + +#include // strlen +#include // mbtowc +#ifndef BOOST_NO_CWCHAR +#include // wcslen +#endif + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::strlen; + #if ! defined(BOOST_NO_INTRINSIC_WCHAR_T) + using ::mbtowc; + using ::wcslen; + #endif +} // namespace std +#endif + +#include +#include + +#include + +#include +#include +#include +#include + +namespace boost { +namespace archive { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// implemenations of functions specific to wide char archives + +// copy chars to output escaping to xml and widening characters as we go +template +void save_iterator(std::wostream &os, InputIterator begin, InputIterator end){ + typedef iterators::wchar_from_mb< + iterators::xml_escape + > xmbtows; + std::copy( + xmbtows(begin), + xmbtows(end), + boost::archive::iterators::ostream_iterator(os) + ); +} + +template +BOOST_WARCHIVE_DECL void +xml_woarchive_impl::save(const std::string & s){ + // note: we don't use s.begin() and s.end() because dinkumware + // doesn't have string::value_type defined. So use a wrapper + // around these values to implement the definitions. + const char * begin = s.data(); + const char * end = begin + s.size(); + save_iterator(os, begin, end); +} + +#ifndef BOOST_NO_STD_WSTRING +template +BOOST_WARCHIVE_DECL void +xml_woarchive_impl::save(const std::wstring & ws){ +#if 0 + typedef iterators::xml_escape xmbtows; + std::copy( + xmbtows(ws.begin()), + xmbtows(ws.end()), + boost::archive::iterators::ostream_iterator(os) + ); +#endif + typedef iterators::xml_escape xmbtows; + std::copy( + xmbtows(ws.data()), + xmbtows(ws.data() + ws.size()), + boost::archive::iterators::ostream_iterator(os) + ); +} +#endif //BOOST_NO_STD_WSTRING + +template +BOOST_WARCHIVE_DECL void +xml_woarchive_impl::save(const char * s){ + save_iterator(os, s, s + std::strlen(s)); +} + +#ifndef BOOST_NO_INTRINSIC_WCHAR_T +template +BOOST_WARCHIVE_DECL void +xml_woarchive_impl::save(const wchar_t * ws){ + os << ws; + typedef iterators::xml_escape xmbtows; + std::copy( + xmbtows(ws), + xmbtows(ws + std::wcslen(ws)), + boost::archive::iterators::ostream_iterator(os) + ); +} +#endif + +template +BOOST_WARCHIVE_DECL +xml_woarchive_impl::xml_woarchive_impl( + std::wostream & os_, + unsigned int flags +) : + basic_text_oprimitive( + os_, + true // don't change the codecvt - use the one below + ), + basic_xml_oarchive(flags) +{ + if(0 == (flags & no_codecvt)){ + std::locale l = std::locale( + os_.getloc(), + new boost::archive::detail::utf8_codecvt_facet + ); + os_.flush(); + os_.imbue(l); + } + if(0 == (flags & no_header)) + this->init(); +} + +template +BOOST_WARCHIVE_DECL +xml_woarchive_impl::~xml_woarchive_impl(){ + if(std::uncaught_exception()) + return; + if(0 == (this->get_flags() & no_header)){ + save(L"\n"); + } +} + +template +BOOST_WARCHIVE_DECL void +xml_woarchive_impl::save_binary( + const void *address, + std::size_t count +){ + this->end_preamble(); + #if ! defined(__MWERKS__) + this->basic_text_oprimitive::save_binary( + #else + this->basic_text_oprimitive::save_binary( + #endif + address, + count + ); + this->indent_next = true; +} + +} // namespace archive +} // namespace boost + +#endif //BOOST_NO_STD_WSTREAMBUF diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_exception.hpp new file mode 100644 index 00000000000..8f9208b60ea --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_exception.hpp @@ -0,0 +1,68 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_BASE64_EXCEPTION_HPP +#define BOOST_ARCHIVE_ITERATORS_BASE64_EXCEPTION_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// base64_exception.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifndef BOOST_NO_EXCEPTIONS +#include + +#include + +namespace boost { +namespace archive { +namespace iterators { + +////////////////////////////////////////////////////////////////////// +// exceptions thrown by base64s +// +class base64_exception : public std::exception +{ +public: + typedef enum { + invalid_code, // attempt to encode a value > 6 bits + invalid_character, // decode a value not in base64 char set + other_exception + } exception_code; + exception_code code; + + base64_exception(exception_code c = other_exception) : code(c) + {} + + virtual const char *what( ) const throw( ) + { + const char *msg = "unknown exception code"; + switch(code){ + case invalid_code: + msg = "attempt to encode a value > 6 bits"; + break; + case invalid_character: + msg = "attempt to decode a value not in base64 char set"; + break; + default: + BOOST_ASSERT(false); + break; + } + return msg; + } +}; + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif //BOOST_NO_EXCEPTIONS +#endif //BOOST_ARCHIVE_ITERATORS_ARCHIVE_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_from_binary.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_from_binary.hpp new file mode 100644 index 00000000000..ee849944397 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_from_binary.hpp @@ -0,0 +1,109 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_BASE64_FROM_BINARY_HPP +#define BOOST_ARCHIVE_ITERATORS_BASE64_FROM_BINARY_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// base64_from_binary.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include // size_t +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include +#include + +namespace boost { +namespace archive { +namespace iterators { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// convert binary integers to base64 characters + +namespace detail { + +template +struct from_6_bit { + typedef CharType result_type; + CharType operator()(CharType t) const{ + static const char * lookup_table = + "ABCDEFGHIJKLMNOPQRSTUVWXYZ" + "abcdefghijklmnopqrstuvwxyz" + "0123456789" + "+/"; + BOOST_ASSERT(t < 64); + return lookup_table[static_cast(t)]; + } +}; + +} // namespace detail + +// note: what we would like to do is +// template +// typedef transform_iterator< +// from_6_bit, +// transform_width +// > base64_from_binary; +// but C++ won't accept this. Rather than using a "type generator" and +// using a different syntax, make a derivation which should be equivalent. +// +// Another issue addressed here is that the transform_iterator doesn't have +// a templated constructor. This makes it incompatible with the dataflow +// ideal. This is also addressed here. + +//template +template< + class Base, + class CharType = typename boost::iterator_value::type +> +class base64_from_binary : + public transform_iterator< + detail::from_6_bit, + Base + > +{ + friend class boost::iterator_core_access; + typedef transform_iterator< + typename detail::from_6_bit, + Base + > super_t; + +public: + // make composible buy using templated constructor + template + base64_from_binary(T start) : + super_t( + Base(static_cast< T >(start)), + detail::from_6_bit() + ) + {} + // intel 7.1 doesn't like default copy constructor + base64_from_binary(const base64_from_binary & rhs) : + super_t( + Base(rhs.base_reference()), + detail::from_6_bit() + ) + {} +// base64_from_binary(){}; +}; + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_BASE64_FROM_BINARY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/binary_from_base64.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/binary_from_base64.hpp new file mode 100644 index 00000000000..89b8f889da3 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/binary_from_base64.hpp @@ -0,0 +1,118 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_BINARY_FROM_BASE64_HPP +#define BOOST_ARCHIVE_ITERATORS_BINARY_FROM_BASE64_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// binary_from_base64.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include + +#include +#include + +namespace boost { +namespace archive { +namespace iterators { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// convert base64 characters to binary data + +namespace detail { + +template +struct to_6_bit { + typedef CharType result_type; + CharType operator()(CharType t) const{ + static const signed char lookup_table[] = { + -1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1, + -1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1, + -1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,62,-1,-1,-1,63, + 52,53,54,55,56,57,58,59,60,61,-1,-1,-1, 0,-1,-1, // render '=' as 0 + -1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9,10,11,12,13,14, + 15,16,17,18,19,20,21,22,23,24,25,-1,-1,-1,-1,-1, + -1,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,-1,-1,-1,-1,-1 + }; + // metrowerks trips this assertion - how come? + #if ! defined(__MWERKS__) + BOOST_STATIC_ASSERT(128 == sizeof(lookup_table)); + #endif + signed char value = -1; + if((unsigned)t <= 127) + value = lookup_table[(unsigned)t]; + if(-1 == value) + boost::serialization::throw_exception( + dataflow_exception(dataflow_exception::invalid_base64_character) + ); + return value; + } +}; + +} // namespace detail + +// note: what we would like to do is +// template +// typedef transform_iterator< +// from_6_bit, +// transform_width +// > base64_from_binary; +// but C++ won't accept this. Rather than using a "type generator" and +// using a different syntax, make a derivation which should be equivalent. +// +// Another issue addressed here is that the transform_iterator doesn't have +// a templated constructor. This makes it incompatible with the dataflow +// ideal. This is also addressed here. + +template< + class Base, + class CharType = typename boost::iterator_value::type +> +class binary_from_base64 : public + transform_iterator< + detail::to_6_bit, + Base + > +{ + friend class boost::iterator_core_access; + typedef transform_iterator< + detail::to_6_bit, + Base + > super_t; +public: + // make composible buy using templated constructor + template + binary_from_base64(T start) : + super_t( + Base(static_cast< T >(start)), + detail::to_6_bit() + ) + {} + // intel 7.1 doesn't like default copy constructor + binary_from_base64(const binary_from_base64 & rhs) : + super_t( + Base(rhs.base_reference()), + detail::to_6_bit() + ) + {} +// binary_from_base64(){}; +}; + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_BINARY_FROM_BASE64_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow.hpp new file mode 100644 index 00000000000..07733d5fd62 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow.hpp @@ -0,0 +1,102 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_DATAFLOW_HPP +#define BOOST_ARCHIVE_ITERATORS_DATAFLOW_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// dataflow.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace boost { +namespace archive { +namespace iterators { + +// poor man's tri-state +struct tri_state { + enum state_enum { + is_false = false, + is_true = true, + is_indeterminant + } m_state; + // convert to bool + operator bool (){ + BOOST_ASSERT(is_indeterminant != m_state); + return is_true == m_state ? true : false; + } + // assign from bool + tri_state & operator=(bool rhs) { + m_state = rhs ? is_true : is_false; + return *this; + } + tri_state(bool rhs) : + m_state(rhs ? is_true : is_false) + {} + tri_state(state_enum state) : + m_state(state) + {} + bool operator==(const tri_state & rhs) const { + return m_state == rhs.m_state; + } + bool operator!=(const tri_state & rhs) const { + return m_state != rhs.m_state; + } +}; + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// implement functions common to dataflow iterators +template +class dataflow { + bool m_eoi; +protected: + // test for iterator equality + tri_state equal(const Derived & rhs) const { + if(m_eoi && rhs.m_eoi) + return true; + if(m_eoi || rhs.m_eoi) + return false; + return tri_state(tri_state::is_indeterminant); + } + void eoi(bool tf){ + m_eoi = tf; + } + bool eoi() const { + return m_eoi; + } +public: + dataflow(bool tf) : + m_eoi(tf) + {} + dataflow() : // used for iterator end + m_eoi(true) + {} +}; + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_DATAFLOW_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow_exception.hpp new file mode 100644 index 00000000000..e3e18605b38 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow_exception.hpp @@ -0,0 +1,80 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_DATAFLOW_EXCEPTION_HPP +#define BOOST_ARCHIVE_ITERATORS_DATAFLOW_EXCEPTION_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// dataflow_exception.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifndef BOOST_NO_EXCEPTIONS +#include +#endif //BOOST_NO_EXCEPTIONS + +#include + +namespace boost { +namespace archive { +namespace iterators { + +////////////////////////////////////////////////////////////////////// +// exceptions thrown by dataflows +// +class dataflow_exception : public std::exception +{ +public: + typedef enum { + invalid_6_bitcode, + invalid_base64_character, + invalid_xml_escape_sequence, + comparison_not_permitted, + invalid_conversion, + other_exception + } exception_code; + exception_code code; + + dataflow_exception(exception_code c = other_exception) : code(c) + {} + + virtual const char *what( ) const throw( ) + { + const char *msg = "unknown exception code"; + switch(code){ + case invalid_6_bitcode: + msg = "attempt to encode a value > 6 bits"; + break; + case invalid_base64_character: + msg = "attempt to decode a value not in base64 char set"; + break; + case invalid_xml_escape_sequence: + msg = "invalid xml escape_sequence"; + break; + case comparison_not_permitted: + msg = "cannot invoke iterator comparison now"; + break; + case invalid_conversion: + msg = "invalid multbyte/wide char conversion"; + break; + default: + BOOST_ASSERT(false); + break; + } + return msg; + } +}; + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif //BOOST_ARCHIVE_ITERATORS_DATAFLOW_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/escape.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/escape.hpp new file mode 100644 index 00000000000..103b31e0fef --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/escape.hpp @@ -0,0 +1,115 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_ESCAPE_HPP +#define BOOST_ARCHIVE_ITERATORS_ESCAPE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// escape.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include // NULL + +#include +#include + +namespace boost { +namespace archive { +namespace iterators { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// insert escapes into text + +template +class escape : + public boost::iterator_adaptor< + Derived, + Base, + typename boost::iterator_value::type, + single_pass_traversal_tag, + typename boost::iterator_value::type + > +{ + typedef typename boost::iterator_value::type base_value_type; + typedef typename boost::iterator_reference::type reference_type; + friend class boost::iterator_core_access; + + typedef typename boost::iterator_adaptor< + Derived, + Base, + base_value_type, + single_pass_traversal_tag, + base_value_type + > super_t; + + typedef escape this_t; + + void dereference_impl() { + m_current_value = static_cast(this)->fill(m_bnext, m_bend); + m_full = true; + } + + //Access the value referred to + reference_type dereference() const { + if(!m_full) + const_cast(this)->dereference_impl(); + return m_current_value; + } + + bool equal(const this_t & rhs) const { + if(m_full){ + if(! rhs.m_full) + const_cast(& rhs)->dereference_impl(); + } + else{ + if(rhs.m_full) + const_cast(this)->dereference_impl(); + } + if(m_bnext != rhs.m_bnext) + return false; + if(this->base_reference() != rhs.base_reference()) + return false; + return true; + } + + void increment(){ + if(++m_bnext < m_bend){ + m_current_value = *m_bnext; + return; + } + ++(this->base_reference()); + m_bnext = NULL; + m_bend = NULL; + m_full = false; + } + + // buffer to handle pending characters + const base_value_type *m_bnext; + const base_value_type *m_bend; + bool m_full; + base_value_type m_current_value; +public: + escape(Base base) : + super_t(base), + m_bnext(NULL), + m_bend(NULL), + m_full(false), + m_current_value(0) + { + } +}; + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_ESCAPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/insert_linebreaks.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/insert_linebreaks.hpp new file mode 100644 index 00000000000..2504b030db1 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/insert_linebreaks.hpp @@ -0,0 +1,99 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_INSERT_LINEBREAKS_HPP +#define BOOST_ARCHIVE_ITERATORS_INSERT_LINEBREAKS_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// insert_linebreaks.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ using ::memcpy; } +#endif + +#include +#include + +namespace boost { +namespace archive { +namespace iterators { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// insert line break every N characters +template< + class Base, + int N, + class CharType = typename boost::iterator_value::type +> +class insert_linebreaks : + public iterator_adaptor< + insert_linebreaks, + Base, + CharType, + single_pass_traversal_tag, + CharType + > +{ +private: + friend class boost::iterator_core_access; + typedef iterator_adaptor< + insert_linebreaks, + Base, + CharType, + single_pass_traversal_tag, + CharType + > super_t; + + bool equal(const insert_linebreaks & rhs) const { + return +// m_count == rhs.m_count +// && base_reference() == rhs.base_reference() + this->base_reference() == rhs.base_reference() + ; + } + + void increment() { + if(m_count == N){ + m_count = 0; + return; + } + ++m_count; + ++(this->base_reference()); + } + CharType dereference() const { + if(m_count == N) + return '\n'; + return * (this->base_reference()); + } + unsigned int m_count; +public: + // make composible buy using templated constructor + template + insert_linebreaks(T start) : + super_t(Base(static_cast< T >(start))), + m_count(0) + {} + // intel 7.1 doesn't like default copy constructor + insert_linebreaks(const insert_linebreaks & rhs) : + super_t(rhs.base_reference()), + m_count(rhs.m_count) + {} +}; + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_INSERT_LINEBREAKS_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/istream_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/istream_iterator.hpp new file mode 100644 index 00000000000..a187f605e69 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/istream_iterator.hpp @@ -0,0 +1,92 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_ISTREAM_ITERATOR_HPP +#define BOOST_ARCHIVE_ITERATORS_ISTREAM_ITERATOR_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// istream_iterator.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// note: this is a custom version of the standard istream_iterator. +// This is necessary as the standard version doesn't work as expected +// for wchar_t based streams on systems for which wchar_t not a true +// type but rather a synonym for some integer type. + +#include // NULL +#include +#include + +namespace boost { +namespace archive { +namespace iterators { + +// given a type, make an input iterator based on a pointer to that type +template +class istream_iterator : + public boost::iterator_facade< + istream_iterator, + Elem, + std::input_iterator_tag, + Elem + > +{ + friend class boost::iterator_core_access; + typedef istream_iterator this_t ; + typedef typename boost::iterator_facade< + istream_iterator, + Elem, + std::input_iterator_tag, + Elem + > super_t; + typedef typename std::basic_istream istream_type; + + bool equal(const this_t & rhs) const { + // note: only works for comparison against end of stream + return m_istream == rhs.m_istream; + } + + //Access the value referred to + Elem dereference() const { + return static_cast(m_istream->peek()); + } + + void increment(){ + if(NULL != m_istream){ + m_istream->ignore(1); + } + } + + istream_type *m_istream; + Elem m_current_value; +public: + istream_iterator(istream_type & is) : + m_istream(& is) + { + //increment(); + } + + istream_iterator() : + m_istream(NULL), + m_current_value(NULL) + {} + + istream_iterator(const istream_iterator & rhs) : + m_istream(rhs.m_istream), + m_current_value(rhs.m_current_value) + {} +}; + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_ISTREAM_ITERATOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/mb_from_wchar.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/mb_from_wchar.hpp new file mode 100644 index 00000000000..05df71c258e --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/mb_from_wchar.hpp @@ -0,0 +1,139 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_MB_FROM_WCHAR_HPP +#define BOOST_ARCHIVE_ITERATORS_MB_FROM_WCHAR_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// mb_from_wchar.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include // size_t +#ifndef BOOST_NO_CWCHAR +#include // mbstate_t +#endif +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::mbstate_t; +} // namespace std +#endif + +#include +#include + +namespace boost { +namespace archive { +namespace iterators { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// class used by text archives to translate wide strings and to char +// strings of the currently selected locale +template // the input iterator +class mb_from_wchar + : public boost::iterator_adaptor< + mb_from_wchar, + Base, + wchar_t, + single_pass_traversal_tag, + char + > +{ + friend class boost::iterator_core_access; + + typedef typename boost::iterator_adaptor< + mb_from_wchar, + Base, + wchar_t, + single_pass_traversal_tag, + char + > super_t; + + typedef mb_from_wchar this_t; + + char dereference_impl() { + if(! m_full){ + fill(); + m_full = true; + } + return m_buffer[m_bnext]; + } + + char dereference() const { + return (const_cast(this))->dereference_impl(); + } + // test for iterator equality + bool equal(const mb_from_wchar & rhs) const { + // once the value is filled, the base_reference has been incremented + // so don't permit comparison anymore. + return + 0 == m_bend + && 0 == m_bnext + && this->base_reference() == rhs.base_reference() + ; + } + + void fill(){ + wchar_t value = * this->base_reference(); + const wchar_t *wend; + char *bend; + std::codecvt_base::result r = m_codecvt_facet.out( + m_mbs, + & value, & value + 1, wend, + m_buffer, m_buffer + sizeof(m_buffer), bend + ); + BOOST_ASSERT(std::codecvt_base::ok == r); + m_bnext = 0; + m_bend = bend - m_buffer; + } + + void increment(){ + if(++m_bnext < m_bend) + return; + m_bend = + m_bnext = 0; + ++(this->base_reference()); + m_full = false; + } + + boost::archive::detail::utf8_codecvt_facet m_codecvt_facet; + std::mbstate_t m_mbs; + // buffer to handle pending characters + char m_buffer[9 /* MB_CUR_MAX */]; + std::size_t m_bend; + std::size_t m_bnext; + bool m_full; + +public: + // make composible buy using templated constructor + template + mb_from_wchar(T start) : + super_t(Base(static_cast< T >(start))), + m_mbs(std::mbstate_t()), + m_bend(0), + m_bnext(0), + m_full(false) + {} + // intel 7.1 doesn't like default copy constructor + mb_from_wchar(const mb_from_wchar & rhs) : + super_t(rhs.base_reference()), + m_bend(rhs.m_bend), + m_bnext(rhs.m_bnext), + m_full(rhs.m_full) + {} +}; + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_MB_FROM_WCHAR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/ostream_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/ostream_iterator.hpp new file mode 100644 index 00000000000..49a9b99034b --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/ostream_iterator.hpp @@ -0,0 +1,83 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_OSTREAM_ITERATOR_HPP +#define BOOST_ARCHIVE_ITERATORS_OSTREAM_ITERATOR_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// ostream_iterator.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// note: this is a custom version of the standard ostream_iterator. +// This is necessary as the standard version doesn't work as expected +// for wchar_t based streams on systems for which wchar_t not a true +// type but rather a synonym for some integer type. + +#include +#include + +namespace boost { +namespace archive { +namespace iterators { + +// given a type, make an input iterator based on a pointer to that type +template +class ostream_iterator : + public boost::iterator_facade< + ostream_iterator, + Elem, + std::output_iterator_tag, + ostream_iterator & + > +{ + friend class boost::iterator_core_access; + typedef ostream_iterator this_t ; + typedef Elem char_type; + typedef std::basic_ostream ostream_type; + + //emulate the behavior of std::ostream + ostream_iterator & dereference() const { + return const_cast(*this); + } + bool equal(const this_t & rhs) const { + return m_ostream == rhs.m_ostream; + } + void increment(){} +protected: + ostream_type *m_ostream; + void put_val(char_type e){ + if(NULL != m_ostream){ + m_ostream->put(e); + if(! m_ostream->good()) + m_ostream = NULL; + } + } +public: + this_t & operator=(char_type c){ + put_val(c); + return *this; + } + ostream_iterator(ostream_type & os) : + m_ostream (& os) + {} + ostream_iterator() : + m_ostream (NULL) + {} + ostream_iterator(const ostream_iterator & rhs) : + m_ostream (rhs.m_ostream) + {} +}; + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_OSTREAM_ITERATOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/remove_whitespace.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/remove_whitespace.hpp new file mode 100644 index 00000000000..c3580ab258a --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/remove_whitespace.hpp @@ -0,0 +1,167 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_REMOVE_WHITESPACE_HPP +#define BOOST_ARCHIVE_ITERATORS_REMOVE_WHITESPACE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// remove_whitespace.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include +#include + +// here is the default standard implementation of the functor used +// by the filter iterator to remove spaces. Unfortunately usage +// of this implementation in combination with spirit trips a bug +// VC 6.5. The only way I can find to work around it is to +// implement a special non-standard version for this platform + +#ifndef BOOST_NO_CWCTYPE +#include // iswspace +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ using ::iswspace; } +#endif +#endif + +#include // isspace +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ using ::isspace; } +#endif + +#if defined(__STD_RWCOMPILER_H__) || defined(_RWSTD_VER) +// this is required for the RW STL on Linux and Tru64. +#undef isspace +#undef iswspace +#endif + +namespace { // anonymous + +template +struct remove_whitespace_predicate; + +template<> +struct remove_whitespace_predicate +{ + bool operator()(unsigned char t){ + return ! std::isspace(t); + } +}; + +#ifndef BOOST_NO_CWCHAR +template<> +struct remove_whitespace_predicate +{ + bool operator()(wchar_t t){ + return ! std::iswspace(t); + } +}; +#endif + +} // namespace anonymous + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// convert base64 file data (including whitespace and padding) to binary + +namespace boost { +namespace archive { +namespace iterators { + +// custom version of filter iterator which doesn't look ahead further than +// necessary + +template +class filter_iterator + : public boost::iterator_adaptor< + filter_iterator, + Base, + use_default, + single_pass_traversal_tag + > +{ + friend class boost::iterator_core_access; + typedef typename boost::iterator_adaptor< + filter_iterator, + Base, + use_default, + single_pass_traversal_tag + > super_t; + typedef filter_iterator this_t; + typedef typename super_t::reference reference_type; + + reference_type dereference_impl(){ + if(! m_full){ + while(! m_predicate(* this->base_reference())) + ++(this->base_reference()); + m_full = true; + } + return * this->base_reference(); + } + + reference_type dereference() const { + return const_cast(this)->dereference_impl(); + } + + Predicate m_predicate; + bool m_full; +public: + // note: this function is public only because comeau compiler complained + // I don't know if this is because the compiler is wrong or what + void increment(){ + m_full = false; + ++(this->base_reference()); + } + filter_iterator(Base start) : + super_t(start), + m_full(false) + {} + filter_iterator(){} +}; + +template +class remove_whitespace : + public filter_iterator< + remove_whitespace_predicate< + typename boost::iterator_value::type + //typename Base::value_type + >, + Base + > +{ + friend class boost::iterator_core_access; + typedef filter_iterator< + remove_whitespace_predicate< + typename boost::iterator_value::type + //typename Base::value_type + >, + Base + > super_t; +public: +// remove_whitespace(){} // why is this needed? + // make composible buy using templated constructor + template + remove_whitespace(T start) : + super_t(Base(static_cast< T >(start))) + {} + // intel 7.1 doesn't like default copy constructor + remove_whitespace(const remove_whitespace & rhs) : + super_t(rhs.base_reference()) + {} +}; + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_REMOVE_WHITESPACE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/transform_width.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/transform_width.hpp new file mode 100644 index 00000000000..09c050a9274 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/transform_width.hpp @@ -0,0 +1,177 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_TRANSFORM_WIDTH_HPP +#define BOOST_ARCHIVE_ITERATORS_TRANSFORM_WIDTH_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// transform_width.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// iterator which takes elements of x bits and returns elements of y bits. +// used to change streams of 8 bit characters into streams of 6 bit characters. +// and vice-versa for implementing base64 encodeing/decoding. Be very careful +// when using and end iterator. end is only reliable detected when the input +// stream length is some common multiple of x and y. E.G. Base64 6 bit +// character and 8 bit bytes. Lowest common multiple is 24 => 4 6 bit characters +// or 3 8 bit characters + +#include +#include + +#include // std::min + +namespace boost { +namespace archive { +namespace iterators { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// class used by text archives to translate char strings to wchar_t +// strings of the currently selected locale +template< + class Base, + int BitsOut, + int BitsIn, + class CharType = typename boost::iterator_value::type // output character +> +class transform_width : + public boost::iterator_adaptor< + transform_width, + Base, + CharType, + single_pass_traversal_tag, + CharType + > +{ + friend class boost::iterator_core_access; + typedef typename boost::iterator_adaptor< + transform_width, + Base, + CharType, + single_pass_traversal_tag, + CharType + > super_t; + + typedef transform_width this_t; + typedef typename iterator_value::type base_value_type; + + void fill(); + + CharType dereference() const { + if(!m_buffer_out_full) + const_cast(this)->fill(); + return m_buffer_out; + } + + bool equal_impl(const this_t & rhs){ + if(BitsIn < BitsOut) // discard any left over bits + return this->base_reference() == rhs.base_reference(); + else{ + // BitsIn > BitsOut // zero fill + if(this->base_reference() == rhs.base_reference()){ + m_end_of_sequence = true; + return 0 == m_remaining_bits; + } + return false; + } + } + + // standard iterator interface + bool equal(const this_t & rhs) const { + return const_cast(this)->equal_impl(rhs); + } + + void increment(){ + m_buffer_out_full = false; + } + + bool m_buffer_out_full; + CharType m_buffer_out; + + // last read element from input + base_value_type m_buffer_in; + + // number of bits to left in the input buffer. + unsigned int m_remaining_bits; + + // flag to indicate we've reached end of data. + bool m_end_of_sequence; + +public: + // make composible buy using templated constructor + template + transform_width(T start) : + super_t(Base(static_cast< T >(start))), + m_buffer_out_full(false), + m_buffer_out(0), + // To disable GCC warning, but not truly necessary + //(m_buffer_in will be initialized later before being + //used because m_remaining_bits == 0) + m_buffer_in(0), + m_remaining_bits(0), + m_end_of_sequence(false) + {} + // intel 7.1 doesn't like default copy constructor + transform_width(const transform_width & rhs) : + super_t(rhs.base_reference()), + m_buffer_out_full(rhs.m_buffer_out_full), + m_buffer_out(rhs.m_buffer_out), + m_buffer_in(rhs.m_buffer_in), + m_remaining_bits(rhs.m_remaining_bits), + m_end_of_sequence(false) + {} +}; + +template< + class Base, + int BitsOut, + int BitsIn, + class CharType +> +void transform_width::fill() { + unsigned int missing_bits = BitsOut; + m_buffer_out = 0; + do{ + if(0 == m_remaining_bits){ + if(m_end_of_sequence){ + m_buffer_in = 0; + m_remaining_bits = missing_bits; + } + else{ + m_buffer_in = * this->base_reference()++; + m_remaining_bits = BitsIn; + } + } + + // append these bits to the next output + // up to the size of the output + unsigned int i = (std::min)(missing_bits, m_remaining_bits); + // shift interesting bits to least significant position + base_value_type j = m_buffer_in >> (m_remaining_bits - i); + // and mask off the un interesting higher bits + // note presumption of twos complement notation + j &= (1 << i) - 1; + // append then interesting bits to the output value + m_buffer_out <<= i; + m_buffer_out |= j; + + // and update counters + missing_bits -= i; + m_remaining_bits -= i; + }while(0 < missing_bits); + m_buffer_out_full = true; +} + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_TRANSFORM_WIDTH_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/unescape.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/unescape.hpp new file mode 100644 index 00000000000..abf62406088 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/unescape.hpp @@ -0,0 +1,89 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_UNESCAPE_HPP +#define BOOST_ARCHIVE_ITERATORS_UNESCAPE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// unescape.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include + +namespace boost { +namespace archive { +namespace iterators { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// class used by text archives to translate char strings to wchar_t +// strings of the currently selected locale +template +class unescape + : public boost::iterator_adaptor< + unescape, + Base, + typename pointee::type, + single_pass_traversal_tag, + typename pointee::type + > +{ + friend class boost::iterator_core_access; + typedef typename boost::iterator_adaptor< + unescape, + Base, + typename pointee::type, + single_pass_traversal_tag, + typename pointee::type + > super_t; + + typedef unescape this_t; +public: + typedef typename this_t::value_type value_type; + typedef typename this_t::reference reference; +private: + value_type dereference_impl() { + if(! m_full){ + m_current_value = static_cast(this)->drain(); + m_full = true; + } + return m_current_value; + } + + reference dereference() const { + return const_cast(this)->dereference_impl(); + } + + value_type m_current_value; + bool m_full; + + void increment(){ + ++(this->base_reference()); + dereference_impl(); + m_full = false; + }; + +public: + + unescape(Base base) : + super_t(base), + m_full(false) + {} + +}; + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_UNESCAPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/wchar_from_mb.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/wchar_from_mb.hpp new file mode 100644 index 00000000000..2af8f6401f2 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/wchar_from_mb.hpp @@ -0,0 +1,194 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_WCHAR_FROM_MB_HPP +#define BOOST_ARCHIVE_ITERATORS_WCHAR_FROM_MB_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// wchar_from_mb.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include // size_t +#ifndef BOOST_NO_CWCHAR +#include // mbstate_t +#endif +#include // copy + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::mbstate_t; +} // namespace std +#endif +#include +#include +#include +#include +#include + +#include + +namespace boost { +namespace archive { +namespace iterators { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// class used by text archives to translate char strings to wchar_t +// strings of the currently selected locale +template +class wchar_from_mb + : public boost::iterator_adaptor< + wchar_from_mb, + Base, + wchar_t, + single_pass_traversal_tag, + wchar_t + > +{ + friend class boost::iterator_core_access; + typedef typename boost::iterator_adaptor< + wchar_from_mb, + Base, + wchar_t, + single_pass_traversal_tag, + wchar_t + > super_t; + + typedef wchar_from_mb this_t; + + void drain(); + + wchar_t dereference() const { + if(m_output.m_next == m_output.m_next_available) + return static_cast(0); + return * m_output.m_next; + } + + void increment(){ + if(m_output.m_next == m_output.m_next_available) + return; + if(++m_output.m_next == m_output.m_next_available){ + if(m_input.m_done) + return; + drain(); + } + } + + bool equal(this_t const & rhs) const { + return dereference() == rhs.dereference(); + } + + boost::archive::detail::utf8_codecvt_facet m_codecvt_facet; + std::mbstate_t m_mbs; + + template + struct sliding_buffer { + boost::array m_buffer; + typename boost::array::const_iterator m_next_available; + typename boost::array::iterator m_next; + bool m_done; + // default ctor + sliding_buffer() : + m_next_available(m_buffer.begin()), + m_next(m_buffer.begin()), + m_done(false) + {} + // copy ctor + sliding_buffer(const sliding_buffer & rhs) : + m_next_available( + std::copy( + rhs.m_buffer.begin(), + rhs.m_next_available, + m_buffer.begin() + ) + ), + m_next( + m_buffer.begin() + (rhs.m_next - rhs.m_buffer.begin()) + ), + m_done(rhs.m_done) + {} + }; + + sliding_buffer::type> m_input; + sliding_buffer::type> m_output; + +public: + // make composible buy using templated constructor + template + wchar_from_mb(T start) : + super_t(Base(static_cast< T >(start))), + m_mbs(std::mbstate_t()) + { + BOOST_ASSERT(std::mbsinit(&m_mbs)); + drain(); + } + // default constructor used as an end iterator + wchar_from_mb(){} + + // copy ctor + wchar_from_mb(const wchar_from_mb & rhs) : + super_t(rhs.base_reference()), + m_mbs(rhs.m_mbs), + m_input(rhs.m_input), + m_output(rhs.m_output) + {} +}; + +template +void wchar_from_mb::drain(){ + BOOST_ASSERT(! m_input.m_done); + for(;;){ + typename boost::iterators::iterator_reference::type c = *(this->base_reference()); + // a null character in a multibyte stream is takes as end of string + if(0 == c){ + m_input.m_done = true; + break; + } + ++(this->base_reference()); + * const_cast::type *>( + (m_input.m_next_available++) + ) = c; + // if input buffer is full - we're done for now + if(m_input.m_buffer.end() == m_input.m_next_available) + break; + } + const typename boost::iterators::iterator_value::type * input_new_start; + typename iterator_value::type * next_available; + + std::codecvt_base::result r = m_codecvt_facet.in( + m_mbs, + m_input.m_buffer.begin(), + m_input.m_next_available, + input_new_start, + m_output.m_buffer.begin(), + m_output.m_buffer.end(), + next_available + ); + BOOST_ASSERT(std::codecvt_base::ok == r); + m_output.m_next_available = next_available; + m_output.m_next = m_output.m_buffer.begin(); + + // we're done with some of the input so shift left. + m_input.m_next_available = std::copy( + input_new_start, + m_input.m_next_available, + m_input.m_buffer.begin() + ); + m_input.m_next = m_input.m_buffer.begin(); +} + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_WCHAR_FROM_MB_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_escape.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_escape.hpp new file mode 100644 index 00000000000..c838a73b864 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_escape.hpp @@ -0,0 +1,121 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_XML_ESCAPE_HPP +#define BOOST_ARCHIVE_ITERATORS_XML_ESCAPE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// xml_escape.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include + +namespace boost { +namespace archive { +namespace iterators { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// insert escapes into xml text + +template +class xml_escape + : public escape, Base> +{ + friend class boost::iterator_core_access; + + typedef escape, Base> super_t; + +public: + char fill(const char * & bstart, const char * & bend); + wchar_t fill(const wchar_t * & bstart, const wchar_t * & bend); + + template + xml_escape(T start) : + super_t(Base(static_cast< T >(start))) + {} + // intel 7.1 doesn't like default copy constructor + xml_escape(const xml_escape & rhs) : + super_t(rhs.base_reference()) + {} +}; + +template +char xml_escape::fill( + const char * & bstart, + const char * & bend +){ + char current_value = * this->base_reference(); + switch(current_value){ + case '<': + bstart = "<"; + bend = bstart + 4; + break; + case '>': + bstart = ">"; + bend = bstart + 4; + break; + case '&': + bstart = "&"; + bend = bstart + 5; + break; + case '"': + bstart = """; + bend = bstart + 6; + break; + case '\'': + bstart = "'"; + bend = bstart + 6; + break; + default: + return current_value; + } + return *bstart; +} + +template +wchar_t xml_escape::fill( + const wchar_t * & bstart, + const wchar_t * & bend +){ + wchar_t current_value = * this->base_reference(); + switch(current_value){ + case '<': + bstart = L"<"; + bend = bstart + 4; + break; + case '>': + bstart = L">"; + bend = bstart + 4; + break; + case '&': + bstart = L"&"; + bend = bstart + 5; + break; + case '"': + bstart = L"""; + bend = bstart + 6; + break; + case '\'': + bstart = L"'"; + bend = bstart + 6; + break; + default: + return current_value; + } + return *bstart; +} + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_XML_ESCAPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape.hpp new file mode 100644 index 00000000000..69977404567 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape.hpp @@ -0,0 +1,125 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_HPP +#define BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// xml_unescape.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include + +#include +#include + +namespace boost { +namespace archive { +namespace iterators { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// replace &??? xml escape sequences with the corresponding characters +template +class xml_unescape + : public unescape, Base> +{ + friend class boost::iterator_core_access; + typedef xml_unescape this_t; + typedef unescape super_t; + typedef typename boost::iterator_reference reference_type; + + reference_type dereference() const { + return unescape, Base>::dereference(); + } +public: + // workaround msvc 7.1 ICU crash + #if defined(BOOST_MSVC) + typedef int value_type; + #else + typedef typename this_t::value_type value_type; + #endif + + void drain_residue(const char *literal); + value_type drain(); + + template + xml_unescape(T start) : + super_t(Base(static_cast< T >(start))) + {} + // intel 7.1 doesn't like default copy constructor + xml_unescape(const xml_unescape & rhs) : + super_t(rhs.base_reference()) + {} +}; + +template +void xml_unescape::drain_residue(const char * literal){ + do{ + if(* literal != * ++(this->base_reference())) + boost::serialization::throw_exception( + dataflow_exception( + dataflow_exception::invalid_xml_escape_sequence + ) + ); + } + while('\0' != * ++literal); +} + +// note key constraint on this function is that can't "look ahead" any +// more than necessary into base iterator. Doing so would alter the base +// iterator refenence which would make subsequent iterator comparisons +// incorrect and thereby break the composiblity of iterators. +template +typename xml_unescape::value_type +//int +xml_unescape::drain(){ + value_type retval = * this->base_reference(); + if('&' != retval){ + return retval; + } + retval = * ++(this->base_reference()); + switch(retval){ + case 'l': // < + drain_residue("t;"); + retval = '<'; + break; + case 'g': // > + drain_residue("t;"); + retval = '>'; + break; + case 'a': + retval = * ++(this->base_reference()); + switch(retval){ + case 'p': // ' + drain_residue("os;"); + retval = '\''; + break; + case 'm': // & + drain_residue("p;"); + retval = '&'; + break; + } + break; + case 'q': + drain_residue("uot;"); + retval = '"'; + break; + } + return retval; +} + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif // BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape_exception.hpp new file mode 100644 index 00000000000..71a64378c20 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape_exception.hpp @@ -0,0 +1,49 @@ +#ifndef BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_EXCEPTION_HPP +#define BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_EXCEPTION_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// xml_unescape_exception.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifndef BOOST_NO_EXCEPTIONS +#include + +#include + +namespace boost { +namespace archive { +namespace iterators { + +////////////////////////////////////////////////////////////////////// +// exceptions thrown by xml_unescapes +// +class xml_unescape_exception : public std::exception +{ +public: + xml_unescape_exception() + {} + + virtual const char *what( ) const throw( ) + { + return "xml contained un-recognized escape code"; + } +}; + +} // namespace iterators +} // namespace archive +} // namespace boost + +#endif //BOOST_NO_EXCEPTIONS +#endif //BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_iarchive.hpp new file mode 100644 index 00000000000..4a898a8ad16 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_iarchive.hpp @@ -0,0 +1,54 @@ +#ifndef BOOST_ARCHIVE_POLYMORPHIC_BINARY_IARCHIVE_HPP +#define BOOST_ARCHIVE_POLYMORPHIC_BINARY_IARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_binary_iarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +class polymorphic_binary_iarchive : + public detail::polymorphic_iarchive_route +{ +public: + polymorphic_binary_iarchive(std::istream & is, unsigned int flags = 0) : + detail::polymorphic_iarchive_route(is, flags) + {} + ~polymorphic_binary_iarchive(){} +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE( + boost::archive::polymorphic_binary_iarchive +) + +#endif // BOOST_ARCHIVE_POLYMORPHIC_BINARY_IARCHIVE_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_oarchive.hpp new file mode 100644 index 00000000000..931b243feb8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_oarchive.hpp @@ -0,0 +1,43 @@ +#ifndef BOOST_ARCHIVE_POLYMORPHIC_BINARY_OARCHIVE_HPP +#define BOOST_ARCHIVE_POLYMORPHIC_BINARY_OARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_binary_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include + +namespace boost { +namespace archive { + +typedef detail::polymorphic_oarchive_route< + binary_oarchive_impl< + binary_oarchive, + std::ostream::char_type, + std::ostream::traits_type + > + > polymorphic_binary_oarchive; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE( + boost::archive::polymorphic_binary_oarchive +) + +#endif // BOOST_ARCHIVE_POLYMORPHIC_BINARY_OARCHIVE_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_iarchive.hpp new file mode 100644 index 00000000000..d3c59a9f0f4 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_iarchive.hpp @@ -0,0 +1,168 @@ +#ifndef BOOST_ARCHIVE_POLYMORPHIC_IARCHIVE_HPP +#define BOOST_ARCHIVE_POLYMORPHIC_IARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_iarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // std::size_t +#include // ULONG_MAX +#include + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include + +#include +#include +#include +#include + +#include +#include // must be the last header + +namespace boost { +namespace serialization { + class extended_type_info; +} // namespace serialization +namespace archive { +namespace detail { + class basic_iarchive; + class basic_iserializer; +} + +class polymorphic_iarchive; + +class BOOST_SYMBOL_VISIBLE polymorphic_iarchive_impl : + public detail::interface_iarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else + friend class detail::interface_iarchive; + friend class load_access; +#endif + // primitive types the only ones permitted by polymorphic archives + virtual void load(bool & t) = 0; + + virtual void load(char & t) = 0; + virtual void load(signed char & t) = 0; + virtual void load(unsigned char & t) = 0; + #ifndef BOOST_NO_CWCHAR + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + virtual void load(wchar_t & t) = 0; + #endif + #endif + virtual void load(short & t) = 0; + virtual void load(unsigned short & t) = 0; + virtual void load(int & t) = 0; + virtual void load(unsigned int & t) = 0; + virtual void load(long & t) = 0; + virtual void load(unsigned long & t) = 0; + + #if defined(BOOST_HAS_LONG_LONG) + virtual void load(boost::long_long_type & t) = 0; + virtual void load(boost::ulong_long_type & t) = 0; + #elif defined(BOOST_HAS_MS_INT64) + virtual void load(__int64 & t) = 0; + virtual void load(unsigned __int64 & t) = 0; + #endif + + virtual void load(float & t) = 0; + virtual void load(double & t) = 0; + + // string types are treated as primitives + virtual void load(std::string & t) = 0; + #ifndef BOOST_NO_STD_WSTRING + virtual void load(std::wstring & t) = 0; + #endif + + // used for xml and other tagged formats + virtual void load_start(const char * name) = 0; + virtual void load_end(const char * name) = 0; + virtual void register_basic_serializer(const detail::basic_iserializer & bis) = 0; + virtual detail::helper_collection & get_helper_collection() = 0; + + // msvc and borland won't automatically pass these to the base class so + // make it explicit here + template + void load_override(T & t) + { + archive::load(* this->This(), t); + } + // special treatment for name-value pairs. + template + void load_override( + const boost::serialization::nvp< T > & t + ){ + load_start(t.name()); + archive::load(* this->This(), t.value()); + load_end(t.name()); + } +protected: + virtual ~polymorphic_iarchive_impl(){}; +public: + // utility function implemented by all legal archives + virtual void set_library_version(library_version_type archive_library_version) = 0; + virtual library_version_type get_library_version() const = 0; + virtual unsigned int get_flags() const = 0; + virtual void delete_created_pointers() = 0; + virtual void reset_object_address( + const void * new_address, + const void * old_address + ) = 0; + + virtual void load_binary(void * t, std::size_t size) = 0; + + // these are used by the serialization library implementation. + virtual void load_object( + void *t, + const detail::basic_iserializer & bis + ) = 0; + virtual const detail::basic_pointer_iserializer * load_pointer( + void * & t, + const detail::basic_pointer_iserializer * bpis_ptr, + const detail::basic_pointer_iserializer * (*finder)( + const boost::serialization::extended_type_info & type + ) + ) = 0; +}; + +} // namespace archive +} // namespace boost + +#include // pops abi_suffix.hpp pragmas + +namespace boost { +namespace archive { + +class BOOST_SYMBOL_VISIBLE polymorphic_iarchive : + public polymorphic_iarchive_impl +{ +public: + virtual ~polymorphic_iarchive(){}; +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::polymorphic_iarchive) + +#endif // BOOST_ARCHIVE_POLYMORPHIC_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_oarchive.hpp new file mode 100644 index 00000000000..edac4edb1e8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_oarchive.hpp @@ -0,0 +1,154 @@ +#ifndef BOOST_ARCHIVE_POLYMORPHIC_OARCHIVE_HPP +#define BOOST_ARCHIVE_POLYMORPHIC_OARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // size_t +#include // ULONG_MAX +#include + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include +#include +#include +#include +#include + +#include +#include // must be the last header + +namespace boost { +namespace serialization { + class extended_type_info; +} // namespace serialization +namespace archive { +namespace detail { + class basic_oarchive; + class basic_oserializer; +} + +class polymorphic_oarchive; + +class BOOST_SYMBOL_VISIBLE polymorphic_oarchive_impl : + public detail::interface_oarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else + friend class detail::interface_oarchive; + friend class save_access; +#endif + // primitive types the only ones permitted by polymorphic archives + virtual void save(const bool t) = 0; + + virtual void save(const char t) = 0; + virtual void save(const signed char t) = 0; + virtual void save(const unsigned char t) = 0; + #ifndef BOOST_NO_CWCHAR + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + virtual void save(const wchar_t t) = 0; + #endif + #endif + virtual void save(const short t) = 0; + virtual void save(const unsigned short t) = 0; + virtual void save(const int t) = 0; + virtual void save(const unsigned int t) = 0; + virtual void save(const long t) = 0; + virtual void save(const unsigned long t) = 0; + + #if defined(BOOST_HAS_LONG_LONG) + virtual void save(const boost::long_long_type t) = 0; + virtual void save(const boost::ulong_long_type t) = 0; + #elif defined(BOOST_HAS_MS_INT64) + virtual void save(const __int64 t) = 0; + virtual void save(const unsigned __int64 t) = 0; + #endif + + virtual void save(const float t) = 0; + virtual void save(const double t) = 0; + + // string types are treated as primitives + virtual void save(const std::string & t) = 0; + #ifndef BOOST_NO_STD_WSTRING + virtual void save(const std::wstring & t) = 0; + #endif + + virtual void save_null_pointer() = 0; + // used for xml and other tagged formats + virtual void save_start(const char * name) = 0; + virtual void save_end(const char * name) = 0; + virtual void register_basic_serializer(const detail::basic_oserializer & bos) = 0; + virtual detail::helper_collection & get_helper_collection() = 0; + + virtual void end_preamble() = 0; + + // msvc and borland won't automatically pass these to the base class so + // make it explicit here + template + void save_override(T & t) + { + archive::save(* this->This(), t); + } + // special treatment for name-value pairs. + template + void save_override( + const ::boost::serialization::nvp< T > & t + ){ + save_start(t.name()); + archive::save(* this->This(), t.const_value()); + save_end(t.name()); + } +protected: + virtual ~polymorphic_oarchive_impl(){}; +public: + // utility functions implemented by all legal archives + virtual unsigned int get_flags() const = 0; + virtual library_version_type get_library_version() const = 0; + virtual void save_binary(const void * t, std::size_t size) = 0; + + virtual void save_object( + const void *x, + const detail::basic_oserializer & bos + ) = 0; + virtual void save_pointer( + const void * t, + const detail::basic_pointer_oserializer * bpos_ptr + ) = 0; +}; + +// note: preserve naming symmetry +class BOOST_SYMBOL_VISIBLE polymorphic_oarchive : + public polymorphic_oarchive_impl +{ +public: + virtual ~polymorphic_oarchive(){}; +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::polymorphic_oarchive) + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_POLYMORPHIC_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_iarchive.hpp new file mode 100644 index 00000000000..7bef2927865 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_iarchive.hpp @@ -0,0 +1,54 @@ +#ifndef BOOST_ARCHIVE_POLYMORPHIC_TEXT_IARCHIVE_HPP +#define BOOST_ARCHIVE_POLYMORPHIC_TEXT_IARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_text_iarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +class polymorphic_text_iarchive : + public detail::polymorphic_iarchive_route +{ +public: + polymorphic_text_iarchive(std::istream & is, unsigned int flags = 0) : + detail::polymorphic_iarchive_route(is, flags) + {} + ~polymorphic_text_iarchive(){} +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE( + boost::archive::polymorphic_text_iarchive +) + +#endif // BOOST_ARCHIVE_POLYMORPHIC_TEXT_IARCHIVE_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_oarchive.hpp new file mode 100644 index 00000000000..457aad9fd75 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_oarchive.hpp @@ -0,0 +1,39 @@ +#ifndef BOOST_ARCHIVE_POLYMORPHIC_TEXT_OARCHIVE_HPP +#define BOOST_ARCHIVE_POLYMORPHIC_TEXT_OARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_text_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include + +namespace boost { +namespace archive { + +typedef detail::polymorphic_oarchive_route< + text_oarchive_impl +> polymorphic_text_oarchive; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE( + boost::archive::polymorphic_text_oarchive +) + +#endif // BOOST_ARCHIVE_POLYMORPHIC_TEXT_OARCHIVE_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_wiarchive.hpp new file mode 100644 index 00000000000..8466f05d6a6 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_wiarchive.hpp @@ -0,0 +1,59 @@ +#ifndef BOOST_ARCHIVE_POLYMORPHIC_TEXT_WIARCHIVE_HPP +#define BOOST_ARCHIVE_POLYMORPHIC_TEXT_WIARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_text_wiarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifdef BOOST_NO_STD_WSTREAMBUF +#error "wide char i/o not supported on this platform" +#else + +#include +#include + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +class polymorphic_text_wiarchive : + public detail::polymorphic_iarchive_route +{ +public: + polymorphic_text_wiarchive(std::wistream & is, unsigned int flags = 0) : + detail::polymorphic_iarchive_route(is, flags) + {} + ~polymorphic_text_wiarchive(){} +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE( + boost::archive::polymorphic_text_wiarchive +) + +#endif // BOOST_NO_STD_WSTREAMBUF +#endif // BOOST_ARCHIVE_POLYMORPHIC_TEXT_WIARCHIVE_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_woarchive.hpp new file mode 100644 index 00000000000..295625d1bcf --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_woarchive.hpp @@ -0,0 +1,44 @@ +#ifndef BOOST_ARCHIVE_POLYMORPHIC_TEXT_WOARCHIVE_HPP +#define BOOST_ARCHIVE_POLYMORPHIC_TEXT_WOARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_text_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifdef BOOST_NO_STD_WSTREAMBUF +#error "wide char i/o not supported on this platform" +#else + +#include +#include + +namespace boost { +namespace archive { + +typedef detail::polymorphic_oarchive_route< + text_woarchive_impl +> polymorphic_text_woarchive; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE( + boost::archive::polymorphic_text_woarchive +) + +#endif // BOOST_NO_STD_WSTREAMBUF +#endif // BOOST_ARCHIVE_POLYMORPHIC_TEXT_WOARCHIVE_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_iarchive.hpp new file mode 100644 index 00000000000..4dc3f894b38 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_iarchive.hpp @@ -0,0 +1,54 @@ +#ifndef BOOST_ARCHIVE_POLYMORPHIC_XML_IARCHIVE_HPP +#define BOOST_ARCHIVE_POLYMORPHIC_XML_IARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_xml_iarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +class polymorphic_xml_iarchive : + public detail::polymorphic_iarchive_route +{ +public: + polymorphic_xml_iarchive(std::istream & is, unsigned int flags = 0) : + detail::polymorphic_iarchive_route(is, flags) + {} + ~polymorphic_xml_iarchive(){} +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE( + boost::archive::polymorphic_xml_iarchive +) + +#endif // BOOST_ARCHIVE_POLYMORPHIC_XML_IARCHIVE_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_oarchive.hpp new file mode 100644 index 00000000000..514f9e530a8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_oarchive.hpp @@ -0,0 +1,39 @@ +#ifndef BOOST_ARCHIVE_POLYMORPHIC_XML_OARCHIVE_HPP +#define BOOST_ARCHIVE_POLYMORPHIC_XML_OARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_xml_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include + +namespace boost { +namespace archive { + +typedef detail::polymorphic_oarchive_route< + xml_oarchive_impl +> polymorphic_xml_oarchive; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE( + boost::archive::polymorphic_xml_oarchive +) + +#endif // BOOST_ARCHIVE_POLYMORPHIC_XML_OARCHIVE_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_wiarchive.hpp new file mode 100644 index 00000000000..d4ab731267f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_wiarchive.hpp @@ -0,0 +1,50 @@ +#ifndef BOOST_ARCHIVE_POLYMORPHIC_XML_WIARCHIVE_HPP +#define BOOST_ARCHIVE_POLYMORPHIC_XML_WIARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_xml_wiarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifdef BOOST_NO_STD_WSTREAMBUF +#error "wide char i/o not supported on this platform" +#else + +#include +#include + +namespace boost { +namespace archive { + +class polymorphic_xml_wiarchive : + public detail::polymorphic_iarchive_route +{ +public: + polymorphic_xml_wiarchive(std::wistream & is, unsigned int flags = 0) : + detail::polymorphic_iarchive_route(is, flags) + {} + ~polymorphic_xml_wiarchive(){} +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE( + boost::archive::polymorphic_xml_wiarchive +) + +#endif // BOOST_NO_STD_WSTREAMBUF +#endif // BOOST_ARCHIVE_POLYMORPHIC_XML_WIARCHIVE_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_woarchive.hpp new file mode 100644 index 00000000000..dd8963fbb14 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_woarchive.hpp @@ -0,0 +1,44 @@ +#ifndef BOOST_ARCHIVE_POLYMORPHIC_XML_WOARCHIVE_HPP +#define BOOST_ARCHIVE_POLYMORPHIC_XML_WOARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// polymorphic_xml_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifdef BOOST_NO_STD_WSTREAMBUF +#error "wide char i/o not supported on this platform" +#else + +#include +#include + +namespace boost { +namespace archive { + +typedef detail::polymorphic_oarchive_route< + xml_woarchive_impl +> polymorphic_xml_woarchive; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE( + boost::archive::polymorphic_xml_woarchive +) + +#endif // BOOST_NO_STD_WSTREAMBUF +#endif // BOOST_ARCHIVE_POLYMORPHIC_XML_WOARCHIVE_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/archive/text_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/text_iarchive.hpp new file mode 100644 index 00000000000..d9d60adf0b8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/text_iarchive.hpp @@ -0,0 +1,132 @@ +#ifndef BOOST_ARCHIVE_TEXT_IARCHIVE_HPP +#define BOOST_ARCHIVE_TEXT_IARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// text_iarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include +#include +#include +#include +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_iarchive; +} // namespace detail + +template +class BOOST_SYMBOL_VISIBLE text_iarchive_impl : + public basic_text_iprimitive, + public basic_text_iarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + friend class detail::interface_iarchive; + friend class load_access; +#endif + template + void load(T & t){ + basic_text_iprimitive::load(t); + } + void load(version_type & t){ + unsigned int v; + load(v); + t = version_type(v); + } + void load(boost::serialization::item_version_type & t){ + unsigned int v; + load(v); + t = boost::serialization::item_version_type(v); + } + BOOST_ARCHIVE_DECL void + load(char * t); + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + BOOST_ARCHIVE_DECL void + load(wchar_t * t); + #endif + BOOST_ARCHIVE_DECL void + load(std::string &s); + #ifndef BOOST_NO_STD_WSTRING + BOOST_ARCHIVE_DECL void + load(std::wstring &ws); + #endif + template + void load_override(T & t){ + basic_text_iarchive::load_override(t); + } + BOOST_ARCHIVE_DECL void + load_override(class_name_type & t); + BOOST_ARCHIVE_DECL void + init(); + BOOST_ARCHIVE_DECL + text_iarchive_impl(std::istream & is, unsigned int flags); + // don't import inline definitions! leave this as a reminder. + //BOOST_ARCHIVE_DECL + ~text_iarchive_impl(){}; +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +class BOOST_SYMBOL_VISIBLE text_iarchive : + public text_iarchive_impl{ +public: + text_iarchive(std::istream & is_, unsigned int flags = 0) : + // note: added _ to suppress useless gcc warning + text_iarchive_impl(is_, flags) + {} + ~text_iarchive(){} +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::text_iarchive) + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_ARCHIVE_TEXT_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/text_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/text_oarchive.hpp new file mode 100644 index 00000000000..9ba0dafffb4 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/text_oarchive.hpp @@ -0,0 +1,121 @@ +#ifndef BOOST_ARCHIVE_TEXT_OARCHIVE_HPP +#define BOOST_ARCHIVE_TEXT_OARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// text_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include // std::size_t + +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include +#include +#include +#include +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_oarchive; +} // namespace detail + +template +class BOOST_SYMBOL_VISIBLE text_oarchive_impl : + /* protected ? */ public basic_text_oprimitive, + public basic_text_oarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + friend class detail::interface_oarchive; + friend class basic_text_oarchive; + friend class save_access; +#endif + template + void save(const T & t){ + this->newtoken(); + basic_text_oprimitive::save(t); + } + void save(const version_type & t){ + save(static_cast(t)); + } + void save(const boost::serialization::item_version_type & t){ + save(static_cast(t)); + } + BOOST_ARCHIVE_DECL void + save(const char * t); + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + BOOST_ARCHIVE_DECL void + save(const wchar_t * t); + #endif + BOOST_ARCHIVE_DECL void + save(const std::string &s); + #ifndef BOOST_NO_STD_WSTRING + BOOST_ARCHIVE_DECL void + save(const std::wstring &ws); + #endif + BOOST_ARCHIVE_DECL + text_oarchive_impl(std::ostream & os, unsigned int flags); + // don't import inline definitions! leave this as a reminder. + //BOOST_ARCHIVE_DECL + ~text_oarchive_impl(){}; +public: + BOOST_ARCHIVE_DECL void + save_binary(const void *address, std::size_t count); +}; + +// do not derive from this class. If you want to extend this functionality +// via inhertance, derived from text_oarchive_impl instead. This will +// preserve correct static polymorphism. +class BOOST_SYMBOL_VISIBLE text_oarchive : + public text_oarchive_impl +{ +public: + text_oarchive(std::ostream & os_, unsigned int flags = 0) : + // note: added _ to suppress useless gcc warning + text_oarchive_impl(os_, flags) + {} + ~text_oarchive(){} +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::text_oarchive) + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_ARCHIVE_TEXT_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/text_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/text_wiarchive.hpp new file mode 100644 index 00000000000..3adf068a51a --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/text_wiarchive.hpp @@ -0,0 +1,137 @@ +#ifndef BOOST_ARCHIVE_TEXT_WIARCHIVE_HPP +#define BOOST_ARCHIVE_TEXT_WIARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// text_wiarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifdef BOOST_NO_STD_WSTREAMBUF +#error "wide char i/o not supported on this platform" +#else + +#include + +#include +#include +#include +#include +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_iarchive; +} // namespace detail + +template +class BOOST_SYMBOL_VISIBLE text_wiarchive_impl : + public basic_text_iprimitive, + public basic_text_iarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) + // for some inexplicable reason insertion of "class" generates compile erro + // on msvc 7.1 + friend detail::interface_iarchive; + friend load_access; + #else + friend class detail::interface_iarchive; + friend class load_access; + #endif +#endif + template + void load(T & t){ + basic_text_iprimitive::load(t); + } + void load(version_type & t){ + unsigned int v; + load(v); + t = version_type(v); + } + void load(boost::serialization::item_version_type & t){ + unsigned int v; + load(v); + t = boost::serialization::item_version_type(v); + } + BOOST_WARCHIVE_DECL void + load(char * t); + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + BOOST_WARCHIVE_DECL void + load(wchar_t * t); + #endif + BOOST_WARCHIVE_DECL void + load(std::string &s); + #ifndef BOOST_NO_STD_WSTRING + BOOST_WARCHIVE_DECL void + load(std::wstring &ws); + #endif + template + void load_override(T & t){ + basic_text_iarchive::load_override(t); + } + BOOST_WARCHIVE_DECL + text_wiarchive_impl(std::wistream & is, unsigned int flags); + ~text_wiarchive_impl(){}; +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +class BOOST_SYMBOL_VISIBLE text_wiarchive : + public text_wiarchive_impl{ +public: + text_wiarchive(std::wistream & is, unsigned int flags = 0) : + text_wiarchive_impl(is, flags) + {} + ~text_wiarchive(){} +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::text_wiarchive) + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_NO_STD_WSTREAMBUF +#endif // BOOST_ARCHIVE_TEXT_WIARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/text_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/text_woarchive.hpp new file mode 100644 index 00000000000..b6b4f8ed59a --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/text_woarchive.hpp @@ -0,0 +1,155 @@ +#ifndef BOOST_ARCHIVE_TEXT_WOARCHIVE_HPP +#define BOOST_ARCHIVE_TEXT_WOARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// text_woarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#ifdef BOOST_NO_STD_WSTREAMBUF +#error "wide char i/o not supported on this platform" +#else + +#include +#include // size_t + +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include +#include +#include +#include +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_oarchive; +} // namespace detail + +template +class BOOST_SYMBOL_VISIBLE text_woarchive_impl : + public basic_text_oprimitive, + public basic_text_oarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) + // for some inexplicable reason insertion of "class" generates compile erro + // on msvc 7.1 + friend detail::interface_oarchive; + friend basic_text_oarchive; + friend save_access; + #else + friend class detail::interface_oarchive; + friend class basic_text_oarchive; + friend class save_access; + #endif +#endif + template + void save(const T & t){ + this->newtoken(); + basic_text_oprimitive::save(t); + } + void save(const version_type & t){ + save(static_cast(t)); + } + void save(const boost::serialization::item_version_type & t){ + save(static_cast(t)); + } + BOOST_WARCHIVE_DECL void + save(const char * t); + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + BOOST_WARCHIVE_DECL void + save(const wchar_t * t); + #endif + BOOST_WARCHIVE_DECL void + save(const std::string &s); + #ifndef BOOST_NO_STD_WSTRING + BOOST_WARCHIVE_DECL void + save(const std::wstring &ws); + #endif + text_woarchive_impl(std::wostream & os, unsigned int flags) : + basic_text_oprimitive( + os, + 0 != (flags & no_codecvt) + ), + basic_text_oarchive(flags) + { + if(0 == (flags & no_header)) + basic_text_oarchive::init(); + } +public: + void save_binary(const void *address, std::size_t count){ + put(static_cast('\n')); + this->end_preamble(); + #if ! defined(__MWERKS__) + this->basic_text_oprimitive::save_binary( + #else + this->basic_text_oprimitive::save_binary( + #endif + address, + count + ); + put(static_cast('\n')); + this->delimiter = this->none; + } + +}; + +// we use the following because we can't use +// typedef text_oarchive_impl > text_oarchive; + +// do not derive from this class. If you want to extend this functionality +// via inhertance, derived from text_oarchive_impl instead. This will +// preserve correct static polymorphism. +class BOOST_SYMBOL_VISIBLE text_woarchive : + public text_woarchive_impl +{ +public: + text_woarchive(std::wostream & os, unsigned int flags = 0) : + text_woarchive_impl(os, flags) + {} + ~text_woarchive(){} +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::text_woarchive) + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_NO_STD_WSTREAMBUF +#endif // BOOST_ARCHIVE_TEXT_WOARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/tmpdir.hpp b/contrib/libboost/boost_1_65_0/boost/archive/tmpdir.hpp new file mode 100644 index 00000000000..400d23b9f68 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/tmpdir.hpp @@ -0,0 +1,50 @@ +#ifndef BOOST_ARCHIVE_TMPDIR_HPP +#define BOOST_ARCHIVE_TMPDIR_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// tmpdir.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // getenv +#include // NULL +//#include + +#include +#ifdef BOOST_NO_STDC_NAMESPACE +namespace std { + using ::getenv; +} +#endif + +namespace boost { +namespace archive { + +inline const char * tmpdir(){ + const char *dirname; + dirname = std::getenv("TMP"); + if(NULL == dirname) + dirname = std::getenv("TMPDIR"); + if(NULL == dirname) + dirname = std::getenv("TEMP"); + if(NULL == dirname){ + //BOOST_ASSERT(false); // no temp directory found + dirname = "."; + } + return dirname; +} + +} // archive +} // boost + +#endif // BOOST_ARCHIVE_TMPDIR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/wcslen.hpp b/contrib/libboost/boost_1_65_0/boost/archive/wcslen.hpp new file mode 100644 index 00000000000..0b60004f095 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/wcslen.hpp @@ -0,0 +1,58 @@ +#ifndef BOOST_ARCHIVE_WCSLEN_HPP +#define BOOST_ARCHIVE_WCSLEN_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// wcslen.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // size_t +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#ifndef BOOST_NO_CWCHAR + +// a couple of libraries which include wchar_t don't include +// wcslen + +#if defined(BOOST_DINKUMWARE_STDLIB) && BOOST_DINKUMWARE_STDLIB < 306 \ +|| defined(__LIBCOMO__) + +namespace std { +inline std::size_t wcslen(const wchar_t * ws) +{ + const wchar_t * eows = ws; + while(* eows != 0) + ++eows; + return eows - ws; +} +} // namespace std + +#else + +#ifndef BOOST_NO_CWCHAR +#include +#endif +#ifdef BOOST_NO_STDC_NAMESPACE +namespace std{ using ::wcslen; } +#endif + +#endif // wcslen + +#endif //BOOST_NO_CWCHAR + +#endif //BOOST_ARCHIVE_WCSLEN_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_archive_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_archive_exception.hpp new file mode 100644 index 00000000000..82c53ef5d3e --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/xml_archive_exception.hpp @@ -0,0 +1,57 @@ +#ifndef BOOST_ARCHIVE_XML_ARCHIVE_EXCEPTION_HPP +#define BOOST_ARCHIVE_XML_ARCHIVE_EXCEPTION_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// xml_archive_exception.hpp: + +// (C) Copyright 2007 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include + +#include +#include +#include + +#include // must be the last header + +namespace boost { +namespace archive { + +////////////////////////////////////////////////////////////////////// +// exceptions thrown by xml archives +// +class BOOST_SYMBOL_VISIBLE xml_archive_exception : + public virtual boost::archive::archive_exception +{ +public: + typedef enum { + xml_archive_parsing_error, // see save_register + xml_archive_tag_mismatch, + xml_archive_tag_name_error + } exception_code; + BOOST_ARCHIVE_DECL xml_archive_exception( + exception_code c, + const char * e1 = NULL, + const char * e2 = NULL + ); + BOOST_ARCHIVE_DECL xml_archive_exception(xml_archive_exception const &) ; + virtual BOOST_ARCHIVE_DECL ~xml_archive_exception() BOOST_NOEXCEPT_OR_NOTHROW ; +}; + +}// namespace archive +}// namespace boost + +#include // pops abi_suffix.hpp pragmas + +#endif //BOOST_XML_ARCHIVE_ARCHIVE_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_iarchive.hpp new file mode 100644 index 00000000000..abd2f9fc4e3 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/xml_iarchive.hpp @@ -0,0 +1,142 @@ +#ifndef BOOST_ARCHIVE_XML_IARCHIVE_HPP +#define BOOST_ARCHIVE_XML_IARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// xml_iarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include +#include +#include +#include +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_iarchive; +} // namespace detail + +template +class basic_xml_grammar; +typedef basic_xml_grammar xml_grammar; + +template +class BOOST_SYMBOL_VISIBLE xml_iarchive_impl : + public basic_text_iprimitive, + public basic_xml_iarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + friend class detail::interface_iarchive; + friend class basic_xml_iarchive; + friend class load_access; +#endif + // use boost:scoped_ptr to implement automatic deletion; + boost::scoped_ptr gimpl; + + std::istream & get_is(){ + return is; + } + template + void load(T & t){ + basic_text_iprimitive::load(t); + } + void + load(version_type & t){ + unsigned int v; + load(v); + t = version_type(v); + } + void + load(boost::serialization::item_version_type & t){ + unsigned int v; + load(v); + t = boost::serialization::item_version_type(v); + } + BOOST_ARCHIVE_DECL void + load(char * t); + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + BOOST_ARCHIVE_DECL void + load(wchar_t * t); + #endif + BOOST_ARCHIVE_DECL void + load(std::string &s); + #ifndef BOOST_NO_STD_WSTRING + BOOST_ARCHIVE_DECL void + load(std::wstring &ws); + #endif + template + void load_override(T & t){ + basic_xml_iarchive::load_override(t); + } + BOOST_ARCHIVE_DECL void + load_override(class_name_type & t); + BOOST_ARCHIVE_DECL void + init(); + BOOST_ARCHIVE_DECL + xml_iarchive_impl(std::istream & is, unsigned int flags); + BOOST_ARCHIVE_DECL + ~xml_iarchive_impl(); +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +class BOOST_SYMBOL_VISIBLE xml_iarchive : + public xml_iarchive_impl{ +public: + xml_iarchive(std::istream & is, unsigned int flags = 0) : + xml_iarchive_impl(is, flags) + {} + ~xml_iarchive(){}; +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::xml_iarchive) + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_ARCHIVE_XML_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_oarchive.hpp new file mode 100644 index 00000000000..eea12680372 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/xml_oarchive.hpp @@ -0,0 +1,137 @@ +#ifndef BOOST_ARCHIVE_XML_OARCHIVE_HPP +#define BOOST_ARCHIVE_XML_OARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// xml_oarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include // size_t +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include +#include +#include +#include +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_oarchive; +} // namespace detail + +template +class BOOST_SYMBOL_VISIBLE xml_oarchive_impl : + public basic_text_oprimitive, + public basic_xml_oarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + friend class detail::interface_oarchive; + friend class basic_xml_oarchive; + friend class save_access; +#endif + template + void save(const T & t){ + basic_text_oprimitive::save(t); + } + void + save(const version_type & t){ + save(static_cast(t)); + } + void + save(const boost::serialization::item_version_type & t){ + save(static_cast(t)); + } + BOOST_ARCHIVE_DECL void + save(const char * t); + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + BOOST_ARCHIVE_DECL void + save(const wchar_t * t); + #endif + BOOST_ARCHIVE_DECL void + save(const std::string &s); + #ifndef BOOST_NO_STD_WSTRING + BOOST_ARCHIVE_DECL void + save(const std::wstring &ws); + #endif + BOOST_ARCHIVE_DECL + xml_oarchive_impl(std::ostream & os, unsigned int flags); + BOOST_ARCHIVE_DECL + ~xml_oarchive_impl(); +public: + BOOST_ARCHIVE_DECL + void save_binary(const void *address, std::size_t count); +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +// we use the following because we can't use +// typedef xml_oarchive_impl > xml_oarchive; + +// do not derive from this class. If you want to extend this functionality +// via inhertance, derived from xml_oarchive_impl instead. This will +// preserve correct static polymorphism. +class BOOST_SYMBOL_VISIBLE xml_oarchive : + public xml_oarchive_impl +{ +public: + xml_oarchive(std::ostream & os, unsigned int flags = 0) : + xml_oarchive_impl(os, flags) + {} + ~xml_oarchive(){} +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::xml_oarchive) + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_ARCHIVE_XML_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_wiarchive.hpp new file mode 100644 index 00000000000..ac24289ac11 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/xml_wiarchive.hpp @@ -0,0 +1,149 @@ +#ifndef BOOST_ARCHIVE_XML_WIARCHIVE_HPP +#define BOOST_ARCHIVE_XML_WIARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// xml_wiarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifdef BOOST_NO_STD_WSTREAMBUF +#error "wide char i/o not supported on this platform" +#else + +#include + +#include +#include +#include +#include +#include +#include +// #include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_iarchive; +} // namespace detail + +template +class basic_xml_grammar; +typedef basic_xml_grammar xml_wgrammar; + +template +class BOOST_SYMBOL_VISIBLE xml_wiarchive_impl : + public basic_text_iprimitive, + public basic_xml_iarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + friend class detail::interface_iarchive; + friend class basic_xml_iarchive; + friend class load_access; +#endif + boost::scoped_ptr gimpl; + std::wistream & get_is(){ + return is; + } + template + void + load(T & t){ + basic_text_iprimitive::load(t); + } + void + load(version_type & t){ + unsigned int v; + load(v); + t = version_type(v); + } + void + load(boost::serialization::item_version_type & t){ + unsigned int v; + load(v); + t = boost::serialization::item_version_type(v); + } + BOOST_WARCHIVE_DECL void + load(char * t); + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + BOOST_WARCHIVE_DECL void + load(wchar_t * t); + #endif + BOOST_WARCHIVE_DECL void + load(std::string &s); + #ifndef BOOST_NO_STD_WSTRING + BOOST_WARCHIVE_DECL void + load(std::wstring &ws); + #endif + template + void load_override(T & t){ + basic_xml_iarchive::load_override(t); + } + BOOST_WARCHIVE_DECL void + load_override(class_name_type & t); + BOOST_WARCHIVE_DECL void + init(); + BOOST_WARCHIVE_DECL + xml_wiarchive_impl(std::wistream & is, unsigned int flags) ; + BOOST_WARCHIVE_DECL + ~xml_wiarchive_impl(); +}; + +} // namespace archive +} // namespace boost + +#ifdef BOOST_MSVC +# pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +class BOOST_SYMBOL_VISIBLE xml_wiarchive : + public xml_wiarchive_impl{ +public: + xml_wiarchive(std::wistream & is, unsigned int flags = 0) : + xml_wiarchive_impl(is, flags) + {} + ~xml_wiarchive(){} +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::xml_wiarchive) + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_NO_STD_WSTREAMBUF +#endif // BOOST_ARCHIVE_XML_WIARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_woarchive.hpp new file mode 100644 index 00000000000..cb7ce68cb6f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/archive/xml_woarchive.hpp @@ -0,0 +1,134 @@ +#ifndef BOOST_ARCHIVE_XML_WOARCHIVE_HPP +#define BOOST_ARCHIVE_XML_WOARCHIVE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// xml_woarchive.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifdef BOOST_NO_STD_WSTREAMBUF +#error "wide char i/o not supported on this platform" +#else +#include // size_t +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include + +//#include +#include +#include +#include +#include +#include +//#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace archive { + +namespace detail { + template class interface_oarchive; +} // namespace detail + +template +class BOOST_SYMBOL_VISIBLE xml_woarchive_impl : + public basic_text_oprimitive, + public basic_xml_oarchive +{ +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else +protected: + friend class detail::interface_oarchive; + friend class basic_xml_oarchive; + friend class save_access; +#endif + //void end_preamble(){ + // basic_xml_oarchive::end_preamble(); + //} + template + void + save(const T & t){ + basic_text_oprimitive::save(t); + } + void + save(const version_type & t){ + save(static_cast(t)); + } + void + save(const boost::serialization::item_version_type & t){ + save(static_cast(t)); + } + BOOST_WARCHIVE_DECL void + save(const char * t); + #ifndef BOOST_NO_INTRINSIC_WCHAR_T + BOOST_WARCHIVE_DECL void + save(const wchar_t * t); + #endif + BOOST_WARCHIVE_DECL void + save(const std::string &s); + #ifndef BOOST_NO_STD_WSTRING + BOOST_WARCHIVE_DECL void + save(const std::wstring &ws); + #endif + BOOST_WARCHIVE_DECL + xml_woarchive_impl(std::wostream & os, unsigned int flags); + BOOST_WARCHIVE_DECL + ~xml_woarchive_impl(); +public: + BOOST_WARCHIVE_DECL void + save_binary(const void *address, std::size_t count); + +}; + +// we use the following because we can't use +// typedef xml_woarchive_impl > xml_woarchive; + +// do not derive from this class. If you want to extend this functionality +// via inhertance, derived from xml_woarchive_impl instead. This will +// preserve correct static polymorphism. +class BOOST_SYMBOL_VISIBLE xml_woarchive : + public xml_woarchive_impl +{ +public: + xml_woarchive(std::wostream & os, unsigned int flags = 0) : + xml_woarchive_impl(os, flags) + {} + ~xml_woarchive(){} +}; + +} // namespace archive +} // namespace boost + +// required by export +BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::xml_woarchive) + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_NO_STD_WSTREAMBUF +#endif // BOOST_ARCHIVE_XML_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/foreach_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/foreach_fwd.hpp new file mode 100644 index 00000000000..4e0bb370c2f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/foreach_fwd.hpp @@ -0,0 +1,51 @@ +/////////////////////////////////////////////////////////////////////////////// +// foreach.hpp header file +// +// Copyright 2010 Eric Niebler. +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) +// See http://www.boost.org/libs/foreach for documentation +// +// Credits: +// Kazutoshi Satoda: for suggesting the need for a _fwd header for foreach's +// customization points. + +#ifndef BOOST_FOREACH_FWD_HPP +#define BOOST_FOREACH_FWD_HPP + +// This must be at global scope, hence the uglified name +enum boost_foreach_argument_dependent_lookup_hack +{ + boost_foreach_argument_dependent_lookup_hack_value +}; + +namespace boost +{ + +namespace foreach +{ + /////////////////////////////////////////////////////////////////////////////// + // boost::foreach::tag + // + typedef boost_foreach_argument_dependent_lookup_hack tag; + + /////////////////////////////////////////////////////////////////////////////// + // boost::foreach::is_lightweight_proxy + // Specialize this for user-defined collection types if they are inexpensive to copy. + // This tells BOOST_FOREACH it can avoid the rvalue/lvalue detection stuff. + template + struct is_lightweight_proxy; + + /////////////////////////////////////////////////////////////////////////////// + // boost::foreach::is_noncopyable + // Specialize this for user-defined collection types if they cannot be copied. + // This also tells BOOST_FOREACH to avoid the rvalue/lvalue detection stuff. + template + struct is_noncopyable; + +} // namespace foreach + +} // namespace boost + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/composite_key.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/composite_key.hpp new file mode 100644 index 00000000000..787cdf83195 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/composite_key.hpp @@ -0,0 +1,1513 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_COMPOSITE_KEY_HPP +#define BOOST_MULTI_INDEX_COMPOSITE_KEY_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if !defined(BOOST_NO_FUNCTION_TEMPLATE_ORDERING) +#include +#endif + +#if !defined(BOOST_NO_SFINAE) +#include +#endif + +#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ + !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) +#include +#endif + +/* A composite key stores n key extractors and "computes" the + * result on a given value as a packed reference to the value and + * the composite key itself. Actual invocations to the component + * key extractors are lazily performed when executing an operation + * on composite_key results (equality, comparison, hashing.) + * As the other key extractors in Boost.MultiIndex, composite_key + * is overloaded to work on chained pointers to T and reference_wrappers + * of T. + */ + +/* This user_definable macro limits the number of elements of a composite + * key; useful for shortening resulting symbol names (MSVC++ 6.0, for + * instance has problems coping with very long symbol names.) + * NB: This cannot exceed the maximum number of arguments of + * boost::tuple. In Boost 1.32, the limit is 10. + */ + +#if !defined(BOOST_MULTI_INDEX_LIMIT_COMPOSITE_KEY_SIZE) +#define BOOST_MULTI_INDEX_LIMIT_COMPOSITE_KEY_SIZE 10 +#endif + +/* maximum number of key extractors in a composite key */ + +#if BOOST_MULTI_INDEX_LIMIT_COMPOSITE_KEY_SIZE<10 /* max length of a tuple */ +#define BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE \ + BOOST_MULTI_INDEX_LIMIT_COMPOSITE_KEY_SIZE +#else +#define BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE 10 +#endif + +/* BOOST_PP_ENUM of BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE elements */ + +#define BOOST_MULTI_INDEX_CK_ENUM(macro,data) \ + BOOST_PP_ENUM(BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE,macro,data) + +/* BOOST_PP_ENUM_PARAMS of BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE elements */ + +#define BOOST_MULTI_INDEX_CK_ENUM_PARAMS(param) \ + BOOST_PP_ENUM_PARAMS(BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE,param) + +/* if n==0 -> text0 + * otherwise -> textn=tuples::null_type + */ + +#define BOOST_MULTI_INDEX_CK_TEMPLATE_PARM(z,n,text) \ + typename BOOST_PP_CAT(text,n) BOOST_PP_EXPR_IF(n,=tuples::null_type) + +/* const textn& kn=textn() */ + +#define BOOST_MULTI_INDEX_CK_CTOR_ARG(z,n,text) \ + const BOOST_PP_CAT(text,n)& BOOST_PP_CAT(k,n) = BOOST_PP_CAT(text,n)() + +/* typename list(0)::type */ + +#define BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N(z,n,list) \ + BOOST_DEDUCED_TYPENAME BOOST_PP_LIST_AT(list,0)< \ + BOOST_PP_LIST_AT(list,1),n \ + >::type + +namespace boost{ + +template class reference_wrapper; /* fwd decl. */ + +namespace multi_index{ + +namespace detail{ + +/* n-th key extractor of a composite key */ + +template +struct nth_key_from_value +{ + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename mpl::eval_if_c< + N::value, + tuples::element, + mpl::identity + >::type type; +}; + +/* nth_composite_key_##name::type yields + * functor >, or tuples::null_type + * if N exceeds the length of the composite key. + */ + +#define BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(name,functor) \ +template \ +struct BOOST_PP_CAT(key_,name) \ +{ \ + typedef functor type; \ +}; \ + \ +template<> \ +struct BOOST_PP_CAT(key_,name) \ +{ \ + typedef tuples::null_type type; \ +}; \ + \ +template \ +struct BOOST_PP_CAT(nth_composite_key_,name) \ +{ \ + typedef typename nth_key_from_value::type key_from_value; \ + typedef typename BOOST_PP_CAT(key_,name)::type type; \ +}; + +/* nth_composite_key_equal_to + * nth_composite_key_less + * nth_composite_key_greater + * nth_composite_key_hash + */ + +BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(equal_to,std::equal_to) +BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(less,std::less) +BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(greater,std::greater) +BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(hash,boost::hash) + +/* used for defining equality and comparison ops of composite_key_result */ + +#define BOOST_MULTI_INDEX_CK_IDENTITY_ENUM_MACRO(z,n,text) text + +struct generic_operator_equal +{ + template + bool operator()(const T& x,const Q& y)const{return x==y;} +}; + +typedef tuple< + BOOST_MULTI_INDEX_CK_ENUM( + BOOST_MULTI_INDEX_CK_IDENTITY_ENUM_MACRO, + detail::generic_operator_equal)> generic_operator_equal_tuple; + +struct generic_operator_less +{ + template + bool operator()(const T& x,const Q& y)const{return x generic_operator_less_tuple; + +/* Metaprogramming machinery for implementing equality, comparison and + * hashing operations of composite_key_result. + * + * equal_* checks for equality between composite_key_results and + * between those and tuples, accepting a tuple of basic equality functors. + * compare_* does lexicographical comparison. + * hash_* computes a combination of elementwise hash values. + */ + +template +< + typename KeyCons1,typename Value1, + typename KeyCons2, typename Value2, + typename EqualCons +> +struct equal_ckey_ckey; /* fwd decl. */ + +template +< + typename KeyCons1,typename Value1, + typename KeyCons2, typename Value2, + typename EqualCons +> +struct equal_ckey_ckey_terminal +{ + static bool compare( + const KeyCons1&,const Value1&, + const KeyCons2&,const Value2&, + const EqualCons&) + { + return true; + } +}; + +template +< + typename KeyCons1,typename Value1, + typename KeyCons2, typename Value2, + typename EqualCons +> +struct equal_ckey_ckey_normal +{ + static bool compare( + const KeyCons1& c0,const Value1& v0, + const KeyCons2& c1,const Value2& v1, + const EqualCons& eq) + { + if(!eq.get_head()(c0.get_head()(v0),c1.get_head()(v1)))return false; + return equal_ckey_ckey< + BOOST_DEDUCED_TYPENAME KeyCons1::tail_type,Value1, + BOOST_DEDUCED_TYPENAME KeyCons2::tail_type,Value2, + BOOST_DEDUCED_TYPENAME EqualCons::tail_type + >::compare(c0.get_tail(),v0,c1.get_tail(),v1,eq.get_tail()); + } +}; + +template +< + typename KeyCons1,typename Value1, + typename KeyCons2, typename Value2, + typename EqualCons +> +struct equal_ckey_ckey: + mpl::if_< + mpl::or_< + is_same, + is_same + >, + equal_ckey_ckey_terminal, + equal_ckey_ckey_normal + >::type +{ +}; + +template +< + typename KeyCons,typename Value, + typename ValCons,typename EqualCons +> +struct equal_ckey_cval; /* fwd decl. */ + +template +< + typename KeyCons,typename Value, + typename ValCons,typename EqualCons +> +struct equal_ckey_cval_terminal +{ + static bool compare( + const KeyCons&,const Value&,const ValCons&,const EqualCons&) + { + return true; + } + + static bool compare( + const ValCons&,const KeyCons&,const Value&,const EqualCons&) + { + return true; + } +}; + +template +< + typename KeyCons,typename Value, + typename ValCons,typename EqualCons +> +struct equal_ckey_cval_normal +{ + static bool compare( + const KeyCons& c,const Value& v,const ValCons& vc, + const EqualCons& eq) + { + if(!eq.get_head()(c.get_head()(v),vc.get_head()))return false; + return equal_ckey_cval< + BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, + BOOST_DEDUCED_TYPENAME ValCons::tail_type, + BOOST_DEDUCED_TYPENAME EqualCons::tail_type + >::compare(c.get_tail(),v,vc.get_tail(),eq.get_tail()); + } + + static bool compare( + const ValCons& vc,const KeyCons& c,const Value& v, + const EqualCons& eq) + { + if(!eq.get_head()(vc.get_head(),c.get_head()(v)))return false; + return equal_ckey_cval< + BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, + BOOST_DEDUCED_TYPENAME ValCons::tail_type, + BOOST_DEDUCED_TYPENAME EqualCons::tail_type + >::compare(vc.get_tail(),c.get_tail(),v,eq.get_tail()); + } +}; + +template +< + typename KeyCons,typename Value, + typename ValCons,typename EqualCons +> +struct equal_ckey_cval: + mpl::if_< + mpl::or_< + is_same, + is_same + >, + equal_ckey_cval_terminal, + equal_ckey_cval_normal + >::type +{ +}; + +template +< + typename KeyCons1,typename Value1, + typename KeyCons2, typename Value2, + typename CompareCons +> +struct compare_ckey_ckey; /* fwd decl. */ + +template +< + typename KeyCons1,typename Value1, + typename KeyCons2, typename Value2, + typename CompareCons +> +struct compare_ckey_ckey_terminal +{ + static bool compare( + const KeyCons1&,const Value1&, + const KeyCons2&,const Value2&, + const CompareCons&) + { + return false; + } +}; + +template +< + typename KeyCons1,typename Value1, + typename KeyCons2, typename Value2, + typename CompareCons +> +struct compare_ckey_ckey_normal +{ + static bool compare( + const KeyCons1& c0,const Value1& v0, + const KeyCons2& c1,const Value2& v1, + const CompareCons& comp) + { + if(comp.get_head()(c0.get_head()(v0),c1.get_head()(v1)))return true; + if(comp.get_head()(c1.get_head()(v1),c0.get_head()(v0)))return false; + return compare_ckey_ckey< + BOOST_DEDUCED_TYPENAME KeyCons1::tail_type,Value1, + BOOST_DEDUCED_TYPENAME KeyCons2::tail_type,Value2, + BOOST_DEDUCED_TYPENAME CompareCons::tail_type + >::compare(c0.get_tail(),v0,c1.get_tail(),v1,comp.get_tail()); + } +}; + +template +< + typename KeyCons1,typename Value1, + typename KeyCons2, typename Value2, + typename CompareCons +> +struct compare_ckey_ckey: + mpl::if_< + mpl::or_< + is_same, + is_same + >, + compare_ckey_ckey_terminal, + compare_ckey_ckey_normal + >::type +{ +}; + +template +< + typename KeyCons,typename Value, + typename ValCons,typename CompareCons +> +struct compare_ckey_cval; /* fwd decl. */ + +template +< + typename KeyCons,typename Value, + typename ValCons,typename CompareCons +> +struct compare_ckey_cval_terminal +{ + static bool compare( + const KeyCons&,const Value&,const ValCons&,const CompareCons&) + { + return false; + } + + static bool compare( + const ValCons&,const KeyCons&,const Value&,const CompareCons&) + { + return false; + } +}; + +template +< + typename KeyCons,typename Value, + typename ValCons,typename CompareCons +> +struct compare_ckey_cval_normal +{ + static bool compare( + const KeyCons& c,const Value& v,const ValCons& vc, + const CompareCons& comp) + { + if(comp.get_head()(c.get_head()(v),vc.get_head()))return true; + if(comp.get_head()(vc.get_head(),c.get_head()(v)))return false; + return compare_ckey_cval< + BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, + BOOST_DEDUCED_TYPENAME ValCons::tail_type, + BOOST_DEDUCED_TYPENAME CompareCons::tail_type + >::compare(c.get_tail(),v,vc.get_tail(),comp.get_tail()); + } + + static bool compare( + const ValCons& vc,const KeyCons& c,const Value& v, + const CompareCons& comp) + { + if(comp.get_head()(vc.get_head(),c.get_head()(v)))return true; + if(comp.get_head()(c.get_head()(v),vc.get_head()))return false; + return compare_ckey_cval< + BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, + BOOST_DEDUCED_TYPENAME ValCons::tail_type, + BOOST_DEDUCED_TYPENAME CompareCons::tail_type + >::compare(vc.get_tail(),c.get_tail(),v,comp.get_tail()); + } +}; + +template +< + typename KeyCons,typename Value, + typename ValCons,typename CompareCons +> +struct compare_ckey_cval: + mpl::if_< + mpl::or_< + is_same, + is_same + >, + compare_ckey_cval_terminal, + compare_ckey_cval_normal + >::type +{ +}; + +template +struct hash_ckey; /* fwd decl. */ + +template +struct hash_ckey_terminal +{ + static std::size_t hash( + const KeyCons&,const Value&,const HashCons&,std::size_t carry) + { + return carry; + } +}; + +template +struct hash_ckey_normal +{ + static std::size_t hash( + const KeyCons& c,const Value& v,const HashCons& h,std::size_t carry=0) + { + /* same hashing formula as boost::hash_combine */ + + carry^=h.get_head()(c.get_head()(v))+0x9e3779b9+(carry<<6)+(carry>>2); + return hash_ckey< + BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, + BOOST_DEDUCED_TYPENAME HashCons::tail_type + >::hash(c.get_tail(),v,h.get_tail(),carry); + } +}; + +template +struct hash_ckey: + mpl::if_< + is_same, + hash_ckey_terminal, + hash_ckey_normal + >::type +{ +}; + +template +struct hash_cval; /* fwd decl. */ + +template +struct hash_cval_terminal +{ + static std::size_t hash(const ValCons&,const HashCons&,std::size_t carry) + { + return carry; + } +}; + +template +struct hash_cval_normal +{ + static std::size_t hash( + const ValCons& vc,const HashCons& h,std::size_t carry=0) + { + carry^=h.get_head()(vc.get_head())+0x9e3779b9+(carry<<6)+(carry>>2); + return hash_cval< + BOOST_DEDUCED_TYPENAME ValCons::tail_type, + BOOST_DEDUCED_TYPENAME HashCons::tail_type + >::hash(vc.get_tail(),h.get_tail(),carry); + } +}; + +template +struct hash_cval: + mpl::if_< + is_same, + hash_cval_terminal, + hash_cval_normal + >::type +{ +}; + +} /* namespace multi_index::detail */ + +/* composite_key_result */ + +#if defined(BOOST_MSVC) +#pragma warning(push) +#pragma warning(disable:4512) +#endif + +template +struct composite_key_result +{ + typedef CompositeKey composite_key_type; + typedef typename composite_key_type::value_type value_type; + + composite_key_result( + const composite_key_type& composite_key_,const value_type& value_): + composite_key(composite_key_),value(value_) + {} + + const composite_key_type& composite_key; + const value_type& value; +}; + +#if defined(BOOST_MSVC) +#pragma warning(pop) +#endif + +/* composite_key */ + +template< + typename Value, + BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_TEMPLATE_PARM,KeyFromValue) +> +struct composite_key: + private tuple +{ +private: + typedef tuple super; + +public: + typedef super key_extractor_tuple; + typedef Value value_type; + typedef composite_key_result result_type; + + composite_key( + BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_CTOR_ARG,KeyFromValue)): + super(BOOST_MULTI_INDEX_CK_ENUM_PARAMS(k)) + {} + + composite_key(const key_extractor_tuple& x):super(x){} + + const key_extractor_tuple& key_extractors()const{return *this;} + key_extractor_tuple& key_extractors(){return *this;} + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if< + is_convertible,result_type>::type +#else + result_type +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + result_type operator()(const value_type& x)const + { + return result_type(*this,x); + } + + result_type operator()(const reference_wrapper& x)const + { + return result_type(*this,x.get()); + } + + result_type operator()(const reference_wrapper& x)const + { + return result_type(*this,x.get()); + } +}; + +/* comparison operators */ + +/* == */ + +template +inline bool operator==( + const composite_key_result& x, + const composite_key_result& y) +{ + typedef typename CompositeKey1::key_extractor_tuple key_extractor_tuple1; + typedef typename CompositeKey1::value_type value_type1; + typedef typename CompositeKey2::key_extractor_tuple key_extractor_tuple2; + typedef typename CompositeKey2::value_type value_type2; + + BOOST_STATIC_ASSERT( + tuples::length::value== + tuples::length::value); + + return detail::equal_ckey_ckey< + key_extractor_tuple1,value_type1, + key_extractor_tuple2,value_type2, + detail::generic_operator_equal_tuple + >::compare( + x.composite_key.key_extractors(),x.value, + y.composite_key.key_extractors(),y.value, + detail::generic_operator_equal_tuple()); +} + +template< + typename CompositeKey, + BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value) +> +inline bool operator==( + const composite_key_result& x, + const tuple& y) +{ + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef tuple key_tuple; + + BOOST_STATIC_ASSERT( + tuples::length::value== + tuples::length::value); + + return detail::equal_ckey_cval< + key_extractor_tuple,value_type, + key_tuple,detail::generic_operator_equal_tuple + >::compare( + x.composite_key.key_extractors(),x.value, + y,detail::generic_operator_equal_tuple()); +} + +template +< + BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), + typename CompositeKey +> +inline bool operator==( + const tuple& x, + const composite_key_result& y) +{ + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef tuple key_tuple; + + BOOST_STATIC_ASSERT( + tuples::length::value== + tuples::length::value); + + return detail::equal_ckey_cval< + key_extractor_tuple,value_type, + key_tuple,detail::generic_operator_equal_tuple + >::compare( + x,y.composite_key.key_extractors(), + y.value,detail::generic_operator_equal_tuple()); +} + +#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ + !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) +template +inline bool operator==( + const composite_key_result& x, + const std::tuple& y) +{ + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef std::tuple key_tuple; + typedef typename detail::cons_stdtuple_ctor< + key_tuple>::result_type cons_key_tuple; + + BOOST_STATIC_ASSERT( + static_cast(tuples::length::value)== + std::tuple_size::value); + + return detail::equal_ckey_cval< + key_extractor_tuple,value_type, + cons_key_tuple,detail::generic_operator_equal_tuple + >::compare( + x.composite_key.key_extractors(),x.value, + detail::make_cons_stdtuple(y),detail::generic_operator_equal_tuple()); +} + +template +inline bool operator==( + const std::tuple& x, + const composite_key_result& y) +{ + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef std::tuple key_tuple; + typedef typename detail::cons_stdtuple_ctor< + key_tuple>::result_type cons_key_tuple; + + BOOST_STATIC_ASSERT( + static_cast(tuples::length::value)== + std::tuple_size::value); + + return detail::equal_ckey_cval< + key_extractor_tuple,value_type, + cons_key_tuple,detail::generic_operator_equal_tuple + >::compare( + detail::make_cons_stdtuple(x),y.composite_key.key_extractors(), + y.value,detail::generic_operator_equal_tuple()); +} +#endif + +/* < */ + +template +inline bool operator<( + const composite_key_result& x, + const composite_key_result& y) +{ + typedef typename CompositeKey1::key_extractor_tuple key_extractor_tuple1; + typedef typename CompositeKey1::value_type value_type1; + typedef typename CompositeKey2::key_extractor_tuple key_extractor_tuple2; + typedef typename CompositeKey2::value_type value_type2; + + return detail::compare_ckey_ckey< + key_extractor_tuple1,value_type1, + key_extractor_tuple2,value_type2, + detail::generic_operator_less_tuple + >::compare( + x.composite_key.key_extractors(),x.value, + y.composite_key.key_extractors(),y.value, + detail::generic_operator_less_tuple()); +} + +template +< + typename CompositeKey, + BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value) +> +inline bool operator<( + const composite_key_result& x, + const tuple& y) +{ + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef tuple key_tuple; + + return detail::compare_ckey_cval< + key_extractor_tuple,value_type, + key_tuple,detail::generic_operator_less_tuple + >::compare( + x.composite_key.key_extractors(),x.value, + y,detail::generic_operator_less_tuple()); +} + +template +< + BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), + typename CompositeKey +> +inline bool operator<( + const tuple& x, + const composite_key_result& y) +{ + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef tuple key_tuple; + + return detail::compare_ckey_cval< + key_extractor_tuple,value_type, + key_tuple,detail::generic_operator_less_tuple + >::compare( + x,y.composite_key.key_extractors(), + y.value,detail::generic_operator_less_tuple()); +} + +#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ + !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) +template +inline bool operator<( + const composite_key_result& x, + const std::tuple& y) +{ + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef std::tuple key_tuple; + typedef typename detail::cons_stdtuple_ctor< + key_tuple>::result_type cons_key_tuple; + + return detail::compare_ckey_cval< + key_extractor_tuple,value_type, + cons_key_tuple,detail::generic_operator_less_tuple + >::compare( + x.composite_key.key_extractors(),x.value, + detail::make_cons_stdtuple(y),detail::generic_operator_less_tuple()); +} + +template +inline bool operator<( + const std::tuple& x, + const composite_key_result& y) +{ + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef std::tuple key_tuple; + typedef typename detail::cons_stdtuple_ctor< + key_tuple>::result_type cons_key_tuple; + + return detail::compare_ckey_cval< + key_extractor_tuple,value_type, + cons_key_tuple,detail::generic_operator_less_tuple + >::compare( + detail::make_cons_stdtuple(x),y.composite_key.key_extractors(), + y.value,detail::generic_operator_less_tuple()); +} +#endif + +/* rest of comparison operators */ + +#define BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS(t1,t2,a1,a2) \ +template inline bool operator!=(const a1& x,const a2& y) \ +{ \ + return !(x==y); \ +} \ + \ +template inline bool operator>(const a1& x,const a2& y) \ +{ \ + return y inline bool operator>=(const a1& x,const a2& y) \ +{ \ + return !(x inline bool operator<=(const a1& x,const a2& y) \ +{ \ + return !(y, + composite_key_result +) + +BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS( + typename CompositeKey, + BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), + composite_key_result, + tuple +) + +BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS( + BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), + typename CompositeKey, + tuple, + composite_key_result +) + +#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ + !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) +BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS( + typename CompositeKey, + typename... Values, + composite_key_result, + std::tuple +) + +BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS( + typename CompositeKey, + typename... Values, + std::tuple, + composite_key_result +) +#endif + +/* composite_key_equal_to */ + +template +< + BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_TEMPLATE_PARM,Pred) +> +struct composite_key_equal_to: + private tuple +{ +private: + typedef tuple super; + +public: + typedef super key_eq_tuple; + + composite_key_equal_to( + BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_CTOR_ARG,Pred)): + super(BOOST_MULTI_INDEX_CK_ENUM_PARAMS(k)) + {} + + composite_key_equal_to(const key_eq_tuple& x):super(x){} + + const key_eq_tuple& key_eqs()const{return *this;} + key_eq_tuple& key_eqs(){return *this;} + + template + bool operator()( + const composite_key_result & x, + const composite_key_result & y)const + { + typedef typename CompositeKey1::key_extractor_tuple key_extractor_tuple1; + typedef typename CompositeKey1::value_type value_type1; + typedef typename CompositeKey2::key_extractor_tuple key_extractor_tuple2; + typedef typename CompositeKey2::value_type value_type2; + + BOOST_STATIC_ASSERT( + tuples::length::value<= + tuples::length::value&& + tuples::length::value== + tuples::length::value); + + return detail::equal_ckey_ckey< + key_extractor_tuple1,value_type1, + key_extractor_tuple2,value_type2, + key_eq_tuple + >::compare( + x.composite_key.key_extractors(),x.value, + y.composite_key.key_extractors(),y.value, + key_eqs()); + } + + template + < + typename CompositeKey, + BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value) + > + bool operator()( + const composite_key_result& x, + const tuple& y)const + { + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef tuple key_tuple; + + BOOST_STATIC_ASSERT( + tuples::length::value<= + tuples::length::value&& + tuples::length::value== + tuples::length::value); + + return detail::equal_ckey_cval< + key_extractor_tuple,value_type, + key_tuple,key_eq_tuple + >::compare(x.composite_key.key_extractors(),x.value,y,key_eqs()); + } + + template + < + BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), + typename CompositeKey + > + bool operator()( + const tuple& x, + const composite_key_result& y)const + { + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef tuple key_tuple; + + BOOST_STATIC_ASSERT( + tuples::length::value<= + tuples::length::value&& + tuples::length::value== + tuples::length::value); + + return detail::equal_ckey_cval< + key_extractor_tuple,value_type, + key_tuple,key_eq_tuple + >::compare(x,y.composite_key.key_extractors(),y.value,key_eqs()); + } + +#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ + !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) + template + bool operator()( + const composite_key_result& x, + const std::tuple& y)const + { + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef std::tuple key_tuple; + typedef typename detail::cons_stdtuple_ctor< + key_tuple>::result_type cons_key_tuple; + + BOOST_STATIC_ASSERT( + tuples::length::value<= + tuples::length::value&& + static_cast(tuples::length::value)== + std::tuple_size::value); + + return detail::equal_ckey_cval< + key_extractor_tuple,value_type, + cons_key_tuple,key_eq_tuple + >::compare( + x.composite_key.key_extractors(),x.value, + detail::make_cons_stdtuple(y),key_eqs()); + } + + template + bool operator()( + const std::tuple& x, + const composite_key_result& y)const + { + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef std::tuple key_tuple; + typedef typename detail::cons_stdtuple_ctor< + key_tuple>::result_type cons_key_tuple; + + BOOST_STATIC_ASSERT( + std::tuple_size::value<= + static_cast(tuples::length::value)&& + std::tuple_size::value== + static_cast(tuples::length::value)); + + return detail::equal_ckey_cval< + key_extractor_tuple,value_type, + cons_key_tuple,key_eq_tuple + >::compare( + detail::make_cons_stdtuple(x),y.composite_key.key_extractors(), + y.value,key_eqs()); + } +#endif +}; + +/* composite_key_compare */ + +template +< + BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_TEMPLATE_PARM,Compare) +> +struct composite_key_compare: + private tuple +{ +private: + typedef tuple super; + +public: + typedef super key_comp_tuple; + + composite_key_compare( + BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_CTOR_ARG,Compare)): + super(BOOST_MULTI_INDEX_CK_ENUM_PARAMS(k)) + {} + + composite_key_compare(const key_comp_tuple& x):super(x){} + + const key_comp_tuple& key_comps()const{return *this;} + key_comp_tuple& key_comps(){return *this;} + + template + bool operator()( + const composite_key_result & x, + const composite_key_result & y)const + { + typedef typename CompositeKey1::key_extractor_tuple key_extractor_tuple1; + typedef typename CompositeKey1::value_type value_type1; + typedef typename CompositeKey2::key_extractor_tuple key_extractor_tuple2; + typedef typename CompositeKey2::value_type value_type2; + + BOOST_STATIC_ASSERT( + tuples::length::value<= + tuples::length::value|| + tuples::length::value<= + tuples::length::value); + + return detail::compare_ckey_ckey< + key_extractor_tuple1,value_type1, + key_extractor_tuple2,value_type2, + key_comp_tuple + >::compare( + x.composite_key.key_extractors(),x.value, + y.composite_key.key_extractors(),y.value, + key_comps()); + } + +#if !defined(BOOST_NO_FUNCTION_TEMPLATE_ORDERING) + template + bool operator()( + const composite_key_result& x, + const Value& y)const + { + return operator()(x,boost::make_tuple(boost::cref(y))); + } +#endif + + template + < + typename CompositeKey, + BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value) + > + bool operator()( + const composite_key_result& x, + const tuple& y)const + { + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef tuple key_tuple; + + BOOST_STATIC_ASSERT( + tuples::length::value<= + tuples::length::value|| + tuples::length::value<= + tuples::length::value); + + return detail::compare_ckey_cval< + key_extractor_tuple,value_type, + key_tuple,key_comp_tuple + >::compare(x.composite_key.key_extractors(),x.value,y,key_comps()); + } + +#if !defined(BOOST_NO_FUNCTION_TEMPLATE_ORDERING) + template + bool operator()( + const Value& x, + const composite_key_result& y)const + { + return operator()(boost::make_tuple(boost::cref(x)),y); + } +#endif + + template + < + BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), + typename CompositeKey + > + bool operator()( + const tuple& x, + const composite_key_result& y)const + { + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef tuple key_tuple; + + BOOST_STATIC_ASSERT( + tuples::length::value<= + tuples::length::value|| + tuples::length::value<= + tuples::length::value); + + return detail::compare_ckey_cval< + key_extractor_tuple,value_type, + key_tuple,key_comp_tuple + >::compare(x,y.composite_key.key_extractors(),y.value,key_comps()); + } + +#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ + !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) + template + bool operator()( + const composite_key_result& x, + const std::tuple& y)const + { + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef std::tuple key_tuple; + typedef typename detail::cons_stdtuple_ctor< + key_tuple>::result_type cons_key_tuple; + + BOOST_STATIC_ASSERT( + tuples::length::value<= + tuples::length::value|| + std::tuple_size::value<= + static_cast(tuples::length::value)); + + return detail::compare_ckey_cval< + key_extractor_tuple,value_type, + cons_key_tuple,key_comp_tuple + >::compare( + x.composite_key.key_extractors(),x.value, + detail::make_cons_stdtuple(y),key_comps()); + } + + template + bool operator()( + const std::tuple& x, + const composite_key_result& y)const + { + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + typedef std::tuple key_tuple; + typedef typename detail::cons_stdtuple_ctor< + key_tuple>::result_type cons_key_tuple; + + BOOST_STATIC_ASSERT( + std::tuple_size::value<= + static_cast(tuples::length::value)|| + tuples::length::value<= + tuples::length::value); + + return detail::compare_ckey_cval< + key_extractor_tuple,value_type, + cons_key_tuple,key_comp_tuple + >::compare( + detail::make_cons_stdtuple(x),y.composite_key.key_extractors(), + y.value,key_comps()); + } +#endif +}; + +/* composite_key_hash */ + +template +< + BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_TEMPLATE_PARM,Hash) +> +struct composite_key_hash: + private tuple +{ +private: + typedef tuple super; + +public: + typedef super key_hasher_tuple; + + composite_key_hash( + BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_CTOR_ARG,Hash)): + super(BOOST_MULTI_INDEX_CK_ENUM_PARAMS(k)) + {} + + composite_key_hash(const key_hasher_tuple& x):super(x){} + + const key_hasher_tuple& key_hash_functions()const{return *this;} + key_hasher_tuple& key_hash_functions(){return *this;} + + template + std::size_t operator()(const composite_key_result & x)const + { + typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; + typedef typename CompositeKey::value_type value_type; + + BOOST_STATIC_ASSERT( + tuples::length::value== + tuples::length::value); + + return detail::hash_ckey< + key_extractor_tuple,value_type, + key_hasher_tuple + >::hash(x.composite_key.key_extractors(),x.value,key_hash_functions()); + } + + template + std::size_t operator()( + const tuple& x)const + { + typedef tuple key_tuple; + + BOOST_STATIC_ASSERT( + tuples::length::value== + tuples::length::value); + + return detail::hash_cval< + key_tuple,key_hasher_tuple + >::hash(x,key_hash_functions()); + } + +#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ + !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) + template + std::size_t operator()(const std::tuple& x)const + { + typedef std::tuple key_tuple; + typedef typename detail::cons_stdtuple_ctor< + key_tuple>::result_type cons_key_tuple; + + BOOST_STATIC_ASSERT( + std::tuple_size::value== + static_cast(tuples::length::value)); + + return detail::hash_cval< + cons_key_tuple,key_hasher_tuple + >::hash(detail::make_cons_stdtuple(x),key_hash_functions()); + } +#endif +}; + +/* Instantiations of the former functors with "natural" basic components: + * composite_key_result_equal_to uses std::equal_to of the values. + * composite_key_result_less uses std::less. + * composite_key_result_greater uses std::greater. + * composite_key_result_hash uses boost::hash. + */ + +#define BOOST_MULTI_INDEX_CK_RESULT_EQUAL_TO_SUPER \ +composite_key_equal_to< \ + BOOST_MULTI_INDEX_CK_ENUM( \ + BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N, \ + /* the argument is a PP list */ \ + (detail::nth_composite_key_equal_to, \ + (BOOST_DEDUCED_TYPENAME CompositeKeyResult::composite_key_type, \ + BOOST_PP_NIL))) \ + > + +template +struct composite_key_result_equal_to: +BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS +BOOST_MULTI_INDEX_CK_RESULT_EQUAL_TO_SUPER +{ +private: + typedef BOOST_MULTI_INDEX_CK_RESULT_EQUAL_TO_SUPER super; + +public: + typedef CompositeKeyResult first_argument_type; + typedef first_argument_type second_argument_type; + typedef bool result_type; + + using super::operator(); +}; + +#define BOOST_MULTI_INDEX_CK_RESULT_LESS_SUPER \ +composite_key_compare< \ + BOOST_MULTI_INDEX_CK_ENUM( \ + BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N, \ + /* the argument is a PP list */ \ + (detail::nth_composite_key_less, \ + (BOOST_DEDUCED_TYPENAME CompositeKeyResult::composite_key_type, \ + BOOST_PP_NIL))) \ + > + +template +struct composite_key_result_less: +BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS +BOOST_MULTI_INDEX_CK_RESULT_LESS_SUPER +{ +private: + typedef BOOST_MULTI_INDEX_CK_RESULT_LESS_SUPER super; + +public: + typedef CompositeKeyResult first_argument_type; + typedef first_argument_type second_argument_type; + typedef bool result_type; + + using super::operator(); +}; + +#define BOOST_MULTI_INDEX_CK_RESULT_GREATER_SUPER \ +composite_key_compare< \ + BOOST_MULTI_INDEX_CK_ENUM( \ + BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N, \ + /* the argument is a PP list */ \ + (detail::nth_composite_key_greater, \ + (BOOST_DEDUCED_TYPENAME CompositeKeyResult::composite_key_type, \ + BOOST_PP_NIL))) \ + > + +template +struct composite_key_result_greater: +BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS +BOOST_MULTI_INDEX_CK_RESULT_GREATER_SUPER +{ +private: + typedef BOOST_MULTI_INDEX_CK_RESULT_GREATER_SUPER super; + +public: + typedef CompositeKeyResult first_argument_type; + typedef first_argument_type second_argument_type; + typedef bool result_type; + + using super::operator(); +}; + +#define BOOST_MULTI_INDEX_CK_RESULT_HASH_SUPER \ +composite_key_hash< \ + BOOST_MULTI_INDEX_CK_ENUM( \ + BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N, \ + /* the argument is a PP list */ \ + (detail::nth_composite_key_hash, \ + (BOOST_DEDUCED_TYPENAME CompositeKeyResult::composite_key_type, \ + BOOST_PP_NIL))) \ + > + +template +struct composite_key_result_hash: +BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS +BOOST_MULTI_INDEX_CK_RESULT_HASH_SUPER +{ +private: + typedef BOOST_MULTI_INDEX_CK_RESULT_HASH_SUPER super; + +public: + typedef CompositeKeyResult argument_type; + typedef std::size_t result_type; + + using super::operator(); +}; + +} /* namespace multi_index */ + +} /* namespace boost */ + +/* Specializations of std::equal_to, std::less, std::greater and boost::hash + * for composite_key_results enabling interoperation with tuples of values. + */ + +namespace std{ + +template +struct equal_to >: + boost::multi_index::composite_key_result_equal_to< + boost::multi_index::composite_key_result + > +{ +}; + +template +struct less >: + boost::multi_index::composite_key_result_less< + boost::multi_index::composite_key_result + > +{ +}; + +template +struct greater >: + boost::multi_index::composite_key_result_greater< + boost::multi_index::composite_key_result + > +{ +}; + +} /* namespace std */ + +namespace boost{ + +template +struct hash >: + boost::multi_index::composite_key_result_hash< + boost::multi_index::composite_key_result + > +{ +}; + +} /* namespace boost */ + +#undef BOOST_MULTI_INDEX_CK_RESULT_HASH_SUPER +#undef BOOST_MULTI_INDEX_CK_RESULT_GREATER_SUPER +#undef BOOST_MULTI_INDEX_CK_RESULT_LESS_SUPER +#undef BOOST_MULTI_INDEX_CK_RESULT_EQUAL_TO_SUPER +#undef BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS +#undef BOOST_MULTI_INDEX_CK_IDENTITY_ENUM_MACRO +#undef BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR +#undef BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N +#undef BOOST_MULTI_INDEX_CK_CTOR_ARG +#undef BOOST_MULTI_INDEX_CK_TEMPLATE_PARM +#undef BOOST_MULTI_INDEX_CK_ENUM_PARAMS +#undef BOOST_MULTI_INDEX_CK_ENUM +#undef BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/access_specifier.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/access_specifier.hpp new file mode 100644 index 00000000000..f3346e836d4 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/access_specifier.hpp @@ -0,0 +1,54 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_ACCESS_SPECIFIER_HPP +#define BOOST_MULTI_INDEX_DETAIL_ACCESS_SPECIFIER_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include +#include + +/* In those compilers that do not accept the member template friend syntax, + * some protected and private sections might need to be specified as + * public. + */ + +#if defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) +#define BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS public +#define BOOST_MULTI_INDEX_PRIVATE_IF_MEMBER_TEMPLATE_FRIENDS public +#else +#define BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS protected +#define BOOST_MULTI_INDEX_PRIVATE_IF_MEMBER_TEMPLATE_FRIENDS private +#endif + +/* GCC does not correctly support in-class using declarations for template + * functions. See http://gcc.gnu.org/bugzilla/show_bug.cgi?id=9810 + * MSVC 7.1/8.0 seem to have a similar problem, though the conditions in + * which the error happens are not that simple. I have yet to isolate this + * into a snippet suitable for bug reporting. + * Sun Studio also has this problem, which might be related, from the + * information gathered at Sun forums, with a known issue notified at the + * internal bug report 6421933. The bug is present up to Studio Express 2, + * the latest preview version of the future Sun Studio 12. As of this writing + * (October 2006) it is not known whether a fix will finally make it into the + * official Sun Studio 12. + */ + +#if BOOST_WORKAROUND(__GNUC__,==3)&&(__GNUC_MINOR__<4)||\ + BOOST_WORKAROUND(BOOST_MSVC,==1310)||\ + BOOST_WORKAROUND(BOOST_MSVC,==1400)||\ + BOOST_WORKAROUND(__SUNPRO_CC,BOOST_TESTED_AT(0x590)) +#define BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS public +#else +#define BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS private +#endif + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/adl_swap.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/adl_swap.hpp new file mode 100644 index 00000000000..02b06442290 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/adl_swap.hpp @@ -0,0 +1,44 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_ADL_SWAP_HPP +#define BOOST_MULTI_INDEX_DETAIL_ADL_SWAP_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +template +void adl_swap(T& x,T& y) +{ + +#if !defined(BOOST_FUNCTION_SCOPE_USING_DECLARATION_BREAKS_ADL) + using std::swap; + swap(x,y); +#else + std::swap(x,y); +#endif + +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/archive_constructed.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/archive_constructed.hpp new file mode 100644 index 00000000000..0a7a26e0d4e --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/archive_constructed.hpp @@ -0,0 +1,83 @@ +/* Copyright 2003-2016 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_ARCHIVE_CONSTRUCTED_HPP +#define BOOST_MULTI_INDEX_DETAIL_ARCHIVE_CONSTRUCTED_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* constructs a stack-based object from a serialization archive */ + +template +struct archive_constructed:private noncopyable +{ + template + archive_constructed(Archive& ar,const unsigned int version) + { + serialization::load_construct_data_adl(ar,&get(),version); + BOOST_TRY{ + ar>>get(); + } + BOOST_CATCH(...){ + (&get())->~T(); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + template + archive_constructed(const char* name,Archive& ar,const unsigned int version) + { + serialization::load_construct_data_adl(ar,&get(),version); + BOOST_TRY{ + ar>>serialization::make_nvp(name,get()); + } + BOOST_CATCH(...){ + (&get())->~T(); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + ~archive_constructed() + { + (&get())->~T(); + } + +#include + + T& get(){return *reinterpret_cast(&space);} + +#include + +private: + typename aligned_storage::value>::type space; +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/auto_space.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/auto_space.hpp new file mode 100644 index 00000000000..9d78c3a363f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/auto_space.hpp @@ -0,0 +1,91 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_AUTO_SPACE_HPP +#define BOOST_MULTI_INDEX_DETAIL_AUTO_SPACE_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* auto_space provides uninitialized space suitably to store + * a given number of elements of a given type. + */ + +/* NB: it is not clear whether using an allocator to handle + * zero-sized arrays of elements is conformant or not. GCC 3.3.1 + * and prior fail here, other stdlibs handle the issue gracefully. + * To be on the safe side, the case n==0 is given special treatment. + * References: + * GCC Bugzilla, "standard allocator crashes when deallocating segment + * "of zero length", http://gcc.gnu.org/bugzilla/show_bug.cgi?id=14176 + * C++ Standard Library Defect Report List (Revision 28), issue 199 + * "What does allocate(0) return?", + * http://www.open-std.org/jtc1/sc22/wg21/docs/lwg-defects.html#199 + */ + +template > +struct auto_space:private noncopyable +{ + typedef typename boost::detail::allocator::rebind_to< + Allocator,T + >::type::pointer pointer; + + explicit auto_space(const Allocator& al=Allocator(),std::size_t n=1): + al_(al),n_(n),data_(n_?al_.allocate(n_):pointer(0)) + {} + + ~auto_space() + { + if(n_)al_.deallocate(data_,n_); + } + + Allocator get_allocator()const{return al_;} + + pointer data()const{return data_;} + + void swap(auto_space& x) + { + if(al_!=x.al_)adl_swap(al_,x.al_); + std::swap(n_,x.n_); + std::swap(data_,x.data_); + } + +private: + typename boost::detail::allocator::rebind_to< + Allocator,T>::type al_; + std::size_t n_; + pointer data_; +}; + +template +void swap(auto_space& x,auto_space& y) +{ + x.swap(y); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/base_type.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/base_type.hpp new file mode 100644 index 00000000000..8c9b62b716a --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/base_type.hpp @@ -0,0 +1,74 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_BASE_TYPE_HPP +#define BOOST_MULTI_INDEX_DETAIL_BASE_TYPE_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* MPL machinery to construct a linear hierarchy of indices out of + * a index list. + */ + +struct index_applier +{ + template + struct apply + { + typedef typename IndexSpecifierMeta::type index_specifier; + typedef typename index_specifier:: + BOOST_NESTED_TEMPLATE index_class::type type; + }; +}; + +template +struct nth_layer +{ + BOOST_STATIC_CONSTANT(int,length=mpl::size::value); + + typedef typename mpl::eval_if_c< + N==length, + mpl::identity >, + mpl::apply2< + index_applier, + mpl::at_c, + nth_layer + > + >::type type; +}; + +template +struct multi_index_base_type:nth_layer<0,Value,IndexSpecifierList,Allocator> +{ + BOOST_STATIC_ASSERT(detail::is_index_list::value); +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bidir_node_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bidir_node_iterator.hpp new file mode 100644 index 00000000000..9be5ec84b43 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bidir_node_iterator.hpp @@ -0,0 +1,114 @@ +/* Copyright 2003-2014 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_BIDIR_NODE_ITERATOR_HPP +#define BOOST_MULTI_INDEX_DETAIL_BIDIR_NODE_ITERATOR_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +#include +#include +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Iterator class for node-based indices with bidirectional + * iterators (ordered and sequenced indices.) + */ + +template +class bidir_node_iterator: + public bidirectional_iterator_helper< + bidir_node_iterator, + typename Node::value_type, + std::ptrdiff_t, + const typename Node::value_type*, + const typename Node::value_type&> +{ +public: + /* coverity[uninit_ctor]: suppress warning */ + bidir_node_iterator(){} + explicit bidir_node_iterator(Node* node_):node(node_){} + + const typename Node::value_type& operator*()const + { + return node->value(); + } + + bidir_node_iterator& operator++() + { + Node::increment(node); + return *this; + } + + bidir_node_iterator& operator--() + { + Node::decrement(node); + return *this; + } + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + /* Serialization. As for why the following is public, + * see explanation in safe_mode_iterator notes in safe_mode.hpp. + */ + + BOOST_SERIALIZATION_SPLIT_MEMBER() + + typedef typename Node::base_type node_base_type; + + template + void save(Archive& ar,const unsigned int)const + { + node_base_type* bnode=node; + ar< + void load(Archive& ar,const unsigned int) + { + node_base_type* bnode; + ar>>serialization::make_nvp("pointer",bnode); + node=static_cast(bnode); + } +#endif + + /* get_node is not to be used by the user */ + + typedef Node node_type; + + Node* get_node()const{return node;} + +private: + Node* node; +}; + +template +bool operator==( + const bidir_node_iterator& x, + const bidir_node_iterator& y) +{ + return x.get_node()==y.get_node(); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bucket_array.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bucket_array.hpp new file mode 100644 index 00000000000..d9fa434d9a9 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bucket_array.hpp @@ -0,0 +1,243 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_BUCKET_ARRAY_HPP +#define BOOST_MULTI_INDEX_DETAIL_BUCKET_ARRAY_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +#include +#include +#include +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* bucket structure for use by hashed indices */ + +#define BOOST_MULTI_INDEX_BA_SIZES_32BIT \ +(53ul)(97ul)(193ul)(389ul)(769ul) \ +(1543ul)(3079ul)(6151ul)(12289ul)(24593ul) \ +(49157ul)(98317ul)(196613ul)(393241ul)(786433ul) \ +(1572869ul)(3145739ul)(6291469ul)(12582917ul)(25165843ul) \ +(50331653ul)(100663319ul)(201326611ul)(402653189ul)(805306457ul) \ +(1610612741ul)(3221225473ul) + +#if ((((ULONG_MAX>>16)>>16)>>16)>>15)==0 /* unsigned long less than 64 bits */ +#define BOOST_MULTI_INDEX_BA_SIZES \ +BOOST_MULTI_INDEX_BA_SIZES_32BIT \ +(4294967291ul) +#else + /* obtained with aid from + * http://javaboutique.internet.com/prime_numb/ + * http://www.rsok.com/~jrm/next_ten_primes.html + * and verified with + * http://www.alpertron.com.ar/ECM.HTM + */ + +#define BOOST_MULTI_INDEX_BA_SIZES \ +BOOST_MULTI_INDEX_BA_SIZES_32BIT \ +(6442450939ul)(12884901893ul)(25769803751ul)(51539607551ul) \ +(103079215111ul)(206158430209ul)(412316860441ul)(824633720831ul) \ +(1649267441651ul)(3298534883309ul)(6597069766657ul)(13194139533299ul) \ +(26388279066623ul)(52776558133303ul)(105553116266489ul)(211106232532969ul) \ +(422212465066001ul)(844424930131963ul)(1688849860263953ul) \ +(3377699720527861ul)(6755399441055731ul)(13510798882111483ul) \ +(27021597764222939ul)(54043195528445957ul)(108086391056891903ul) \ +(216172782113783843ul)(432345564227567621ul)(864691128455135207ul) \ +(1729382256910270481ul)(3458764513820540933ul)(6917529027641081903ul) \ +(13835058055282163729ul)(18446744073709551557ul) +#endif + +template /* templatized to have in-header static var defs */ +class bucket_array_base:private noncopyable +{ +protected: + static const std::size_t sizes[ + BOOST_PP_SEQ_SIZE(BOOST_MULTI_INDEX_BA_SIZES)]; + + static std::size_t size_index(std::size_t n) + { + const std::size_t *bound=std::lower_bound(sizes,sizes+sizes_length,n); + if(bound==sizes+sizes_length)--bound; + return bound-sizes; + } + +#define BOOST_MULTI_INDEX_BA_POSITION_CASE(z,n,_) \ + case n:return hash%BOOST_PP_SEQ_ELEM(n,BOOST_MULTI_INDEX_BA_SIZES); + + static std::size_t position(std::size_t hash,std::size_t size_index_) + { + /* Accelerate hash%sizes[size_index_] by replacing with a switch on + * hash%Ci expressions, each Ci a compile-time constant, which the + * compiler can implement without using integer division. + */ + + switch(size_index_){ + default: /* never used */ + BOOST_PP_REPEAT( + BOOST_PP_SEQ_SIZE(BOOST_MULTI_INDEX_BA_SIZES), + BOOST_MULTI_INDEX_BA_POSITION_CASE,~) + } + } + +private: + static const std::size_t sizes_length; +}; + +template +const std::size_t bucket_array_base<_>::sizes[]={ + BOOST_PP_SEQ_ENUM(BOOST_MULTI_INDEX_BA_SIZES) +}; + +template +const std::size_t bucket_array_base<_>::sizes_length= + sizeof(bucket_array_base<_>::sizes)/ + sizeof(bucket_array_base<_>::sizes[0]); + +#undef BOOST_MULTI_INDEX_BA_POSITION_CASE +#undef BOOST_MULTI_INDEX_BA_SIZES +#undef BOOST_MULTI_INDEX_BA_SIZES_32BIT + +template +class bucket_array:bucket_array_base<> +{ + typedef bucket_array_base<> super; + typedef hashed_index_base_node_impl< + typename boost::detail::allocator::rebind_to< + Allocator, + char + >::type + > base_node_impl_type; + +public: + typedef typename base_node_impl_type::base_pointer base_pointer; + typedef typename base_node_impl_type::pointer pointer; + + bucket_array(const Allocator& al,pointer end_,std::size_t size_): + size_index_(super::size_index(size_)), + spc(al,super::sizes[size_index_]+1) + { + clear(end_); + } + + std::size_t size()const + { + return super::sizes[size_index_]; + } + + std::size_t position(std::size_t hash)const + { + return super::position(hash,size_index_); + } + + base_pointer begin()const{return buckets();} + base_pointer end()const{return buckets()+size();} + base_pointer at(std::size_t n)const{return buckets()+n;} + + void clear(pointer end_) + { + for(base_pointer x=begin(),y=end();x!=y;++x)x->prior()=pointer(0); + end()->prior()=end_->prior()=end_; + end_->next()=end(); + } + + void swap(bucket_array& x) + { + std::swap(size_index_,x.size_index_); + spc.swap(x.spc); + } + +private: + std::size_t size_index_; + auto_space spc; + + base_pointer buckets()const + { + return spc.data(); + } + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + friend class boost::serialization::access; + + /* bucket_arrays do not emit any kind of serialization info. They are + * fed to Boost.Serialization as hashed index iterators need to track + * them during serialization. + */ + + template + void serialize(Archive&,const unsigned int) + { + } +#endif +}; + +template +void swap(bucket_array& x,bucket_array& y) +{ + x.swap(y); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +/* bucket_arrays never get constructed directly by Boost.Serialization, + * as archives are always fed pointers to previously existent + * arrays. So, if this is called it means we are dealing with a + * somehow invalid archive. + */ + +#if defined(BOOST_NO_ARGUMENT_DEPENDENT_LOOKUP) +namespace serialization{ +#else +namespace multi_index{ +namespace detail{ +#endif + +template +inline void load_construct_data( + Archive&,boost::multi_index::detail::bucket_array*, + const unsigned int) +{ + throw_exception( + archive::archive_exception(archive::archive_exception::other_exception)); +} + +#if defined(BOOST_NO_ARGUMENT_DEPENDENT_LOOKUP) +} /* namespace serialization */ +#else +} /* namespace multi_index::detail */ +} /* namespace multi_index */ +#endif + +#endif + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/cons_stdtuple.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/cons_stdtuple.hpp new file mode 100644 index 00000000000..855c5e06aa9 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/cons_stdtuple.hpp @@ -0,0 +1,93 @@ +/* Copyright 2003-2014 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_CONS_STDTUPLE_HPP +#define BOOST_MULTI_INDEX_DETAIL_CONS_STDTUPLE_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* std::tuple wrapper providing the cons-based interface of boost::tuple for + * composite_key interoperability. + */ + +template +struct cons_stdtuple; + +struct cons_stdtuple_ctor_terminal +{ + typedef boost::tuples::null_type result_type; + + template + static result_type create(const StdTuple&) + { + return boost::tuples::null_type(); + } +}; + +template +struct cons_stdtuple_ctor_normal +{ + typedef cons_stdtuple result_type; + + static result_type create(const StdTuple& t) + { + return result_type(t); + } +}; + +template +struct cons_stdtuple_ctor: + boost::mpl::if_c< + N::value, + cons_stdtuple_ctor_normal, + cons_stdtuple_ctor_terminal + >::type +{}; + +template +struct cons_stdtuple +{ + typedef typename std::tuple_element::type head_type; + typedef cons_stdtuple_ctor tail_ctor; + typedef typename tail_ctor::result_type tail_type; + + cons_stdtuple(const StdTuple& t_):t(t_){} + + const head_type& get_head()const{return std::get(t);} + tail_type get_tail()const{return tail_ctor::create(t);} + + const StdTuple& t; +}; + +template +typename cons_stdtuple_ctor::result_type +make_cons_stdtuple(const StdTuple& t) +{ + return cons_stdtuple_ctor::create(t); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/converter.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/converter.hpp new file mode 100644 index 00000000000..3e04a3e8295 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/converter.hpp @@ -0,0 +1,52 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_CONVERTER_HPP +#define BOOST_MULTI_INDEX_DETAIL_CONVERTER_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* converter offers means to access indices of a given multi_index_container + * and for convertibilty between index iterators, so providing a + * localized access point for get() and project() functions. + */ + +template +struct converter +{ + static const Index& index(const MultiIndexContainer& x){return x;} + static Index& index(MultiIndexContainer& x){return x;} + + static typename Index::const_iterator const_iterator( + const MultiIndexContainer& x,typename MultiIndexContainer::node_type* node) + { + return x.Index::make_iterator(node); + } + + static typename Index::iterator iterator( + MultiIndexContainer& x,typename MultiIndexContainer::node_type* node) + { + return x.Index::make_iterator(node); + } +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/copy_map.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/copy_map.hpp new file mode 100644 index 00000000000..9a34b259cf3 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/copy_map.hpp @@ -0,0 +1,142 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_COPY_MAP_HPP +#define BOOST_MULTI_INDEX_DETAIL_COPY_MAP_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* copy_map is used as an auxiliary structure during copy_() operations. + * When a container with n nodes is replicated, node_map holds the pairings + * between original and copied nodes, and provides a fast way to find a + * copied node from an original one. + * The semantics of the class are not simple, and no attempt has been made + * to enforce it: multi_index_container handles it right. On the other hand, + * the const interface, which is the one provided to index implementations, + * only allows for: + * - Enumeration of pairs of (original,copied) nodes (excluding the headers), + * - fast retrieval of copied nodes (including the headers.) + */ + +template +struct copy_map_entry +{ + copy_map_entry(Node* f,Node* s):first(f),second(s){} + + Node* first; + Node* second; + + bool operator<(const copy_map_entry& x)const + { + return std::less()(first,x.first); + } +}; + +template +class copy_map:private noncopyable +{ +public: + typedef const copy_map_entry* const_iterator; + + copy_map( + const Allocator& al,std::size_t size,Node* header_org,Node* header_cpy): + al_(al),size_(size),spc(al_,size_),n(0), + header_org_(header_org),header_cpy_(header_cpy),released(false) + {} + + ~copy_map() + { + if(!released){ + for(std::size_t i=0;isecond->value()); + deallocate((spc.data()+i)->second); + } + } + } + + const_iterator begin()const{return raw_ptr(spc.data());} + const_iterator end()const{return raw_ptr(spc.data()+n);} + + void clone(Node* node) + { + (spc.data()+n)->first=node; + (spc.data()+n)->second=raw_ptr(al_.allocate(1)); + BOOST_TRY{ + boost::detail::allocator::construct( + &(spc.data()+n)->second->value(),node->value()); + } + BOOST_CATCH(...){ + deallocate((spc.data()+n)->second); + BOOST_RETHROW; + } + BOOST_CATCH_END + ++n; + + if(n==size_){ + std::sort( + raw_ptr*>(spc.data()), + raw_ptr*>(spc.data())+size_); + } + } + + Node* find(Node* node)const + { + if(node==header_org_)return header_cpy_; + return std::lower_bound( + begin(),end(),copy_map_entry(node,0))->second; + } + + void release() + { + released=true; + } + +private: + typedef typename boost::detail::allocator::rebind_to< + Allocator,Node + >::type allocator_type; + typedef typename allocator_type::pointer allocator_pointer; + + allocator_type al_; + std::size_t size_; + auto_space,Allocator> spc; + std::size_t n; + Node* header_org_; + Node* header_cpy_; + bool released; + + void deallocate(Node* node) + { + al_.deallocate(static_cast(node),1); + } +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/do_not_copy_elements_tag.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/do_not_copy_elements_tag.hpp new file mode 100644 index 00000000000..f0fa7304253 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/do_not_copy_elements_tag.hpp @@ -0,0 +1,34 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_DO_NOT_COPY_ELEMENTS_TAG_HPP +#define BOOST_MULTI_INDEX_DETAIL_DO_NOT_COPY_ELEMENTS_TAG_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Used to mark a special ctor variant that copies the internal objects of + * a container but not its elements. + */ + +struct do_not_copy_elements_tag{}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/duplicates_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/duplicates_iterator.hpp new file mode 100644 index 00000000000..cbebf264045 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/duplicates_iterator.hpp @@ -0,0 +1,120 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_DUPLICATES_ITERATOR_HPP +#define BOOST_MULTI_INDEX_DETAIL_DUPLICATES_ITERATOR_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* duplicates_operator is given a range of ordered elements and + * passes only over those which are duplicated. + */ + +template +class duplicates_iterator +{ +public: + typedef typename Node::value_type value_type; + typedef std::ptrdiff_t difference_type; + typedef const typename Node::value_type* pointer; + typedef const typename Node::value_type& reference; + typedef std::forward_iterator_tag iterator_category; + + duplicates_iterator(Node* node_,Node* end_,Predicate pred_): + node(node_),begin_chunk(0),end(end_),pred(pred_) + { + advance(); + } + + duplicates_iterator(Node* end_,Predicate pred_): + node(end_),begin_chunk(end_),end(end_),pred(pred_) + { + } + + reference operator*()const + { + return node->value(); + } + + pointer operator->()const + { + return &node->value(); + } + + duplicates_iterator& operator++() + { + Node::increment(node); + sync(); + return *this; + } + + duplicates_iterator operator++(int) + { + duplicates_iterator tmp(*this); + ++(*this); + return tmp; + } + + Node* get_node()const{return node;} + +private: + void sync() + { + if(node!=end&&pred(begin_chunk->value(),node->value()))advance(); + } + + void advance() + { + for(Node* node2=node;node!=end;node=node2){ + Node::increment(node2); + if(node2!=end&&!pred(node->value(),node2->value()))break; + } + begin_chunk=node; + } + + Node* node; + Node* begin_chunk; + Node* end; + Predicate pred; +}; + +template +bool operator==( + const duplicates_iterator& x, + const duplicates_iterator& y) +{ + return x.get_node()==y.get_node(); +} + +template +bool operator!=( + const duplicates_iterator& x, + const duplicates_iterator& y) +{ + return !(x==y); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/has_tag.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/has_tag.hpp new file mode 100644 index 00000000000..217b61143af --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/has_tag.hpp @@ -0,0 +1,42 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_HAS_TAG_HPP +#define BOOST_MULTI_INDEX_DETAIL_HAS_TAG_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* determines whether an index type has a given tag in its tag list */ + +template +struct has_tag +{ + template + struct apply:mpl::contains + { + }; +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_args.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_args.hpp new file mode 100644 index 00000000000..81902f5a4a5 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_args.hpp @@ -0,0 +1,105 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_ARGS_HPP +#define BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_ARGS_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Hashed index specifiers can be instantiated in two forms: + * + * (hashed_unique|hashed_non_unique)< + * KeyFromValue, + * Hash=boost::hash, + * Pred=std::equal_to > + * (hashed_unique|hashed_non_unique)< + * TagList, + * KeyFromValue, + * Hash=boost::hash, + * Pred=std::equal_to > + * + * hashed_index_args implements the machinery to accept this + * argument-dependent polymorphism. + */ + +template +struct index_args_default_hash +{ + typedef ::boost::hash type; +}; + +template +struct index_args_default_pred +{ + typedef std::equal_to type; +}; + +template +struct hashed_index_args +{ + typedef is_tag full_form; + + typedef typename mpl::if_< + full_form, + Arg1, + tag< > >::type tag_list_type; + typedef typename mpl::if_< + full_form, + Arg2, + Arg1>::type key_from_value_type; + typedef typename mpl::if_< + full_form, + Arg3, + Arg2>::type supplied_hash_type; + typedef typename mpl::eval_if< + mpl::is_na, + index_args_default_hash, + mpl::identity + >::type hash_type; + typedef typename mpl::if_< + full_form, + Arg4, + Arg3>::type supplied_pred_type; + typedef typename mpl::eval_if< + mpl::is_na, + index_args_default_pred, + mpl::identity + >::type pred_type; + + BOOST_STATIC_ASSERT(is_tag::value); + BOOST_STATIC_ASSERT(!mpl::is_na::value); + BOOST_STATIC_ASSERT(!mpl::is_na::value); + BOOST_STATIC_ASSERT(!mpl::is_na::value); +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_iterator.hpp new file mode 100644 index 00000000000..8d063002a1d --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_iterator.hpp @@ -0,0 +1,166 @@ +/* Copyright 2003-2014 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_ITERATOR_HPP +#define BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_ITERATOR_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +#include +#include +#include +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Iterator class for hashed indices. + */ + +struct hashed_index_global_iterator_tag{}; +struct hashed_index_local_iterator_tag{}; + +template +class hashed_index_iterator: + public forward_iterator_helper< + hashed_index_iterator, + typename Node::value_type, + std::ptrdiff_t, + const typename Node::value_type*, + const typename Node::value_type&> +{ +public: + /* coverity[uninit_ctor]: suppress warning */ + hashed_index_iterator(){} + hashed_index_iterator(Node* node_):node(node_){} + + const typename Node::value_type& operator*()const + { + return node->value(); + } + + hashed_index_iterator& operator++() + { + this->increment(Category()); + return *this; + } + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + /* Serialization. As for why the following is public, + * see explanation in safe_mode_iterator notes in safe_mode.hpp. + */ + + BOOST_SERIALIZATION_SPLIT_MEMBER() + + typedef typename Node::base_type node_base_type; + + template + void save(Archive& ar,const unsigned int)const + { + node_base_type* bnode=node; + ar< + void load(Archive& ar,const unsigned int version) + { + load(ar,version,Category()); + } + + template + void load( + Archive& ar,const unsigned int version,hashed_index_global_iterator_tag) + { + node_base_type* bnode; + ar>>serialization::make_nvp("pointer",bnode); + node=static_cast(bnode); + if(version<1){ + BucketArray* throw_away; /* consume unused ptr */ + ar>>serialization::make_nvp("pointer",throw_away); + } + } + + template + void load( + Archive& ar,const unsigned int version,hashed_index_local_iterator_tag) + { + node_base_type* bnode; + ar>>serialization::make_nvp("pointer",bnode); + node=static_cast(bnode); + if(version<1){ + BucketArray* buckets; + ar>>serialization::make_nvp("pointer",buckets); + if(buckets&&node&&node->impl()==buckets->end()->prior()){ + /* end local_iterators used to point to end node, now they are null */ + node=0; + } + } + } +#endif + + /* get_node is not to be used by the user */ + + typedef Node node_type; + + Node* get_node()const{return node;} + +private: + + void increment(hashed_index_global_iterator_tag) + { + Node::increment(node); + } + + void increment(hashed_index_local_iterator_tag) + { + Node::increment_local(node); + } + + Node* node; +}; + +template +bool operator==( + const hashed_index_iterator& x, + const hashed_index_iterator& y) +{ + return x.get_node()==y.get_node(); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +/* class version = 1 : hashed_index_iterator does no longer serialize a bucket + * array pointer. + */ + +namespace serialization { +template +struct version< + boost::multi_index::detail::hashed_index_iterator +> +{ + BOOST_STATIC_CONSTANT(int,value=1); +}; +} /* namespace serialization */ +#endif + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_node.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_node.hpp new file mode 100644 index 00000000000..7788e810ac9 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_node.hpp @@ -0,0 +1,778 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_NODE_HPP +#define BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_NODE_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Certain C++ requirements on unordered associative containers (see LWG issue + * #579) imply a data structure where nodes are linked in a single list, which + * in its turn forces implementors to add additional overhed per node to + * associate each with its corresponding bucket. Others resort to storing hash + * values, we use an alternative structure providing unconditional O(1) + * manipulation, even in situations of unfair hash distribution, plus some + * lookup speedups. For unique indices we maintain a doubly linked list of + * nodes except that if N is the first node of a bucket its associated + * bucket node is embedded between N and the preceding node in the following + * manner: + * + * +---+ +---+ +---+ +---+ + * <--+ |<--+ | <--+ |<--+ | + * ... | B0| | B1| ... | B1| | B2| ... + * | |-+ | +--> | |-+ | +--> + * +-+-+ | +---+ +-+-+ | +---+ + * | ^ | ^ + * | | | | + * | +-+ | +-+ + * | | | | + * v | v | + * --+---+---+---+-- --+---+---+---+-- + * ... | | B1| | ... | | B2| | ... + * --+---+---+---+-- --+---+---+---+-- + * + * + * The fist and last nodes of buckets can be checked with + * + * first node of a bucket: Npn != N + * last node of a bucket: Nnp != N + * + * (n and p short for ->next(), ->prior(), bucket nodes have prior pointers + * only). Pure insert and erase (without lookup) can be unconditionally done + * in O(1). + * For non-unique indices we add the following additional complexity: when + * there is a group of 3 or more equivalent elements, they are linked as + * follows: + * + * +-----------------------+ + * | v + * +---+ | +---+ +---+ +---+ + * | | +-+ | | |<--+ | + * | F | | S | ... | P | | L | + * | +-->| | | +-+ | | + * +---+ +---+ +---+ | +---+ + * ^ | + * +-----------------------+ + * + * F, S, P and L are the first, second, penultimate and last node in the + * group, respectively (S and P can coincide if the group has size 3.) This + * arrangement is used to skip equivalent elements in O(1) when doing lookup, + * while preserving O(1) insert/erase. The following invariants identify + * special positions (some of the operations have to be carefully implemented + * as Xnn is not valid if Xn points to a bucket): + * + * first node of a bucket: Npnp == N + * last node of a bucket: Nnpp == N + * first node of a group: Nnp != N && Nnppn == N + * second node of a group: Npn != N && Nppnn == N + * n-1 node of a group: Nnp != N && Nnnpp == N + * last node of a group: Npn != N && Npnnp == N + * + * The memory overhead is one pointer per bucket plus two pointers per node, + * probably unbeatable. The resulting structure is bidirectonally traversable, + * though currently we are just providing forward iteration. + */ + +template +struct hashed_index_node_impl; + +/* half-header (only prior() pointer) to use for the bucket array */ + +template +struct hashed_index_base_node_impl +{ + typedef typename + boost::detail::allocator::rebind_to< + Allocator,hashed_index_base_node_impl + >::type::pointer base_pointer; + typedef typename + boost::detail::allocator::rebind_to< + Allocator,hashed_index_base_node_impl + >::type::const_pointer const_base_pointer; + typedef typename + boost::detail::allocator::rebind_to< + Allocator, + hashed_index_node_impl + >::type::pointer pointer; + typedef typename + boost::detail::allocator::rebind_to< + Allocator, + hashed_index_node_impl + >::type::const_pointer const_pointer; + + pointer& prior(){return prior_;} + pointer prior()const{return prior_;} + +private: + pointer prior_; +}; + +/* full header (prior() and next()) for the nodes */ + +template +struct hashed_index_node_impl:hashed_index_base_node_impl +{ +private: + typedef hashed_index_base_node_impl super; + +public: + typedef typename super::base_pointer base_pointer; + typedef typename super::const_base_pointer const_base_pointer; + typedef typename super::pointer pointer; + typedef typename super::const_pointer const_pointer; + + base_pointer& next(){return next_;} + base_pointer next()const{return next_;} + + static pointer pointer_from(base_pointer x) + { + return static_cast( + static_cast( + raw_ptr(x))); + } + + static base_pointer base_pointer_from(pointer x) + { + return static_cast( + raw_ptr(x)); + } + +private: + base_pointer next_; +}; + +/* Boost.MultiIndex requires machinery to reverse unlink operations. A simple + * way to make a pointer-manipulation function undoable is to templatize + * its internal pointer assignments with a functor that, besides doing the + * assignment, keeps track of the original pointer values and can later undo + * the operations in reverse order. + */ + +struct default_assigner +{ + template void operator()(T& x,const T& val){x=val;} +}; + +template +struct unlink_undo_assigner +{ + typedef typename Node::base_pointer base_pointer; + typedef typename Node::pointer pointer; + + unlink_undo_assigner():pointer_track_count(0),base_pointer_track_count(0){} + + void operator()(pointer& x,pointer val) + { + pointer_tracks[pointer_track_count].x=&x; + pointer_tracks[pointer_track_count++].val=x; + x=val; + } + + void operator()(base_pointer& x,base_pointer val) + { + base_pointer_tracks[base_pointer_track_count].x=&x; + base_pointer_tracks[base_pointer_track_count++].val=x; + x=val; + } + + void operator()() /* undo op */ + { + /* in the absence of aliasing, restitution order is immaterial */ + + while(pointer_track_count--){ + *(pointer_tracks[pointer_track_count].x)= + pointer_tracks[pointer_track_count].val; + } + while(base_pointer_track_count--){ + *(base_pointer_tracks[base_pointer_track_count].x)= + base_pointer_tracks[base_pointer_track_count].val; + } + } + + struct pointer_track {pointer* x; pointer val;}; + struct base_pointer_track{base_pointer* x; base_pointer val;}; + + /* We know the maximum number of pointer and base pointer assignments that + * the two unlink versions do, so we can statically reserve the needed + * storage. + */ + + pointer_track pointer_tracks[3]; + int pointer_track_count; + base_pointer_track base_pointer_tracks[2]; + int base_pointer_track_count; +}; + +/* algorithmic stuff for unique and non-unique variants */ + +struct hashed_unique_tag{}; +struct hashed_non_unique_tag{}; + +template +struct hashed_index_node_alg; + +template +struct hashed_index_node_alg +{ + typedef typename Node::base_pointer base_pointer; + typedef typename Node::const_base_pointer const_base_pointer; + typedef typename Node::pointer pointer; + typedef typename Node::const_pointer const_pointer; + + static bool is_first_of_bucket(pointer x) + { + return x->prior()->next()!=base_pointer_from(x); + } + + static pointer after(pointer x) + { + return is_last_of_bucket(x)?x->next()->prior():pointer_from(x->next()); + } + + static pointer after_local(pointer x) + { + return is_last_of_bucket(x)?pointer(0):pointer_from(x->next()); + } + + static pointer next_to_inspect(pointer x) + { + return is_last_of_bucket(x)?pointer(0):pointer_from(x->next()); + } + + static void link(pointer x,base_pointer buc,pointer end) + { + if(buc->prior()==pointer(0)){ /* empty bucket */ + x->prior()=end->prior(); + x->next()=end->prior()->next(); + x->prior()->next()=buc; + buc->prior()=x; + end->prior()=x; + } + else{ + x->prior()=buc->prior()->prior(); + x->next()=base_pointer_from(buc->prior()); + buc->prior()=x; + x->next()->prior()=x; + } + } + + static void unlink(pointer x) + { + default_assigner assign; + unlink(x,assign); + } + + typedef unlink_undo_assigner unlink_undo; + + template + static void unlink(pointer x,Assigner& assign) + { + if(is_first_of_bucket(x)){ + if(is_last_of_bucket(x)){ + assign(x->prior()->next()->prior(),pointer(0)); + assign(x->prior()->next(),x->next()); + assign(x->next()->prior()->prior(),x->prior()); + } + else{ + assign(x->prior()->next()->prior(),pointer_from(x->next())); + assign(x->next()->prior(),x->prior()); + } + } + else if(is_last_of_bucket(x)){ + assign(x->prior()->next(),x->next()); + assign(x->next()->prior()->prior(),x->prior()); + } + else{ + assign(x->prior()->next(),x->next()); + assign(x->next()->prior(),x->prior()); + } + } + + /* used only at rehashing */ + + static void append(pointer x,pointer end) + { + x->prior()=end->prior(); + x->next()=end->prior()->next(); + x->prior()->next()=base_pointer_from(x); + end->prior()=x; + } + + static bool unlink_last(pointer end) + { + /* returns true iff bucket is emptied */ + + pointer x=end->prior(); + if(x->prior()->next()==base_pointer_from(x)){ + x->prior()->next()=x->next(); + end->prior()=x->prior(); + return false; + } + else{ + x->prior()->next()->prior()=pointer(0); + x->prior()->next()=x->next(); + end->prior()=x->prior(); + return true; + } + } + +private: + static pointer pointer_from(base_pointer x) + { + return Node::pointer_from(x); + } + + static base_pointer base_pointer_from(pointer x) + { + return Node::base_pointer_from(x); + } + + static bool is_last_of_bucket(pointer x) + { + return x->next()->prior()!=x; + } +}; + +template +struct hashed_index_node_alg +{ + typedef typename Node::base_pointer base_pointer; + typedef typename Node::const_base_pointer const_base_pointer; + typedef typename Node::pointer pointer; + typedef typename Node::const_pointer const_pointer; + + static bool is_first_of_bucket(pointer x) + { + return x->prior()->next()->prior()==x; + } + + static bool is_first_of_group(pointer x) + { + return + x->next()->prior()!=x&& + x->next()->prior()->prior()->next()==base_pointer_from(x); + } + + static pointer after(pointer x) + { + if(x->next()->prior()==x)return pointer_from(x->next()); + if(x->next()->prior()->prior()==x)return x->next()->prior(); + if(x->next()->prior()->prior()->next()==base_pointer_from(x)) + return pointer_from(x->next()); + return pointer_from(x->next())->next()->prior(); + } + + static pointer after_local(pointer x) + { + if(x->next()->prior()==x)return pointer_from(x->next()); + if(x->next()->prior()->prior()==x)return pointer(0); + if(x->next()->prior()->prior()->next()==base_pointer_from(x)) + return pointer_from(x->next()); + return pointer_from(x->next())->next()->prior(); + } + + static pointer next_to_inspect(pointer x) + { + if(x->next()->prior()==x)return pointer_from(x->next()); + if(x->next()->prior()->prior()==x)return pointer(0); + if(x->next()->prior()->next()->prior()!=x->next()->prior()) + return pointer(0); + return pointer_from(x->next()->prior()->next()); + } + + static void link(pointer x,base_pointer buc,pointer end) + { + if(buc->prior()==pointer(0)){ /* empty bucket */ + x->prior()=end->prior(); + x->next()=end->prior()->next(); + x->prior()->next()=buc; + buc->prior()=x; + end->prior()=x; + } + else{ + x->prior()=buc->prior()->prior(); + x->next()=base_pointer_from(buc->prior()); + buc->prior()=x; + x->next()->prior()=x; + } + }; + + static void link(pointer x,pointer first,pointer last) + { + x->prior()=first->prior(); + x->next()=base_pointer_from(first); + if(is_first_of_bucket(first)){ + x->prior()->next()->prior()=x; + } + else{ + x->prior()->next()=base_pointer_from(x); + } + + if(first==last){ + last->prior()=x; + } + else if(first->next()==base_pointer_from(last)){ + first->prior()=last; + first->next()=base_pointer_from(x); + } + else{ + pointer second=pointer_from(first->next()), + lastbutone=last->prior(); + second->prior()=first; + first->prior()=last; + lastbutone->next()=base_pointer_from(x); + } + } + + static void unlink(pointer x) + { + default_assigner assign; + unlink(x,assign); + } + + typedef unlink_undo_assigner unlink_undo; + + template + static void unlink(pointer x,Assigner& assign) + { + if(x->prior()->next()==base_pointer_from(x)){ + if(x->next()->prior()==x){ + left_unlink(x,assign); + right_unlink(x,assign); + } + else if(x->next()->prior()->prior()==x){ /* last of bucket */ + left_unlink(x,assign); + right_unlink_last_of_bucket(x,assign); + } + else if(x->next()->prior()->prior()->next()== + base_pointer_from(x)){ /* first of group size */ + left_unlink(x,assign); + right_unlink_first_of_group(x,assign); + } + else{ /* n-1 of group */ + unlink_last_but_one_of_group(x,assign); + } + } + else if(x->prior()->next()->prior()==x){ /* first of bucket */ + if(x->next()->prior()==x){ + left_unlink_first_of_bucket(x,assign); + right_unlink(x,assign); + } + else if(x->next()->prior()->prior()==x){ /* last of bucket */ + assign(x->prior()->next()->prior(),pointer(0)); + assign(x->prior()->next(),x->next()); + assign(x->next()->prior()->prior(),x->prior()); + } + else{ /* first of group */ + left_unlink_first_of_bucket(x,assign); + right_unlink_first_of_group(x,assign); + } + } + else if(x->next()->prior()->prior()==x){ /* last of group and bucket */ + left_unlink_last_of_group(x,assign); + right_unlink_last_of_bucket(x,assign); + } + else if(pointer_from(x->prior()->prior()->next()) + ->next()==base_pointer_from(x)){ /* second of group */ + unlink_second_of_group(x,assign); + } + else{ /* last of group, ~(last of bucket) */ + left_unlink_last_of_group(x,assign); + right_unlink(x,assign); + } + } + + /* used only at rehashing */ + + static void link_range( + pointer first,pointer last,base_pointer buc,pointer cend) + { + if(buc->prior()==pointer(0)){ /* empty bucket */ + first->prior()=cend->prior(); + last->next()=cend->prior()->next(); + first->prior()->next()=buc; + buc->prior()=first; + cend->prior()=last; + } + else{ + first->prior()=buc->prior()->prior(); + last->next()=base_pointer_from(buc->prior()); + buc->prior()=first; + last->next()->prior()=last; + } + } + + static void append_range(pointer first,pointer last,pointer cend) + { + first->prior()=cend->prior(); + last->next()=cend->prior()->next(); + first->prior()->next()=base_pointer_from(first); + cend->prior()=last; + } + + static std::pair unlink_last_group(pointer end) + { + /* returns first of group true iff bucket is emptied */ + + pointer x=end->prior(); + if(x->prior()->next()==base_pointer_from(x)){ + x->prior()->next()=x->next(); + end->prior()=x->prior(); + return std::make_pair(x,false); + } + else if(x->prior()->next()->prior()==x){ + x->prior()->next()->prior()=pointer(0); + x->prior()->next()=x->next(); + end->prior()=x->prior(); + return std::make_pair(x,true); + } + else{ + pointer y=pointer_from(x->prior()->next()); + + if(y->prior()->next()==base_pointer_from(y)){ + y->prior()->next()=x->next(); + end->prior()=y->prior(); + return std::make_pair(y,false); + } + else{ + y->prior()->next()->prior()=pointer(0); + y->prior()->next()=x->next(); + end->prior()=y->prior(); + return std::make_pair(y,true); + } + } + } + + static void unlink_range(pointer first,pointer last) + { + if(is_first_of_bucket(first)){ + if(is_last_of_bucket(last)){ + first->prior()->next()->prior()=pointer(0); + first->prior()->next()=last->next(); + last->next()->prior()->prior()=first->prior(); + } + else{ + first->prior()->next()->prior()=pointer_from(last->next()); + last->next()->prior()=first->prior(); + } + } + else if(is_last_of_bucket(last)){ + first->prior()->next()=last->next(); + last->next()->prior()->prior()=first->prior(); + } + else{ + first->prior()->next()=last->next(); + last->next()->prior()=first->prior(); + } + } + +private: + static pointer pointer_from(base_pointer x) + { + return Node::pointer_from(x); + } + + static base_pointer base_pointer_from(pointer x) + { + return Node::base_pointer_from(x); + } + + static bool is_last_of_bucket(pointer x) + { + return x->next()->prior()->prior()==x; + } + + template + static void left_unlink(pointer x,Assigner& assign) + { + assign(x->prior()->next(),x->next()); + } + + template + static void right_unlink(pointer x,Assigner& assign) + { + assign(x->next()->prior(),x->prior()); + } + + template + static void left_unlink_first_of_bucket(pointer x,Assigner& assign) + { + assign(x->prior()->next()->prior(),pointer_from(x->next())); + } + + template + static void right_unlink_last_of_bucket(pointer x,Assigner& assign) + { + assign(x->next()->prior()->prior(),x->prior()); + } + + template + static void right_unlink_first_of_group(pointer x,Assigner& assign) + { + pointer second=pointer_from(x->next()), + last=second->prior(), + lastbutone=last->prior(); + if(second==lastbutone){ + assign(second->next(),base_pointer_from(last)); + assign(second->prior(),x->prior()); + } + else{ + assign(lastbutone->next(),base_pointer_from(second)); + assign(second->next()->prior(),last); + assign(second->prior(),x->prior()); + } + } + + template + static void left_unlink_last_of_group(pointer x,Assigner& assign) + { + pointer lastbutone=x->prior(), + first=pointer_from(lastbutone->next()), + second=pointer_from(first->next()); + if(lastbutone==second){ + assign(lastbutone->prior(),first); + assign(lastbutone->next(),x->next()); + } + else{ + assign(second->prior(),lastbutone); + assign(lastbutone->prior()->next(),base_pointer_from(first)); + assign(lastbutone->next(),x->next()); + } + } + + template + static void unlink_last_but_one_of_group(pointer x,Assigner& assign) + { + pointer first=pointer_from(x->next()), + second=pointer_from(first->next()), + last=second->prior(); + if(second==x){ + assign(last->prior(),first); + assign(first->next(),base_pointer_from(last)); + } + else{ + assign(last->prior(),x->prior()); + assign(x->prior()->next(),base_pointer_from(first)); + } + } + + template + static void unlink_second_of_group(pointer x,Assigner& assign) + { + pointer last=x->prior(), + lastbutone=last->prior(), + first=pointer_from(lastbutone->next()); + if(lastbutone==x){ + assign(first->next(),base_pointer_from(last)); + assign(last->prior(),first); + } + else{ + assign(first->next(),x->next()); + assign(x->next()->prior(),last); + } + } +}; + +template +struct hashed_index_node_trampoline: + hashed_index_node_impl< + typename boost::detail::allocator::rebind_to< + typename Super::allocator_type, + char + >::type + > +{ + typedef typename boost::detail::allocator::rebind_to< + typename Super::allocator_type, + char + >::type impl_allocator_type; + typedef hashed_index_node_impl impl_type; +}; + +template +struct hashed_index_node: + Super,hashed_index_node_trampoline +{ +private: + typedef hashed_index_node_trampoline trampoline; + +public: + typedef typename trampoline::impl_type impl_type; + typedef hashed_index_node_alg< + impl_type,Category> node_alg; + typedef typename trampoline::base_pointer impl_base_pointer; + typedef typename trampoline::const_base_pointer const_impl_base_pointer; + typedef typename trampoline::pointer impl_pointer; + typedef typename trampoline::const_pointer const_impl_pointer; + + impl_pointer& prior(){return trampoline::prior();} + impl_pointer prior()const{return trampoline::prior();} + impl_base_pointer& next(){return trampoline::next();} + impl_base_pointer next()const{return trampoline::next();} + + impl_pointer impl() + { + return static_cast( + static_cast(static_cast(this))); + } + + const_impl_pointer impl()const + { + return static_cast( + static_cast(static_cast(this))); + } + + static hashed_index_node* from_impl(impl_pointer x) + { + return + static_cast( + static_cast( + raw_ptr(x))); + } + + static const hashed_index_node* from_impl(const_impl_pointer x) + { + return + static_cast( + static_cast( + raw_ptr(x))); + } + + /* interoperability with hashed_index_iterator */ + + static void increment(hashed_index_node*& x) + { + x=from_impl(node_alg::after(x->impl())); + } + + static void increment_local(hashed_index_node*& x) + { + x=from_impl(node_alg::after_local(x->impl())); + } +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/header_holder.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/header_holder.hpp new file mode 100644 index 00000000000..ca8a9b2edb1 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/header_holder.hpp @@ -0,0 +1,50 @@ +/* Copyright 2003-2008 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_HEADER_HOLDER_HPP +#define BOOST_MULTI_INDEX_DETAIL_HEADER_HOLDER_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* A utility class used to hold a pointer to the header node. + * The base from member idiom is used because index classes, which are + * superclasses of multi_index_container, need this header in construction + * time. The allocation is made by the allocator of the multi_index_container + * class --hence, this allocator needs also be stored resorting + * to the base from member trick. + */ + +template +struct header_holder:private noncopyable +{ + header_holder():member(final().allocate_node()){} + ~header_holder(){final().deallocate_node(&*member);} + + NodeTypePtr member; + +private: + Final& final(){return *static_cast(this);} +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ignore_wstrict_aliasing.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ignore_wstrict_aliasing.hpp new file mode 100644 index 00000000000..ae398456d1f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ignore_wstrict_aliasing.hpp @@ -0,0 +1,18 @@ +/* Copyright 2003-2016 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#include + +#if defined(BOOST_GCC)&&(BOOST_GCC>=4*10000+6*100) +#if !defined(BOOST_MULTI_INDEX_DETAIL_RESTORE_WSTRICT_ALIASING) +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wstrict-aliasing" +#else +#pragma GCC diagnostic pop +#endif +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_base.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_base.hpp new file mode 100644 index 00000000000..99000ed4813 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_base.hpp @@ -0,0 +1,293 @@ +/* Copyright 2003-2014 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_BASE_HPP +#define BOOST_MULTI_INDEX_DETAIL_INDEX_BASE_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +#include +#include +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* The role of this class is threefold: + * - tops the linear hierarchy of indices. + * - terminates some cascading backbone function calls (insert_, etc.), + * - grants access to the backbone functions of the final + * multi_index_container class (for access restriction reasons, these + * cannot be called directly from the index classes.) + */ + +struct lvalue_tag{}; +struct rvalue_tag{}; +struct emplaced_tag{}; + +template +class index_base +{ +protected: + typedef index_node_base node_type; + typedef typename multi_index_node_type< + Value,IndexSpecifierList,Allocator>::type final_node_type; + typedef multi_index_container< + Value,IndexSpecifierList,Allocator> final_type; + typedef tuples::null_type ctor_args_list; + typedef typename + boost::detail::allocator::rebind_to< + Allocator, + typename Allocator::value_type + >::type final_allocator_type; + typedef mpl::vector0<> index_type_list; + typedef mpl::vector0<> iterator_type_list; + typedef mpl::vector0<> const_iterator_type_list; + typedef copy_map< + final_node_type, + final_allocator_type> copy_map_type; + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + typedef index_saver< + node_type, + final_allocator_type> index_saver_type; + typedef index_loader< + node_type, + final_node_type, + final_allocator_type> index_loader_type; +#endif + +private: + typedef Value value_type; + +protected: + explicit index_base(const ctor_args_list&,const Allocator&){} + + index_base( + const index_base&, + do_not_copy_elements_tag) + {} + + void copy_( + const index_base&,const copy_map_type&) + {} + + final_node_type* insert_(const value_type& v,final_node_type*& x,lvalue_tag) + { + x=final().allocate_node(); + BOOST_TRY{ + boost::detail::allocator::construct(&x->value(),v); + } + BOOST_CATCH(...){ + final().deallocate_node(x); + BOOST_RETHROW; + } + BOOST_CATCH_END + return x; + } + + final_node_type* insert_(const value_type& v,final_node_type*& x,rvalue_tag) + { + x=final().allocate_node(); + BOOST_TRY{ + /* This shoud have used a modified, T&&-compatible version of + * boost::detail::allocator::construct, but + * is too old and venerable to + * mess with; besides, it is a general internal utility and the imperfect + * perfect forwarding emulation of Boost.Move might break other libs. + */ + + new (&x->value()) value_type(boost::move(const_cast(v))); + } + BOOST_CATCH(...){ + final().deallocate_node(x); + BOOST_RETHROW; + } + BOOST_CATCH_END + return x; + } + + final_node_type* insert_(const value_type&,final_node_type*& x,emplaced_tag) + { + return x; + } + + final_node_type* insert_( + const value_type& v,node_type*,final_node_type*& x,lvalue_tag) + { + return insert_(v,x,lvalue_tag()); + } + + final_node_type* insert_( + const value_type& v,node_type*,final_node_type*& x,rvalue_tag) + { + return insert_(v,x,rvalue_tag()); + } + + final_node_type* insert_( + const value_type&,node_type*,final_node_type*& x,emplaced_tag) + { + return x; + } + + void erase_(node_type* x) + { + boost::detail::allocator::destroy(&x->value()); + } + + void delete_node_(node_type* x) + { + boost::detail::allocator::destroy(&x->value()); + } + + void clear_(){} + + void swap_(index_base&){} + + void swap_elements_(index_base&){} + + bool replace_(const value_type& v,node_type* x,lvalue_tag) + { + x->value()=v; + return true; + } + + bool replace_(const value_type& v,node_type* x,rvalue_tag) + { + x->value()=boost::move(const_cast(v)); + return true; + } + + bool modify_(node_type*){return true;} + + bool modify_rollback_(node_type*){return true;} + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + /* serialization */ + + template + void save_(Archive&,const unsigned int,const index_saver_type&)const{} + + template + void load_(Archive&,const unsigned int,const index_loader_type&){} +#endif + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) + /* invariant stuff */ + + bool invariant_()const{return true;} +#endif + + /* access to backbone memfuns of Final class */ + + final_type& final(){return *static_cast(this);} + const final_type& final()const{return *static_cast(this);} + + final_node_type* final_header()const{return final().header();} + + bool final_empty_()const{return final().empty_();} + std::size_t final_size_()const{return final().size_();} + std::size_t final_max_size_()const{return final().max_size_();} + + std::pair final_insert_(const value_type& x) + {return final().insert_(x);} + std::pair final_insert_rv_(const value_type& x) + {return final().insert_rv_(x);} + template + std::pair final_insert_ref_(const T& t) + {return final().insert_ref_(t);} + template + std::pair final_insert_ref_(T& t) + {return final().insert_ref_(t);} + + template + std::pair final_emplace_( + BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + return final().emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + } + + std::pair final_insert_( + const value_type& x,final_node_type* position) + {return final().insert_(x,position);} + std::pair final_insert_rv_( + const value_type& x,final_node_type* position) + {return final().insert_rv_(x,position);} + template + std::pair final_insert_ref_( + const T& t,final_node_type* position) + {return final().insert_ref_(t,position);} + template + std::pair final_insert_ref_( + T& t,final_node_type* position) + {return final().insert_ref_(t,position);} + + template + std::pair final_emplace_hint_( + final_node_type* position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + return final().emplace_hint_( + position,BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + } + + void final_erase_(final_node_type* x){final().erase_(x);} + + void final_delete_node_(final_node_type* x){final().delete_node_(x);} + void final_delete_all_nodes_(){final().delete_all_nodes_();} + void final_clear_(){final().clear_();} + + void final_swap_(final_type& x){final().swap_(x);} + + bool final_replace_( + const value_type& k,final_node_type* x) + {return final().replace_(k,x);} + bool final_replace_rv_( + const value_type& k,final_node_type* x) + {return final().replace_rv_(k,x);} + + template + bool final_modify_(Modifier& mod,final_node_type* x) + {return final().modify_(mod,x);} + + template + bool final_modify_(Modifier& mod,Rollback& back,final_node_type* x) + {return final().modify_(mod,back,x);} + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) + void final_check_invariant_()const{final().check_invariant_();} +#endif +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_loader.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_loader.hpp new file mode 100644 index 00000000000..71418a10e19 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_loader.hpp @@ -0,0 +1,139 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_LOADER_HPP +#define BOOST_MULTI_INDEX_DETAIL_INDEX_LOADER_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Counterpart of index_saver (check index_saver.hpp for serialization + * details.)* multi_index_container is in charge of supplying the info about + * the base sequence, and each index can subsequently load itself using the + * const interface of index_loader. + */ + +template +class index_loader:private noncopyable +{ +public: + index_loader(const Allocator& al,std::size_t size): + spc(al,size),size_(size),n(0),sorted(false) + { + } + + template + void add(Node* node,Archive& ar,const unsigned int) + { + ar>>serialization::make_nvp("position",*node); + entries()[n++]=node; + } + + template + void add_track(Node* node,Archive& ar,const unsigned int) + { + ar>>serialization::make_nvp("position",*node); + } + + /* A rearranger is passed two nodes, and is expected to + * reposition the second after the first. + * If the first node is 0, then the second should be moved + * to the beginning of the sequence. + */ + + template + void load(Rearranger r,Archive& ar,const unsigned int)const + { + FinalNode* prev=unchecked_load_node(ar); + if(!prev)return; + + if(!sorted){ + std::sort(entries(),entries()+size_); + sorted=true; + } + + check_node(prev); + + for(;;){ + for(;;){ + FinalNode* node=load_node(ar); + if(!node)break; + + if(node==prev)prev=0; + r(prev,node); + + prev=node; + } + prev=load_node(ar); + if(!prev)break; + } + } + +private: + Node** entries()const{return raw_ptr(spc.data());} + + /* We try to delay sorting as much as possible just in case it + * is not necessary, hence this version of load_node. + */ + + template + FinalNode* unchecked_load_node(Archive& ar)const + { + Node* node=0; + ar>>serialization::make_nvp("pointer",node); + return static_cast(node); + } + + template + FinalNode* load_node(Archive& ar)const + { + Node* node=0; + ar>>serialization::make_nvp("pointer",node); + check_node(node); + return static_cast(node); + } + + void check_node(Node* node)const + { + if(node!=0&&!std::binary_search(entries(),entries()+size_,node)){ + throw_exception( + archive::archive_exception( + archive::archive_exception::other_exception)); + } + } + + auto_space spc; + std::size_t size_; + std::size_t n; + mutable bool sorted; +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_matcher.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_matcher.hpp new file mode 100644 index 00000000000..34d1f9d5a8d --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_matcher.hpp @@ -0,0 +1,249 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_MATCHER_HPP +#define BOOST_MULTI_INDEX_DETAIL_INDEX_MATCHER_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* index_matcher compares a sequence of elements against a + * base sequence, identifying those elements that belong to the + * longest subsequence which is ordered with respect to the base. + * For instance, if the base sequence is: + * + * 0 1 2 3 4 5 6 7 8 9 + * + * and the compared sequence (not necesarilly the same length): + * + * 1 4 2 3 0 7 8 9 + * + * the elements of the longest ordered subsequence are: + * + * 1 2 3 7 8 9 + * + * The algorithm for obtaining such a subsequence is called + * Patience Sorting, described in ch. 1 of: + * Aldous, D., Diaconis, P.: "Longest increasing subsequences: from + * patience sorting to the Baik-Deift-Johansson Theorem", Bulletin + * of the American Mathematical Society, vol. 36, no 4, pp. 413-432, + * July 1999. + * http://www.ams.org/bull/1999-36-04/S0273-0979-99-00796-X/ + * S0273-0979-99-00796-X.pdf + * + * This implementation is not fully generic since it assumes that + * the sequences given are pointed to by index iterators (having a + * get_node() memfun.) + */ + +namespace index_matcher{ + +/* The algorithm stores the nodes of the base sequence and a number + * of "piles" that are dynamically updated during the calculation + * stage. From a logical point of view, nodes form an independent + * sequence from piles. They are stored together so as to minimize + * allocated memory. + */ + +struct entry +{ + entry(void* node_,std::size_t pos_=0):node(node_),pos(pos_){} + + /* node stuff */ + + void* node; + std::size_t pos; + entry* previous; + bool ordered; + + struct less_by_node + { + bool operator()( + const entry& x,const entry& y)const + { + return std::less()(x.node,y.node); + } + }; + + /* pile stuff */ + + std::size_t pile_top; + entry* pile_top_entry; + + struct less_by_pile_top + { + bool operator()( + const entry& x,const entry& y)const + { + return x.pile_top +class algorithm_base:private noncopyable +{ +protected: + algorithm_base(const Allocator& al,std::size_t size): + spc(al,size),size_(size),n_(0),sorted(false) + { + } + + void add(void* node) + { + entries()[n_]=entry(node,n_); + ++n_; + } + + void begin_algorithm()const + { + if(!sorted){ + std::sort(entries(),entries()+size_,entry::less_by_node()); + sorted=true; + } + num_piles=0; + } + + void add_node_to_algorithm(void* node)const + { + entry* ent= + std::lower_bound( + entries(),entries()+size_, + entry(node),entry::less_by_node()); /* localize entry */ + ent->ordered=false; + std::size_t n=ent->pos; /* get its position */ + + entry dummy(0); + dummy.pile_top=n; + + entry* pile_ent= /* find the first available pile */ + std::lower_bound( /* to stack the entry */ + entries(),entries()+num_piles, + dummy,entry::less_by_pile_top()); + + pile_ent->pile_top=n; /* stack the entry */ + pile_ent->pile_top_entry=ent; + + /* if not the first pile, link entry to top of the preceding pile */ + if(pile_ent>&entries()[0]){ + ent->previous=(pile_ent-1)->pile_top_entry; + } + + if(pile_ent==&entries()[num_piles]){ /* new pile? */ + ++num_piles; + } + } + + void finish_algorithm()const + { + if(num_piles>0){ + /* Mark those elements which are in their correct position, i.e. those + * belonging to the longest increasing subsequence. These are those + * elements linked from the top of the last pile. + */ + + entry* ent=entries()[num_piles-1].pile_top_entry; + for(std::size_t n=num_piles;n--;){ + ent->ordered=true; + ent=ent->previous; + } + } + } + + bool is_ordered(void * node)const + { + return std::lower_bound( + entries(),entries()+size_, + entry(node),entry::less_by_node())->ordered; + } + +private: + entry* entries()const{return raw_ptr(spc.data());} + + auto_space spc; + std::size_t size_; + std::size_t n_; + mutable bool sorted; + mutable std::size_t num_piles; +}; + +/* The algorithm has three phases: + * - Initialization, during which the nodes of the base sequence are added. + * - Execution. + * - Results querying, through the is_ordered memfun. + */ + +template +class algorithm:private algorithm_base +{ + typedef algorithm_base super; + +public: + algorithm(const Allocator& al,std::size_t size):super(al,size){} + + void add(Node* node) + { + super::add(node); + } + + template + void execute(IndexIterator first,IndexIterator last)const + { + super::begin_algorithm(); + + for(IndexIterator it=first;it!=last;++it){ + add_node_to_algorithm(get_node(it)); + } + + super::finish_algorithm(); + } + + bool is_ordered(Node* node)const + { + return super::is_ordered(node); + } + +private: + void add_node_to_algorithm(Node* node)const + { + super::add_node_to_algorithm(node); + } + + template + static Node* get_node(IndexIterator it) + { + return static_cast(it.get_node()); + } +}; + +} /* namespace multi_index::detail::index_matcher */ + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_node_base.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_node_base.hpp new file mode 100644 index 00000000000..1a1f0cae4be --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_node_base.hpp @@ -0,0 +1,135 @@ +/* Copyright 2003-2016 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_NODE_BASE_HPP +#define BOOST_MULTI_INDEX_DETAIL_INDEX_NODE_BASE_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +#include +#include +#include +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* index_node_base tops the node hierarchy of multi_index_container. It holds + * the value of the element contained. + */ + +template +struct pod_value_holder +{ + typename aligned_storage< + sizeof(Value), + alignment_of::value + >::type space; +}; + +template +struct index_node_base:private pod_value_holder +{ + typedef index_node_base base_type; /* used for serialization purposes */ + typedef Value value_type; + typedef Allocator allocator_type; + +#include + + value_type& value() + { + return *reinterpret_cast(&this->space); + } + + const value_type& value()const + { + return *reinterpret_cast(&this->space); + } + +#include + + static index_node_base* from_value(const value_type* p) + { + return static_cast( + reinterpret_cast*>( /* std 9.2.17 */ + const_cast(p))); + } + +private: +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + friend class boost::serialization::access; + + /* nodes do not emit any kind of serialization info. They are + * fed to Boost.Serialization so that pointers to nodes are + * tracked correctly. + */ + + template + void serialize(Archive&,const unsigned int) + { + } +#endif +}; + +template +Node* node_from_value(const Value* p) +{ + typedef typename Node::allocator_type allocator_type; + return static_cast( + index_node_base::from_value(p)); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +/* Index nodes never get constructed directly by Boost.Serialization, + * as archives are always fed pointers to previously existent + * nodes. So, if this is called it means we are dealing with a + * somehow invalid archive. + */ + +#if defined(BOOST_NO_ARGUMENT_DEPENDENT_LOOKUP) +namespace serialization{ +#else +namespace multi_index{ +namespace detail{ +#endif + +template +inline void load_construct_data( + Archive&,boost::multi_index::detail::index_node_base*, + const unsigned int) +{ + throw_exception( + archive::archive_exception(archive::archive_exception::other_exception)); +} + +#if defined(BOOST_NO_ARGUMENT_DEPENDENT_LOOKUP) +} /* namespace serialization */ +#else +} /* namespace multi_index::detail */ +} /* namespace multi_index */ +#endif + +#endif + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_saver.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_saver.hpp new file mode 100644 index 00000000000..ae09d4eba4f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_saver.hpp @@ -0,0 +1,135 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_SAVER_HPP +#define BOOST_MULTI_INDEX_DETAIL_INDEX_SAVER_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* index_saver accepts a base sequence of previously saved elements + * and saves a possibly reordered subsequence in an efficient manner, + * serializing only the information needed to rearrange the subsequence + * based on the original order of the base. + * multi_index_container is in charge of supplying the info about the + * base sequence, and each index can subsequently save itself using the + * const interface of index_saver. + */ + +template +class index_saver:private noncopyable +{ +public: + index_saver(const Allocator& al,std::size_t size):alg(al,size){} + + template + void add(Node* node,Archive& ar,const unsigned int) + { + ar< + void add_track(Node* node,Archive& ar,const unsigned int) + { + ar< + void save( + IndexIterator first,IndexIterator last,Archive& ar, + const unsigned int)const + { + /* calculate ordered positions */ + + alg.execute(first,last); + + /* Given a consecutive subsequence of displaced elements + * x1,...,xn, the following information is serialized: + * + * p0,p1,...,pn,0 + * + * where pi is a pointer to xi and p0 is a pointer to the element + * preceding x1. Crealy, from this information is possible to + * restore the original order on loading time. If x1 is the first + * element in the sequence, the following is serialized instead: + * + * p1,p1,...,pn,0 + * + * For each subsequence of n elements, n+2 pointers are serialized. + * An optimization policy is applied: consider for instance the + * sequence + * + * a,B,c,D + * + * where B and D are displaced, but c is in its correct position. + * Applying the schema described above we would serialize 6 pointers: + * + * p(a),p(B),0 + * p(c),p(D),0 + * + * but this can be reduced to 5 pointers by treating c as a displaced + * element: + * + * p(a),p(B),p(c),p(D),0 + */ + + std::size_t last_saved=3; /* distance to last pointer saved */ + for(IndexIterator it=first,prev=first;it!=last;prev=it++,++last_saved){ + if(!alg.is_ordered(get_node(it))){ + if(last_saved>1)save_node(get_node(prev),ar); + save_node(get_node(it),ar); + last_saved=0; + } + else if(last_saved==2)save_node(null_node(),ar); + } + if(last_saved<=2)save_node(null_node(),ar); + + /* marks the end of the serialization info for [first,last) */ + + save_node(null_node(),ar); + } + +private: + template + static Node* get_node(IndexIterator it) + { + return it.get_node(); + } + + static Node* null_node(){return 0;} + + template + static void save_node(Node* node,Archive& ar) + { + ar< alg; +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/invariant_assert.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/invariant_assert.hpp new file mode 100644 index 00000000000..c6c547c7c33 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/invariant_assert.hpp @@ -0,0 +1,21 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_INVARIANT_ASSERT_HPP +#define BOOST_MULTI_INDEX_DETAIL_INVARIANT_ASSERT_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#if !defined(BOOST_MULTI_INDEX_INVARIANT_ASSERT) +#include +#define BOOST_MULTI_INDEX_INVARIANT_ASSERT BOOST_ASSERT +#endif + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_index_list.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_index_list.hpp new file mode 100644 index 00000000000..f6a24218b81 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_index_list.hpp @@ -0,0 +1,40 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_IS_INDEX_LIST_HPP +#define BOOST_MULTI_INDEX_DETAIL_IS_INDEX_LIST_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +template +struct is_index_list +{ + BOOST_STATIC_CONSTANT(bool,mpl_sequence=mpl::is_sequence::value); + BOOST_STATIC_CONSTANT(bool,non_empty=!mpl::empty::value); + BOOST_STATIC_CONSTANT(bool,value=mpl_sequence&&non_empty); +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_transparent.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_transparent.hpp new file mode 100644 index 00000000000..72036d257e2 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_transparent.hpp @@ -0,0 +1,135 @@ +/* Copyright 2003-2014 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_IS_TRANSPARENT_HPP +#define BOOST_MULTI_INDEX_DETAIL_IS_TRANSPARENT_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Metafunction that checks if f(arg,arg2) executes without argument type + * conversion. By default (i.e. when it cannot be determined) it evaluates to + * true. + */ + +template +struct is_transparent:mpl::true_{}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#if !defined(BOOST_NO_SFINAE)&&!defined(BOOST_NO_SFINAE_EXPR)&& \ + !defined(BOOST_NO_CXX11_DECLTYPE)&& \ + (defined(BOOST_NO_CXX11_FINAL)||defined(BOOST_IS_FINAL)) + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +struct not_is_transparent_result_type{}; + +template +struct is_transparent_class_helper:F +{ + using F::operator(); + template + not_is_transparent_result_type operator()(const T&,const Q&)const; +}; + +template +struct is_transparent_class:mpl::true_{}; + +template +struct is_transparent_class< + F,Arg1,Arg2, + typename enable_if< + is_same< + decltype( + declval >()( + declval(),declval()) + ), + not_is_transparent_result_type + > + >::type +>:mpl::false_{}; + +template +struct is_transparent< + F,Arg1,Arg2, + typename enable_if< + mpl::and_< + is_class, + mpl::not_ > /* is_transparent_class_helper derives from F */ + > + >::type +>:is_transparent_class{}; + +template +struct is_transparent_function:mpl::true_{}; + +template +struct is_transparent_function< + F,Arg1,Arg2, + typename enable_if< + mpl::or_< + mpl::not_::arg1_type,const Arg1&>, + is_same::arg1_type,Arg1> + > >, + mpl::not_::arg2_type,const Arg2&>, + is_same::arg2_type,Arg2> + > > + > + >::type +>:mpl::false_{}; + +template +struct is_transparent< + F,Arg1,Arg2, + typename enable_if< + is_function::type> + >::type +>:is_transparent_function::type,Arg1,Arg2>{}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/iter_adaptor.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/iter_adaptor.hpp new file mode 100644 index 00000000000..7a032350b36 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/iter_adaptor.hpp @@ -0,0 +1,321 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_ITER_ADAPTOR_HPP +#define BOOST_MULTI_INDEX_DETAIL_ITER_ADAPTOR_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Poor man's version of boost::iterator_adaptor. Used instead of the + * original as compile times for the latter are significantly higher. + * The interface is not replicated exactly, only to the extent necessary + * for internal consumption. + */ + +/* NB. The purpose of the (non-inclass) global operators ==, < and - defined + * above is to partially alleviate a problem of MSVC++ 6.0 by * which + * friend-injected operators on T are not visible if T is instantiated only + * in template code where T is a dependent type. + */ + +class iter_adaptor_access +{ +public: + template + static typename Class::reference dereference(const Class& x) + { + return x.dereference(); + } + + template + static bool equal(const Class& x,const Class& y) + { + return x.equal(y); + } + + template + static void increment(Class& x) + { + x.increment(); + } + + template + static void decrement(Class& x) + { + x.decrement(); + } + + template + static void advance(Class& x,typename Class::difference_type n) + { + x.advance(n); + } + + template + static typename Class::difference_type distance_to( + const Class& x,const Class& y) + { + return x.distance_to(y); + } +}; + +template +struct iter_adaptor_selector; + +template +class forward_iter_adaptor_base: + public forward_iterator_helper< + Derived, + typename Base::value_type, + typename Base::difference_type, + typename Base::pointer, + typename Base::reference> +{ +public: + typedef typename Base::reference reference; + + reference operator*()const + { + return iter_adaptor_access::dereference(final()); + } + + friend bool operator==(const Derived& x,const Derived& y) + { + return iter_adaptor_access::equal(x,y); + } + + Derived& operator++() + { + iter_adaptor_access::increment(final()); + return final(); + } + +private: + Derived& final(){return *static_cast(this);} + const Derived& final()const{return *static_cast(this);} +}; + +template +bool operator==( + const forward_iter_adaptor_base& x, + const forward_iter_adaptor_base& y) +{ + return iter_adaptor_access::equal( + static_cast(x),static_cast(y)); +} + +template<> +struct iter_adaptor_selector +{ + template + struct apply + { + typedef forward_iter_adaptor_base type; + }; +}; + +template +class bidirectional_iter_adaptor_base: + public bidirectional_iterator_helper< + Derived, + typename Base::value_type, + typename Base::difference_type, + typename Base::pointer, + typename Base::reference> +{ +public: + typedef typename Base::reference reference; + + reference operator*()const + { + return iter_adaptor_access::dereference(final()); + } + + friend bool operator==(const Derived& x,const Derived& y) + { + return iter_adaptor_access::equal(x,y); + } + + Derived& operator++() + { + iter_adaptor_access::increment(final()); + return final(); + } + + Derived& operator--() + { + iter_adaptor_access::decrement(final()); + return final(); + } + +private: + Derived& final(){return *static_cast(this);} + const Derived& final()const{return *static_cast(this);} +}; + +template +bool operator==( + const bidirectional_iter_adaptor_base& x, + const bidirectional_iter_adaptor_base& y) +{ + return iter_adaptor_access::equal( + static_cast(x),static_cast(y)); +} + +template<> +struct iter_adaptor_selector +{ + template + struct apply + { + typedef bidirectional_iter_adaptor_base type; + }; +}; + +template +class random_access_iter_adaptor_base: + public random_access_iterator_helper< + Derived, + typename Base::value_type, + typename Base::difference_type, + typename Base::pointer, + typename Base::reference> +{ +public: + typedef typename Base::reference reference; + typedef typename Base::difference_type difference_type; + + reference operator*()const + { + return iter_adaptor_access::dereference(final()); + } + + friend bool operator==(const Derived& x,const Derived& y) + { + return iter_adaptor_access::equal(x,y); + } + + friend bool operator<(const Derived& x,const Derived& y) + { + return iter_adaptor_access::distance_to(x,y)>0; + } + + Derived& operator++() + { + iter_adaptor_access::increment(final()); + return final(); + } + + Derived& operator--() + { + iter_adaptor_access::decrement(final()); + return final(); + } + + Derived& operator+=(difference_type n) + { + iter_adaptor_access::advance(final(),n); + return final(); + } + + Derived& operator-=(difference_type n) + { + iter_adaptor_access::advance(final(),-n); + return final(); + } + + friend difference_type operator-(const Derived& x,const Derived& y) + { + return iter_adaptor_access::distance_to(y,x); + } + +private: + Derived& final(){return *static_cast(this);} + const Derived& final()const{return *static_cast(this);} +}; + +template +bool operator==( + const random_access_iter_adaptor_base& x, + const random_access_iter_adaptor_base& y) +{ + return iter_adaptor_access::equal( + static_cast(x),static_cast(y)); +} + +template +bool operator<( + const random_access_iter_adaptor_base& x, + const random_access_iter_adaptor_base& y) +{ + return iter_adaptor_access::distance_to( + static_cast(x),static_cast(y))>0; +} + +template +typename random_access_iter_adaptor_base::difference_type +operator-( + const random_access_iter_adaptor_base& x, + const random_access_iter_adaptor_base& y) +{ + return iter_adaptor_access::distance_to( + static_cast(y),static_cast(x)); +} + +template<> +struct iter_adaptor_selector +{ + template + struct apply + { + typedef random_access_iter_adaptor_base type; + }; +}; + +template +struct iter_adaptor_base +{ + typedef iter_adaptor_selector< + typename Base::iterator_category> selector; + typedef typename mpl::apply2< + selector,Derived,Base>::type type; +}; + +template +class iter_adaptor:public iter_adaptor_base::type +{ +protected: + iter_adaptor(){} + explicit iter_adaptor(const Base& b_):b(b_){} + + const Base& base_reference()const{return b;} + Base& base_reference(){return b;} + +private: + Base b; +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/modify_key_adaptor.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/modify_key_adaptor.hpp new file mode 100644 index 00000000000..6df89b18386 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/modify_key_adaptor.hpp @@ -0,0 +1,49 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_MODIFY_KEY_ADAPTOR_HPP +#define BOOST_MULTI_INDEX_DETAIL_MODIFY_KEY_ADAPTOR_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Functional adaptor to resolve modify_key as a call to modify. + * Preferred over compose_f_gx and stuff cause it eliminates problems + * with references to references, dealing with function pointers, etc. + */ + +template +struct modify_key_adaptor +{ + + modify_key_adaptor(Fun f_,KeyFromValue kfv_):f(f_),kfv(kfv_){} + + void operator()(Value& x) + { + f(kfv(x)); + } + +private: + Fun f; + KeyFromValue kfv; +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/no_duplicate_tags.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/no_duplicate_tags.hpp new file mode 100644 index 00000000000..ba216ed82cf --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/no_duplicate_tags.hpp @@ -0,0 +1,97 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_NO_DUPLICATE_TAGS_HPP +#define BOOST_MULTI_INDEX_DETAIL_NO_DUPLICATE_TAGS_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* no_duplicate_tags check at compile-time that a tag list + * has no duplicate tags. + * The algorithm deserves some explanation: tags + * are sequentially inserted into a mpl::set if they were + * not already present. Due to the magic of mpl::set + * (mpl::has_key is contant time), this operation takes linear + * time, and even MSVC++ 6.5 handles it gracefully (other obvious + * solutions are quadratic.) + */ + +struct duplicate_tag_mark{}; + +struct duplicate_tag_marker +{ + template + struct apply + { + typedef mpl::s_item< + typename mpl::if_,duplicate_tag_mark,Tag>::type, + MplSet + > type; + }; +}; + +template +struct no_duplicate_tags +{ + typedef typename mpl::fold< + TagList, + mpl::set0<>, + duplicate_tag_marker + >::type aux; + + BOOST_STATIC_CONSTANT( + bool,value=!(mpl::has_key::value)); +}; + +/* Variant for an index list: duplication is checked + * across all the indices. + */ + +struct duplicate_tag_list_marker +{ + template + struct apply:mpl::fold< + BOOST_DEDUCED_TYPENAME Index::tag_list, + MplSet, + duplicate_tag_marker> + { + }; +}; + +template +struct no_duplicate_tags_in_index_list +{ + typedef typename mpl::fold< + IndexList, + mpl::set0<>, + duplicate_tag_list_marker + >::type aux; + + BOOST_STATIC_CONSTANT( + bool,value=!(mpl::has_key::value)); +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/node_type.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/node_type.hpp new file mode 100644 index 00000000000..7fe85cf968b --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/node_type.hpp @@ -0,0 +1,66 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_NODE_TYPE_HPP +#define BOOST_MULTI_INDEX_DETAIL_NODE_TYPE_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* MPL machinery to construct the internal node type associated to an + * index list. + */ + +struct index_node_applier +{ + template + struct apply + { + typedef typename mpl::deref::type index_specifier; + typedef typename index_specifier:: + BOOST_NESTED_TEMPLATE node_class::type type; + }; +}; + +template +struct multi_index_node_type +{ + BOOST_STATIC_ASSERT(detail::is_index_list::value); + + typedef typename mpl::reverse_iter_fold< + IndexSpecifierList, + index_node_base, + mpl::bind2 + >::type type; +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_args.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_args.hpp new file mode 100644 index 00000000000..3e2641f2f4d --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_args.hpp @@ -0,0 +1,83 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_ARGS_HPP +#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_ARGS_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Oredered index specifiers can be instantiated in two forms: + * + * (ordered_unique|ordered_non_unique)< + * KeyFromValue,Compare=std::less > + * (ordered_unique|ordered_non_unique)< + * TagList,KeyFromValue,Compare=std::less > + * + * index_args implements the machinery to accept this argument-dependent + * polymorphism. + */ + +template +struct index_args_default_compare +{ + typedef std::less type; +}; + +template +struct ordered_index_args +{ + typedef is_tag full_form; + + typedef typename mpl::if_< + full_form, + Arg1, + tag< > >::type tag_list_type; + typedef typename mpl::if_< + full_form, + Arg2, + Arg1>::type key_from_value_type; + typedef typename mpl::if_< + full_form, + Arg3, + Arg2>::type supplied_compare_type; + typedef typename mpl::eval_if< + mpl::is_na, + index_args_default_compare, + mpl::identity + >::type compare_type; + + BOOST_STATIC_ASSERT(is_tag::value); + BOOST_STATIC_ASSERT(!mpl::is_na::value); + BOOST_STATIC_ASSERT(!mpl::is_na::value); +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl.hpp new file mode 100644 index 00000000000..040cb989630 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl.hpp @@ -0,0 +1,1567 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + * + * The internal implementation of red-black trees is based on that of SGI STL + * stl_tree.h file: + * + * Copyright (c) 1996,1997 + * Silicon Graphics Computer Systems, Inc. + * + * Permission to use, copy, modify, distribute and sell this software + * and its documentation for any purpose is hereby granted without fee, + * provided that the above copyright notice appear in all copies and + * that both that copyright notice and this permission notice appear + * in supporting documentation. Silicon Graphics makes no + * representations about the suitability of this software for any + * purpose. It is provided "as is" without express or implied warranty. + * + * + * Copyright (c) 1994 + * Hewlett-Packard Company + * + * Permission to use, copy, modify, distribute and sell this software + * and its documentation for any purpose is hereby granted without fee, + * provided that the above copyright notice appear in all copies and + * that both that copyright notice and this permission notice appear + * in supporting documentation. Hewlett-Packard Company makes no + * representations about the suitability of this software for any + * purpose. It is provided "as is" without express or implied warranty. + * + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_IMPL_HPP +#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_IMPL_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) +#include +#endif + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +#include +#include +#include +#include +#endif + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) +#define BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF(x) \ + detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ + detail::make_obj_guard(x,&ordered_index_impl::check_invariant_); \ + BOOST_JOIN(check_invariant_,__LINE__).touch(); +#define BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT \ + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF(*this) +#else +#define BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF(x) +#define BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* ordered_index adds a layer of ordered indexing to a given Super and accepts + * an augmenting policy for optional addition of order statistics. + */ + +/* Most of the implementation of unique and non-unique indices is + * shared. We tell from one another on instantiation time by using + * these tags. + */ + +struct ordered_unique_tag{}; +struct ordered_non_unique_tag{}; + +template< + typename KeyFromValue,typename Compare, + typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy +> +class ordered_index; + +template< + typename KeyFromValue,typename Compare, + typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy +> +class ordered_index_impl: + BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS SuperMeta::type + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + ,public safe_mode::safe_container< + ordered_index_impl< + KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy> > +#endif + +{ +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ + BOOST_WORKAROUND(__MWERKS__,<=0x3003) +/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the + * lifetime of const references bound to temporaries --precisely what + * scopeguards are. + */ + +#pragma parse_mfunc_templ off +#endif + + typedef typename SuperMeta::type super; + +protected: + typedef ordered_index_node< + AugmentPolicy,typename super::node_type> node_type; + +protected: /* for the benefit of AugmentPolicy::augmented_interface */ + typedef typename node_type::impl_type node_impl_type; + typedef typename node_impl_type::pointer node_impl_pointer; + +public: + /* types */ + + typedef typename KeyFromValue::result_type key_type; + typedef typename node_type::value_type value_type; + typedef KeyFromValue key_from_value; + typedef Compare key_compare; + typedef value_comparison< + value_type,KeyFromValue,Compare> value_compare; + typedef tuple ctor_args; + typedef typename super::final_allocator_type allocator_type; + typedef typename allocator_type::reference reference; + typedef typename allocator_type::const_reference const_reference; + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + typedef safe_mode::safe_iterator< + bidir_node_iterator, + ordered_index_impl> iterator; +#else + typedef bidir_node_iterator iterator; +#endif + + typedef iterator const_iterator; + + typedef std::size_t size_type; + typedef std::ptrdiff_t difference_type; + typedef typename allocator_type::pointer pointer; + typedef typename allocator_type::const_pointer const_pointer; + typedef typename + boost::reverse_iterator reverse_iterator; + typedef typename + boost::reverse_iterator const_reverse_iterator; + typedef TagList tag_list; + +protected: + typedef typename super::final_node_type final_node_type; + typedef tuples::cons< + ctor_args, + typename super::ctor_args_list> ctor_args_list; + typedef typename mpl::push_front< + typename super::index_type_list, + ordered_index< + KeyFromValue,Compare, + SuperMeta,TagList,Category,AugmentPolicy + > >::type index_type_list; + typedef typename mpl::push_front< + typename super::iterator_type_list, + iterator>::type iterator_type_list; + typedef typename mpl::push_front< + typename super::const_iterator_type_list, + const_iterator>::type const_iterator_type_list; + typedef typename super::copy_map_type copy_map_type; + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + typedef typename super::index_saver_type index_saver_type; + typedef typename super::index_loader_type index_loader_type; +#endif + +protected: +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + typedef safe_mode::safe_container< + ordered_index_impl> safe_super; +#endif + + typedef typename call_traits< + value_type>::param_type value_param_type; + typedef typename call_traits< + key_type>::param_type key_param_type; + + /* Needed to avoid commas in BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL + * expansion. + */ + + typedef std::pair emplace_return_type; + +public: + + /* construct/copy/destroy + * Default and copy ctors are in the protected section as indices are + * not supposed to be created on their own. No range ctor either. + * Assignment operators defined at ordered_index rather than here. + */ + + allocator_type get_allocator()const BOOST_NOEXCEPT + { + return this->final().get_allocator(); + } + + /* iterators */ + + iterator + begin()BOOST_NOEXCEPT{return make_iterator(leftmost());} + const_iterator + begin()const BOOST_NOEXCEPT{return make_iterator(leftmost());} + iterator + end()BOOST_NOEXCEPT{return make_iterator(header());} + const_iterator + end()const BOOST_NOEXCEPT{return make_iterator(header());} + reverse_iterator + rbegin()BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} + const_reverse_iterator + rbegin()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} + reverse_iterator + rend()BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} + const_reverse_iterator + rend()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} + const_iterator + cbegin()const BOOST_NOEXCEPT{return begin();} + const_iterator + cend()const BOOST_NOEXCEPT{return end();} + const_reverse_iterator + crbegin()const BOOST_NOEXCEPT{return rbegin();} + const_reverse_iterator + crend()const BOOST_NOEXCEPT{return rend();} + + iterator iterator_to(const value_type& x) + { + return make_iterator(node_from_value(&x)); + } + + const_iterator iterator_to(const value_type& x)const + { + return make_iterator(node_from_value(&x)); + } + + /* capacity */ + + bool empty()const BOOST_NOEXCEPT{return this->final_empty_();} + size_type size()const BOOST_NOEXCEPT{return this->final_size_();} + size_type max_size()const BOOST_NOEXCEPT{return this->final_max_size_();} + + /* modifiers */ + + BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( + emplace_return_type,emplace,emplace_impl) + + BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( + iterator,emplace_hint,emplace_hint_impl,iterator,position) + + std::pair insert(const value_type& x) + { + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + std::pair p=this->final_insert_(x); + return std::pair(make_iterator(p.first),p.second); + } + + std::pair insert(BOOST_RV_REF(value_type) x) + { + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + std::pair p=this->final_insert_rv_(x); + return std::pair(make_iterator(p.first),p.second); + } + + iterator insert(iterator position,const value_type& x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + std::pair p=this->final_insert_( + x,static_cast(position.get_node())); + return make_iterator(p.first); + } + + iterator insert(iterator position,BOOST_RV_REF(value_type) x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + std::pair p=this->final_insert_rv_( + x,static_cast(position.get_node())); + return make_iterator(p.first); + } + + template + void insert(InputIterator first,InputIterator last) + { + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + node_type* hint=header(); /* end() */ + for(;first!=last;++first){ + hint=this->final_insert_ref_( + *first,static_cast(hint)).first; + node_type::increment(hint); + } + } + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) + void insert(std::initializer_list list) + { + insert(list.begin(),list.end()); + } +#endif + + iterator erase(iterator position) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + this->final_erase_(static_cast(position++.get_node())); + return position; + } + + size_type erase(key_param_type x) + { + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + std::pair p=equal_range(x); + size_type s=0; + while(p.first!=p.second){ + p.first=erase(p.first); + ++s; + } + return s; + } + + iterator erase(iterator first,iterator last) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); + BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + while(first!=last){ + first=erase(first); + } + return first; + } + + bool replace(iterator position,const value_type& x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + return this->final_replace_( + x,static_cast(position.get_node())); + } + + bool replace(iterator position,BOOST_RV_REF(value_type) x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + return this->final_replace_rv_( + x,static_cast(position.get_node())); + } + + template + bool modify(iterator position,Modifier mod) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + /* MSVC++ 6.0 optimizer on safe mode code chokes if this + * this is not added. Left it for all compilers as it does no + * harm. + */ + + position.detach(); +#endif + + return this->final_modify_( + mod,static_cast(position.get_node())); + } + + template + bool modify(iterator position,Modifier mod,Rollback back_) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + /* MSVC++ 6.0 optimizer on safe mode code chokes if this + * this is not added. Left it for all compilers as it does no + * harm. + */ + + position.detach(); +#endif + + return this->final_modify_( + mod,back_,static_cast(position.get_node())); + } + + template + bool modify_key(iterator position,Modifier mod) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + return modify( + position,modify_key_adaptor(mod,key)); + } + + template + bool modify_key(iterator position,Modifier mod,Rollback back_) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + return modify( + position, + modify_key_adaptor(mod,key), + modify_key_adaptor(back_,key)); + } + + void swap( + ordered_index< + KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x) + { + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF(x); + this->final_swap_(x.final()); + } + + void clear()BOOST_NOEXCEPT + { + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + this->final_clear_(); + } + + /* observers */ + + key_from_value key_extractor()const{return key;} + key_compare key_comp()const{return comp_;} + value_compare value_comp()const{return value_compare(key,comp_);} + + /* set operations */ + + /* Internally, these ops rely on const_iterator being the same + * type as iterator. + */ + + template + iterator find(const CompatibleKey& x)const + { + return make_iterator(ordered_index_find(root(),header(),key,x,comp_)); + } + + template + iterator find( + const CompatibleKey& x,const CompatibleCompare& comp)const + { + return make_iterator(ordered_index_find(root(),header(),key,x,comp)); + } + + template + size_type count(const CompatibleKey& x)const + { + return count(x,comp_); + } + + template + size_type count(const CompatibleKey& x,const CompatibleCompare& comp)const + { + std::pair p=equal_range(x,comp); + size_type n=std::distance(p.first,p.second); + return n; + } + + template + iterator lower_bound(const CompatibleKey& x)const + { + return make_iterator( + ordered_index_lower_bound(root(),header(),key,x,comp_)); + } + + template + iterator lower_bound( + const CompatibleKey& x,const CompatibleCompare& comp)const + { + return make_iterator( + ordered_index_lower_bound(root(),header(),key,x,comp)); + } + + template + iterator upper_bound(const CompatibleKey& x)const + { + return make_iterator( + ordered_index_upper_bound(root(),header(),key,x,comp_)); + } + + template + iterator upper_bound( + const CompatibleKey& x,const CompatibleCompare& comp)const + { + return make_iterator( + ordered_index_upper_bound(root(),header(),key,x,comp)); + } + + template + std::pair equal_range( + const CompatibleKey& x)const + { + std::pair p= + ordered_index_equal_range(root(),header(),key,x,comp_); + return std::pair( + make_iterator(p.first),make_iterator(p.second)); + } + + template + std::pair equal_range( + const CompatibleKey& x,const CompatibleCompare& comp)const + { + std::pair p= + ordered_index_equal_range(root(),header(),key,x,comp); + return std::pair( + make_iterator(p.first),make_iterator(p.second)); + } + + /* range */ + + template + std::pair + range(LowerBounder lower,UpperBounder upper)const + { + typedef typename mpl::if_< + is_same, + BOOST_DEDUCED_TYPENAME mpl::if_< + is_same, + both_unbounded_tag, + lower_unbounded_tag + >::type, + BOOST_DEDUCED_TYPENAME mpl::if_< + is_same, + upper_unbounded_tag, + none_unbounded_tag + >::type + >::type dispatch; + + return range(lower,upper,dispatch()); + } + +BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: + ordered_index_impl(const ctor_args_list& args_list,const allocator_type& al): + super(args_list.get_tail(),al), + key(tuples::get<0>(args_list.get_head())), + comp_(tuples::get<1>(args_list.get_head())) + { + empty_initialize(); + } + + ordered_index_impl( + const ordered_index_impl< + KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x): + super(x), + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super(), +#endif + + key(x.key), + comp_(x.comp_) + { + /* Copy ctor just takes the key and compare objects from x. The rest is + * done in a subsequent call to copy_(). + */ + } + + ordered_index_impl( + const ordered_index_impl< + KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x, + do_not_copy_elements_tag): + super(x,do_not_copy_elements_tag()), + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super(), +#endif + + key(x.key), + comp_(x.comp_) + { + empty_initialize(); + } + + ~ordered_index_impl() + { + /* the container is guaranteed to be empty by now */ + } + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + iterator make_iterator(node_type* node){return iterator(node,this);} + const_iterator make_iterator(node_type* node)const + {return const_iterator(node,const_cast(this));} +#else + iterator make_iterator(node_type* node){return iterator(node);} + const_iterator make_iterator(node_type* node)const + {return const_iterator(node);} +#endif + + void copy_( + const ordered_index_impl< + KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x, + const copy_map_type& map) + { + if(!x.root()){ + empty_initialize(); + } + else{ + header()->color()=x.header()->color(); + AugmentPolicy::copy(x.header()->impl(),header()->impl()); + + node_type* root_cpy=map.find(static_cast(x.root())); + header()->parent()=root_cpy->impl(); + + node_type* leftmost_cpy=map.find( + static_cast(x.leftmost())); + header()->left()=leftmost_cpy->impl(); + + node_type* rightmost_cpy=map.find( + static_cast(x.rightmost())); + header()->right()=rightmost_cpy->impl(); + + typedef typename copy_map_type::const_iterator copy_map_iterator; + for(copy_map_iterator it=map.begin(),it_end=map.end();it!=it_end;++it){ + node_type* org=it->first; + node_type* cpy=it->second; + + cpy->color()=org->color(); + AugmentPolicy::copy(org->impl(),cpy->impl()); + + node_impl_pointer parent_org=org->parent(); + if(parent_org==node_impl_pointer(0))cpy->parent()=node_impl_pointer(0); + else{ + node_type* parent_cpy=map.find( + static_cast(node_type::from_impl(parent_org))); + cpy->parent()=parent_cpy->impl(); + if(parent_org->left()==org->impl()){ + parent_cpy->left()=cpy->impl(); + } + else if(parent_org->right()==org->impl()){ + /* header() does not satisfy this nor the previous check */ + parent_cpy->right()=cpy->impl(); + } + } + + if(org->left()==node_impl_pointer(0)) + cpy->left()=node_impl_pointer(0); + if(org->right()==node_impl_pointer(0)) + cpy->right()=node_impl_pointer(0); + } + } + + super::copy_(x,map); + } + + template + final_node_type* insert_( + value_param_type v,final_node_type*& x,Variant variant) + { + link_info inf; + if(!link_point(key(v),inf,Category())){ + return static_cast(node_type::from_impl(inf.pos)); + } + + final_node_type* res=super::insert_(v,x,variant); + if(res==x){ + node_impl_type::link( + static_cast(x)->impl(),inf.side,inf.pos,header()->impl()); + } + return res; + } + + template + final_node_type* insert_( + value_param_type v,node_type* position,final_node_type*& x,Variant variant) + { + link_info inf; + if(!hinted_link_point(key(v),position,inf,Category())){ + return static_cast(node_type::from_impl(inf.pos)); + } + + final_node_type* res=super::insert_(v,position,x,variant); + if(res==x){ + node_impl_type::link( + static_cast(x)->impl(),inf.side,inf.pos,header()->impl()); + } + return res; + } + + void erase_(node_type* x) + { + node_impl_type::rebalance_for_erase( + x->impl(),header()->parent(),header()->left(),header()->right()); + super::erase_(x); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + } + + void delete_all_nodes_() + { + delete_all_nodes(root()); + } + + void clear_() + { + super::clear_(); + empty_initialize(); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super::detach_dereferenceable_iterators(); +#endif + } + + void swap_( + ordered_index_impl< + KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x) + { + std::swap(key,x.key); + std::swap(comp_,x.comp_); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super::swap(x); +#endif + + super::swap_(x); + } + + void swap_elements_( + ordered_index_impl< + KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x) + { +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super::swap(x); +#endif + + super::swap_elements_(x); + } + + template + bool replace_(value_param_type v,node_type* x,Variant variant) + { + if(in_place(v,x,Category())){ + return super::replace_(v,x,variant); + } + + node_type* next=x; + node_type::increment(next); + + node_impl_type::rebalance_for_erase( + x->impl(),header()->parent(),header()->left(),header()->right()); + + BOOST_TRY{ + link_info inf; + if(link_point(key(v),inf,Category())&&super::replace_(v,x,variant)){ + node_impl_type::link(x->impl(),inf.side,inf.pos,header()->impl()); + return true; + } + node_impl_type::restore(x->impl(),next->impl(),header()->impl()); + return false; + } + BOOST_CATCH(...){ + node_impl_type::restore(x->impl(),next->impl(),header()->impl()); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + bool modify_(node_type* x) + { + bool b; + BOOST_TRY{ + b=in_place(x->value(),x,Category()); + } + BOOST_CATCH(...){ + erase_(x); + BOOST_RETHROW; + } + BOOST_CATCH_END + if(!b){ + node_impl_type::rebalance_for_erase( + x->impl(),header()->parent(),header()->left(),header()->right()); + BOOST_TRY{ + link_info inf; + if(!link_point(key(x->value()),inf,Category())){ + super::erase_(x); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + return false; + } + node_impl_type::link(x->impl(),inf.side,inf.pos,header()->impl()); + } + BOOST_CATCH(...){ + super::erase_(x); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + BOOST_TRY{ + if(!super::modify_(x)){ + node_impl_type::rebalance_for_erase( + x->impl(),header()->parent(),header()->left(),header()->right()); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + + return false; + } + else return true; + } + BOOST_CATCH(...){ + node_impl_type::rebalance_for_erase( + x->impl(),header()->parent(),header()->left(),header()->right()); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + bool modify_rollback_(node_type* x) + { + if(in_place(x->value(),x,Category())){ + return super::modify_rollback_(x); + } + + node_type* next=x; + node_type::increment(next); + + node_impl_type::rebalance_for_erase( + x->impl(),header()->parent(),header()->left(),header()->right()); + + BOOST_TRY{ + link_info inf; + if(link_point(key(x->value()),inf,Category())&& + super::modify_rollback_(x)){ + node_impl_type::link(x->impl(),inf.side,inf.pos,header()->impl()); + return true; + } + node_impl_type::restore(x->impl(),next->impl(),header()->impl()); + return false; + } + BOOST_CATCH(...){ + node_impl_type::restore(x->impl(),next->impl(),header()->impl()); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + /* serialization */ + + template + void save_( + Archive& ar,const unsigned int version,const index_saver_type& sm)const + { + save_(ar,version,sm,Category()); + } + + template + void load_(Archive& ar,const unsigned int version,const index_loader_type& lm) + { + load_(ar,version,lm,Category()); + } +#endif + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) + /* invariant stuff */ + + bool invariant_()const + { + if(size()==0||begin()==end()){ + if(size()!=0||begin()!=end()|| + header()->left()!=header()->impl()|| + header()->right()!=header()->impl())return false; + } + else{ + if((size_type)std::distance(begin(),end())!=size())return false; + + std::size_t len=node_impl_type::black_count( + leftmost()->impl(),root()->impl()); + for(const_iterator it=begin(),it_end=end();it!=it_end;++it){ + node_type* x=it.get_node(); + node_type* left_x=node_type::from_impl(x->left()); + node_type* right_x=node_type::from_impl(x->right()); + + if(x->color()==red){ + if((left_x&&left_x->color()==red)|| + (right_x&&right_x->color()==red))return false; + } + if(left_x&&comp_(key(x->value()),key(left_x->value())))return false; + if(right_x&&comp_(key(right_x->value()),key(x->value())))return false; + if(!left_x&&!right_x&& + node_impl_type::black_count(x->impl(),root()->impl())!=len) + return false; + if(!AugmentPolicy::invariant(x->impl()))return false; + } + + if(leftmost()->impl()!=node_impl_type::minimum(root()->impl())) + return false; + if(rightmost()->impl()!=node_impl_type::maximum(root()->impl())) + return false; + } + + return super::invariant_(); + } + + + /* This forwarding function eases things for the boost::mem_fn construct + * in BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT. Actually, + * final_check_invariant is already an inherited member function of + * ordered_index_impl. + */ + void check_invariant_()const{this->final_check_invariant_();} +#endif + +protected: /* for the benefit of AugmentPolicy::augmented_interface */ + node_type* header()const{return this->final_header();} + node_type* root()const{return node_type::from_impl(header()->parent());} + node_type* leftmost()const{return node_type::from_impl(header()->left());} + node_type* rightmost()const{return node_type::from_impl(header()->right());} + +private: + void empty_initialize() + { + header()->color()=red; + /* used to distinguish header() from root, in iterator.operator++ */ + + header()->parent()=node_impl_pointer(0); + header()->left()=header()->impl(); + header()->right()=header()->impl(); + } + + struct link_info + { + /* coverity[uninit_ctor]: suppress warning */ + link_info():side(to_left){} + + ordered_index_side side; + node_impl_pointer pos; + }; + + bool link_point(key_param_type k,link_info& inf,ordered_unique_tag) + { + node_type* y=header(); + node_type* x=root(); + bool c=true; + while(x){ + y=x; + c=comp_(k,key(x->value())); + x=node_type::from_impl(c?x->left():x->right()); + } + node_type* yy=y; + if(c){ + if(yy==leftmost()){ + inf.side=to_left; + inf.pos=y->impl(); + return true; + } + else node_type::decrement(yy); + } + + if(comp_(key(yy->value()),k)){ + inf.side=c?to_left:to_right; + inf.pos=y->impl(); + return true; + } + else{ + inf.pos=yy->impl(); + return false; + } + } + + bool link_point(key_param_type k,link_info& inf,ordered_non_unique_tag) + { + node_type* y=header(); + node_type* x=root(); + bool c=true; + while (x){ + y=x; + c=comp_(k,key(x->value())); + x=node_type::from_impl(c?x->left():x->right()); + } + inf.side=c?to_left:to_right; + inf.pos=y->impl(); + return true; + } + + bool lower_link_point(key_param_type k,link_info& inf,ordered_non_unique_tag) + { + node_type* y=header(); + node_type* x=root(); + bool c=false; + while (x){ + y=x; + c=comp_(key(x->value()),k); + x=node_type::from_impl(c?x->right():x->left()); + } + inf.side=c?to_right:to_left; + inf.pos=y->impl(); + return true; + } + + bool hinted_link_point( + key_param_type k,node_type* position,link_info& inf,ordered_unique_tag) + { + if(position->impl()==header()->left()){ + if(size()>0&&comp_(k,key(position->value()))){ + inf.side=to_left; + inf.pos=position->impl(); + return true; + } + else return link_point(k,inf,ordered_unique_tag()); + } + else if(position==header()){ + if(comp_(key(rightmost()->value()),k)){ + inf.side=to_right; + inf.pos=rightmost()->impl(); + return true; + } + else return link_point(k,inf,ordered_unique_tag()); + } + else{ + node_type* before=position; + node_type::decrement(before); + if(comp_(key(before->value()),k)&&comp_(k,key(position->value()))){ + if(before->right()==node_impl_pointer(0)){ + inf.side=to_right; + inf.pos=before->impl(); + return true; + } + else{ + inf.side=to_left; + inf.pos=position->impl(); + return true; + } + } + else return link_point(k,inf,ordered_unique_tag()); + } + } + + bool hinted_link_point( + key_param_type k,node_type* position,link_info& inf,ordered_non_unique_tag) + { + if(position->impl()==header()->left()){ + if(size()>0&&!comp_(key(position->value()),k)){ + inf.side=to_left; + inf.pos=position->impl(); + return true; + } + else return lower_link_point(k,inf,ordered_non_unique_tag()); + } + else if(position==header()){ + if(!comp_(k,key(rightmost()->value()))){ + inf.side=to_right; + inf.pos=rightmost()->impl(); + return true; + } + else return link_point(k,inf,ordered_non_unique_tag()); + } + else{ + node_type* before=position; + node_type::decrement(before); + if(!comp_(k,key(before->value()))){ + if(!comp_(key(position->value()),k)){ + if(before->right()==node_impl_pointer(0)){ + inf.side=to_right; + inf.pos=before->impl(); + return true; + } + else{ + inf.side=to_left; + inf.pos=position->impl(); + return true; + } + } + else return lower_link_point(k,inf,ordered_non_unique_tag()); + } + else return link_point(k,inf,ordered_non_unique_tag()); + } + } + + void delete_all_nodes(node_type* x) + { + if(!x)return; + + delete_all_nodes(node_type::from_impl(x->left())); + delete_all_nodes(node_type::from_impl(x->right())); + this->final_delete_node_(static_cast(x)); + } + + bool in_place(value_param_type v,node_type* x,ordered_unique_tag) + { + node_type* y; + if(x!=leftmost()){ + y=x; + node_type::decrement(y); + if(!comp_(key(y->value()),key(v)))return false; + } + + y=x; + node_type::increment(y); + return y==header()||comp_(key(v),key(y->value())); + } + + bool in_place(value_param_type v,node_type* x,ordered_non_unique_tag) + { + node_type* y; + if(x!=leftmost()){ + y=x; + node_type::decrement(y); + if(comp_(key(v),key(y->value())))return false; + } + + y=x; + node_type::increment(y); + return y==header()||!comp_(key(y->value()),key(v)); + } + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + void detach_iterators(node_type* x) + { + iterator it=make_iterator(x); + safe_mode::detach_equivalent_iterators(it); + } +#endif + + template + std::pair emplace_impl(BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + std::pairp= + this->final_emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + return std::pair(make_iterator(p.first),p.second); + } + + template + iterator emplace_hint_impl( + iterator position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; + std::pairp= + this->final_emplace_hint_( + static_cast(position.get_node()), + BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + return make_iterator(p.first); + } + + template + std::pair + range(LowerBounder lower,UpperBounder upper,none_unbounded_tag)const + { + node_type* y=header(); + node_type* z=root(); + + while(z){ + if(!lower(key(z->value()))){ + z=node_type::from_impl(z->right()); + } + else if(!upper(key(z->value()))){ + y=z; + z=node_type::from_impl(z->left()); + } + else{ + return std::pair( + make_iterator( + lower_range(node_type::from_impl(z->left()),z,lower)), + make_iterator( + upper_range(node_type::from_impl(z->right()),y,upper))); + } + } + + return std::pair(make_iterator(y),make_iterator(y)); + } + + template + std::pair + range(LowerBounder,UpperBounder upper,lower_unbounded_tag)const + { + return std::pair( + begin(), + make_iterator(upper_range(root(),header(),upper))); + } + + template + std::pair + range(LowerBounder lower,UpperBounder,upper_unbounded_tag)const + { + return std::pair( + make_iterator(lower_range(root(),header(),lower)), + end()); + } + + template + std::pair + range(LowerBounder,UpperBounder,both_unbounded_tag)const + { + return std::pair(begin(),end()); + } + + template + node_type * lower_range(node_type* top,node_type* y,LowerBounder lower)const + { + while(top){ + if(lower(key(top->value()))){ + y=top; + top=node_type::from_impl(top->left()); + } + else top=node_type::from_impl(top->right()); + } + + return y; + } + + template + node_type * upper_range(node_type* top,node_type* y,UpperBounder upper)const + { + while(top){ + if(!upper(key(top->value()))){ + y=top; + top=node_type::from_impl(top->left()); + } + else top=node_type::from_impl(top->right()); + } + + return y; + } + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + template + void save_( + Archive& ar,const unsigned int version,const index_saver_type& sm, + ordered_unique_tag)const + { + super::save_(ar,version,sm); + } + + template + void load_( + Archive& ar,const unsigned int version,const index_loader_type& lm, + ordered_unique_tag) + { + super::load_(ar,version,lm); + } + + template + void save_( + Archive& ar,const unsigned int version,const index_saver_type& sm, + ordered_non_unique_tag)const + { + typedef duplicates_iterator dup_iterator; + + sm.save( + dup_iterator(begin().get_node(),end().get_node(),value_comp()), + dup_iterator(end().get_node(),value_comp()), + ar,version); + super::save_(ar,version,sm); + } + + template + void load_( + Archive& ar,const unsigned int version,const index_loader_type& lm, + ordered_non_unique_tag) + { + lm.load( + ::boost::bind( + &ordered_index_impl::rearranger,this, + ::boost::arg<1>(),::boost::arg<2>()), + ar,version); + super::load_(ar,version,lm); + } + + void rearranger(node_type* position,node_type *x) + { + if(!position||comp_(key(position->value()),key(x->value()))){ + position=lower_bound(key(x->value())).get_node(); + } + else if(comp_(key(x->value()),key(position->value()))){ + /* inconsistent rearrangement */ + throw_exception( + archive::archive_exception( + archive::archive_exception::other_exception)); + } + else node_type::increment(position); + + if(position!=x){ + node_impl_type::rebalance_for_erase( + x->impl(),header()->parent(),header()->left(),header()->right()); + node_impl_type::restore( + x->impl(),position->impl(),header()->impl()); + } + } +#endif /* serialization */ + +protected: /* for the benefit of AugmentPolicy::augmented_interface */ + key_from_value key; + key_compare comp_; + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ + BOOST_WORKAROUND(__MWERKS__,<=0x3003) +#pragma parse_mfunc_templ reset +#endif +}; + +template< + typename KeyFromValue,typename Compare, + typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy +> +class ordered_index: + public AugmentPolicy::template augmented_interface< + ordered_index_impl< + KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy + > + >::type +{ + typedef typename AugmentPolicy::template + augmented_interface< + ordered_index_impl< + KeyFromValue,Compare, + SuperMeta,TagList,Category,AugmentPolicy + > + >::type super; +public: + typedef typename super::ctor_args_list ctor_args_list; + typedef typename super::allocator_type allocator_type; + typedef typename super::iterator iterator; + + /* construct/copy/destroy + * Default and copy ctors are in the protected section as indices are + * not supposed to be created on their own. No range ctor either. + */ + + ordered_index& operator=(const ordered_index& x) + { + this->final()=x.final(); + return *this; + } + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) + ordered_index& operator=( + std::initializer_list list) + { + this->final()=list; + return *this; + } +#endif + +protected: + ordered_index( + const ctor_args_list& args_list,const allocator_type& al): + super(args_list,al){} + + ordered_index(const ordered_index& x):super(x){}; + + ordered_index(const ordered_index& x,do_not_copy_elements_tag): + super(x,do_not_copy_elements_tag()){}; +}; + +/* comparison */ + +template< + typename KeyFromValue1,typename Compare1, + typename SuperMeta1,typename TagList1,typename Category1, + typename AugmentPolicy1, + typename KeyFromValue2,typename Compare2, + typename SuperMeta2,typename TagList2,typename Category2, + typename AugmentPolicy2 +> +bool operator==( + const ordered_index< + KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, + const ordered_index< + KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) +{ + return x.size()==y.size()&&std::equal(x.begin(),x.end(),y.begin()); +} + +template< + typename KeyFromValue1,typename Compare1, + typename SuperMeta1,typename TagList1,typename Category1, + typename AugmentPolicy1, + typename KeyFromValue2,typename Compare2, + typename SuperMeta2,typename TagList2,typename Category2, + typename AugmentPolicy2 +> +bool operator<( + const ordered_index< + KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, + const ordered_index< + KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) +{ + return std::lexicographical_compare(x.begin(),x.end(),y.begin(),y.end()); +} + +template< + typename KeyFromValue1,typename Compare1, + typename SuperMeta1,typename TagList1,typename Category1, + typename AugmentPolicy1, + typename KeyFromValue2,typename Compare2, + typename SuperMeta2,typename TagList2,typename Category2, + typename AugmentPolicy2 +> +bool operator!=( + const ordered_index< + KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, + const ordered_index< + KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) +{ + return !(x==y); +} + +template< + typename KeyFromValue1,typename Compare1, + typename SuperMeta1,typename TagList1,typename Category1, + typename AugmentPolicy1, + typename KeyFromValue2,typename Compare2, + typename SuperMeta2,typename TagList2,typename Category2, + typename AugmentPolicy2 +> +bool operator>( + const ordered_index< + KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, + const ordered_index< + KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) +{ + return y +bool operator>=( + const ordered_index< + KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, + const ordered_index< + KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) +{ + return !(x +bool operator<=( + const ordered_index< + KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, + const ordered_index< + KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) +{ + return !(x>y); +} + +/* specialized algorithms */ + +template< + typename KeyFromValue,typename Compare, + typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy +> +void swap( + ordered_index< + KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x, + ordered_index< + KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& y) +{ + x.swap(y); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +/* Boost.Foreach compatibility */ + +template< + typename KeyFromValue,typename Compare, + typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy +> +inline boost::mpl::true_* boost_foreach_is_noncopyable( + boost::multi_index::detail::ordered_index< + KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>*&, + boost_foreach_argument_dependent_lookup_hack) +{ + return 0; +} + +#undef BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT +#undef BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl_fwd.hpp new file mode 100644 index 00000000000..6590ef05fdd --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl_fwd.hpp @@ -0,0 +1,128 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_IMPL_FWD_HPP +#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_IMPL_FWD_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +template< + typename KeyFromValue,typename Compare, + typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy +> +class ordered_index; + +template< + typename KeyFromValue1,typename Compare1, + typename SuperMeta1,typename TagList1,typename Category1, + typename AugmentPolicy1, + typename KeyFromValue2,typename Compare2, + typename SuperMeta2,typename TagList2,typename Category2, + typename AugmentPolicy2 +> +bool operator==( + const ordered_index< + KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, + const ordered_index< + KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); + +template< + typename KeyFromValue1,typename Compare1, + typename SuperMeta1,typename TagList1,typename Category1, + typename AugmentPolicy1, + typename KeyFromValue2,typename Compare2, + typename SuperMeta2,typename TagList2,typename Category2, + typename AugmentPolicy2 +> +bool operator<( + const ordered_index< + KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, + const ordered_index< + KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); + +template< + typename KeyFromValue1,typename Compare1, + typename SuperMeta1,typename TagList1,typename Category1, + typename AugmentPolicy1, + typename KeyFromValue2,typename Compare2, + typename SuperMeta2,typename TagList2,typename Category2, + typename AugmentPolicy2 +> +bool operator!=( + const ordered_index< + KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, + const ordered_index< + KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); + +template< + typename KeyFromValue1,typename Compare1, + typename SuperMeta1,typename TagList1,typename Category1, + typename AugmentPolicy1, + typename KeyFromValue2,typename Compare2, + typename SuperMeta2,typename TagList2,typename Category2, + typename AugmentPolicy2 +> +bool operator>( + const ordered_index< + KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, + const ordered_index< + KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); + +template< + typename KeyFromValue1,typename Compare1, + typename SuperMeta1,typename TagList1,typename Category1, + typename AugmentPolicy1, + typename KeyFromValue2,typename Compare2, + typename SuperMeta2,typename TagList2,typename Category2, + typename AugmentPolicy2 +> +bool operator>=( + const ordered_index< + KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, + const ordered_index< + KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); + +template< + typename KeyFromValue1,typename Compare1, + typename SuperMeta1,typename TagList1,typename Category1, + typename AugmentPolicy1, + typename KeyFromValue2,typename Compare2, + typename SuperMeta2,typename TagList2,typename Category2, + typename AugmentPolicy2 +> +bool operator<=( + const ordered_index< + KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, + const ordered_index< + KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); + +template< + typename KeyFromValue,typename Compare, + typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy +> +void swap( + ordered_index< + KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x, + ordered_index< + KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& y); + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_node.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_node.hpp new file mode 100644 index 00000000000..e7af0377fb9 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_node.hpp @@ -0,0 +1,658 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + * + * The internal implementation of red-black trees is based on that of SGI STL + * stl_tree.h file: + * + * Copyright (c) 1996,1997 + * Silicon Graphics Computer Systems, Inc. + * + * Permission to use, copy, modify, distribute and sell this software + * and its documentation for any purpose is hereby granted without fee, + * provided that the above copyright notice appear in all copies and + * that both that copyright notice and this permission notice appear + * in supporting documentation. Silicon Graphics makes no + * representations about the suitability of this software for any + * purpose. It is provided "as is" without express or implied warranty. + * + * + * Copyright (c) 1994 + * Hewlett-Packard Company + * + * Permission to use, copy, modify, distribute and sell this software + * and its documentation for any purpose is hereby granted without fee, + * provided that the above copyright notice appear in all copies and + * that both that copyright notice and this permission notice appear + * in supporting documentation. Hewlett-Packard Company makes no + * representations about the suitability of this software for any + * purpose. It is provided "as is" without express or implied warranty. + * + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_NODE_HPP +#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_NODE_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include + +#if !defined(BOOST_MULTI_INDEX_DISABLE_COMPRESSED_ORDERED_INDEX_NODES) +#include +#include +#include +#include +#include +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* definition of red-black nodes for ordered_index */ + +enum ordered_index_color{red=false,black=true}; +enum ordered_index_side{to_left=false,to_right=true}; + +template +struct ordered_index_node_impl; /* fwd decl. */ + +template +struct ordered_index_node_std_base +{ + typedef typename + boost::detail::allocator::rebind_to< + Allocator, + ordered_index_node_impl + >::type::pointer pointer; + typedef typename + boost::detail::allocator::rebind_to< + Allocator, + ordered_index_node_impl + >::type::const_pointer const_pointer; + typedef ordered_index_color& color_ref; + typedef pointer& parent_ref; + + ordered_index_color& color(){return color_;} + ordered_index_color color()const{return color_;} + pointer& parent(){return parent_;} + pointer parent()const{return parent_;} + pointer& left(){return left_;} + pointer left()const{return left_;} + pointer& right(){return right_;} + pointer right()const{return right_;} + +private: + ordered_index_color color_; + pointer parent_; + pointer left_; + pointer right_; +}; + +#if !defined(BOOST_MULTI_INDEX_DISABLE_COMPRESSED_ORDERED_INDEX_NODES) +/* If ordered_index_node_impl has even alignment, we can use the least + * significant bit of one of the ordered_index_node_impl pointers to + * store color information. This typically reduces the size of + * ordered_index_node_impl by 25%. + */ + +#if defined(BOOST_MSVC) +/* This code casts pointers to an integer type that has been computed + * to be large enough to hold the pointer, however the metaprogramming + * logic is not always spotted by the VC++ code analyser that issues a + * long list of warnings. + */ + +#pragma warning(push) +#pragma warning(disable:4312 4311) +#endif + +template +struct ordered_index_node_compressed_base +{ + typedef ordered_index_node_impl< + AugmentPolicy,Allocator>* pointer; + typedef const ordered_index_node_impl< + AugmentPolicy,Allocator>* const_pointer; + + struct color_ref + { + color_ref(uintptr_type* r_):r(r_){} + + operator ordered_index_color()const + { + return ordered_index_color(*r&uintptr_type(1)); + } + + color_ref& operator=(ordered_index_color c) + { + *r&=~uintptr_type(1); + *r|=uintptr_type(c); + return *this; + } + + color_ref& operator=(const color_ref& x) + { + return operator=(x.operator ordered_index_color()); + } + + private: + uintptr_type* r; + }; + + struct parent_ref + { + parent_ref(uintptr_type* r_):r(r_){} + + operator pointer()const + { + return (pointer)(void*)(*r&~uintptr_type(1)); + } + + parent_ref& operator=(pointer p) + { + *r=((uintptr_type)(void*)p)|(*r&uintptr_type(1)); + return *this; + } + + parent_ref& operator=(const parent_ref& x) + { + return operator=(x.operator pointer()); + } + + pointer operator->()const + { + return operator pointer(); + } + + private: + uintptr_type* r; + }; + + color_ref color(){return color_ref(&parentcolor_);} + ordered_index_color color()const + { + return ordered_index_color(parentcolor_&uintptr_type(1)); + } + + parent_ref parent(){return parent_ref(&parentcolor_);} + pointer parent()const + { + return (pointer)(void*)(parentcolor_&~uintptr_type(1)); + } + + pointer& left(){return left_;} + pointer left()const{return left_;} + pointer& right(){return right_;} + pointer right()const{return right_;} + +private: + uintptr_type parentcolor_; + pointer left_; + pointer right_; +}; +#if defined(BOOST_MSVC) +#pragma warning(pop) +#endif +#endif + +template +struct ordered_index_node_impl_base: + +#if !defined(BOOST_MULTI_INDEX_DISABLE_COMPRESSED_ORDERED_INDEX_NODES) + AugmentPolicy::template augmented_node< + typename mpl::if_c< + !(has_uintptr_type::value)|| + (alignment_of< + ordered_index_node_compressed_base + >::value%2)|| + !(is_same< + typename boost::detail::allocator::rebind_to< + Allocator, + ordered_index_node_impl + >::type::pointer, + ordered_index_node_impl*>::value), + ordered_index_node_std_base, + ordered_index_node_compressed_base + >::type + >::type +#else + AugmentPolicy::template augmented_node< + ordered_index_node_std_base + >::type +#endif + +{}; + +template +struct ordered_index_node_impl: + ordered_index_node_impl_base +{ +private: + typedef ordered_index_node_impl_base super; + +public: + typedef typename super::color_ref color_ref; + typedef typename super::parent_ref parent_ref; + typedef typename super::pointer pointer; + typedef typename super::const_pointer const_pointer; + + /* interoperability with bidir_node_iterator */ + + static void increment(pointer& x) + { + if(x->right()!=pointer(0)){ + x=x->right(); + while(x->left()!=pointer(0))x=x->left(); + } + else{ + pointer y=x->parent(); + while(x==y->right()){ + x=y; + y=y->parent(); + } + if(x->right()!=y)x=y; + } + } + + static void decrement(pointer& x) + { + if(x->color()==red&&x->parent()->parent()==x){ + x=x->right(); + } + else if(x->left()!=pointer(0)){ + pointer y=x->left(); + while(y->right()!=pointer(0))y=y->right(); + x=y; + }else{ + pointer y=x->parent(); + while(x==y->left()){ + x=y; + y=y->parent(); + } + x=y; + } + } + + /* algorithmic stuff */ + + static void rotate_left(pointer x,parent_ref root) + { + pointer y=x->right(); + x->right()=y->left(); + if(y->left()!=pointer(0))y->left()->parent()=x; + y->parent()=x->parent(); + + if(x==root) root=y; + else if(x==x->parent()->left())x->parent()->left()=y; + else x->parent()->right()=y; + y->left()=x; + x->parent()=y; + AugmentPolicy::rotate_left(x,y); + } + + static pointer minimum(pointer x) + { + while(x->left()!=pointer(0))x=x->left(); + return x; + } + + static pointer maximum(pointer x) + { + while(x->right()!=pointer(0))x=x->right(); + return x; + } + + static void rotate_right(pointer x,parent_ref root) + { + pointer y=x->left(); + x->left()=y->right(); + if(y->right()!=pointer(0))y->right()->parent()=x; + y->parent()=x->parent(); + + if(x==root) root=y; + else if(x==x->parent()->right())x->parent()->right()=y; + else x->parent()->left()=y; + y->right()=x; + x->parent()=y; + AugmentPolicy::rotate_right(x,y); + } + + static void rebalance(pointer x,parent_ref root) + { + x->color()=red; + while(x!=root&&x->parent()->color()==red){ + if(x->parent()==x->parent()->parent()->left()){ + pointer y=x->parent()->parent()->right(); + if(y!=pointer(0)&&y->color()==red){ + x->parent()->color()=black; + y->color()=black; + x->parent()->parent()->color()=red; + x=x->parent()->parent(); + } + else{ + if(x==x->parent()->right()){ + x=x->parent(); + rotate_left(x,root); + } + x->parent()->color()=black; + x->parent()->parent()->color()=red; + rotate_right(x->parent()->parent(),root); + } + } + else{ + pointer y=x->parent()->parent()->left(); + if(y!=pointer(0)&&y->color()==red){ + x->parent()->color()=black; + y->color()=black; + x->parent()->parent()->color()=red; + x=x->parent()->parent(); + } + else{ + if(x==x->parent()->left()){ + x=x->parent(); + rotate_right(x,root); + } + x->parent()->color()=black; + x->parent()->parent()->color()=red; + rotate_left(x->parent()->parent(),root); + } + } + } + root->color()=black; + } + + static void link( + pointer x,ordered_index_side side,pointer position,pointer header) + { + if(side==to_left){ + position->left()=x; /* also makes leftmost=x when parent==header */ + if(position==header){ + header->parent()=x; + header->right()=x; + } + else if(position==header->left()){ + header->left()=x; /* maintain leftmost pointing to min node */ + } + } + else{ + position->right()=x; + if(position==header->right()){ + header->right()=x; /* maintain rightmost pointing to max node */ + } + } + x->parent()=position; + x->left()=pointer(0); + x->right()=pointer(0); + AugmentPolicy::add(x,pointer(header->parent())); + ordered_index_node_impl::rebalance(x,header->parent()); + } + + static pointer rebalance_for_erase( + pointer z,parent_ref root,pointer& leftmost,pointer& rightmost) + { + pointer y=z; + pointer x=pointer(0); + pointer x_parent=pointer(0); + if(y->left()==pointer(0)){ /* z has at most one non-null child. y==z. */ + x=y->right(); /* x might be null */ + } + else{ + if(y->right()==pointer(0)){ /* z has exactly one non-null child. y==z. */ + x=y->left(); /* x is not null */ + } + else{ /* z has two non-null children. Set y to */ + y=y->right(); /* z's successor. x might be null. */ + while(y->left()!=pointer(0))y=y->left(); + x=y->right(); + } + } + AugmentPolicy::remove(y,pointer(root)); + if(y!=z){ + AugmentPolicy::copy(z,y); + z->left()->parent()=y; /* relink y in place of z. y is z's successor */ + y->left()=z->left(); + if(y!=z->right()){ + x_parent=y->parent(); + if(x!=pointer(0))x->parent()=y->parent(); + y->parent()->left()=x; /* y must be a child of left */ + y->right()=z->right(); + z->right()->parent()=y; + } + else{ + x_parent=y; + } + + if(root==z) root=y; + else if(z->parent()->left()==z)z->parent()->left()=y; + else z->parent()->right()=y; + y->parent()=z->parent(); + ordered_index_color c=y->color(); + y->color()=z->color(); + z->color()=c; + y=z; /* y now points to node to be actually deleted */ + } + else{ /* y==z */ + x_parent=y->parent(); + if(x!=pointer(0))x->parent()=y->parent(); + if(root==z){ + root=x; + } + else{ + if(z->parent()->left()==z)z->parent()->left()=x; + else z->parent()->right()=x; + } + if(leftmost==z){ + if(z->right()==pointer(0)){ /* z->left() must be null also */ + leftmost=z->parent(); + } + else{ + leftmost=minimum(x); /* makes leftmost==header if z==root */ + } + } + if(rightmost==z){ + if(z->left()==pointer(0)){ /* z->right() must be null also */ + rightmost=z->parent(); + } + else{ /* x==z->left() */ + rightmost=maximum(x); /* makes rightmost==header if z==root */ + } + } + } + if(y->color()!=red){ + while(x!=root&&(x==pointer(0)|| x->color()==black)){ + if(x==x_parent->left()){ + pointer w=x_parent->right(); + if(w->color()==red){ + w->color()=black; + x_parent->color()=red; + rotate_left(x_parent,root); + w=x_parent->right(); + } + if((w->left()==pointer(0)||w->left()->color()==black) && + (w->right()==pointer(0)||w->right()->color()==black)){ + w->color()=red; + x=x_parent; + x_parent=x_parent->parent(); + } + else{ + if(w->right()==pointer(0 ) + || w->right()->color()==black){ + if(w->left()!=pointer(0)) w->left()->color()=black; + w->color()=red; + rotate_right(w,root); + w=x_parent->right(); + } + w->color()=x_parent->color(); + x_parent->color()=black; + if(w->right()!=pointer(0))w->right()->color()=black; + rotate_left(x_parent,root); + break; + } + } + else{ /* same as above,with right <-> left */ + pointer w=x_parent->left(); + if(w->color()==red){ + w->color()=black; + x_parent->color()=red; + rotate_right(x_parent,root); + w=x_parent->left(); + } + if((w->right()==pointer(0)||w->right()->color()==black) && + (w->left()==pointer(0)||w->left()->color()==black)){ + w->color()=red; + x=x_parent; + x_parent=x_parent->parent(); + } + else{ + if(w->left()==pointer(0)||w->left()->color()==black){ + if(w->right()!=pointer(0))w->right()->color()=black; + w->color()=red; + rotate_left(w,root); + w=x_parent->left(); + } + w->color()=x_parent->color(); + x_parent->color()=black; + if(w->left()!=pointer(0))w->left()->color()=black; + rotate_right(x_parent,root); + break; + } + } + } + if(x!=pointer(0))x->color()=black; + } + return y; + } + + static void restore(pointer x,pointer position,pointer header) + { + if(position->left()==pointer(0)||position->left()==header){ + link(x,to_left,position,header); + } + else{ + decrement(position); + link(x,to_right,position,header); + } + } + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) + /* invariant stuff */ + + static std::size_t black_count(pointer node,pointer root) + { + if(node==pointer(0))return 0; + std::size_t sum=0; + for(;;){ + if(node->color()==black)++sum; + if(node==root)break; + node=node->parent(); + } + return sum; + } +#endif +}; + +template +struct ordered_index_node_trampoline: + ordered_index_node_impl< + AugmentPolicy, + typename boost::detail::allocator::rebind_to< + typename Super::allocator_type, + char + >::type + > +{ + typedef ordered_index_node_impl< + AugmentPolicy, + typename boost::detail::allocator::rebind_to< + typename Super::allocator_type, + char + >::type + > impl_type; +}; + +template +struct ordered_index_node: + Super,ordered_index_node_trampoline +{ +private: + typedef ordered_index_node_trampoline trampoline; + +public: + typedef typename trampoline::impl_type impl_type; + typedef typename trampoline::color_ref impl_color_ref; + typedef typename trampoline::parent_ref impl_parent_ref; + typedef typename trampoline::pointer impl_pointer; + typedef typename trampoline::const_pointer const_impl_pointer; + + impl_color_ref color(){return trampoline::color();} + ordered_index_color color()const{return trampoline::color();} + impl_parent_ref parent(){return trampoline::parent();} + impl_pointer parent()const{return trampoline::parent();} + impl_pointer& left(){return trampoline::left();} + impl_pointer left()const{return trampoline::left();} + impl_pointer& right(){return trampoline::right();} + impl_pointer right()const{return trampoline::right();} + + impl_pointer impl() + { + return static_cast( + static_cast(static_cast(this))); + } + + const_impl_pointer impl()const + { + return static_cast( + static_cast(static_cast(this))); + } + + static ordered_index_node* from_impl(impl_pointer x) + { + return + static_cast( + static_cast( + raw_ptr(x))); + } + + static const ordered_index_node* from_impl(const_impl_pointer x) + { + return + static_cast( + static_cast( + raw_ptr(x))); + } + + /* interoperability with bidir_node_iterator */ + + static void increment(ordered_index_node*& x) + { + impl_pointer xi=x->impl(); + trampoline::increment(xi); + x=from_impl(xi); + } + + static void decrement(ordered_index_node*& x) + { + impl_pointer xi=x->impl(); + trampoline::decrement(xi); + x=from_impl(xi); + } +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_ops.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_ops.hpp new file mode 100644 index 00000000000..84d5cacae19 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_ops.hpp @@ -0,0 +1,266 @@ +/* Copyright 2003-2014 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + * + * The internal implementation of red-black trees is based on that of SGI STL + * stl_tree.h file: + * + * Copyright (c) 1996,1997 + * Silicon Graphics Computer Systems, Inc. + * + * Permission to use, copy, modify, distribute and sell this software + * and its documentation for any purpose is hereby granted without fee, + * provided that the above copyright notice appear in all copies and + * that both that copyright notice and this permission notice appear + * in supporting documentation. Silicon Graphics makes no + * representations about the suitability of this software for any + * purpose. It is provided "as is" without express or implied warranty. + * + * + * Copyright (c) 1994 + * Hewlett-Packard Company + * + * Permission to use, copy, modify, distribute and sell this software + * and its documentation for any purpose is hereby granted without fee, + * provided that the above copyright notice appear in all copies and + * that both that copyright notice and this permission notice appear + * in supporting documentation. Hewlett-Packard Company makes no + * representations about the suitability of this software for any + * purpose. It is provided "as is" without express or implied warranty. + * + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_OPS_HPP +#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_OPS_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Common code for index memfuns having templatized and + * non-templatized versions. + * Implementation note: When CompatibleKey is consistently promoted to + * KeyFromValue::result_type for comparison, the promotion is made once in + * advance to increase efficiency. + */ + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline Node* ordered_index_find( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp) +{ + typedef typename KeyFromValue::result_type key_type; + + return ordered_index_find( + top,y,key,x,comp, + mpl::and_< + promotes_1st_arg, + promotes_2nd_arg >()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleCompare +> +inline Node* ordered_index_find( + Node* top,Node* y,const KeyFromValue& key, + const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, + const CompatibleCompare& comp,mpl::true_) +{ + return ordered_index_find(top,y,key,x,comp,mpl::false_()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline Node* ordered_index_find( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp,mpl::false_) +{ + Node* y0=y; + + while (top){ + if(!comp(key(top->value()),x)){ + y=top; + top=Node::from_impl(top->left()); + } + else top=Node::from_impl(top->right()); + } + + return (y==y0||comp(x,key(y->value())))?y0:y; +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline Node* ordered_index_lower_bound( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp) +{ + typedef typename KeyFromValue::result_type key_type; + + return ordered_index_lower_bound( + top,y,key,x,comp, + promotes_2nd_arg()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleCompare +> +inline Node* ordered_index_lower_bound( + Node* top,Node* y,const KeyFromValue& key, + const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, + const CompatibleCompare& comp,mpl::true_) +{ + return ordered_index_lower_bound(top,y,key,x,comp,mpl::false_()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline Node* ordered_index_lower_bound( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp,mpl::false_) +{ + while(top){ + if(!comp(key(top->value()),x)){ + y=top; + top=Node::from_impl(top->left()); + } + else top=Node::from_impl(top->right()); + } + + return y; +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline Node* ordered_index_upper_bound( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp) +{ + typedef typename KeyFromValue::result_type key_type; + + return ordered_index_upper_bound( + top,y,key,x,comp, + promotes_1st_arg()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleCompare +> +inline Node* ordered_index_upper_bound( + Node* top,Node* y,const KeyFromValue& key, + const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, + const CompatibleCompare& comp,mpl::true_) +{ + return ordered_index_upper_bound(top,y,key,x,comp,mpl::false_()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline Node* ordered_index_upper_bound( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp,mpl::false_) +{ + while(top){ + if(comp(x,key(top->value()))){ + y=top; + top=Node::from_impl(top->left()); + } + else top=Node::from_impl(top->right()); + } + + return y; +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline std::pair ordered_index_equal_range( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp) +{ + typedef typename KeyFromValue::result_type key_type; + + return ordered_index_equal_range( + top,y,key,x,comp, + mpl::and_< + promotes_1st_arg, + promotes_2nd_arg >()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleCompare +> +inline std::pair ordered_index_equal_range( + Node* top,Node* y,const KeyFromValue& key, + const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, + const CompatibleCompare& comp,mpl::true_) +{ + return ordered_index_equal_range(top,y,key,x,comp,mpl::false_()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline std::pair ordered_index_equal_range( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp,mpl::false_) +{ + while(top){ + if(comp(key(top->value()),x)){ + top=Node::from_impl(top->right()); + } + else if(comp(x,key(top->value()))){ + y=top; + top=Node::from_impl(top->left()); + } + else{ + return std::pair( + ordered_index_lower_bound( + Node::from_impl(top->left()),top,key,x,comp,mpl::false_()), + ordered_index_upper_bound( + Node::from_impl(top->right()),y,key,x,comp,mpl::false_())); + } + } + + return std::pair(y,y); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/promotes_arg.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/promotes_arg.hpp new file mode 100644 index 00000000000..7a11b6e9fbe --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/promotes_arg.hpp @@ -0,0 +1,83 @@ +/* Copyright 2003-2017 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_PROMOTES_ARG_HPP +#define BOOST_MULTI_INDEX_DETAIL_PROMOTES_ARG_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include + +/* Metafunctions to check if f(arg1,arg2) promotes either arg1 to the type of + * arg2 or viceversa. By default, (i.e. if it cannot be determined), no + * promotion is assumed. + */ + +#if BOOST_WORKAROUND(BOOST_MSVC,<1400) + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +template +struct promotes_1st_arg:mpl::false_{}; + +template +struct promotes_2nd_arg:mpl::false_{}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#else + +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +template +struct promotes_1st_arg: + mpl::and_< + mpl::not_ >, + is_convertible, + is_transparent + > +{}; + +template +struct promotes_2nd_arg: + mpl::and_< + mpl::not_ >, + is_convertible, + is_transparent + > +{}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/raw_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/raw_ptr.hpp new file mode 100644 index 00000000000..c32007435c0 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/raw_ptr.hpp @@ -0,0 +1,52 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_RAW_PTR_HPP +#define BOOST_MULTI_INDEX_DETAIL_RAW_PTR_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* gets the underlying pointer of a pointer-like value */ + +template +inline RawPointer raw_ptr(RawPointer const& p,mpl::true_) +{ + return p; +} + +template +inline RawPointer raw_ptr(Pointer const& p,mpl::false_) +{ + return p==Pointer(0)?0:&*p; +} + +template +inline RawPointer raw_ptr(Pointer const& p) +{ + return raw_ptr(p,is_same()); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/restore_wstrict_aliasing.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/restore_wstrict_aliasing.hpp new file mode 100644 index 00000000000..ee2c799d5a8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/restore_wstrict_aliasing.hpp @@ -0,0 +1,11 @@ +/* Copyright 2003-2016 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#define BOOST_MULTI_INDEX_DETAIL_RESTORE_WSTRICT_ALIASING +#include +#undef BOOST_MULTI_INDEX_DETAIL_RESTORE_WSTRICT_ALIASING diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_loader.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_loader.hpp new file mode 100644 index 00000000000..4b00345a6d9 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_loader.hpp @@ -0,0 +1,173 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_RND_INDEX_LOADER_HPP +#define BOOST_MULTI_INDEX_DETAIL_RND_INDEX_LOADER_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* This class implements a serialization rearranger for random access + * indices. In order to achieve O(n) performance, the following strategy + * is followed: the nodes of the index are handled as if in a bidirectional + * list, where the next pointers are stored in the original + * random_access_index_ptr_array and the prev pointers are stored in + * an auxiliary array. Rearranging of nodes in such a bidirectional list + * is constant time. Once all the arrangements are performed (on destruction + * time) the list is traversed in reverse order and + * pointers are swapped and set accordingly so that they recover its + * original semantics ( *(node->up())==node ) while retaining the + * new order. + */ + +template +class random_access_index_loader_base:private noncopyable +{ +protected: + typedef random_access_index_node_impl< + typename boost::detail::allocator::rebind_to< + Allocator, + char + >::type + > node_impl_type; + typedef typename node_impl_type::pointer node_impl_pointer; + typedef random_access_index_ptr_array ptr_array; + + random_access_index_loader_base(const Allocator& al_,ptr_array& ptrs_): + al(al_), + ptrs(ptrs_), + header(*ptrs.end()), + prev_spc(al,0), + preprocessed(false) + {} + + ~random_access_index_loader_base() + { + if(preprocessed) + { + node_impl_pointer n=header; + next(n)=n; + + for(std::size_t i=ptrs.size();i--;){ + n=prev(n); + std::size_t d=position(n); + if(d!=i){ + node_impl_pointer m=prev(next_at(i)); + std::swap(m->up(),n->up()); + next_at(d)=next_at(i); + std::swap(prev_at(d),prev_at(i)); + } + next(n)=n; + } + } + } + + void rearrange(node_impl_pointer position_,node_impl_pointer x) + { + preprocess(); /* only incur this penalty if rearrange() is ever called */ + if(position_==node_impl_pointer(0))position_=header; + next(prev(x))=next(x); + prev(next(x))=prev(x); + prev(x)=position_; + next(x)=next(position_); + next(prev(x))=prev(next(x))=x; + } + +private: + void preprocess() + { + if(!preprocessed){ + /* get space for the auxiliary prev array */ + auto_space tmp(al,ptrs.size()+1); + prev_spc.swap(tmp); + + /* prev_spc elements point to the prev nodes */ + std::rotate_copy( + &*ptrs.begin(),&*ptrs.end(),&*ptrs.end()+1,&*prev_spc.data()); + + /* ptrs elements point to the next nodes */ + std::rotate(&*ptrs.begin(),&*ptrs.begin()+1,&*ptrs.end()+1); + + preprocessed=true; + } + } + + std::size_t position(node_impl_pointer x)const + { + return (std::size_t)(x->up()-ptrs.begin()); + } + + node_impl_pointer& next_at(std::size_t n)const + { + return *ptrs.at(n); + } + + node_impl_pointer& prev_at(std::size_t n)const + { + return *(prev_spc.data()+n); + } + + node_impl_pointer& next(node_impl_pointer x)const + { + return *(x->up()); + } + + node_impl_pointer& prev(node_impl_pointer x)const + { + return prev_at(position(x)); + } + + Allocator al; + ptr_array& ptrs; + node_impl_pointer header; + auto_space prev_spc; + bool preprocessed; +}; + +template +class random_access_index_loader: + private random_access_index_loader_base +{ + typedef random_access_index_loader_base super; + typedef typename super::node_impl_pointer node_impl_pointer; + typedef typename super::ptr_array ptr_array; + +public: + random_access_index_loader(const Allocator& al_,ptr_array& ptrs_): + super(al_,ptrs_) + {} + + void rearrange(Node* position_,Node *x) + { + super::rearrange( + position_?position_->impl():node_impl_pointer(0),x->impl()); + } +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_node.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_node.hpp new file mode 100644 index 00000000000..ad61ea25dda --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_node.hpp @@ -0,0 +1,273 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_RND_INDEX_NODE_HPP +#define BOOST_MULTI_INDEX_DETAIL_RND_INDEX_NODE_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +template +struct random_access_index_node_impl +{ + typedef typename + boost::detail::allocator::rebind_to< + Allocator,random_access_index_node_impl + >::type::pointer pointer; + typedef typename + boost::detail::allocator::rebind_to< + Allocator,random_access_index_node_impl + >::type::const_pointer const_pointer; + typedef typename + boost::detail::allocator::rebind_to< + Allocator,pointer + >::type::pointer ptr_pointer; + + ptr_pointer& up(){return up_;} + ptr_pointer up()const{return up_;} + + /* interoperability with rnd_node_iterator */ + + static void increment(pointer& x) + { + x=*(x->up()+1); + } + + static void decrement(pointer& x) + { + x=*(x->up()-1); + } + + static void advance(pointer& x,std::ptrdiff_t n) + { + x=*(x->up()+n); + } + + static std::ptrdiff_t distance(pointer x,pointer y) + { + return y->up()-x->up(); + } + + /* algorithmic stuff */ + + static void relocate(ptr_pointer pos,ptr_pointer x) + { + pointer n=*x; + if(xup()=pos-1; + } + else{ + while(x!=pos){ + *x=*(x-1); + (*x)->up()=x; + --x; + } + *pos=n; + n->up()=pos; + } + }; + + static void relocate(ptr_pointer pos,ptr_pointer first,ptr_pointer last) + { + ptr_pointer begin,middle,end; + if(posup()=begin+j; + break; + } + else{ + *(begin+j)=*(begin+k); + (*(begin+j))->up()=begin+j; + } + + if(kup()=begin+k; + break; + } + else{ + *(begin+k)=*(begin+j); + (*(begin+k))->up()=begin+k; + } + } + } + }; + + static void extract(ptr_pointer x,ptr_pointer pend) + { + --pend; + while(x!=pend){ + *x=*(x+1); + (*x)->up()=x; + ++x; + } + } + + static void transfer( + ptr_pointer pbegin0,ptr_pointer pend0,ptr_pointer pbegin1) + { + while(pbegin0!=pend0){ + *pbegin1=*pbegin0++; + (*pbegin1)->up()=pbegin1; + ++pbegin1; + } + } + + static void reverse(ptr_pointer pbegin,ptr_pointer pend) + { + std::ptrdiff_t d=(pend-pbegin)/2; + for(std::ptrdiff_t i=0;iup()=pbegin; + (*pend)->up()=pend; + ++pbegin; + } + } + +private: + ptr_pointer up_; +}; + +template +struct random_access_index_node_trampoline: + random_access_index_node_impl< + typename boost::detail::allocator::rebind_to< + typename Super::allocator_type, + char + >::type + > +{ + typedef random_access_index_node_impl< + typename boost::detail::allocator::rebind_to< + typename Super::allocator_type, + char + >::type + > impl_type; +}; + +template +struct random_access_index_node: + Super,random_access_index_node_trampoline +{ +private: + typedef random_access_index_node_trampoline trampoline; + +public: + typedef typename trampoline::impl_type impl_type; + typedef typename trampoline::pointer impl_pointer; + typedef typename trampoline::const_pointer const_impl_pointer; + typedef typename trampoline::ptr_pointer impl_ptr_pointer; + + impl_ptr_pointer& up(){return trampoline::up();} + impl_ptr_pointer up()const{return trampoline::up();} + + impl_pointer impl() + { + return static_cast( + static_cast(static_cast(this))); + } + + const_impl_pointer impl()const + { + return static_cast( + static_cast(static_cast(this))); + } + + static random_access_index_node* from_impl(impl_pointer x) + { + return + static_cast( + static_cast( + raw_ptr(x))); + } + + static const random_access_index_node* from_impl(const_impl_pointer x) + { + return + static_cast( + static_cast( + raw_ptr(x))); + } + + /* interoperability with rnd_node_iterator */ + + static void increment(random_access_index_node*& x) + { + impl_pointer xi=x->impl(); + trampoline::increment(xi); + x=from_impl(xi); + } + + static void decrement(random_access_index_node*& x) + { + impl_pointer xi=x->impl(); + trampoline::decrement(xi); + x=from_impl(xi); + } + + static void advance(random_access_index_node*& x,std::ptrdiff_t n) + { + impl_pointer xi=x->impl(); + trampoline::advance(xi,n); + x=from_impl(xi); + } + + static std::ptrdiff_t distance( + random_access_index_node* x,random_access_index_node* y) + { + return trampoline::distance(x->impl(),y->impl()); + } +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ops.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ops.hpp new file mode 100644 index 00000000000..f5e76e4441f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ops.hpp @@ -0,0 +1,203 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_RND_INDEX_OPS_HPP +#define BOOST_MULTI_INDEX_DETAIL_RND_INDEX_OPS_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Common code for random_access_index memfuns having templatized and + * non-templatized versions. + */ + +template +Node* random_access_index_remove( + random_access_index_ptr_array& ptrs,Predicate pred) +{ + typedef typename Node::value_type value_type; + typedef typename Node::impl_ptr_pointer impl_ptr_pointer; + + impl_ptr_pointer first=ptrs.begin(), + res=first, + last=ptrs.end(); + for(;first!=last;++first){ + if(!pred( + const_cast(Node::from_impl(*first)->value()))){ + if(first!=res){ + std::swap(*first,*res); + (*first)->up()=first; + (*res)->up()=res; + } + ++res; + } + } + return Node::from_impl(*res); +} + +template +Node* random_access_index_unique( + random_access_index_ptr_array& ptrs,BinaryPredicate binary_pred) +{ + typedef typename Node::value_type value_type; + typedef typename Node::impl_ptr_pointer impl_ptr_pointer; + + impl_ptr_pointer first=ptrs.begin(), + res=first, + last=ptrs.end(); + if(first!=last){ + for(;++first!=last;){ + if(!binary_pred( + const_cast(Node::from_impl(*res)->value()), + const_cast(Node::from_impl(*first)->value()))){ + ++res; + if(first!=res){ + std::swap(*first,*res); + (*first)->up()=first; + (*res)->up()=res; + } + } + } + ++res; + } + return Node::from_impl(*res); +} + +template +void random_access_index_inplace_merge( + const Allocator& al, + random_access_index_ptr_array& ptrs, + BOOST_DEDUCED_TYPENAME Node::impl_ptr_pointer first1,Compare comp) +{ + typedef typename Node::value_type value_type; + typedef typename Node::impl_pointer impl_pointer; + typedef typename Node::impl_ptr_pointer impl_ptr_pointer; + + auto_space spc(al,ptrs.size()); + + impl_ptr_pointer first0=ptrs.begin(), + last0=first1, + last1=ptrs.end(), + out=spc.data(); + while(first0!=last0&&first1!=last1){ + if(comp( + const_cast(Node::from_impl(*first1)->value()), + const_cast(Node::from_impl(*first0)->value()))){ + *out++=*first1++; + } + else{ + *out++=*first0++; + } + } + std::copy(&*first0,&*last0,&*out); + std::copy(&*first1,&*last1,&*out); + + first1=ptrs.begin(); + out=spc.data(); + while(first1!=last1){ + *first1=*out++; + (*first1)->up()=first1; + ++first1; + } +} + +/* sorting */ + +/* auxiliary stuff */ + +template +struct random_access_index_sort_compare +{ + typedef typename Node::impl_pointer first_argument_type; + typedef typename Node::impl_pointer second_argument_type; + typedef bool result_type; + + random_access_index_sort_compare(Compare comp_=Compare()):comp(comp_){} + + bool operator()( + typename Node::impl_pointer x,typename Node::impl_pointer y)const + { + typedef typename Node::value_type value_type; + + return comp( + const_cast(Node::from_impl(x)->value()), + const_cast(Node::from_impl(y)->value())); + } + +private: + Compare comp; +}; + +template +void random_access_index_sort( + const Allocator& al, + random_access_index_ptr_array& ptrs, + Compare comp) +{ + /* The implementation is extremely simple: an auxiliary + * array of pointers is sorted using stdlib facilities and + * then used to rearrange the index. This is suboptimal + * in space and time, but has some advantages over other + * possible approaches: + * - Use std::stable_sort() directly on ptrs using some + * special iterator in charge of maintaining pointers + * and up() pointers in sync: we cannot guarantee + * preservation of the container invariants in the face of + * exceptions, if, for instance, std::stable_sort throws + * when ptrs transitorily contains duplicate elements. + * - Rewrite the internal algorithms of std::stable_sort + * adapted for this case: besides being a fair amount of + * work, making a stable sort compatible with Boost.MultiIndex + * invariants (basically, no duplicates or missing elements + * even if an exception is thrown) is complicated, error-prone + * and possibly won't perform much better than the + * solution adopted. + */ + + if(ptrs.size()<=1)return; + + typedef typename Node::impl_pointer impl_pointer; + typedef typename Node::impl_ptr_pointer impl_ptr_pointer; + typedef random_access_index_sort_compare< + Node,Compare> ptr_compare; + + impl_ptr_pointer first=ptrs.begin(); + impl_ptr_pointer last=ptrs.end(); + auto_space< + impl_pointer, + Allocator> spc(al,ptrs.size()); + impl_ptr_pointer buf=spc.data(); + + std::copy(&*first,&*last,&*buf); + std::stable_sort(&*buf,&*buf+ptrs.size(),ptr_compare(comp)); + + while(first!=last){ + *first=*buf++; + (*first)->up()=first; + ++first; + } +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ptr_array.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ptr_array.hpp new file mode 100644 index 00000000000..bae1c851b8e --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ptr_array.hpp @@ -0,0 +1,144 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_RND_INDEX_PTR_ARRAY_HPP +#define BOOST_MULTI_INDEX_DETAIL_RND_INDEX_PTR_ARRAY_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* pointer structure for use by random access indices */ + +template +class random_access_index_ptr_array:private noncopyable +{ + typedef random_access_index_node_impl< + typename boost::detail::allocator::rebind_to< + Allocator, + char + >::type + > node_impl_type; + +public: + typedef typename node_impl_type::pointer value_type; + typedef typename boost::detail::allocator::rebind_to< + Allocator,value_type + >::type::pointer pointer; + + random_access_index_ptr_array( + const Allocator& al,value_type end_,std::size_t sz): + size_(sz), + capacity_(sz), + spc(al,capacity_+1) + { + *end()=end_; + end_->up()=end(); + } + + std::size_t size()const{return size_;} + std::size_t capacity()const{return capacity_;} + + void room_for_one() + { + if(size_==capacity_){ + reserve(capacity_<=10?15:capacity_+capacity_/2); + } + } + + void reserve(std::size_t c) + { + if(c>capacity_)set_capacity(c); + } + + void shrink_to_fit() + { + if(capacity_>size_)set_capacity(size_); + } + + pointer begin()const{return ptrs();} + pointer end()const{return ptrs()+size_;} + pointer at(std::size_t n)const{return ptrs()+n;} + + void push_back(value_type x) + { + *(end()+1)=*end(); + (*(end()+1))->up()=end()+1; + *end()=x; + (*end())->up()=end(); + ++size_; + } + + void erase(value_type x) + { + node_impl_type::extract(x->up(),end()+1); + --size_; + } + + void clear() + { + *begin()=*end(); + (*begin())->up()=begin(); + size_=0; + } + + void swap(random_access_index_ptr_array& x) + { + std::swap(size_,x.size_); + std::swap(capacity_,x.capacity_); + spc.swap(x.spc); + } + +private: + std::size_t size_; + std::size_t capacity_; + auto_space spc; + + pointer ptrs()const + { + return spc.data(); + } + + void set_capacity(std::size_t c) + { + auto_space spc1(spc.get_allocator(),c+1); + node_impl_type::transfer(begin(),end()+1,spc1.data()); + spc.swap(spc1); + capacity_=c; + } +}; + +template +void swap( + random_access_index_ptr_array& x, + random_access_index_ptr_array& y) +{ + x.swap(y); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_node_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_node_iterator.hpp new file mode 100644 index 00000000000..48026132fb7 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_node_iterator.hpp @@ -0,0 +1,140 @@ +/* Copyright 2003-2014 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_RND_NODE_ITERATOR_HPP +#define BOOST_MULTI_INDEX_DETAIL_RND_NODE_ITERATOR_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +#include +#include +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Iterator class for node-based indices with random access iterators. */ + +template +class rnd_node_iterator: + public random_access_iterator_helper< + rnd_node_iterator, + typename Node::value_type, + std::ptrdiff_t, + const typename Node::value_type*, + const typename Node::value_type&> +{ +public: + /* coverity[uninit_ctor]: suppress warning */ + rnd_node_iterator(){} + explicit rnd_node_iterator(Node* node_):node(node_){} + + const typename Node::value_type& operator*()const + { + return node->value(); + } + + rnd_node_iterator& operator++() + { + Node::increment(node); + return *this; + } + + rnd_node_iterator& operator--() + { + Node::decrement(node); + return *this; + } + + rnd_node_iterator& operator+=(std::ptrdiff_t n) + { + Node::advance(node,n); + return *this; + } + + rnd_node_iterator& operator-=(std::ptrdiff_t n) + { + Node::advance(node,-n); + return *this; + } + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + /* Serialization. As for why the following is public, + * see explanation in safe_mode_iterator notes in safe_mode.hpp. + */ + + BOOST_SERIALIZATION_SPLIT_MEMBER() + + typedef typename Node::base_type node_base_type; + + template + void save(Archive& ar,const unsigned int)const + { + node_base_type* bnode=node; + ar< + void load(Archive& ar,const unsigned int) + { + node_base_type* bnode; + ar>>serialization::make_nvp("pointer",bnode); + node=static_cast(bnode); + } +#endif + + /* get_node is not to be used by the user */ + + typedef Node node_type; + + Node* get_node()const{return node;} + +private: + Node* node; +}; + +template +bool operator==( + const rnd_node_iterator& x, + const rnd_node_iterator& y) +{ + return x.get_node()==y.get_node(); +} + +template +bool operator<( + const rnd_node_iterator& x, + const rnd_node_iterator& y) +{ + return Node::distance(x.get_node(),y.get_node())>0; +} + +template +std::ptrdiff_t operator-( + const rnd_node_iterator& x, + const rnd_node_iterator& y) +{ + return Node::distance(y.get_node(),x.get_node()); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnk_index_ops.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnk_index_ops.hpp new file mode 100644 index 00000000000..fb233cf4973 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnk_index_ops.hpp @@ -0,0 +1,300 @@ +/* Copyright 2003-2017 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_RNK_INDEX_OPS_HPP +#define BOOST_MULTI_INDEX_DETAIL_RNK_INDEX_OPS_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Common code for ranked_index memfuns having templatized and + * non-templatized versions. + */ + +template +inline std::size_t ranked_node_size(Pointer x) +{ + return x!=Pointer(0)?x->size:0; +} + +template +inline Pointer ranked_index_nth(std::size_t n,Pointer end_) +{ + Pointer top=end_->parent(); + if(top==Pointer(0)||n>=top->size)return end_; + + for(;;){ + std::size_t s=ranked_node_size(top->left()); + if(n==s)return top; + if(nleft(); + else{ + top=top->right(); + n-=s+1; + } + } +} + +template +inline std::size_t ranked_index_rank(Pointer x,Pointer end_) +{ + Pointer top=end_->parent(); + if(top==Pointer(0))return 0; + if(x==end_)return top->size; + + std::size_t s=ranked_node_size(x->left()); + while(x!=top){ + Pointer z=x->parent(); + if(x==z->right()){ + s+=ranked_node_size(z->left())+1; + } + x=z; + } + return s; +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline std::size_t ranked_index_find_rank( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp) +{ + typedef typename KeyFromValue::result_type key_type; + + return ranked_index_find_rank( + top,y,key,x,comp, + mpl::and_< + promotes_1st_arg, + promotes_2nd_arg >()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleCompare +> +inline std::size_t ranked_index_find_rank( + Node* top,Node* y,const KeyFromValue& key, + const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, + const CompatibleCompare& comp,mpl::true_) +{ + return ranked_index_find_rank(top,y,key,x,comp,mpl::false_()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline std::size_t ranked_index_find_rank( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp,mpl::false_) +{ + if(!top)return 0; + + std::size_t s=top->impl()->size, + s0=s; + Node* y0=y; + + do{ + if(!comp(key(top->value()),x)){ + y=top; + s-=ranked_node_size(y->right())+1; + top=Node::from_impl(top->left()); + } + else top=Node::from_impl(top->right()); + }while(top); + + return (y==y0||comp(x,key(y->value())))?s0:s; +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline std::size_t ranked_index_lower_bound_rank( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp) +{ + typedef typename KeyFromValue::result_type key_type; + + return ranked_index_lower_bound_rank( + top,y,key,x,comp, + promotes_2nd_arg()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleCompare +> +inline std::size_t ranked_index_lower_bound_rank( + Node* top,Node* y,const KeyFromValue& key, + const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, + const CompatibleCompare& comp,mpl::true_) +{ + return ranked_index_lower_bound_rank(top,y,key,x,comp,mpl::false_()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline std::size_t ranked_index_lower_bound_rank( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp,mpl::false_) +{ + if(!top)return 0; + + std::size_t s=top->impl()->size; + + do{ + if(!comp(key(top->value()),x)){ + y=top; + s-=ranked_node_size(y->right())+1; + top=Node::from_impl(top->left()); + } + else top=Node::from_impl(top->right()); + }while(top); + + return s; +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline std::size_t ranked_index_upper_bound_rank( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp) +{ + typedef typename KeyFromValue::result_type key_type; + + return ranked_index_upper_bound_rank( + top,y,key,x,comp, + promotes_1st_arg()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleCompare +> +inline std::size_t ranked_index_upper_bound_rank( + Node* top,Node* y,const KeyFromValue& key, + const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, + const CompatibleCompare& comp,mpl::true_) +{ + return ranked_index_upper_bound_rank(top,y,key,x,comp,mpl::false_()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline std::size_t ranked_index_upper_bound_rank( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp,mpl::false_) +{ + if(!top)return 0; + + std::size_t s=top->impl()->size; + + do{ + if(comp(x,key(top->value()))){ + y=top; + s-=ranked_node_size(y->right())+1; + top=Node::from_impl(top->left()); + } + else top=Node::from_impl(top->right()); + }while(top); + + return s; +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline std::pair ranked_index_equal_range_rank( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp) +{ + typedef typename KeyFromValue::result_type key_type; + + return ranked_index_equal_range_rank( + top,y,key,x,comp, + mpl::and_< + promotes_1st_arg, + promotes_2nd_arg >()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleCompare +> +inline std::pair ranked_index_equal_range_rank( + Node* top,Node* y,const KeyFromValue& key, + const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, + const CompatibleCompare& comp,mpl::true_) +{ + return ranked_index_equal_range_rank(top,y,key,x,comp,mpl::false_()); +} + +template< + typename Node,typename KeyFromValue, + typename CompatibleKey,typename CompatibleCompare +> +inline std::pair ranked_index_equal_range_rank( + Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, + const CompatibleCompare& comp,mpl::false_) +{ + if(!top)return std::pair(0,0); + + std::size_t s=top->impl()->size; + + do{ + if(comp(key(top->value()),x)){ + top=Node::from_impl(top->right()); + } + else if(comp(x,key(top->value()))){ + y=top; + s-=ranked_node_size(y->right())+1; + top=Node::from_impl(top->left()); + } + else{ + return std::pair( + s-top->impl()->size+ + ranked_index_lower_bound_rank( + Node::from_impl(top->left()),top,key,x,comp,mpl::false_()), + s-ranked_node_size(top->right())+ + ranked_index_upper_bound_rank( + Node::from_impl(top->right()),y,key,x,comp,mpl::false_())); + } + }while(top); + + return std::pair(s,s); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/safe_mode.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/safe_mode.hpp new file mode 100644 index 00000000000..905270e9fb3 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/safe_mode.hpp @@ -0,0 +1,588 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_SAFE_MODE_HPP +#define BOOST_MULTI_INDEX_DETAIL_SAFE_MODE_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +/* Safe mode machinery, in the spirit of Cay Hortmann's "Safe STL" + * (http://www.horstmann.com/safestl.html). + * In this mode, containers of type Container are derived from + * safe_container, and their corresponding iterators + * are wrapped with safe_iterator. These classes provide + * an internal record of which iterators are at a given moment associated + * to a given container, and properly mark the iterators as invalid + * when the container gets destroyed. + * Iterators are chained in a single attached list, whose header is + * kept by the container. More elaborate data structures would yield better + * performance, but I decided to keep complexity to a minimum since + * speed is not an issue here. + * Safe mode iterators automatically check that only proper operations + * are performed on them: for instance, an invalid iterator cannot be + * dereferenced. Additionally, a set of utilty macros and functions are + * provided that serve to implement preconditions and cooperate with + * the framework within the container. + * Iterators can also be unchecked, i.e. they do not have info about + * which container they belong in. This situation arises when the iterator + * is restored from a serialization archive: only information on the node + * is available, and it is not possible to determine to which container + * the iterator is associated to. The only sensible policy is to assume + * unchecked iterators are valid, though this can certainly generate false + * positive safe mode checks. + * This is not a full-fledged safe mode framework, and is only intended + * for use within the limits of Boost.MultiIndex. + */ + +/* Assertion macros. These resolve to no-ops if + * !defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE). + */ + +#if !defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) +#undef BOOST_MULTI_INDEX_SAFE_MODE_ASSERT +#define BOOST_MULTI_INDEX_SAFE_MODE_ASSERT(expr,error_code) ((void)0) +#else +#if !defined(BOOST_MULTI_INDEX_SAFE_MODE_ASSERT) +#include +#define BOOST_MULTI_INDEX_SAFE_MODE_ASSERT(expr,error_code) BOOST_ASSERT(expr) +#endif +#endif + +#define BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it) \ + BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ + safe_mode::check_valid_iterator(it), \ + safe_mode::invalid_iterator); + +#define BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(it) \ + BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ + safe_mode::check_dereferenceable_iterator(it), \ + safe_mode::not_dereferenceable_iterator); + +#define BOOST_MULTI_INDEX_CHECK_INCREMENTABLE_ITERATOR(it) \ + BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ + safe_mode::check_incrementable_iterator(it), \ + safe_mode::not_incrementable_iterator); + +#define BOOST_MULTI_INDEX_CHECK_DECREMENTABLE_ITERATOR(it) \ + BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ + safe_mode::check_decrementable_iterator(it), \ + safe_mode::not_decrementable_iterator); + +#define BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,cont) \ + BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ + safe_mode::check_is_owner(it,cont), \ + safe_mode::not_owner); + +#define BOOST_MULTI_INDEX_CHECK_SAME_OWNER(it0,it1) \ + BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ + safe_mode::check_same_owner(it0,it1), \ + safe_mode::not_same_owner); + +#define BOOST_MULTI_INDEX_CHECK_VALID_RANGE(it0,it1) \ + BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ + safe_mode::check_valid_range(it0,it1), \ + safe_mode::invalid_range); + +#define BOOST_MULTI_INDEX_CHECK_OUTSIDE_RANGE(it,it0,it1) \ + BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ + safe_mode::check_outside_range(it,it0,it1), \ + safe_mode::inside_range); + +#define BOOST_MULTI_INDEX_CHECK_IN_BOUNDS(it,n) \ + BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ + safe_mode::check_in_bounds(it,n), \ + safe_mode::out_of_bounds); + +#define BOOST_MULTI_INDEX_CHECK_DIFFERENT_CONTAINER(cont0,cont1) \ + BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ + safe_mode::check_different_container(cont0,cont1), \ + safe_mode::same_container); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +#include +#include +#endif + +#if defined(BOOST_HAS_THREADS) +#include +#endif + +namespace boost{ + +namespace multi_index{ + +namespace safe_mode{ + +/* Checking routines. Assume the best for unchecked iterators + * (i.e. they pass the checking when there is not enough info + * to know.) + */ + +template +inline bool check_valid_iterator(const Iterator& it) +{ + return it.valid()||it.unchecked(); +} + +template +inline bool check_dereferenceable_iterator(const Iterator& it) +{ + return (it.valid()&&it!=it.owner()->end())||it.unchecked(); +} + +template +inline bool check_incrementable_iterator(const Iterator& it) +{ + return (it.valid()&&it!=it.owner()->end())||it.unchecked(); +} + +template +inline bool check_decrementable_iterator(const Iterator& it) +{ + return (it.valid()&&it!=it.owner()->begin())||it.unchecked(); +} + +template +inline bool check_is_owner( + const Iterator& it,const typename Iterator::container_type& cont) +{ + return (it.valid()&&it.owner()==&cont)||it.unchecked(); +} + +template +inline bool check_same_owner(const Iterator& it0,const Iterator& it1) +{ + return (it0.valid()&&it1.valid()&&it0.owner()==it1.owner())|| + it0.unchecked()||it1.unchecked(); +} + +template +inline bool check_valid_range(const Iterator& it0,const Iterator& it1) +{ + if(!check_same_owner(it0,it1))return false; + + if(it0.valid()){ + Iterator last=it0.owner()->end(); + if(it1==last)return true; + + for(Iterator first=it0;first!=last;++first){ + if(first==it1)return true; + } + return false; + } + return true; +} + +template +inline bool check_outside_range( + const Iterator& it,const Iterator& it0,const Iterator& it1) +{ + if(!check_same_owner(it0,it1))return false; + + if(it0.valid()){ + Iterator last=it0.owner()->end(); + bool found=false; + + Iterator first=it0; + for(;first!=last;++first){ + if(first==it1)break; + + /* crucial that this check goes after previous break */ + + if(first==it)found=true; + } + if(first!=it1)return false; + return !found; + } + return true; +} + +template +inline bool check_in_bounds(const Iterator& it,Difference n) +{ + if(it.unchecked())return true; + if(!it.valid()) return false; + if(n>0) return it.owner()->end()-it>=n; + else return it.owner()->begin()-it<=n; +} + +template +inline bool check_different_container( + const Container& cont0,const Container& cont1) +{ + return &cont0!=&cont1; +} + +/* Invalidates all iterators equivalent to that given. Safe containers + * must call this when deleting elements: the safe mode framework cannot + * perform this operation automatically without outside help. + */ + +template +inline void detach_equivalent_iterators(Iterator& it) +{ + if(it.valid()){ + { +#if defined(BOOST_HAS_THREADS) + boost::detail::lightweight_mutex::scoped_lock lock(it.cont->mutex); +#endif + + Iterator *prev_,*next_; + for( + prev_=static_cast(&it.cont->header); + (next_=static_cast(prev_->next))!=0;){ + if(next_!=&it&&*next_==it){ + prev_->next=next_->next; + next_->cont=0; + } + else prev_=next_; + } + } + it.detach(); + } +} + +template class safe_container; /* fwd decl. */ + +} /* namespace multi_index::safe_mode */ + +namespace detail{ + +class safe_container_base; /* fwd decl. */ + +class safe_iterator_base +{ +public: + bool valid()const{return cont!=0;} + bool unchecked()const{return unchecked_;} + + inline void detach(); + + void uncheck() + { + detach(); + unchecked_=true; + } + +protected: + safe_iterator_base():cont(0),next(0),unchecked_(false){} + + explicit safe_iterator_base(safe_container_base* cont_): + unchecked_(false) + { + attach(cont_); + } + + safe_iterator_base(const safe_iterator_base& it): + unchecked_(it.unchecked_) + { + attach(it.cont); + } + + safe_iterator_base& operator=(const safe_iterator_base& it) + { + unchecked_=it.unchecked_; + safe_container_base* new_cont=it.cont; + if(cont!=new_cont){ + detach(); + attach(new_cont); + } + return *this; + } + + ~safe_iterator_base() + { + detach(); + } + + const safe_container_base* owner()const{return cont;} + +BOOST_MULTI_INDEX_PRIVATE_IF_MEMBER_TEMPLATE_FRIENDS: + friend class safe_container_base; + +#if !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) + template friend class safe_mode::safe_container; + template friend + void safe_mode::detach_equivalent_iterators(Iterator&); +#endif + + inline void attach(safe_container_base* cont_); + + safe_container_base* cont; + safe_iterator_base* next; + bool unchecked_; +}; + +class safe_container_base:private noncopyable +{ +public: + safe_container_base(){} + +BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: + friend class safe_iterator_base; + +#if !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) + template friend + void safe_mode::detach_equivalent_iterators(Iterator&); +#endif + + ~safe_container_base() + { + /* Detaches all remaining iterators, which by now will + * be those pointing to the end of the container. + */ + + for(safe_iterator_base* it=header.next;it;it=it->next)it->cont=0; + header.next=0; + } + + void swap(safe_container_base& x) + { + for(safe_iterator_base* it0=header.next;it0;it0=it0->next)it0->cont=&x; + for(safe_iterator_base* it1=x.header.next;it1;it1=it1->next)it1->cont=this; + std::swap(header.cont,x.header.cont); + std::swap(header.next,x.header.next); + } + + safe_iterator_base header; + +#if defined(BOOST_HAS_THREADS) + boost::detail::lightweight_mutex mutex; +#endif +}; + +void safe_iterator_base::attach(safe_container_base* cont_) +{ + cont=cont_; + if(cont){ +#if defined(BOOST_HAS_THREADS) + boost::detail::lightweight_mutex::scoped_lock lock(cont->mutex); +#endif + + next=cont->header.next; + cont->header.next=this; + } +} + +void safe_iterator_base::detach() +{ + if(cont){ +#if defined(BOOST_HAS_THREADS) + boost::detail::lightweight_mutex::scoped_lock lock(cont->mutex); +#endif + + safe_iterator_base *prev_,*next_; + for(prev_=&cont->header;(next_=prev_->next)!=this;prev_=next_){} + prev_->next=next; + cont=0; + } +} + +} /* namespace multi_index::detail */ + +namespace safe_mode{ + +/* In order to enable safe mode on a container: + * - The container must derive from safe_container, + * - iterators must be generated via safe_iterator, which adapts a + * preexistent unsafe iterator class. + */ + +template +class safe_container; + +template +class safe_iterator: + public detail::iter_adaptor,Iterator>, + public detail::safe_iterator_base +{ + typedef detail::iter_adaptor super; + typedef detail::safe_iterator_base safe_super; + +public: + typedef Container container_type; + typedef typename Iterator::reference reference; + typedef typename Iterator::difference_type difference_type; + + safe_iterator(){} + explicit safe_iterator(safe_container* cont_): + safe_super(cont_){} + template + safe_iterator(const T0& t0,safe_container* cont_): + super(Iterator(t0)),safe_super(cont_){} + template + safe_iterator( + const T0& t0,const T1& t1,safe_container* cont_): + super(Iterator(t0,t1)),safe_super(cont_){} + + safe_iterator& operator=(const safe_iterator& x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(x); + this->base_reference()=x.base_reference(); + safe_super::operator=(x); + return *this; + } + + const container_type* owner()const + { + return + static_cast( + static_cast*>( + this->safe_super::owner())); + } + + /* get_node is not to be used by the user */ + + typedef typename Iterator::node_type node_type; + + node_type* get_node()const{return this->base_reference().get_node();} + +private: + friend class boost::multi_index::detail::iter_adaptor_access; + + reference dereference()const + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(*this); + return *(this->base_reference()); + } + + bool equal(const safe_iterator& x)const + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(x); + BOOST_MULTI_INDEX_CHECK_SAME_OWNER(*this,x); + return this->base_reference()==x.base_reference(); + } + + void increment() + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); + BOOST_MULTI_INDEX_CHECK_INCREMENTABLE_ITERATOR(*this); + ++(this->base_reference()); + } + + void decrement() + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); + BOOST_MULTI_INDEX_CHECK_DECREMENTABLE_ITERATOR(*this); + --(this->base_reference()); + } + + void advance(difference_type n) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); + BOOST_MULTI_INDEX_CHECK_IN_BOUNDS(*this,n); + this->base_reference()+=n; + } + + difference_type distance_to(const safe_iterator& x)const + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(x); + BOOST_MULTI_INDEX_CHECK_SAME_OWNER(*this,x); + return x.base_reference()-this->base_reference(); + } + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + /* Serialization. Note that Iterator::save and Iterator:load + * are assumed to be defined and public: at first sight it seems + * like we could have resorted to the public serialization interface + * for doing the forwarding to the adapted iterator class: + * ar<>base_reference(); + * but this would cause incompatibilities if a saving + * program is in safe mode and the loading program is not, or + * viceversa --in safe mode, the archived iterator data is one layer + * deeper, this is especially relevant with XML archives. + * It'd be nice if Boost.Serialization provided some forwarding + * facility for use by adaptor classes. + */ + + friend class boost::serialization::access; + + BOOST_SERIALIZATION_SPLIT_MEMBER() + + template + void save(Archive& ar,const unsigned int version)const + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); + this->base_reference().save(ar,version); + } + + template + void load(Archive& ar,const unsigned int version) + { + this->base_reference().load(ar,version); + safe_super::uncheck(); + } +#endif +}; + +template +class safe_container:public detail::safe_container_base +{ + typedef detail::safe_container_base super; + +public: + void detach_dereferenceable_iterators() + { + typedef typename Container::iterator iterator; + + iterator end_=static_cast(this)->end(); + iterator *prev_,*next_; + for( + prev_=static_cast(&this->header); + (next_=static_cast(prev_->next))!=0;){ + if(*next_!=end_){ + prev_->next=next_->next; + next_->cont=0; + } + else prev_=next_; + } + } + + void swap(safe_container& x) + { + super::swap(x); + } +}; + +} /* namespace multi_index::safe_mode */ + +} /* namespace multi_index */ + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +namespace serialization{ +template +struct version< + boost::multi_index::safe_mode::safe_iterator +> +{ + BOOST_STATIC_CONSTANT( + int,value=boost::serialization::version::value); +}; +} /* namespace serialization */ +#endif + +} /* namespace boost */ + +#endif /* BOOST_MULTI_INDEX_ENABLE_SAFE_MODE */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/scope_guard.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/scope_guard.hpp new file mode 100644 index 00000000000..116f8f50415 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/scope_guard.hpp @@ -0,0 +1,453 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_SCOPE_GUARD_HPP +#define BOOST_MULTI_INDEX_DETAIL_SCOPE_GUARD_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Until some official version of the ScopeGuard idiom makes it into Boost, + * we locally define our own. This is a merely reformated version of + * ScopeGuard.h as defined in: + * Alexandrescu, A., Marginean, P.:"Generic: Change the Way You + * Write Exception-Safe Code - Forever", C/C++ Users Jornal, Dec 2000, + * http://www.drdobbs.com/184403758 + * with the following modifications: + * - General pretty formatting (pretty to my taste at least.) + * - Naming style changed to standard C++ library requirements. + * - Added scope_guard_impl4 and obj_scope_guard_impl3, (Boost.MultiIndex + * needs them). A better design would provide guards for many more + * arguments through the Boost Preprocessor Library. + * - Added scope_guard_impl_base::touch (see below.) + * - Removed RefHolder and ByRef, whose functionality is provided + * already by Boost.Ref. + * - Removed static make_guard's and make_obj_guard's, so that the code + * will work even if BOOST_NO_MEMBER_TEMPLATES is defined. This forces + * us to move some private ctors to public, though. + * + * NB: CodeWarrior Pro 8 seems to have problems looking up safe_execute + * without an explicit qualification. + * + * We also define the following variants of the idiom: + * + * - make_guard_if_c( ... ) + * - make_guard_if( ... ) + * - make_obj_guard_if_c( ... ) + * - make_obj_guard_if( ... ) + * which may be used with a compile-time constant to yield + * a "null_guard" if the boolean compile-time parameter is false, + * or conversely, the guard is only constructed if the constant is true. + * This is useful to avoid extra tagging, because the returned + * null_guard can be optimzed comlpetely away by the compiler. + */ + +class scope_guard_impl_base +{ +public: + scope_guard_impl_base():dismissed_(false){} + void dismiss()const{dismissed_=true;} + + /* This helps prevent some "unused variable" warnings under, for instance, + * GCC 3.2. + */ + void touch()const{} + +protected: + ~scope_guard_impl_base(){} + + scope_guard_impl_base(const scope_guard_impl_base& other): + dismissed_(other.dismissed_) + { + other.dismiss(); + } + + template + static void safe_execute(J& j){ + BOOST_TRY{ + if(!j.dismissed_)j.execute(); + } + BOOST_CATCH(...){} + BOOST_CATCH_END + } + + mutable bool dismissed_; + +private: + scope_guard_impl_base& operator=(const scope_guard_impl_base&); +}; + +typedef const scope_guard_impl_base& scope_guard; + +struct null_guard : public scope_guard_impl_base +{ + template< class T1 > + null_guard( const T1& ) + { } + + template< class T1, class T2 > + null_guard( const T1&, const T2& ) + { } + + template< class T1, class T2, class T3 > + null_guard( const T1&, const T2&, const T3& ) + { } + + template< class T1, class T2, class T3, class T4 > + null_guard( const T1&, const T2&, const T3&, const T4& ) + { } + + template< class T1, class T2, class T3, class T4, class T5 > + null_guard( const T1&, const T2&, const T3&, const T4&, const T5& ) + { } +}; + +template< bool cond, class T > +struct null_guard_return +{ + typedef typename boost::mpl::if_c::type type; +}; + +template +class scope_guard_impl0:public scope_guard_impl_base +{ +public: + scope_guard_impl0(F fun):fun_(fun){} + ~scope_guard_impl0(){scope_guard_impl_base::safe_execute(*this);} + void execute(){fun_();} + +protected: + + F fun_; +}; + +template +inline scope_guard_impl0 make_guard(F fun) +{ + return scope_guard_impl0(fun); +} + +template +inline typename null_guard_return >::type +make_guard_if_c(F fun) +{ + return typename null_guard_return >::type(fun); +} + +template +inline typename null_guard_return >::type +make_guard_if(F fun) +{ + return make_guard_if(fun); +} + +template +class scope_guard_impl1:public scope_guard_impl_base +{ +public: + scope_guard_impl1(F fun,P1 p1):fun_(fun),p1_(p1){} + ~scope_guard_impl1(){scope_guard_impl_base::safe_execute(*this);} + void execute(){fun_(p1_);} + +protected: + F fun_; + const P1 p1_; +}; + +template +inline scope_guard_impl1 make_guard(F fun,P1 p1) +{ + return scope_guard_impl1(fun,p1); +} + +template +inline typename null_guard_return >::type +make_guard_if_c(F fun,P1 p1) +{ + return typename null_guard_return >::type(fun,p1); +} + +template +inline typename null_guard_return >::type +make_guard_if(F fun,P1 p1) +{ + return make_guard_if_c(fun,p1); +} + +template +class scope_guard_impl2:public scope_guard_impl_base +{ +public: + scope_guard_impl2(F fun,P1 p1,P2 p2):fun_(fun),p1_(p1),p2_(p2){} + ~scope_guard_impl2(){scope_guard_impl_base::safe_execute(*this);} + void execute(){fun_(p1_,p2_);} + +protected: + F fun_; + const P1 p1_; + const P2 p2_; +}; + +template +inline scope_guard_impl2 make_guard(F fun,P1 p1,P2 p2) +{ + return scope_guard_impl2(fun,p1,p2); +} + +template +inline typename null_guard_return >::type +make_guard_if_c(F fun,P1 p1,P2 p2) +{ + return typename null_guard_return >::type(fun,p1,p2); +} + +template +inline typename null_guard_return >::type +make_guard_if(F fun,P1 p1,P2 p2) +{ + return make_guard_if_c(fun,p1,p2); +} + +template +class scope_guard_impl3:public scope_guard_impl_base +{ +public: + scope_guard_impl3(F fun,P1 p1,P2 p2,P3 p3):fun_(fun),p1_(p1),p2_(p2),p3_(p3){} + ~scope_guard_impl3(){scope_guard_impl_base::safe_execute(*this);} + void execute(){fun_(p1_,p2_,p3_);} + +protected: + F fun_; + const P1 p1_; + const P2 p2_; + const P3 p3_; +}; + +template +inline scope_guard_impl3 make_guard(F fun,P1 p1,P2 p2,P3 p3) +{ + return scope_guard_impl3(fun,p1,p2,p3); +} + +template +inline typename null_guard_return >::type +make_guard_if_c(F fun,P1 p1,P2 p2,P3 p3) +{ + return typename null_guard_return >::type(fun,p1,p2,p3); +} + +template +inline typename null_guard_return< C::value,scope_guard_impl3 >::type +make_guard_if(F fun,P1 p1,P2 p2,P3 p3) +{ + return make_guard_if_c(fun,p1,p2,p3); +} + +template +class scope_guard_impl4:public scope_guard_impl_base +{ +public: + scope_guard_impl4(F fun,P1 p1,P2 p2,P3 p3,P4 p4): + fun_(fun),p1_(p1),p2_(p2),p3_(p3),p4_(p4){} + ~scope_guard_impl4(){scope_guard_impl_base::safe_execute(*this);} + void execute(){fun_(p1_,p2_,p3_,p4_);} + +protected: + F fun_; + const P1 p1_; + const P2 p2_; + const P3 p3_; + const P4 p4_; +}; + +template +inline scope_guard_impl4 make_guard( + F fun,P1 p1,P2 p2,P3 p3,P4 p4) +{ + return scope_guard_impl4(fun,p1,p2,p3,p4); +} + +template +inline typename null_guard_return >::type +make_guard_if_c( + F fun,P1 p1,P2 p2,P3 p3,P4 p4) +{ + return typename null_guard_return >::type(fun,p1,p2,p3,p4); +} + +template +inline typename null_guard_return >::type +make_guard_if( + F fun,P1 p1,P2 p2,P3 p3,P4 p4) +{ + return make_guard_if_c(fun,p1,p2,p3,p4); +} + +template +class obj_scope_guard_impl0:public scope_guard_impl_base +{ +public: + obj_scope_guard_impl0(Obj& obj,MemFun mem_fun):obj_(obj),mem_fun_(mem_fun){} + ~obj_scope_guard_impl0(){scope_guard_impl_base::safe_execute(*this);} + void execute(){(obj_.*mem_fun_)();} + +protected: + Obj& obj_; + MemFun mem_fun_; +}; + +template +inline obj_scope_guard_impl0 make_obj_guard(Obj& obj,MemFun mem_fun) +{ + return obj_scope_guard_impl0(obj,mem_fun); +} + +template +inline typename null_guard_return >::type +make_obj_guard_if_c(Obj& obj,MemFun mem_fun) +{ + return typename null_guard_return >::type(obj,mem_fun); +} + +template +inline typename null_guard_return >::type +make_obj_guard_if(Obj& obj,MemFun mem_fun) +{ + return make_obj_guard_if_c(obj,mem_fun); +} + +template +class obj_scope_guard_impl1:public scope_guard_impl_base +{ +public: + obj_scope_guard_impl1(Obj& obj,MemFun mem_fun,P1 p1): + obj_(obj),mem_fun_(mem_fun),p1_(p1){} + ~obj_scope_guard_impl1(){scope_guard_impl_base::safe_execute(*this);} + void execute(){(obj_.*mem_fun_)(p1_);} + +protected: + Obj& obj_; + MemFun mem_fun_; + const P1 p1_; +}; + +template +inline obj_scope_guard_impl1 make_obj_guard( + Obj& obj,MemFun mem_fun,P1 p1) +{ + return obj_scope_guard_impl1(obj,mem_fun,p1); +} + +template +inline typename null_guard_return >::type +make_obj_guard_if_c( Obj& obj,MemFun mem_fun,P1 p1) +{ + return typename null_guard_return >::type(obj,mem_fun,p1); +} + +template +inline typename null_guard_return >::type +make_obj_guard_if( Obj& obj,MemFun mem_fun,P1 p1) +{ + return make_obj_guard_if_c(obj,mem_fun,p1); +} + +template +class obj_scope_guard_impl2:public scope_guard_impl_base +{ +public: + obj_scope_guard_impl2(Obj& obj,MemFun mem_fun,P1 p1,P2 p2): + obj_(obj),mem_fun_(mem_fun),p1_(p1),p2_(p2) + {} + ~obj_scope_guard_impl2(){scope_guard_impl_base::safe_execute(*this);} + void execute(){(obj_.*mem_fun_)(p1_,p2_);} + +protected: + Obj& obj_; + MemFun mem_fun_; + const P1 p1_; + const P2 p2_; +}; + +template +inline obj_scope_guard_impl2 +make_obj_guard(Obj& obj,MemFun mem_fun,P1 p1,P2 p2) +{ + return obj_scope_guard_impl2(obj,mem_fun,p1,p2); +} + +template +inline typename null_guard_return >::type +make_obj_guard_if_c(Obj& obj,MemFun mem_fun,P1 p1,P2 p2) +{ + return typename null_guard_return >::type(obj,mem_fun,p1,p2); +} + +template +inline typename null_guard_return >::type +make_obj_guard_if(Obj& obj,MemFun mem_fun,P1 p1,P2 p2) +{ + return make_obj_guard_if_c(obj,mem_fun,p1,p2); +} + +template +class obj_scope_guard_impl3:public scope_guard_impl_base +{ +public: + obj_scope_guard_impl3(Obj& obj,MemFun mem_fun,P1 p1,P2 p2,P3 p3): + obj_(obj),mem_fun_(mem_fun),p1_(p1),p2_(p2),p3_(p3) + {} + ~obj_scope_guard_impl3(){scope_guard_impl_base::safe_execute(*this);} + void execute(){(obj_.*mem_fun_)(p1_,p2_,p3_);} + +protected: + Obj& obj_; + MemFun mem_fun_; + const P1 p1_; + const P2 p2_; + const P3 p3_; +}; + +template +inline obj_scope_guard_impl3 +make_obj_guard(Obj& obj,MemFun mem_fun,P1 p1,P2 p2,P3 p3) +{ + return obj_scope_guard_impl3(obj,mem_fun,p1,p2,p3); +} + +template +inline typename null_guard_return >::type +make_obj_guard_if_c(Obj& obj,MemFun mem_fun,P1 p1,P2 p2,P3 p3) +{ + return typename null_guard_return >::type(obj,mem_fun,p1,p2,p3); +} + +template +inline typename null_guard_return >::type +make_obj_guard_if(Obj& obj,MemFun mem_fun,P1 p1,P2 p2,P3 p3) +{ + return make_obj_guard_if_c(obj,mem_fun,p1,p2,p3); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_node.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_node.hpp new file mode 100644 index 00000000000..85b345af938 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_node.hpp @@ -0,0 +1,217 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_SEQ_INDEX_NODE_HPP +#define BOOST_MULTI_INDEX_DETAIL_SEQ_INDEX_NODE_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* doubly-linked node for use by sequenced_index */ + +template +struct sequenced_index_node_impl +{ + typedef typename + boost::detail::allocator::rebind_to< + Allocator,sequenced_index_node_impl + >::type::pointer pointer; + typedef typename + boost::detail::allocator::rebind_to< + Allocator,sequenced_index_node_impl + >::type::const_pointer const_pointer; + + pointer& prior(){return prior_;} + pointer prior()const{return prior_;} + pointer& next(){return next_;} + pointer next()const{return next_;} + + /* interoperability with bidir_node_iterator */ + + static void increment(pointer& x){x=x->next();} + static void decrement(pointer& x){x=x->prior();} + + /* algorithmic stuff */ + + static void link(pointer x,pointer header) + { + x->prior()=header->prior(); + x->next()=header; + x->prior()->next()=x->next()->prior()=x; + }; + + static void unlink(pointer x) + { + x->prior()->next()=x->next(); + x->next()->prior()=x->prior(); + } + + static void relink(pointer position,pointer x) + { + unlink(x); + x->prior()=position->prior(); + x->next()=position; + x->prior()->next()=x->next()->prior()=x; + } + + static void relink(pointer position,pointer x,pointer y) + { + /* position is assumed not to be in [x,y) */ + + if(x!=y){ + pointer z=y->prior(); + x->prior()->next()=y; + y->prior()=x->prior(); + x->prior()=position->prior(); + z->next()=position; + x->prior()->next()=x; + z->next()->prior()=z; + } + } + + static void reverse(pointer header) + { + pointer x=header; + do{ + pointer y=x->next(); + std::swap(x->prior(),x->next()); + x=y; + }while(x!=header); + } + + static void swap(pointer x,pointer y) + { + /* This swap function does not exchange the header nodes, + * but rather their pointers. This is *not* used for implementing + * sequenced_index::swap. + */ + + if(x->next()!=x){ + if(y->next()!=y){ + std::swap(x->next(),y->next()); + std::swap(x->prior(),y->prior()); + x->next()->prior()=x->prior()->next()=x; + y->next()->prior()=y->prior()->next()=y; + } + else{ + y->next()=x->next(); + y->prior()=x->prior(); + x->next()=x->prior()=x; + y->next()->prior()=y->prior()->next()=y; + } + } + else if(y->next()!=y){ + x->next()=y->next(); + x->prior()=y->prior(); + y->next()=y->prior()=y; + x->next()->prior()=x->prior()->next()=x; + } + } + +private: + pointer prior_; + pointer next_; +}; + +template +struct sequenced_index_node_trampoline: + sequenced_index_node_impl< + typename boost::detail::allocator::rebind_to< + typename Super::allocator_type, + char + >::type + > +{ + typedef sequenced_index_node_impl< + typename boost::detail::allocator::rebind_to< + typename Super::allocator_type, + char + >::type + > impl_type; +}; + +template +struct sequenced_index_node:Super,sequenced_index_node_trampoline +{ +private: + typedef sequenced_index_node_trampoline trampoline; + +public: + typedef typename trampoline::impl_type impl_type; + typedef typename trampoline::pointer impl_pointer; + typedef typename trampoline::const_pointer const_impl_pointer; + + impl_pointer& prior(){return trampoline::prior();} + impl_pointer prior()const{return trampoline::prior();} + impl_pointer& next(){return trampoline::next();} + impl_pointer next()const{return trampoline::next();} + + impl_pointer impl() + { + return static_cast( + static_cast(static_cast(this))); + } + + const_impl_pointer impl()const + { + return static_cast( + static_cast(static_cast(this))); + } + + static sequenced_index_node* from_impl(impl_pointer x) + { + return + static_cast( + static_cast( + raw_ptr(x))); + } + + static const sequenced_index_node* from_impl(const_impl_pointer x) + { + return + static_cast( + static_cast( + raw_ptr(x))); + } + + /* interoperability with bidir_node_iterator */ + + static void increment(sequenced_index_node*& x) + { + impl_pointer xi=x->impl(); + trampoline::increment(xi); + x=from_impl(xi); + } + + static void decrement(sequenced_index_node*& x) + { + impl_pointer xi=x->impl(); + trampoline::decrement(xi); + x=from_impl(xi); + } +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_ops.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_ops.hpp new file mode 100644 index 00000000000..142bdd9dd9a --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_ops.hpp @@ -0,0 +1,203 @@ +/* Copyright 2003-2016 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_SEQ_INDEX_OPS_HPP +#define BOOST_MULTI_INDEX_DETAIL_SEQ_INDEX_OPS_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Common code for sequenced_index memfuns having templatized and + * non-templatized versions. + */ + +template +void sequenced_index_remove(SequencedIndex& x,Predicate pred) +{ + typedef typename SequencedIndex::iterator iterator; + iterator first=x.begin(),last=x.end(); + while(first!=last){ + if(pred(*first))x.erase(first++); + else ++first; + } +} + +template +void sequenced_index_unique(SequencedIndex& x,BinaryPredicate binary_pred) +{ + typedef typename SequencedIndex::iterator iterator; + iterator first=x.begin(); + iterator last=x.end(); + if(first!=last){ + for(iterator middle=first;++middle!=last;middle=first){ + if(binary_pred(*middle,*first))x.erase(middle); + else first=middle; + } + } +} + +template +void sequenced_index_merge(SequencedIndex& x,SequencedIndex& y,Compare comp) +{ + typedef typename SequencedIndex::iterator iterator; + if(&x!=&y){ + iterator first0=x.begin(),last0=x.end(); + iterator first1=y.begin(),last1=y.end(); + while(first0!=last0&&first1!=last1){ + if(comp(*first1,*first0))x.splice(first0,y,first1++); + else ++first0; + } + x.splice(last0,y,first1,last1); + } +} + +/* sorting */ + +/* auxiliary stuff */ + +template +void sequenced_index_collate( + BOOST_DEDUCED_TYPENAME Node::impl_type* x, + BOOST_DEDUCED_TYPENAME Node::impl_type* y, + Compare comp) +{ + typedef typename Node::impl_type impl_type; + typedef typename Node::impl_pointer impl_pointer; + + impl_pointer first0=x->next(); + impl_pointer last0=x; + impl_pointer first1=y->next(); + impl_pointer last1=y; + while(first0!=last0&&first1!=last1){ + if(comp( + Node::from_impl(first1)->value(),Node::from_impl(first0)->value())){ + impl_pointer tmp=first1->next(); + impl_type::relink(first0,first1); + first1=tmp; + } + else first0=first0->next(); + } + impl_type::relink(last0,first1,last1); +} + +/* Some versions of CGG require a bogus typename in counter_spc + * inside sequenced_index_sort if the following is defined + * also inside sequenced_index_sort. + */ + +BOOST_STATIC_CONSTANT( + std::size_t, + sequenced_index_sort_max_fill= + (std::size_t)std::numeric_limits::digits+1); + +#include + +template +void sequenced_index_sort(Node* header,Compare comp) +{ + /* Musser's mergesort, see http://www.cs.rpi.edu/~musser/gp/List/lists1.html. + * The implementation is a little convoluted: in the original code + * counter elements and carry are std::lists: here we do not want + * to use multi_index instead, so we do things at a lower level, managing + * directly the internal node representation. + * Incidentally, the implementations I've seen of this algorithm (SGI, + * Dinkumware, STLPort) are not exception-safe: this is. Moreover, we do not + * use any dynamic storage. + */ + + if(header->next()==header->impl()|| + header->next()->next()==header->impl())return; + + typedef typename Node::impl_type impl_type; + typedef typename Node::impl_pointer impl_pointer; + + typedef typename aligned_storage< + sizeof(impl_type), + alignment_of::value + >::type carry_spc_type; + carry_spc_type carry_spc; + impl_type& carry= + *reinterpret_cast(&carry_spc); + typedef typename aligned_storage< + sizeof( + impl_type + [sequenced_index_sort_max_fill]), + alignment_of< + impl_type + [sequenced_index_sort_max_fill] + >::value + >::type counter_spc_type; + counter_spc_type counter_spc; + impl_type* counter= + reinterpret_cast(&counter_spc); + std::size_t fill=0; + + carry.prior()=carry.next()=static_cast(&carry); + counter[0].prior()=counter[0].next()=static_cast(&counter[0]); + + BOOST_TRY{ + while(header->next()!=header->impl()){ + impl_type::relink(carry.next(),header->next()); + std::size_t i=0; + while(i(&counter[i])){ + sequenced_index_collate(&carry,&counter[i++],comp); + } + impl_type::swap( + static_cast(&carry), + static_cast(&counter[i])); + if(i==fill){ + ++fill; + counter[fill].prior()=counter[fill].next()= + static_cast(&counter[fill]); + } + } + + for(std::size_t i=1;i(&counter[i],&counter[i-1],comp); + } + impl_type::swap( + header->impl(),static_cast(&counter[fill-1])); + } + BOOST_CATCH(...) + { + impl_type::relink( + header->impl(),carry.next(),static_cast(&carry)); + for(std::size_t i=0;i<=fill;++i){ + impl_type::relink( + header->impl(),counter[i].next(), + static_cast(&counter[i])); + } + BOOST_RETHROW; + } + BOOST_CATCH_END +} + +#include + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/serialization_version.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/serialization_version.hpp new file mode 100644 index 00000000000..ccd8bb4f791 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/serialization_version.hpp @@ -0,0 +1,73 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_SERIALIZATION_VERSION_HPP +#define BOOST_MULTI_INDEX_DETAIL_SERIALIZATION_VERSION_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* Helper class for storing and retrieving a given type serialization class + * version while avoiding saving the number multiple times in the same + * archive. + * Behavior undefined if template partial specialization is not supported. + */ + +template +struct serialization_version +{ + serialization_version(): + value(boost::serialization::version::value){} + + serialization_version& operator=(unsigned int x){value=x;return *this;}; + + operator unsigned int()const{return value;} + +private: + friend class boost::serialization::access; + + BOOST_SERIALIZATION_SPLIT_MEMBER() + + template + void save(Archive&,const unsigned int)const{} + + template + void load(Archive&,const unsigned int version) + { + this->value=version; + } + + unsigned int value; +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +namespace serialization { +template +struct version > +{ + BOOST_STATIC_CONSTANT(int,value=version::value); +}; +} /* namespace serialization */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/uintptr_type.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/uintptr_type.hpp new file mode 100644 index 00000000000..9c92d01d4de --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/uintptr_type.hpp @@ -0,0 +1,76 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_UINTPTR_TYPE_HPP +#define BOOST_MULTI_INDEX_DETAIL_UINTPTR_TYPE_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* has_uintptr_type is an MPL integral constant determining whether + * there exists an unsigned integral type with the same size as + * void *. + * uintptr_type is such a type if has_uintptr is true, or unsigned int + * otherwise. + * Note that uintptr_type is more restrictive than C99 uintptr_t, + * where an integral type with size greater than that of void * + * would be conformant. + */ + +templatestruct uintptr_candidates; +template<>struct uintptr_candidates<-1>{typedef unsigned int type;}; +template<>struct uintptr_candidates<0> {typedef unsigned int type;}; +template<>struct uintptr_candidates<1> {typedef unsigned short type;}; +template<>struct uintptr_candidates<2> {typedef unsigned long type;}; + +#if defined(BOOST_HAS_LONG_LONG) +template<>struct uintptr_candidates<3> {typedef boost::ulong_long_type type;}; +#else +template<>struct uintptr_candidates<3> {typedef unsigned int type;}; +#endif + +#if defined(BOOST_HAS_MS_INT64) +template<>struct uintptr_candidates<4> {typedef unsigned __int64 type;}; +#else +template<>struct uintptr_candidates<4> {typedef unsigned int type;}; +#endif + +struct uintptr_aux +{ + BOOST_STATIC_CONSTANT(int,index= + sizeof(void*)==sizeof(uintptr_candidates<0>::type)?0: + sizeof(void*)==sizeof(uintptr_candidates<1>::type)?1: + sizeof(void*)==sizeof(uintptr_candidates<2>::type)?2: + sizeof(void*)==sizeof(uintptr_candidates<3>::type)?3: + sizeof(void*)==sizeof(uintptr_candidates<4>::type)?4:-1); + + BOOST_STATIC_CONSTANT(bool,has_uintptr_type=(index>=0)); + + typedef uintptr_candidates::type type; +}; + +typedef mpl::bool_ has_uintptr_type; +typedef uintptr_aux::type uintptr_type; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/unbounded.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/unbounded.hpp new file mode 100644 index 00000000000..dc09be1770d --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/unbounded.hpp @@ -0,0 +1,66 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_UNBOUNDED_HPP +#define BOOST_MULTI_INDEX_DETAIL_UNBOUNDED_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include + +namespace boost{ + +namespace multi_index{ + +/* dummy type and variable for use in ordered_index::range() */ + +/* ODR-abiding technique shown at the example attached to + * http://lists.boost.org/Archives/boost/2006/07/108355.php + */ + +namespace detail{class unbounded_helper;} + +detail::unbounded_helper unbounded(detail::unbounded_helper); + +namespace detail{ + +class unbounded_helper +{ + unbounded_helper(){} + unbounded_helper(const unbounded_helper&){} + friend unbounded_helper multi_index::unbounded(unbounded_helper); +}; + +typedef unbounded_helper (*unbounded_type)(unbounded_helper); + +} /* namespace multi_index::detail */ + +inline detail::unbounded_helper unbounded(detail::unbounded_helper) +{ + return detail::unbounded_helper(); +} + +/* tags used in the implementation of range */ + +namespace detail{ + +struct none_unbounded_tag{}; +struct lower_unbounded_tag{}; +struct upper_unbounded_tag{}; +struct both_unbounded_tag{}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/value_compare.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/value_compare.hpp new file mode 100644 index 00000000000..ac42e8779aa --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/value_compare.hpp @@ -0,0 +1,56 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_VALUE_COMPARE_HPP +#define BOOST_MULTI_INDEX_DETAIL_VALUE_COMPARE_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +template +struct value_comparison +{ + typedef Value first_argument_type; + typedef Value second_argument_type; + typedef bool result_type; + + value_comparison( + const KeyFromValue& key_=KeyFromValue(),const Compare& comp_=Compare()): + key(key_),comp(comp_) + { + } + + bool operator()( + typename call_traits::param_type x, + typename call_traits::param_type y)const + { + return comp(key(x),key(y)); + } + +private: + KeyFromValue key; + Compare comp; +}; + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/vartempl_support.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/vartempl_support.hpp new file mode 100644 index 00000000000..06ff430f4be --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/vartempl_support.hpp @@ -0,0 +1,247 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_DETAIL_VARTEMPL_SUPPORT_HPP +#define BOOST_MULTI_INDEX_DETAIL_VARTEMPL_SUPPORT_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +/* Utilities for emulation of variadic template functions. Variadic packs are + * replaced by lists of BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS parameters: + * + * - typename... Args --> BOOST_MULTI_INDEX_TEMPLATE_PARAM_PACK + * - Args&&... args --> BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK + * - std::forward(args)... --> BOOST_MULTI_INDEX_FORWARD_PARAM_PACK + * + * Forwarding emulated with Boost.Move. A template functions foo_imp + * defined in such way accepts *exactly* BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS + * arguments: variable number of arguments is emulated by providing a set of + * overloads foo forwarding to foo_impl with + * + * BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL + * BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG (initial extra arg) + * + * which fill the extra args with boost::multi_index::detail::noarg's. + * boost::multi_index::detail::vartempl_placement_new works the opposite + * way: it acceps a full a pointer x to Value and a + * BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK and forwards to + * new(x) Value(args) where args is the argument pack after discarding + * noarg's. + * + * Emulation decays to the real thing when the compiler supports variadic + * templates and move semantics natively. + */ + +#include + +#if defined(BOOST_NO_CXX11_RVALUE_REFERENCES)||\ + defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if !defined(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS) +#define BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS 5 +#endif + +#define BOOST_MULTI_INDEX_TEMPLATE_PARAM_PACK \ +BOOST_PP_ENUM_PARAMS( \ + BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,typename T) + +#define BOOST_MULTI_INDEX_VARTEMPL_ARG(z,n,_) \ +BOOST_FWD_REF(BOOST_PP_CAT(T,n)) BOOST_PP_CAT(t,n) + +#define BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK \ +BOOST_PP_ENUM( \ + BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS, \ + BOOST_MULTI_INDEX_VARTEMPL_ARG,~) + +#define BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG(z,n,_) \ +boost::forward(BOOST_PP_CAT(t,n)) + +#define BOOST_MULTI_INDEX_FORWARD_PARAM_PACK \ +BOOST_PP_ENUM( \ + BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS, \ + BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG,~) + +namespace boost{namespace multi_index{namespace detail{ +struct noarg{}; +}}} + +/* call vartempl function without args */ + +#define BOOST_MULTI_INDEX_NULL_PARAM_PACK \ +BOOST_PP_ENUM_PARAMS( \ + BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS, \ + boost::multi_index::detail::noarg() BOOST_PP_INTERCEPT) + +#define BOOST_MULTI_INDEX_TEMPLATE_N(n) \ +template + +#define BOOST_MULTI_INDEX_TEMPLATE_0(n) + +#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_AUX(z,n,data) \ +BOOST_PP_IF(n, \ + BOOST_MULTI_INDEX_TEMPLATE_N, \ + BOOST_MULTI_INDEX_TEMPLATE_0)(n) \ +BOOST_PP_SEQ_ELEM(0,data) /* ret */ \ +BOOST_PP_SEQ_ELEM(1,data) /* name_from */ ( \ + BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_ARG,~)) \ +{ \ + return BOOST_PP_SEQ_ELEM(2,data) /* name_to */ ( \ + BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG,~) \ + BOOST_PP_COMMA_IF( \ + BOOST_PP_AND( \ + n,BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n))) \ + BOOST_PP_ENUM_PARAMS( \ + BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n), \ + boost::multi_index::detail::noarg() BOOST_PP_INTERCEPT) \ + ); \ +} + +#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( \ + ret,name_from,name_to) \ +BOOST_PP_REPEAT_FROM_TO( \ + 0,BOOST_PP_ADD(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,1), \ + BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_AUX, \ + (ret)(name_from)(name_to)) + +#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG_AUX( \ + z,n,data) \ +BOOST_PP_IF(n, \ + BOOST_MULTI_INDEX_TEMPLATE_N, \ + BOOST_MULTI_INDEX_TEMPLATE_0)(n) \ +BOOST_PP_SEQ_ELEM(0,data) /* ret */ \ +BOOST_PP_SEQ_ELEM(1,data) /* name_from */ ( \ + BOOST_PP_SEQ_ELEM(3,data) BOOST_PP_SEQ_ELEM(4,data) /* extra arg */\ + BOOST_PP_COMMA_IF(n) \ + BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_ARG,~)) \ +{ \ + return BOOST_PP_SEQ_ELEM(2,data) /* name_to */ ( \ + BOOST_PP_SEQ_ELEM(4,data) /* extra_arg_name */ \ + BOOST_PP_COMMA_IF(n) \ + BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG,~) \ + BOOST_PP_COMMA_IF( \ + BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n)) \ + BOOST_PP_ENUM_PARAMS( \ + BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n), \ + boost::multi_index::detail::noarg() BOOST_PP_INTERCEPT) \ + ); \ +} + +#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( \ + ret,name_from,name_to,extra_arg_type,extra_arg_name) \ +BOOST_PP_REPEAT_FROM_TO( \ + 0,BOOST_PP_ADD(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,1), \ + BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG_AUX, \ + (ret)(name_from)(name_to)(extra_arg_type)(extra_arg_name)) + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +#define BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW_AUX(z,n,name) \ +template< \ + typename Value \ + BOOST_PP_COMMA_IF(n) \ + BOOST_PP_ENUM_PARAMS(n,typename T) \ +> \ +Value* name( \ + Value* x \ + BOOST_PP_COMMA_IF(n) \ + BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_ARG,~) \ + BOOST_PP_COMMA_IF( \ + BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n)) \ + BOOST_PP_ENUM_PARAMS( \ + BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n), \ + BOOST_FWD_REF(noarg) BOOST_PP_INTERCEPT)) \ +{ \ + return new(x) Value( \ + BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG,~)); \ +} + +#define BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW(name) \ +BOOST_PP_REPEAT_FROM_TO( \ + 0,BOOST_PP_ADD(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,1), \ + BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW_AUX, \ + name) + +BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW(vartempl_placement_new) + +#undef BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW_AUX +#undef BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#else + +/* native variadic templates support */ + +#include + +#define BOOST_MULTI_INDEX_TEMPLATE_PARAM_PACK typename... Args +#define BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK Args&&... args +#define BOOST_MULTI_INDEX_FORWARD_PARAM_PACK std::forward(args)... +#define BOOST_MULTI_INDEX_NULL_PARAM_PACK + +#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( \ + ret,name_from,name_to) \ +template ret name_from(Args&&... args) \ +{ \ + return name_to(std::forward(args)...); \ +} + +#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( \ + ret,name_from,name_to,extra_arg_type,extra_arg_name) \ +template ret name_from( \ + extra_arg_type extra_arg_name,Args&&... args) \ +{ \ + return name_to(extra_arg_name,std::forward(args)...); \ +} + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +template +Value* vartempl_placement_new(Value*x,Args&&... args) +{ + return new(x) Value(std::forward(args)...); +} + +} /* namespace multi_index::detail */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/global_fun.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/global_fun.hpp new file mode 100644 index 00000000000..2c13769100c --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/global_fun.hpp @@ -0,0 +1,185 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_GLOBAL_FUN_HPP +#define BOOST_MULTI_INDEX_GLOBAL_FUN_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include + +#if !defined(BOOST_NO_SFINAE) +#include +#endif + +namespace boost{ + +template class reference_wrapper; /* fwd decl. */ + +namespace multi_index{ + +namespace detail{ + +/* global_fun is a read-only key extractor from Value based on a given global + * (or static member) function with signature: + * + * Type f([const] Value [&]); + * + * Additionally, global_fun and const_global_fun are overloaded to support + * referece_wrappers of Value and "chained pointers" to Value's. By chained + * pointer to T we mean a type P such that, given a p of Type P + * *...n...*x is convertible to T&, for some n>=1. + * Examples of chained pointers are raw and smart pointers, iterators and + * arbitrary combinations of these (vg. T** or unique_ptr.) + */ + +template +struct const_ref_global_fun_base +{ + typedef typename remove_reference::type result_type; + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if< + is_convertible,Type>::type +#else + Type +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + Type operator()(Value x)const + { + return PtrToFunction(x); + } + + Type operator()( + const reference_wrapper< + typename remove_reference::type>& x)const + { + return operator()(x.get()); + } + + Type operator()( + const reference_wrapper< + typename remove_const< + typename remove_reference::type>::type>& x + +#if BOOST_WORKAROUND(BOOST_MSVC,==1310) +/* http://lists.boost.org/Archives/boost/2015/10/226135.php */ + ,int=0 +#endif + + )const + { + return operator()(x.get()); + } +}; + +template +struct non_const_ref_global_fun_base +{ + typedef typename remove_reference::type result_type; + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if< + is_convertible,Type>::type +#else + Type +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + Type operator()(Value x)const + { + return PtrToFunction(x); + } + + Type operator()( + const reference_wrapper< + typename remove_reference::type>& x)const + { + return operator()(x.get()); + } +}; + +template +struct non_ref_global_fun_base +{ + typedef typename remove_reference::type result_type; + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if< + is_convertible,Type>::type +#else + Type +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + Type operator()(const Value& x)const + { + return PtrToFunction(x); + } + + Type operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } + + Type operator()( + const reference_wrapper::type>& x)const + { + return operator()(x.get()); + } +}; + +} /* namespace multi_index::detail */ + +template +struct global_fun: + mpl::if_c< + is_reference::value, + typename mpl::if_c< + is_const::type>::value, + detail::const_ref_global_fun_base, + detail::non_const_ref_global_fun_base + >::type, + detail::non_ref_global_fun_base + >::type +{ +}; + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index.hpp new file mode 100644 index 00000000000..352d0c13f17 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index.hpp @@ -0,0 +1,1725 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_HASHED_INDEX_HPP +#define BOOST_MULTI_INDEX_HASHED_INDEX_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) +#include +#endif + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +#include +#endif + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) +#define BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF(x) \ + detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ + detail::make_obj_guard(x,&hashed_index::check_invariant_); \ + BOOST_JOIN(check_invariant_,__LINE__).touch(); +#define BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT \ + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF(*this) +#else +#define BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF(x) +#define BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* hashed_index adds a layer of hashed indexing to a given Super */ + +/* Most of the implementation of unique and non-unique indices is + * shared. We tell from one another on instantiation time by using + * Category tags defined in hash_index_node.hpp. + */ + +template< + typename KeyFromValue,typename Hash,typename Pred, + typename SuperMeta,typename TagList,typename Category +> +class hashed_index: + BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS SuperMeta::type + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + ,public safe_mode::safe_container< + hashed_index > +#endif + +{ +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ + BOOST_WORKAROUND(__MWERKS__,<=0x3003) +/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the + * lifetime of const references bound to temporaries --precisely what + * scopeguards are. + */ + +#pragma parse_mfunc_templ off +#endif + + typedef typename SuperMeta::type super; + +protected: + typedef hashed_index_node< + typename super::node_type,Category> node_type; + +private: + typedef typename node_type::node_alg node_alg; + typedef typename node_type::impl_type node_impl_type; + typedef typename node_impl_type::pointer node_impl_pointer; + typedef typename node_impl_type::base_pointer node_impl_base_pointer; + typedef bucket_array< + typename super::final_allocator_type> bucket_array_type; + +public: + /* types */ + + typedef typename KeyFromValue::result_type key_type; + typedef typename node_type::value_type value_type; + typedef KeyFromValue key_from_value; + typedef Hash hasher; + typedef Pred key_equal; + typedef tuple ctor_args; + typedef typename super::final_allocator_type allocator_type; + typedef typename allocator_type::pointer pointer; + typedef typename allocator_type::const_pointer const_pointer; + typedef typename allocator_type::reference reference; + typedef typename allocator_type::const_reference const_reference; + typedef std::size_t size_type; + typedef std::ptrdiff_t difference_type; + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + typedef safe_mode::safe_iterator< + hashed_index_iterator< + node_type,bucket_array_type, + hashed_index_global_iterator_tag>, + hashed_index> iterator; +#else + typedef hashed_index_iterator< + node_type,bucket_array_type, + hashed_index_global_iterator_tag> iterator; +#endif + + typedef iterator const_iterator; + + typedef hashed_index_iterator< + node_type,bucket_array_type, + hashed_index_local_iterator_tag> local_iterator; + typedef local_iterator const_local_iterator; + + typedef TagList tag_list; + +protected: + typedef typename super::final_node_type final_node_type; + typedef tuples::cons< + ctor_args, + typename super::ctor_args_list> ctor_args_list; + typedef typename mpl::push_front< + typename super::index_type_list, + hashed_index>::type index_type_list; + typedef typename mpl::push_front< + typename super::iterator_type_list, + iterator>::type iterator_type_list; + typedef typename mpl::push_front< + typename super::const_iterator_type_list, + const_iterator>::type const_iterator_type_list; + typedef typename super::copy_map_type copy_map_type; + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + typedef typename super::index_saver_type index_saver_type; + typedef typename super::index_loader_type index_loader_type; +#endif + +private: +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + typedef safe_mode::safe_container< + hashed_index> safe_super; +#endif + + typedef typename call_traits::param_type value_param_type; + typedef typename call_traits< + key_type>::param_type key_param_type; + + /* Needed to avoid commas in BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL + * expansion. + */ + + typedef std::pair emplace_return_type; + +public: + + /* construct/destroy/copy + * Default and copy ctors are in the protected section as indices are + * not supposed to be created on their own. No range ctor either. + */ + + hashed_index& operator=( + const hashed_index& x) + { + this->final()=x.final(); + return *this; + } + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) + hashed_index& operator=( + std::initializer_list list) + { + this->final()=list; + return *this; + } +#endif + + allocator_type get_allocator()const BOOST_NOEXCEPT + { + return this->final().get_allocator(); + } + + /* size and capacity */ + + bool empty()const BOOST_NOEXCEPT{return this->final_empty_();} + size_type size()const BOOST_NOEXCEPT{return this->final_size_();} + size_type max_size()const BOOST_NOEXCEPT{return this->final_max_size_();} + + /* iterators */ + + iterator begin()BOOST_NOEXCEPT + {return make_iterator(node_type::from_impl(header()->next()->prior()));} + const_iterator begin()const BOOST_NOEXCEPT + {return make_iterator(node_type::from_impl(header()->next()->prior()));} + iterator end()BOOST_NOEXCEPT{return make_iterator(header());} + const_iterator end()const BOOST_NOEXCEPT{return make_iterator(header());} + const_iterator cbegin()const BOOST_NOEXCEPT{return begin();} + const_iterator cend()const BOOST_NOEXCEPT{return end();} + + iterator iterator_to(const value_type& x) + { + return make_iterator(node_from_value(&x)); + } + + const_iterator iterator_to(const value_type& x)const + { + return make_iterator(node_from_value(&x)); + } + + /* modifiers */ + + BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( + emplace_return_type,emplace,emplace_impl) + + BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( + iterator,emplace_hint,emplace_hint_impl,iterator,position) + + std::pair insert(const value_type& x) + { + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + std::pair p=this->final_insert_(x); + return std::pair(make_iterator(p.first),p.second); + } + + std::pair insert(BOOST_RV_REF(value_type) x) + { + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + std::pair p=this->final_insert_rv_(x); + return std::pair(make_iterator(p.first),p.second); + } + + iterator insert(iterator position,const value_type& x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + std::pair p=this->final_insert_( + x,static_cast(position.get_node())); + return make_iterator(p.first); + } + + iterator insert(iterator position,BOOST_RV_REF(value_type) x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + std::pair p=this->final_insert_rv_( + x,static_cast(position.get_node())); + return make_iterator(p.first); + } + + template + void insert(InputIterator first,InputIterator last) + { + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + for(;first!=last;++first)this->final_insert_ref_(*first); + } + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) + void insert(std::initializer_list list) + { + insert(list.begin(),list.end()); + } +#endif + + iterator erase(iterator position) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + this->final_erase_(static_cast(position++.get_node())); + return position; + } + + size_type erase(key_param_type k) + { + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + + std::size_t buc=buckets.position(hash_(k)); + for(node_impl_pointer x=buckets.at(buc)->prior(); + x!=node_impl_pointer(0);x=node_alg::next_to_inspect(x)){ + if(eq_(k,key(node_type::from_impl(x)->value()))){ + node_impl_pointer y=end_of_range(x); + size_type s=0; + do{ + node_impl_pointer z=node_alg::after(x); + this->final_erase_( + static_cast(node_type::from_impl(x))); + x=z; + ++s; + }while(x!=y); + return s; + } + } + return 0; + } + + iterator erase(iterator first,iterator last) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); + BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + while(first!=last){ + first=erase(first); + } + return first; + } + + bool replace(iterator position,const value_type& x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + return this->final_replace_( + x,static_cast(position.get_node())); + } + + bool replace(iterator position,BOOST_RV_REF(value_type) x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + return this->final_replace_rv_( + x,static_cast(position.get_node())); + } + + template + bool modify(iterator position,Modifier mod) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + /* MSVC++ 6.0 optimizer on safe mode code chokes if this + * this is not added. Left it for all compilers as it does no + * harm. + */ + + position.detach(); +#endif + + return this->final_modify_( + mod,static_cast(position.get_node())); + } + + template + bool modify(iterator position,Modifier mod,Rollback back_) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + /* MSVC++ 6.0 optimizer on safe mode code chokes if this + * this is not added. Left it for all compilers as it does no + * harm. + */ + + position.detach(); +#endif + + return this->final_modify_( + mod,back_,static_cast(position.get_node())); + } + + template + bool modify_key(iterator position,Modifier mod) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + return modify( + position,modify_key_adaptor(mod,key)); + } + + template + bool modify_key(iterator position,Modifier mod,Rollback back_) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + return modify( + position, + modify_key_adaptor(mod,key), + modify_key_adaptor(back_,key)); + } + + void clear()BOOST_NOEXCEPT + { + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + this->final_clear_(); + } + + void swap(hashed_index& x) + { + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF(x); + this->final_swap_(x.final()); + } + + /* observers */ + + key_from_value key_extractor()const{return key;} + hasher hash_function()const{return hash_;} + key_equal key_eq()const{return eq_;} + + /* lookup */ + + /* Internally, these ops rely on const_iterator being the same + * type as iterator. + */ + + /* Implementation note: When CompatibleKey is consistently promoted to + * KeyFromValue::result_type for equality comparison, the promotion is made + * once in advance to increase efficiency. + */ + + template + iterator find(const CompatibleKey& k)const + { + return find(k,hash_,eq_); + } + + template< + typename CompatibleKey,typename CompatibleHash,typename CompatiblePred + > + iterator find( + const CompatibleKey& k, + const CompatibleHash& hash,const CompatiblePred& eq)const + { + return find( + k,hash,eq,promotes_1st_arg()); + } + + template + size_type count(const CompatibleKey& k)const + { + return count(k,hash_,eq_); + } + + template< + typename CompatibleKey,typename CompatibleHash,typename CompatiblePred + > + size_type count( + const CompatibleKey& k, + const CompatibleHash& hash,const CompatiblePred& eq)const + { + return count( + k,hash,eq,promotes_1st_arg()); + } + + template + std::pair equal_range(const CompatibleKey& k)const + { + return equal_range(k,hash_,eq_); + } + + template< + typename CompatibleKey,typename CompatibleHash,typename CompatiblePred + > + std::pair equal_range( + const CompatibleKey& k, + const CompatibleHash& hash,const CompatiblePred& eq)const + { + return equal_range( + k,hash,eq,promotes_1st_arg()); + } + + /* bucket interface */ + + size_type bucket_count()const BOOST_NOEXCEPT{return buckets.size();} + size_type max_bucket_count()const BOOST_NOEXCEPT{return static_cast(-1);} + + size_type bucket_size(size_type n)const + { + size_type res=0; + for(node_impl_pointer x=buckets.at(n)->prior(); + x!=node_impl_pointer(0);x=node_alg::after_local(x)){ + ++res; + } + return res; + } + + size_type bucket(key_param_type k)const + { + return buckets.position(hash_(k)); + } + + local_iterator begin(size_type n) + { + return const_cast(this)->begin(n); + } + + const_local_iterator begin(size_type n)const + { + node_impl_pointer x=buckets.at(n)->prior(); + if(x==node_impl_pointer(0))return end(n); + return make_local_iterator(node_type::from_impl(x)); + } + + local_iterator end(size_type n) + { + return const_cast(this)->end(n); + } + + const_local_iterator end(size_type)const + { + return make_local_iterator(0); + } + + const_local_iterator cbegin(size_type n)const{return begin(n);} + const_local_iterator cend(size_type n)const{return end(n);} + + local_iterator local_iterator_to(const value_type& x) + { + return make_local_iterator(node_from_value(&x)); + } + + const_local_iterator local_iterator_to(const value_type& x)const + { + return make_local_iterator(node_from_value(&x)); + } + + /* hash policy */ + + float load_factor()const BOOST_NOEXCEPT + {return static_cast(size())/bucket_count();} + float max_load_factor()const BOOST_NOEXCEPT{return mlf;} + void max_load_factor(float z){mlf=z;calculate_max_load();} + + void rehash(size_type n) + { + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + if(size()<=max_load&&n<=bucket_count())return; + + size_type bc =(std::numeric_limits::max)(); + float fbc=static_cast(1+size()/mlf); + if(bc>fbc){ + bc=static_cast(fbc); + if(bc(std::ceil(static_cast(n)/mlf))); + } + +BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: + hashed_index(const ctor_args_list& args_list,const allocator_type& al): + super(args_list.get_tail(),al), + key(tuples::get<1>(args_list.get_head())), + hash_(tuples::get<2>(args_list.get_head())), + eq_(tuples::get<3>(args_list.get_head())), + buckets(al,header()->impl(),tuples::get<0>(args_list.get_head())), + mlf(1.0f) + { + calculate_max_load(); + } + + hashed_index( + const hashed_index& x): + super(x), + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super(), +#endif + + key(x.key), + hash_(x.hash_), + eq_(x.eq_), + buckets(x.get_allocator(),header()->impl(),x.buckets.size()), + mlf(x.mlf), + max_load(x.max_load) + { + /* Copy ctor just takes the internal configuration objects from x. The rest + * is done in subsequent call to copy_(). + */ + } + + hashed_index( + const hashed_index& x, + do_not_copy_elements_tag): + super(x,do_not_copy_elements_tag()), + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super(), +#endif + + key(x.key), + hash_(x.hash_), + eq_(x.eq_), + buckets(x.get_allocator(),header()->impl(),0), + mlf(1.0f) + { + calculate_max_load(); + } + + ~hashed_index() + { + /* the container is guaranteed to be empty by now */ + } + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + iterator make_iterator(node_type* node) + { + return iterator(node,this); + } + + const_iterator make_iterator(node_type* node)const + { + return const_iterator(node,const_cast(this)); + } +#else + iterator make_iterator(node_type* node) + { + return iterator(node); + } + + const_iterator make_iterator(node_type* node)const + { + return const_iterator(node); + } +#endif + + local_iterator make_local_iterator(node_type* node) + { + return local_iterator(node); + } + + const_local_iterator make_local_iterator(node_type* node)const + { + return const_local_iterator(node); + } + + void copy_( + const hashed_index& x, + const copy_map_type& map) + { + copy_(x,map,Category()); + } + + void copy_( + const hashed_index& x, + const copy_map_type& map,hashed_unique_tag) + { + if(x.size()!=0){ + node_impl_pointer end_org=x.header()->impl(), + org=end_org, + cpy=header()->impl(); + do{ + node_impl_pointer prev_org=org->prior(), + prev_cpy= + static_cast(map.find(static_cast( + node_type::from_impl(prev_org))))->impl(); + cpy->prior()=prev_cpy; + if(node_alg::is_first_of_bucket(org)){ + node_impl_base_pointer buc_org=prev_org->next(), + buc_cpy= + buckets.begin()+(buc_org-x.buckets.begin()); + prev_cpy->next()=buc_cpy; + buc_cpy->prior()=cpy; + } + else{ + prev_cpy->next()=node_impl_type::base_pointer_from(cpy); + } + org=prev_org; + cpy=prev_cpy; + }while(org!=end_org); + } + + super::copy_(x,map); + } + + void copy_( + const hashed_index& x, + const copy_map_type& map,hashed_non_unique_tag) + { + if(x.size()!=0){ + node_impl_pointer end_org=x.header()->impl(), + org=end_org, + cpy=header()->impl(); + do{ + node_impl_pointer next_org=node_alg::after(org), + next_cpy= + static_cast(map.find(static_cast( + node_type::from_impl(next_org))))->impl(); + if(node_alg::is_first_of_bucket(next_org)){ + node_impl_base_pointer buc_org=org->next(), + buc_cpy= + buckets.begin()+(buc_org-x.buckets.begin()); + cpy->next()=buc_cpy; + buc_cpy->prior()=next_cpy; + next_cpy->prior()=cpy; + } + else{ + if(org->next()==node_impl_type::base_pointer_from(next_org)){ + cpy->next()=node_impl_type::base_pointer_from(next_cpy); + } + else{ + cpy->next()= + node_impl_type::base_pointer_from( + static_cast(map.find(static_cast( + node_type::from_impl( + node_impl_type::pointer_from(org->next())))))->impl()); + } + + if(next_org->prior()!=org){ + next_cpy->prior()= + static_cast(map.find(static_cast( + node_type::from_impl(next_org->prior()))))->impl(); + } + else{ + next_cpy->prior()=cpy; + } + } + org=next_org; + cpy=next_cpy; + }while(org!=end_org); + } + + super::copy_(x,map); + } + + template + final_node_type* insert_( + value_param_type v,final_node_type*& x,Variant variant) + { + reserve_for_insert(size()+1); + + std::size_t buc=find_bucket(v); + link_info pos(buckets.at(buc)); + if(!link_point(v,pos)){ + return static_cast( + node_type::from_impl(node_impl_type::pointer_from(pos))); + } + + final_node_type* res=super::insert_(v,x,variant); + if(res==x)link(static_cast(x),pos); + return res; + } + + template + final_node_type* insert_( + value_param_type v,node_type* position,final_node_type*& x,Variant variant) + { + reserve_for_insert(size()+1); + + std::size_t buc=find_bucket(v); + link_info pos(buckets.at(buc)); + if(!link_point(v,pos)){ + return static_cast( + node_type::from_impl(node_impl_type::pointer_from(pos))); + } + + final_node_type* res=super::insert_(v,position,x,variant); + if(res==x)link(static_cast(x),pos); + return res; + } + + void erase_(node_type* x) + { + unlink(x); + super::erase_(x); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + } + + void delete_all_nodes_() + { + delete_all_nodes_(Category()); + } + + void delete_all_nodes_(hashed_unique_tag) + { + for(node_impl_pointer x_end=header()->impl(),x=x_end->prior();x!=x_end;){ + node_impl_pointer y=x->prior(); + this->final_delete_node_( + static_cast(node_type::from_impl(x))); + x=y; + } + } + + void delete_all_nodes_(hashed_non_unique_tag) + { + for(node_impl_pointer x_end=header()->impl(),x=x_end->prior();x!=x_end;){ + node_impl_pointer y=x->prior(); + if(y->next()!=node_impl_type::base_pointer_from(x)&& + y->next()->prior()!=x){ /* n-1 of group */ + /* Make the second node prior() pointer back-linked so that it won't + * refer to a deleted node when the time for its own destruction comes. + */ + + node_impl_pointer first=node_impl_type::pointer_from(y->next()); + first->next()->prior()=first; + } + this->final_delete_node_( + static_cast(node_type::from_impl(x))); + x=y; + } + } + + void clear_() + { + super::clear_(); + buckets.clear(header()->impl()); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super::detach_dereferenceable_iterators(); +#endif + } + + void swap_( + hashed_index& x) + { + std::swap(key,x.key); + std::swap(hash_,x.hash_); + std::swap(eq_,x.eq_); + buckets.swap(x.buckets); + std::swap(mlf,x.mlf); + std::swap(max_load,x.max_load); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super::swap(x); +#endif + + super::swap_(x); + } + + void swap_elements_( + hashed_index& x) + { + buckets.swap(x.buckets); + std::swap(mlf,x.mlf); + std::swap(max_load,x.max_load); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super::swap(x); +#endif + + super::swap_elements_(x); + } + + template + bool replace_(value_param_type v,node_type* x,Variant variant) + { + if(eq_(key(v),key(x->value()))){ + return super::replace_(v,x,variant); + } + + unlink_undo undo; + unlink(x,undo); + + BOOST_TRY{ + std::size_t buc=find_bucket(v); + link_info pos(buckets.at(buc)); + if(link_point(v,pos)&&super::replace_(v,x,variant)){ + link(x,pos); + return true; + } + undo(); + return false; + } + BOOST_CATCH(...){ + undo(); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + bool modify_(node_type* x) + { + std::size_t buc; + bool b; + BOOST_TRY{ + buc=find_bucket(x->value()); + b=in_place(x->impl(),key(x->value()),buc); + } + BOOST_CATCH(...){ + erase_(x); + BOOST_RETHROW; + } + BOOST_CATCH_END + if(!b){ + unlink(x); + BOOST_TRY{ + link_info pos(buckets.at(buc)); + if(!link_point(x->value(),pos)){ + super::erase_(x); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + return false; + } + link(x,pos); + } + BOOST_CATCH(...){ + super::erase_(x); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + BOOST_TRY{ + if(!super::modify_(x)){ + unlink(x); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + return false; + } + else return true; + } + BOOST_CATCH(...){ + unlink(x); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + bool modify_rollback_(node_type* x) + { + std::size_t buc=find_bucket(x->value()); + if(in_place(x->impl(),key(x->value()),buc)){ + return super::modify_rollback_(x); + } + + unlink_undo undo; + unlink(x,undo); + + BOOST_TRY{ + link_info pos(buckets.at(buc)); + if(link_point(x->value(),pos)&&super::modify_rollback_(x)){ + link(x,pos); + return true; + } + undo(); + return false; + } + BOOST_CATCH(...){ + undo(); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + /* comparison */ + +#if !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) + /* defect macro refers to class, not function, templates, but anyway */ + + template + friend bool operator==( + const hashed_index&,const hashed_index& y); +#endif + + bool equals(const hashed_index& x)const{return equals(x,Category());} + + bool equals(const hashed_index& x,hashed_unique_tag)const + { + if(size()!=x.size())return false; + for(const_iterator it=begin(),it_end=end(),it2_end=x.end(); + it!=it_end;++it){ + const_iterator it2=x.find(key(*it)); + if(it2==it2_end||!(*it==*it2))return false; + } + return true; + } + + bool equals(const hashed_index& x,hashed_non_unique_tag)const + { + if(size()!=x.size())return false; + for(const_iterator it=begin(),it_end=end();it!=it_end;){ + const_iterator it2,it2_last; + boost::tie(it2,it2_last)=x.equal_range(key(*it)); + if(it2==it2_last)return false; + + const_iterator it_last=make_iterator( + node_type::from_impl(end_of_range(it.get_node()->impl()))); + if(std::distance(it,it_last)!=std::distance(it2,it2_last))return false; + + /* From is_permutation code in + * http://www.open-std.org/jtc1/sc22/wg21/docs/papers/2010/n3068.pdf + */ + + for(;it!=it_last;++it,++it2){ + if(!(*it==*it2))break; + } + if(it!=it_last){ + for(const_iterator scan=it;scan!=it_last;++scan){ + if(std::find(it,scan,*scan)!=scan)continue; + std::ptrdiff_t matches=std::count(it2,it2_last,*scan); + if(matches==0||matches!=std::count(scan,it_last,*scan))return false; + } + it=it_last; + } + } + return true; + } + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + /* serialization */ + + template + void save_( + Archive& ar,const unsigned int version,const index_saver_type& sm)const + { + ar< + void load_(Archive& ar,const unsigned int version,const index_loader_type& lm) + { + ar>>serialization::make_nvp("position",buckets); + super::load_(ar,version,lm); + } +#endif + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) + /* invariant stuff */ + + bool invariant_()const + { + if(size()==0||begin()==end()){ + if(size()!=0||begin()!=end())return false; + } + else{ + size_type s0=0; + for(const_iterator it=begin(),it_end=end();it!=it_end;++it,++s0){} + if(s0!=size())return false; + + size_type s1=0; + for(size_type buc=0;bucfinal_check_invariant_();} +#endif + +private: + node_type* header()const{return this->final_header();} + + std::size_t find_bucket(value_param_type v)const + { + return bucket(key(v)); + } + + struct link_info_non_unique + { + link_info_non_unique(node_impl_base_pointer pos): + first(pos),last(node_impl_base_pointer(0)){} + + operator const node_impl_base_pointer&()const{return this->first;} + + node_impl_base_pointer first,last; + }; + + typedef typename mpl::if_< + is_same, + node_impl_base_pointer, + link_info_non_unique + >::type link_info; + + bool link_point(value_param_type v,link_info& pos) + { + return link_point(v,pos,Category()); + } + + bool link_point( + value_param_type v,node_impl_base_pointer& pos,hashed_unique_tag) + { + for(node_impl_pointer x=pos->prior();x!=node_impl_pointer(0); + x=node_alg::after_local(x)){ + if(eq_(key(v),key(node_type::from_impl(x)->value()))){ + pos=node_impl_type::base_pointer_from(x); + return false; + } + } + return true; + } + + bool link_point( + value_param_type v,link_info_non_unique& pos,hashed_non_unique_tag) + { + for(node_impl_pointer x=pos.first->prior();x!=node_impl_pointer(0); + x=node_alg::next_to_inspect(x)){ + if(eq_(key(v),key(node_type::from_impl(x)->value()))){ + pos.first=node_impl_type::base_pointer_from(x); + pos.last=node_impl_type::base_pointer_from(last_of_range(x)); + return true; + } + } + return true; + } + + node_impl_pointer last_of_range(node_impl_pointer x)const + { + return last_of_range(x,Category()); + } + + node_impl_pointer last_of_range(node_impl_pointer x,hashed_unique_tag)const + { + return x; + } + + node_impl_pointer last_of_range( + node_impl_pointer x,hashed_non_unique_tag)const + { + node_impl_base_pointer y=x->next(); + node_impl_pointer z=y->prior(); + if(z==x){ /* range of size 1 or 2 */ + node_impl_pointer yy=node_impl_type::pointer_from(y); + return + eq_( + key(node_type::from_impl(x)->value()), + key(node_type::from_impl(yy)->value()))?yy:x; + } + else if(z->prior()==x) /* last of bucket */ + return x; + else /* group of size>2 */ + return z; + } + + node_impl_pointer end_of_range(node_impl_pointer x)const + { + return end_of_range(x,Category()); + } + + node_impl_pointer end_of_range(node_impl_pointer x,hashed_unique_tag)const + { + return node_alg::after(last_of_range(x)); + } + + node_impl_pointer end_of_range( + node_impl_pointer x,hashed_non_unique_tag)const + { + node_impl_base_pointer y=x->next(); + node_impl_pointer z=y->prior(); + if(z==x){ /* range of size 1 or 2 */ + node_impl_pointer yy=node_impl_type::pointer_from(y); + if(!eq_( + key(node_type::from_impl(x)->value()), + key(node_type::from_impl(yy)->value())))yy=x; + return yy->next()->prior()==yy? + node_impl_type::pointer_from(yy->next()): + yy->next()->prior(); + } + else if(z->prior()==x) /* last of bucket */ + return z; + else /* group of size>2 */ + return z->next()->prior()==z? + node_impl_type::pointer_from(z->next()): + z->next()->prior(); + } + + void link(node_type* x,const link_info& pos) + { + link(x,pos,Category()); + } + + void link(node_type* x,node_impl_base_pointer pos,hashed_unique_tag) + { + node_alg::link(x->impl(),pos,header()->impl()); + } + + void link(node_type* x,const link_info_non_unique& pos,hashed_non_unique_tag) + { + if(pos.last==node_impl_base_pointer(0)){ + node_alg::link(x->impl(),pos.first,header()->impl()); + } + else{ + node_alg::link( + x->impl(), + node_impl_type::pointer_from(pos.first), + node_impl_type::pointer_from(pos.last)); + } + } + + void unlink(node_type* x) + { + node_alg::unlink(x->impl()); + } + + typedef typename node_alg::unlink_undo unlink_undo; + + void unlink(node_type* x,unlink_undo& undo) + { + node_alg::unlink(x->impl(),undo); + } + + void calculate_max_load() + { + float fml=static_cast(mlf*static_cast(bucket_count())); + max_load=(std::numeric_limits::max)(); + if(max_load>fml)max_load=static_cast(fml); + } + + void reserve_for_insert(size_type n) + { + if(n>max_load){ + size_type bc =(std::numeric_limits::max)(); + float fbc=static_cast(1+static_cast(n)/mlf); + if(bc>fbc)bc =static_cast(fbc); + unchecked_rehash(bc); + } + } + + void unchecked_rehash(size_type n){unchecked_rehash(n,Category());} + + void unchecked_rehash(size_type n,hashed_unique_tag) + { + node_impl_type cpy_end_node; + node_impl_pointer cpy_end=node_impl_pointer(&cpy_end_node), + end_=header()->impl(); + bucket_array_type buckets_cpy(get_allocator(),cpy_end,n); + + if(size()!=0){ + auto_space< + std::size_t,allocator_type> hashes(get_allocator(),size()); + auto_space< + node_impl_pointer,allocator_type> node_ptrs(get_allocator(),size()); + std::size_t i=0,size_=size(); + bool within_bucket=false; + BOOST_TRY{ + for(;i!=size_;++i){ + node_impl_pointer x=end_->prior(); + + /* only this can possibly throw */ + std::size_t h=hash_(key(node_type::from_impl(x)->value())); + + hashes.data()[i]=h; + node_ptrs.data()[i]=x; + within_bucket=!node_alg::unlink_last(end_); + node_alg::link(x,buckets_cpy.at(buckets_cpy.position(h)),cpy_end); + } + } + BOOST_CATCH(...){ + if(i!=0){ + std::size_t prev_buc=buckets.position(hashes.data()[i-1]); + if(!within_bucket)prev_buc=~prev_buc; + + for(std::size_t j=i;j--;){ + std::size_t buc=buckets.position(hashes.data()[j]); + node_impl_pointer x=node_ptrs.data()[j]; + if(buc==prev_buc)node_alg::append(x,end_); + else node_alg::link(x,buckets.at(buc),end_); + prev_buc=buc; + } + } + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + end_->prior()=cpy_end->prior()!=cpy_end?cpy_end->prior():end_; + end_->next()=cpy_end->next(); + end_->prior()->next()->prior()=end_->next()->prior()->prior()=end_; + buckets.swap(buckets_cpy); + calculate_max_load(); + } + + void unchecked_rehash(size_type n,hashed_non_unique_tag) + { + node_impl_type cpy_end_node; + node_impl_pointer cpy_end=node_impl_pointer(&cpy_end_node), + end_=header()->impl(); + bucket_array_type buckets_cpy(get_allocator(),cpy_end,n); + + if(size()!=0){ + auto_space< + std::size_t,allocator_type> hashes(get_allocator(),size()); + auto_space< + node_impl_pointer,allocator_type> node_ptrs(get_allocator(),size()); + std::size_t i=0; + bool within_bucket=false; + BOOST_TRY{ + for(;;++i){ + node_impl_pointer x=end_->prior(); + if(x==end_)break; + + /* only this can possibly throw */ + std::size_t h=hash_(key(node_type::from_impl(x)->value())); + + hashes.data()[i]=h; + node_ptrs.data()[i]=x; + std::pair p= + node_alg::unlink_last_group(end_); + node_alg::link_range( + p.first,x,buckets_cpy.at(buckets_cpy.position(h)),cpy_end); + within_bucket=!(p.second); + } + } + BOOST_CATCH(...){ + if(i!=0){ + std::size_t prev_buc=buckets.position(hashes.data()[i-1]); + if(!within_bucket)prev_buc=~prev_buc; + + for(std::size_t j=i;j--;){ + std::size_t buc=buckets.position(hashes.data()[j]); + node_impl_pointer x=node_ptrs.data()[j], + y= + x->prior()->next()!=node_impl_type::base_pointer_from(x)&& + x->prior()->next()->prior()!=x? + node_impl_type::pointer_from(x->prior()->next()):x; + node_alg::unlink_range(y,x); + if(buc==prev_buc)node_alg::append_range(y,x,end_); + else node_alg::link_range(y,x,buckets.at(buc),end_); + prev_buc=buc; + } + } + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + end_->prior()=cpy_end->prior()!=cpy_end?cpy_end->prior():end_; + end_->next()=cpy_end->next(); + end_->prior()->next()->prior()=end_->next()->prior()->prior()=end_; + buckets.swap(buckets_cpy); + calculate_max_load(); + } + + bool in_place(node_impl_pointer x,key_param_type k,std::size_t buc)const + { + return in_place(x,k,buc,Category()); + } + + bool in_place( + node_impl_pointer x,key_param_type k,std::size_t buc, + hashed_unique_tag)const + { + bool found=false; + for(node_impl_pointer y=buckets.at(buc)->prior(); + y!=node_impl_pointer(0);y=node_alg::after_local(y)){ + if(y==x)found=true; + else if(eq_(k,key(node_type::from_impl(y)->value())))return false; + } + return found; + } + + bool in_place( + node_impl_pointer x,key_param_type k,std::size_t buc, + hashed_non_unique_tag)const + { + bool found=false; + int range_size=0; + for(node_impl_pointer y=buckets.at(buc)->prior();y!=node_impl_pointer(0);){ + if(node_alg::is_first_of_group(y)){ /* group of 3 or more */ + if(y==x){ + /* in place <-> equal to some other member of the group */ + return eq_( + k, + key(node_type::from_impl( + node_impl_type::pointer_from(y->next()))->value())); + } + else{ + node_impl_pointer z= + node_alg::after_local(y->next()->prior()); /* end of range */ + if(eq_(k,key(node_type::from_impl(y)->value()))){ + if(found)return false; /* x lies outside */ + do{ + if(y==x)return true; + y=node_alg::after_local(y); + }while(y!=z); + return false; /* x not found */ + } + else{ + if(range_size==1&&!found)return false; + if(range_size==2)return found; + range_size=0; + y=z; /* skip range (and potentially x, too, which is fine) */ + } + } + } + else{ /* group of 1 or 2 */ + if(y==x){ + if(range_size==1)return true; + range_size=1; + found=true; + } + else if(eq_(k,key(node_type::from_impl(y)->value()))){ + if(range_size==0&&found)return false; + if(range_size==1&&!found)return false; + if(range_size==2)return false; + ++range_size; + } + else{ + if(range_size==1&&!found)return false; + if(range_size==2)return found; + range_size=0; + } + y=node_alg::after_local(y); + } + } + return found; + } + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + void detach_iterators(node_type* x) + { + iterator it=make_iterator(x); + safe_mode::detach_equivalent_iterators(it); + } +#endif + + template + std::pair emplace_impl(BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + std::pairp= + this->final_emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + return std::pair(make_iterator(p.first),p.second); + } + + template + iterator emplace_hint_impl( + iterator position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; + std::pairp= + this->final_emplace_hint_( + static_cast(position.get_node()), + BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + return make_iterator(p.first); + } + + template< + typename CompatibleHash,typename CompatiblePred + > + iterator find( + const key_type& k, + const CompatibleHash& hash,const CompatiblePred& eq,mpl::true_)const + { + return find(k,hash,eq,mpl::false_()); + } + + template< + typename CompatibleKey,typename CompatibleHash,typename CompatiblePred + > + iterator find( + const CompatibleKey& k, + const CompatibleHash& hash,const CompatiblePred& eq,mpl::false_)const + { + std::size_t buc=buckets.position(hash(k)); + for(node_impl_pointer x=buckets.at(buc)->prior(); + x!=node_impl_pointer(0);x=node_alg::next_to_inspect(x)){ + if(eq(k,key(node_type::from_impl(x)->value()))){ + return make_iterator(node_type::from_impl(x)); + } + } + return end(); + } + + template< + typename CompatibleHash,typename CompatiblePred + > + size_type count( + const key_type& k, + const CompatibleHash& hash,const CompatiblePred& eq,mpl::true_)const + { + return count(k,hash,eq,mpl::false_()); + } + + template< + typename CompatibleKey,typename CompatibleHash,typename CompatiblePred + > + size_type count( + const CompatibleKey& k, + const CompatibleHash& hash,const CompatiblePred& eq,mpl::false_)const + { + std::size_t buc=buckets.position(hash(k)); + for(node_impl_pointer x=buckets.at(buc)->prior(); + x!=node_impl_pointer(0);x=node_alg::next_to_inspect(x)){ + if(eq(k,key(node_type::from_impl(x)->value()))){ + size_type res=0; + node_impl_pointer y=end_of_range(x); + do{ + ++res; + x=node_alg::after(x); + }while(x!=y); + return res; + } + } + return 0; + } + + template< + typename CompatibleHash,typename CompatiblePred + > + std::pair equal_range( + const key_type& k, + const CompatibleHash& hash,const CompatiblePred& eq,mpl::true_)const + { + return equal_range(k,hash,eq,mpl::false_()); + } + + template< + typename CompatibleKey,typename CompatibleHash,typename CompatiblePred + > + std::pair equal_range( + const CompatibleKey& k, + const CompatibleHash& hash,const CompatiblePred& eq,mpl::false_)const + { + std::size_t buc=buckets.position(hash(k)); + for(node_impl_pointer x=buckets.at(buc)->prior(); + x!=node_impl_pointer(0);x=node_alg::next_to_inspect(x)){ + if(eq(k,key(node_type::from_impl(x)->value()))){ + return std::pair( + make_iterator(node_type::from_impl(x)), + make_iterator(node_type::from_impl(end_of_range(x)))); + } + } + return std::pair(end(),end()); + } + + key_from_value key; + hasher hash_; + key_equal eq_; + bucket_array_type buckets; + float mlf; + size_type max_load; + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ + BOOST_WORKAROUND(__MWERKS__,<=0x3003) +#pragma parse_mfunc_templ reset +#endif +}; + +/* comparison */ + +template< + typename KeyFromValue,typename Hash,typename Pred, + typename SuperMeta,typename TagList,typename Category +> +bool operator==( + const hashed_index& x, + const hashed_index& y) +{ + return x.equals(y); +} + +template< + typename KeyFromValue,typename Hash,typename Pred, + typename SuperMeta,typename TagList,typename Category +> +bool operator!=( + const hashed_index& x, + const hashed_index& y) +{ + return !(x==y); +} + +/* specialized algorithms */ + +template< + typename KeyFromValue,typename Hash,typename Pred, + typename SuperMeta,typename TagList,typename Category +> +void swap( + hashed_index& x, + hashed_index& y) +{ + x.swap(y); +} + +} /* namespace multi_index::detail */ + +/* hashed index specifiers */ + +template +struct hashed_unique +{ + typedef typename detail::hashed_index_args< + Arg1,Arg2,Arg3,Arg4> index_args; + typedef typename index_args::tag_list_type::type tag_list_type; + typedef typename index_args::key_from_value_type key_from_value_type; + typedef typename index_args::hash_type hash_type; + typedef typename index_args::pred_type pred_type; + + template + struct node_class + { + typedef detail::hashed_index_node type; + }; + + template + struct index_class + { + typedef detail::hashed_index< + key_from_value_type,hash_type,pred_type, + SuperMeta,tag_list_type,detail::hashed_unique_tag> type; + }; +}; + +template +struct hashed_non_unique +{ + typedef typename detail::hashed_index_args< + Arg1,Arg2,Arg3,Arg4> index_args; + typedef typename index_args::tag_list_type::type tag_list_type; + typedef typename index_args::key_from_value_type key_from_value_type; + typedef typename index_args::hash_type hash_type; + typedef typename index_args::pred_type pred_type; + + template + struct node_class + { + typedef detail::hashed_index_node< + Super,detail::hashed_non_unique_tag> type; + }; + + template + struct index_class + { + typedef detail::hashed_index< + key_from_value_type,hash_type,pred_type, + SuperMeta,tag_list_type,detail::hashed_non_unique_tag> type; + }; +}; + +} /* namespace multi_index */ + +} /* namespace boost */ + +/* Boost.Foreach compatibility */ + +template< + typename KeyFromValue,typename Hash,typename Pred, + typename SuperMeta,typename TagList,typename Category +> +inline boost::mpl::true_* boost_foreach_is_noncopyable( + boost::multi_index::detail::hashed_index< + KeyFromValue,Hash,Pred,SuperMeta,TagList,Category>*&, + boost_foreach_argument_dependent_lookup_hack) +{ + return 0; +} + +#undef BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT +#undef BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index_fwd.hpp new file mode 100644 index 00000000000..d77e36c321b --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index_fwd.hpp @@ -0,0 +1,74 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_HASHED_INDEX_FWD_HPP +#define BOOST_MULTI_INDEX_HASHED_INDEX_FWD_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +template< + typename KeyFromValue,typename Hash,typename Pred, + typename SuperMeta,typename TagList,typename Category +> +class hashed_index; + +template< + typename KeyFromValue,typename Hash,typename Pred, + typename SuperMeta,typename TagList,typename Category +> +bool operator==( + const hashed_index& x, + const hashed_index& y); + +template< + typename KeyFromValue,typename Hash,typename Pred, + typename SuperMeta,typename TagList,typename Category +> +bool operator!=( + const hashed_index& x, + const hashed_index& y); + +template< + typename KeyFromValue,typename Hash,typename Pred, + typename SuperMeta,typename TagList,typename Category +> +void swap( + hashed_index& x, + hashed_index& y); + +} /* namespace multi_index::detail */ + +/* hashed_index specifiers */ + +template< + typename Arg1,typename Arg2=mpl::na, + typename Arg3=mpl::na,typename Arg4=mpl::na +> +struct hashed_unique; + +template< + typename Arg1,typename Arg2=mpl::na, + typename Arg3=mpl::na,typename Arg4=mpl::na +> +struct hashed_non_unique; + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/identity.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/identity.hpp new file mode 100644 index 00000000000..6c832ce1562 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/identity.hpp @@ -0,0 +1,145 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_IDENTITY_HPP +#define BOOST_MULTI_INDEX_IDENTITY_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include +#include +#include +#include +#include +#include +#include + +#if !defined(BOOST_NO_SFINAE) +#include +#endif + +namespace boost{ + +template class reference_wrapper; /* fwd decl. */ + +namespace multi_index{ + +namespace detail{ + +/* identity is a do-nothing key extractor that returns the [const] Type& + * object passed. + * Additionally, identity is overloaded to support referece_wrappers + * of Type and "chained pointers" to Type's. By chained pointer to Type we + * mean a type P such that, given a p of type P + * *...n...*x is convertible to Type&, for some n>=1. + * Examples of chained pointers are raw and smart pointers, iterators and + * arbitrary combinations of these (vg. Type** or unique_ptr.) + */ + +template +struct const_identity_base +{ + typedef Type result_type; + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if,Type&>::type +#else + Type& +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + Type& operator()(Type& x)const + { + return x; + } + + Type& operator()(const reference_wrapper& x)const + { + return x.get(); + } + + Type& operator()( + const reference_wrapper::type>& x + +#if BOOST_WORKAROUND(BOOST_MSVC,==1310) +/* http://lists.boost.org/Archives/boost/2015/10/226135.php */ + ,int=0 +#endif + + )const + { + return x.get(); + } +}; + +template +struct non_const_identity_base +{ + typedef Type result_type; + + /* templatized for pointer-like types */ + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if< + is_convertible,Type&>::type +#else + Type& +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + const Type& operator()(const Type& x)const + { + return x; + } + + Type& operator()(Type& x)const + { + return x; + } + + const Type& operator()(const reference_wrapper& x)const + { + return x.get(); + } + + Type& operator()(const reference_wrapper& x)const + { + return x.get(); + } +}; + +} /* namespace multi_index::detail */ + +template +struct identity: + mpl::if_c< + is_const::value, + detail::const_identity_base,detail::non_const_identity_base + >::type +{ +}; + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/identity_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/identity_fwd.hpp new file mode 100644 index 00000000000..af6bd55ef5f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/identity_fwd.hpp @@ -0,0 +1,26 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_IDENTITY_FWD_HPP +#define BOOST_MULTI_INDEX_IDENTITY_FWD_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +namespace boost{ + +namespace multi_index{ + +template struct identity; + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/indexed_by.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/indexed_by.hpp new file mode 100644 index 00000000000..d2217e39166 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/indexed_by.hpp @@ -0,0 +1,68 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_INDEXED_BY_HPP +#define BOOST_MULTI_INDEX_INDEXED_BY_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include + +/* An alias to mpl::vector used to hide MPL from the user. + * indexed_by contains the index specifiers for instantiation + * of a multi_index_container. + */ + +/* This user_definable macro limits the number of elements of an index list; + * useful for shortening resulting symbol names (MSVC++ 6.0, for instance, + * has problems coping with very long symbol names.) + */ + +#if !defined(BOOST_MULTI_INDEX_LIMIT_INDEXED_BY_SIZE) +#define BOOST_MULTI_INDEX_LIMIT_INDEXED_BY_SIZE BOOST_MPL_LIMIT_VECTOR_SIZE +#endif + +#if BOOST_MULTI_INDEX_LIMIT_INDEXED_BY_SIZE +struct indexed_by: + mpl::vector +{ +}; + +} /* namespace multi_index */ + +} /* namespace boost */ + +#undef BOOST_MULTI_INDEX_INDEXED_BY_TEMPLATE_PARM +#undef BOOST_MULTI_INDEX_INDEXED_BY_SIZE + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/key_extractors.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/key_extractors.hpp new file mode 100644 index 00000000000..60179ba2339 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/key_extractors.hpp @@ -0,0 +1,22 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_KEY_EXTRACTORS_HPP +#define BOOST_MULTI_INDEX_KEY_EXTRACTORS_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include +#include +#include +#include +#include + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/mem_fun.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/mem_fun.hpp new file mode 100644 index 00000000000..111c386c5f5 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/mem_fun.hpp @@ -0,0 +1,205 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_MEM_FUN_HPP +#define BOOST_MULTI_INDEX_MEM_FUN_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include + +#if !defined(BOOST_NO_SFINAE) +#include +#endif + +namespace boost{ + +template class reference_wrapper; /* fwd decl. */ + +namespace multi_index{ + +/* mem_fun implements a read-only key extractor based on a given non-const + * member function of a class. + * const_mem_fun does the same for const member functions. + * Additionally, mem_fun and const_mem_fun are overloaded to support + * referece_wrappers of T and "chained pointers" to T's. By chained pointer + * to T we mean a type P such that, given a p of Type P + * *...n...*x is convertible to T&, for some n>=1. + * Examples of chained pointers are raw and smart pointers, iterators and + * arbitrary combinations of these (vg. T** or unique_ptr.) + */ + +template +struct const_mem_fun +{ + typedef typename remove_reference::type result_type; + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if< + is_convertible,Type>::type +#else + Type +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + Type operator()(const Class& x)const + { + return (x.*PtrToMemberFunction)(); + } + + Type operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } + + Type operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } +}; + +template +struct mem_fun +{ + typedef typename remove_reference::type result_type; + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if< + is_convertible,Type>::type +#else + Type +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + Type operator()(Class& x)const + { + return (x.*PtrToMemberFunction)(); + } + + Type operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } +}; + +/* MSVC++ 6.0 has problems with const member functions as non-type template + * parameters, somehow it takes them as non-const. const_mem_fun_explicit + * workarounds this deficiency by accepting an extra type parameter that + * specifies the signature of the member function. The workaround was found at: + * Daniel, C.:"Re: weird typedef problem in VC", + * news:microsoft.public.vc.language, 21st nov 2002, + * http://groups.google.com/groups? + * hl=en&lr=&ie=UTF-8&selm=ukwvg3O0BHA.1512%40tkmsftngp05 + * + * MSVC++ 6.0 support has been dropped and [const_]mem_fun_explicit is + * deprecated. + */ + +template< + class Class,typename Type, + typename PtrToMemberFunctionType,PtrToMemberFunctionType PtrToMemberFunction> +struct const_mem_fun_explicit +{ + typedef typename remove_reference::type result_type; + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if< + is_convertible,Type>::type +#else + Type +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + Type operator()(const Class& x)const + { + return (x.*PtrToMemberFunction)(); + } + + Type operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } + + Type operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } +}; + +template< + class Class,typename Type, + typename PtrToMemberFunctionType,PtrToMemberFunctionType PtrToMemberFunction> +struct mem_fun_explicit +{ + typedef typename remove_reference::type result_type; + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if< + is_convertible,Type>::type +#else + Type +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + Type operator()(Class& x)const + { + return (x.*PtrToMemberFunction)(); + } + + Type operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } +}; + +/* BOOST_MULTI_INDEX_CONST_MEM_FUN and BOOST_MULTI_INDEX_MEM_FUN used to + * resolve to [const_]mem_fun_explicit for MSVC++ 6.0 and to + * [const_]mem_fun otherwise. Support for this compiler having been dropped, + * they are now just wrappers over [const_]mem_fun kept for backwards- + * compatibility reasons. + */ + +#define BOOST_MULTI_INDEX_CONST_MEM_FUN(Class,Type,MemberFunName) \ +::boost::multi_index::const_mem_fun< Class,Type,&Class::MemberFunName > +#define BOOST_MULTI_INDEX_MEM_FUN(Class,Type,MemberFunName) \ +::boost::multi_index::mem_fun< Class,Type,&Class::MemberFunName > + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/member.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/member.hpp new file mode 100644 index 00000000000..a8e645074a2 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/member.hpp @@ -0,0 +1,262 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_MEMBER_HPP +#define BOOST_MULTI_INDEX_MEMBER_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include + +#if !defined(BOOST_NO_SFINAE) +#include +#endif + +namespace boost{ + +template class reference_wrapper; /* fwd decl. */ + +namespace multi_index{ + +namespace detail{ + +/* member is a read/write key extractor for accessing a given + * member of a class. + * Additionally, member is overloaded to support referece_wrappers + * of T and "chained pointers" to T's. By chained pointer to T we mean + * a type P such that, given a p of Type P + * *...n...*x is convertible to T&, for some n>=1. + * Examples of chained pointers are raw and smart pointers, iterators and + * arbitrary combinations of these (vg. T** or unique_ptr.) + */ + +template +struct const_member_base +{ + typedef Type result_type; + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if< + is_convertible,Type&>::type +#else + Type& +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + Type& operator()(const Class& x)const + { + return x.*PtrToMember; + } + + Type& operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } + + Type& operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } +}; + +template +struct non_const_member_base +{ + typedef Type result_type; + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if< + is_convertible,Type&>::type +#else + Type& +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + const Type& operator()(const Class& x)const + { + return x.*PtrToMember; + } + + Type& operator()(Class& x)const + { + return x.*PtrToMember; + } + + const Type& operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } + + Type& operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } +}; + +} /* namespace multi_index::detail */ + +template +struct member: + mpl::if_c< + is_const::value, + detail::const_member_base, + detail::non_const_member_base + >::type +{ +}; + +namespace detail{ + +/* MSVC++ 6.0 does not support properly pointers to members as + * non-type template arguments, as reported in + * http://support.microsoft.com/default.aspx?scid=kb;EN-US;249045 + * A similar problem (though not identical) is shown by MSVC++ 7.0. + * We provide an alternative to member<> accepting offsets instead + * of pointers to members. This happens to work even for non-POD + * types (although the standard forbids use of offsetof on these), + * so it serves as a workaround in this compiler for all practical + * purposes. + * Surprisingly enough, other compilers, like Intel C++ 7.0/7.1 and + * Visual Age 6.0, have similar bugs. This replacement of member<> + * can be used for them too. + * + * Support for such old compilers is dropped and + * [non_]const_member_offset_base is deprecated. + */ + +template +struct const_member_offset_base +{ + typedef Type result_type; + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if< + is_convertible,Type&>::type +#else + Type& +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + Type& operator()(const Class& x)const + { + return *static_cast( + static_cast( + static_cast( + static_cast(&x))+OffsetOfMember)); + } + + Type& operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } + + Type& operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } +}; + +template +struct non_const_member_offset_base +{ + typedef Type result_type; + + template + +#if !defined(BOOST_NO_SFINAE) + typename disable_if< + is_convertible,Type&>::type +#else + Type& +#endif + + operator()(const ChainedPtr& x)const + { + return operator()(*x); + } + + const Type& operator()(const Class& x)const + { + return *static_cast( + static_cast( + static_cast( + static_cast(&x))+OffsetOfMember)); + } + + Type& operator()(Class& x)const + { + return *static_cast( + static_cast( + static_cast(static_cast(&x))+OffsetOfMember)); + } + + const Type& operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } + + Type& operator()(const reference_wrapper& x)const + { + return operator()(x.get()); + } +}; + +} /* namespace multi_index::detail */ + +template +struct member_offset: + mpl::if_c< + is_const::value, + detail::const_member_offset_base, + detail::non_const_member_offset_base + >::type +{ +}; + +/* BOOST_MULTI_INDEX_MEMBER resolves to member in the normal cases, + * and to member_offset as a workaround in those defective compilers for + * which BOOST_NO_POINTER_TO_MEMBER_TEMPLATE_PARAMETERS is defined. + */ + +#if defined(BOOST_NO_POINTER_TO_MEMBER_TEMPLATE_PARAMETERS) +#define BOOST_MULTI_INDEX_MEMBER(Class,Type,MemberName) \ +::boost::multi_index::member_offset< Class,Type,offsetof(Class,MemberName) > +#else +#define BOOST_MULTI_INDEX_MEMBER(Class,Type,MemberName) \ +::boost::multi_index::member< Class,Type,&Class::MemberName > +#endif + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index.hpp new file mode 100644 index 00000000000..5bcd69de8c9 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index.hpp @@ -0,0 +1,114 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_ORDERED_INDEX_HPP +#define BOOST_MULTI_INDEX_ORDERED_INDEX_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* no augment policy for plain ordered indices */ + +struct null_augment_policy +{ + template + struct augmented_interface + { + typedef OrderedIndexImpl type; + }; + + template + struct augmented_node + { + typedef OrderedIndexNodeImpl type; + }; + + template static void add(Pointer,Pointer){} + template static void remove(Pointer,Pointer){} + template static void copy(Pointer,Pointer){} + template static void rotate_left(Pointer,Pointer){} + template static void rotate_right(Pointer,Pointer){} + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) + /* invariant stuff */ + + template static bool invariant(Pointer){return true;} + +#endif +}; + +} /* namespace multi_index::detail */ + +/* ordered_index specifiers */ + +template +struct ordered_unique +{ + typedef typename detail::ordered_index_args< + Arg1,Arg2,Arg3> index_args; + typedef typename index_args::tag_list_type::type tag_list_type; + typedef typename index_args::key_from_value_type key_from_value_type; + typedef typename index_args::compare_type compare_type; + + template + struct node_class + { + typedef detail::ordered_index_node type; + }; + + template + struct index_class + { + typedef detail::ordered_index< + key_from_value_type,compare_type, + SuperMeta,tag_list_type,detail::ordered_unique_tag, + detail::null_augment_policy> type; + }; +}; + +template +struct ordered_non_unique +{ + typedef detail::ordered_index_args< + Arg1,Arg2,Arg3> index_args; + typedef typename index_args::tag_list_type::type tag_list_type; + typedef typename index_args::key_from_value_type key_from_value_type; + typedef typename index_args::compare_type compare_type; + + template + struct node_class + { + typedef detail::ordered_index_node type; + }; + + template + struct index_class + { + typedef detail::ordered_index< + key_from_value_type,compare_type, + SuperMeta,tag_list_type,detail::ordered_non_unique_tag, + detail::null_augment_policy> type; + }; +}; + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index_fwd.hpp new file mode 100644 index 00000000000..fe44aaf860d --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index_fwd.hpp @@ -0,0 +1,35 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_ORDERED_INDEX_FWD_HPP +#define BOOST_MULTI_INDEX_ORDERED_INDEX_FWD_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include +#include + +namespace boost{ + +namespace multi_index{ + +/* ordered_index specifiers */ + +template +struct ordered_unique; + +template +struct ordered_non_unique; + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index.hpp new file mode 100644 index 00000000000..fe1884ddd38 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index.hpp @@ -0,0 +1,1167 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_RANDOM_ACCESS_INDEX_HPP +#define BOOST_MULTI_INDEX_RANDOM_ACCESS_INDEX_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) +#include +#endif + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +#include +#endif + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) +#define BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF(x) \ + detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ + detail::make_obj_guard(x,&random_access_index::check_invariant_); \ + BOOST_JOIN(check_invariant_,__LINE__).touch(); +#define BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT \ + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF(*this) +#else +#define BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF(x) +#define BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* random_access_index adds a layer of random access indexing + * to a given Super + */ + +template +class random_access_index: + BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS SuperMeta::type + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + ,public safe_mode::safe_container< + random_access_index > +#endif + +{ +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ + BOOST_WORKAROUND(__MWERKS__,<=0x3003) +/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the + * lifetime of const references bound to temporaries --precisely what + * scopeguards are. + */ + +#pragma parse_mfunc_templ off +#endif + + typedef typename SuperMeta::type super; + +protected: + typedef random_access_index_node< + typename super::node_type> node_type; + +private: + typedef typename node_type::impl_type node_impl_type; + typedef random_access_index_ptr_array< + typename super::final_allocator_type> ptr_array; + typedef typename ptr_array::pointer node_impl_ptr_pointer; + +public: + /* types */ + + typedef typename node_type::value_type value_type; + typedef tuples::null_type ctor_args; + typedef typename super::final_allocator_type allocator_type; + typedef typename allocator_type::reference reference; + typedef typename allocator_type::const_reference const_reference; + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + typedef safe_mode::safe_iterator< + rnd_node_iterator, + random_access_index> iterator; +#else + typedef rnd_node_iterator iterator; +#endif + + typedef iterator const_iterator; + + typedef std::size_t size_type; + typedef std::ptrdiff_t difference_type; + typedef typename allocator_type::pointer pointer; + typedef typename allocator_type::const_pointer const_pointer; + typedef typename + boost::reverse_iterator reverse_iterator; + typedef typename + boost::reverse_iterator const_reverse_iterator; + typedef TagList tag_list; + +protected: + typedef typename super::final_node_type final_node_type; + typedef tuples::cons< + ctor_args, + typename super::ctor_args_list> ctor_args_list; + typedef typename mpl::push_front< + typename super::index_type_list, + random_access_index>::type index_type_list; + typedef typename mpl::push_front< + typename super::iterator_type_list, + iterator>::type iterator_type_list; + typedef typename mpl::push_front< + typename super::const_iterator_type_list, + const_iterator>::type const_iterator_type_list; + typedef typename super::copy_map_type copy_map_type; + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + typedef typename super::index_saver_type index_saver_type; + typedef typename super::index_loader_type index_loader_type; +#endif + +private: +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + typedef safe_mode::safe_container< + random_access_index> safe_super; +#endif + + typedef typename call_traits< + value_type>::param_type value_param_type; + + /* Needed to avoid commas in BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL + * expansion. + */ + + typedef std::pair emplace_return_type; + +public: + + /* construct/copy/destroy + * Default and copy ctors are in the protected section as indices are + * not supposed to be created on their own. No range ctor either. + */ + + random_access_index& operator=( + const random_access_index& x) + { + this->final()=x.final(); + return *this; + } + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) + random_access_index& operator=( + std::initializer_list list) + { + this->final()=list; + return *this; + } +#endif + + template + void assign(InputIterator first,InputIterator last) + { + assign_iter(first,last,mpl::not_ >()); + } + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) + void assign(std::initializer_list list) + { + assign(list.begin(),list.end()); + } +#endif + + void assign(size_type n,value_param_type value) + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + clear(); + for(size_type i=0;ifinal().get_allocator(); + } + + /* iterators */ + + iterator begin()BOOST_NOEXCEPT + {return make_iterator(node_type::from_impl(*ptrs.begin()));} + const_iterator begin()const BOOST_NOEXCEPT + {return make_iterator(node_type::from_impl(*ptrs.begin()));} + iterator + end()BOOST_NOEXCEPT{return make_iterator(header());} + const_iterator + end()const BOOST_NOEXCEPT{return make_iterator(header());} + reverse_iterator + rbegin()BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} + const_reverse_iterator + rbegin()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} + reverse_iterator + rend()BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} + const_reverse_iterator + rend()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} + const_iterator + cbegin()const BOOST_NOEXCEPT{return begin();} + const_iterator + cend()const BOOST_NOEXCEPT{return end();} + const_reverse_iterator + crbegin()const BOOST_NOEXCEPT{return rbegin();} + const_reverse_iterator + crend()const BOOST_NOEXCEPT{return rend();} + + iterator iterator_to(const value_type& x) + { + return make_iterator(node_from_value(&x)); + } + + const_iterator iterator_to(const value_type& x)const + { + return make_iterator(node_from_value(&x)); + } + + /* capacity */ + + bool empty()const BOOST_NOEXCEPT{return this->final_empty_();} + size_type size()const BOOST_NOEXCEPT{return this->final_size_();} + size_type max_size()const BOOST_NOEXCEPT{return this->final_max_size_();} + size_type capacity()const BOOST_NOEXCEPT{return ptrs.capacity();} + + void reserve(size_type n) + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + ptrs.reserve(n); + } + + void shrink_to_fit() + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + ptrs.shrink_to_fit(); + } + + void resize(size_type n) + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + if(n>size()) + for(size_type m=n-size();m--;) + this->final_emplace_(BOOST_MULTI_INDEX_NULL_PARAM_PACK); + else if(nsize())for(size_type m=n-size();m--;)this->final_insert_(x); + else if(nvalue(); + } + + const_reference at(size_type n)const + { + if(n>=size())throw_exception(std::out_of_range("random access index")); + return node_type::from_impl(*ptrs.at(n))->value(); + } + + const_reference front()const{return operator[](0);} + const_reference back()const{return operator[](size()-1);} + + /* modifiers */ + + BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( + emplace_return_type,emplace_front,emplace_front_impl) + + std::pair push_front(const value_type& x) + {return insert(begin(),x);} + std::pair push_front(BOOST_RV_REF(value_type) x) + {return insert(begin(),boost::move(x));} + void pop_front(){erase(begin());} + + BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( + emplace_return_type,emplace_back,emplace_back_impl) + + std::pair push_back(const value_type& x) + {return insert(end(),x);} + std::pair push_back(BOOST_RV_REF(value_type) x) + {return insert(end(),boost::move(x));} + void pop_back(){erase(--end());} + + BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( + emplace_return_type,emplace,emplace_impl,iterator,position) + + std::pair insert(iterator position,const value_type& x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + std::pair p=this->final_insert_(x); + if(p.second&&position.get_node()!=header()){ + relocate(position.get_node(),p.first); + } + return std::pair(make_iterator(p.first),p.second); + } + + std::pair insert(iterator position,BOOST_RV_REF(value_type) x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + std::pair p=this->final_insert_rv_(x); + if(p.second&&position.get_node()!=header()){ + relocate(position.get_node(),p.first); + } + return std::pair(make_iterator(p.first),p.second); + } + + void insert(iterator position,size_type n,value_param_type x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + size_type s=0; + BOOST_TRY{ + while(n--){ + if(push_back(x).second)++s; + } + } + BOOST_CATCH(...){ + relocate(position,end()-s,end()); + BOOST_RETHROW; + } + BOOST_CATCH_END + relocate(position,end()-s,end()); + } + + template + void insert(iterator position,InputIterator first,InputIterator last) + { + insert_iter(position,first,last,mpl::not_ >()); + } + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) + void insert(iterator position,std::initializer_list list) + { + insert(position,list.begin(),list.end()); + } +#endif + + iterator erase(iterator position) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + this->final_erase_(static_cast(position++.get_node())); + return position; + } + + iterator erase(iterator first,iterator last) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); + BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + difference_type n=last-first; + relocate(end(),first,last); + while(n--)pop_back(); + return last; + } + + bool replace(iterator position,const value_type& x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + return this->final_replace_( + x,static_cast(position.get_node())); + } + + bool replace(iterator position,BOOST_RV_REF(value_type) x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + return this->final_replace_rv_( + x,static_cast(position.get_node())); + } + + template + bool modify(iterator position,Modifier mod) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + /* MSVC++ 6.0 optimizer on safe mode code chokes if this + * this is not added. Left it for all compilers as it does no + * harm. + */ + + position.detach(); +#endif + + return this->final_modify_( + mod,static_cast(position.get_node())); + } + + template + bool modify(iterator position,Modifier mod,Rollback back_) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + /* MSVC++ 6.0 optimizer on safe mode code chokes if this + * this is not added. Left it for all compilers as it does no + * harm. + */ + + position.detach(); +#endif + + return this->final_modify_( + mod,back_,static_cast(position.get_node())); + } + + void swap(random_access_index& x) + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF(x); + this->final_swap_(x.final()); + } + + void clear()BOOST_NOEXCEPT + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + this->final_clear_(); + } + + /* list operations */ + + void splice(iterator position,random_access_index& x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_CHECK_DIFFERENT_CONTAINER(*this,x); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + iterator first=x.begin(),last=x.end(); + size_type n=0; + BOOST_TRY{ + while(first!=last){ + if(push_back(*first).second){ + first=x.erase(first); + ++n; + } + else ++first; + } + } + BOOST_CATCH(...){ + relocate(position,end()-n,end()); + BOOST_RETHROW; + } + BOOST_CATCH_END + relocate(position,end()-n,end()); + } + + void splice( + iterator position,random_access_index& x,iterator i) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(i); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(i); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(i,x); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + if(&x==this)relocate(position,i); + else{ + if(insert(position,*i).second){ + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + /* MSVC++ 6.0 optimizer has a hard time with safe mode, and the following + * workaround is needed. Left it for all compilers as it does no + * harm. + */ + i.detach(); + x.erase(x.make_iterator(i.get_node())); +#else + x.erase(i); +#endif + + } + } + } + + void splice( + iterator position,random_access_index& x, + iterator first,iterator last) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,x); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,x); + BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + if(&x==this)relocate(position,first,last); + else{ + size_type n=0; + BOOST_TRY{ + while(first!=last){ + if(push_back(*first).second){ + first=x.erase(first); + ++n; + } + else ++first; + } + } + BOOST_CATCH(...){ + relocate(position,end()-n,end()); + BOOST_RETHROW; + } + BOOST_CATCH_END + relocate(position,end()-n,end()); + } + } + + void remove(value_param_type value) + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + difference_type n= + end()-make_iterator( + random_access_index_remove( + ptrs, + ::boost::bind(std::equal_to(),::boost::arg<1>(),value))); + while(n--)pop_back(); + } + + template + void remove_if(Predicate pred) + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + difference_type n= + end()-make_iterator(random_access_index_remove(ptrs,pred)); + while(n--)pop_back(); + } + + void unique() + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + difference_type n= + end()-make_iterator( + random_access_index_unique( + ptrs,std::equal_to())); + while(n--)pop_back(); + } + + template + void unique(BinaryPredicate binary_pred) + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + difference_type n= + end()-make_iterator( + random_access_index_unique(ptrs,binary_pred)); + while(n--)pop_back(); + } + + void merge(random_access_index& x) + { + if(this!=&x){ + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + size_type s=size(); + splice(end(),x); + random_access_index_inplace_merge( + get_allocator(),ptrs,ptrs.at(s),std::less()); + } + } + + template + void merge(random_access_index& x,Compare comp) + { + if(this!=&x){ + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + size_type s=size(); + splice(end(),x); + random_access_index_inplace_merge( + get_allocator(),ptrs,ptrs.at(s),comp); + } + } + + void sort() + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + random_access_index_sort( + get_allocator(),ptrs,std::less()); + } + + template + void sort(Compare comp) + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + random_access_index_sort( + get_allocator(),ptrs,comp); + } + + void reverse()BOOST_NOEXCEPT + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + node_impl_type::reverse(ptrs.begin(),ptrs.end()); + } + + /* rearrange operations */ + + void relocate(iterator position,iterator i) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(i); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(i); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(i,*this); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + if(position!=i)relocate(position.get_node(),i.get_node()); + } + + void relocate(iterator position,iterator first,iterator last) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); + BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); + BOOST_MULTI_INDEX_CHECK_OUTSIDE_RANGE(position,first,last); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + if(position!=last)relocate( + position.get_node(),first.get_node(),last.get_node()); + } + + template + void rearrange(InputIterator first) + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + for(node_impl_ptr_pointer p0=ptrs.begin(),p0_end=ptrs.end(); + p0!=p0_end;++first,++p0){ + const value_type& v1=*first; + node_impl_ptr_pointer p1=node_from_value(&v1)->up(); + + std::swap(*p0,*p1); + (*p0)->up()=p0; + (*p1)->up()=p1; + } + } + +BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: + random_access_index( + const ctor_args_list& args_list,const allocator_type& al): + super(args_list.get_tail(),al), + ptrs(al,header()->impl(),0) + { + } + + random_access_index(const random_access_index& x): + super(x), + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super(), +#endif + + ptrs(x.get_allocator(),header()->impl(),x.size()) + { + /* The actual copying takes place in subsequent call to copy_(). + */ + } + + random_access_index( + const random_access_index& x,do_not_copy_elements_tag): + super(x,do_not_copy_elements_tag()), + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super(), +#endif + + ptrs(x.get_allocator(),header()->impl(),0) + { + } + + ~random_access_index() + { + /* the container is guaranteed to be empty by now */ + } + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + iterator make_iterator(node_type* node){return iterator(node,this);} + const_iterator make_iterator(node_type* node)const + {return const_iterator(node,const_cast(this));} +#else + iterator make_iterator(node_type* node){return iterator(node);} + const_iterator make_iterator(node_type* node)const + {return const_iterator(node);} +#endif + + void copy_( + const random_access_index& x,const copy_map_type& map) + { + for(node_impl_ptr_pointer begin_org=x.ptrs.begin(), + begin_cpy=ptrs.begin(), + end_org=x.ptrs.end(); + begin_org!=end_org;++begin_org,++begin_cpy){ + *begin_cpy= + static_cast( + map.find( + static_cast( + node_type::from_impl(*begin_org))))->impl(); + (*begin_cpy)->up()=begin_cpy; + } + + super::copy_(x,map); + } + + template + final_node_type* insert_( + value_param_type v,final_node_type*& x,Variant variant) + { + ptrs.room_for_one(); + final_node_type* res=super::insert_(v,x,variant); + if(res==x)ptrs.push_back(static_cast(x)->impl()); + return res; + } + + template + final_node_type* insert_( + value_param_type v,node_type* position,final_node_type*& x,Variant variant) + { + ptrs.room_for_one(); + final_node_type* res=super::insert_(v,position,x,variant); + if(res==x)ptrs.push_back(static_cast(x)->impl()); + return res; + } + + void erase_(node_type* x) + { + ptrs.erase(x->impl()); + super::erase_(x); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + } + + void delete_all_nodes_() + { + for(node_impl_ptr_pointer x=ptrs.begin(),x_end=ptrs.end();x!=x_end;++x){ + this->final_delete_node_( + static_cast(node_type::from_impl(*x))); + } + } + + void clear_() + { + super::clear_(); + ptrs.clear(); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super::detach_dereferenceable_iterators(); +#endif + } + + void swap_(random_access_index& x) + { + ptrs.swap(x.ptrs); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super::swap(x); +#endif + + super::swap_(x); + } + + void swap_elements_(random_access_index& x) + { + ptrs.swap(x.ptrs); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super::swap(x); +#endif + + super::swap_elements_(x); + } + + template + bool replace_(value_param_type v,node_type* x,Variant variant) + { + return super::replace_(v,x,variant); + } + + bool modify_(node_type* x) + { + BOOST_TRY{ + if(!super::modify_(x)){ + ptrs.erase(x->impl()); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + + return false; + } + else return true; + } + BOOST_CATCH(...){ + ptrs.erase(x->impl()); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + bool modify_rollback_(node_type* x) + { + return super::modify_rollback_(x); + } + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + /* serialization */ + + template + void save_( + Archive& ar,const unsigned int version,const index_saver_type& sm)const + { + sm.save(begin(),end(),ar,version); + super::save_(ar,version,sm); + } + + template + void load_( + Archive& ar,const unsigned int version,const index_loader_type& lm) + { + { + typedef random_access_index_loader loader; + + loader ld(get_allocator(),ptrs); + lm.load( + ::boost::bind( + &loader::rearrange,&ld,::boost::arg<1>(),::boost::arg<2>()), + ar,version); + } /* exit scope so that ld frees its resources */ + super::load_(ar,version,lm); + } +#endif + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) + /* invariant stuff */ + + bool invariant_()const + { + if(size()>capacity())return false; + if(size()==0||begin()==end()){ + if(size()!=0||begin()!=end())return false; + } + else{ + size_type s=0; + for(const_iterator it=begin(),it_end=end();;++it,++s){ + if(*(it.get_node()->up())!=it.get_node()->impl())return false; + if(it==it_end)break; + } + if(s!=size())return false; + } + + return super::invariant_(); + } + + /* This forwarding function eases things for the boost::mem_fn construct + * in BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT. Actually, + * final_check_invariant is already an inherited member function of index. + */ + void check_invariant_()const{this->final_check_invariant_();} +#endif + +private: + node_type* header()const{return this->final_header();} + + static void relocate(node_type* position,node_type* x) + { + node_impl_type::relocate(position->up(),x->up()); + } + + static void relocate(node_type* position,node_type* first,node_type* last) + { + node_impl_type::relocate( + position->up(),first->up(),last->up()); + } + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + void detach_iterators(node_type* x) + { + iterator it=make_iterator(x); + safe_mode::detach_equivalent_iterators(it); + } +#endif + + template + void assign_iter(InputIterator first,InputIterator last,mpl::true_) + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + clear(); + for(;first!=last;++first)this->final_insert_ref_(*first); + } + + void assign_iter(size_type n,value_param_type value,mpl::false_) + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + clear(); + for(size_type i=0;i + void insert_iter( + iterator position,InputIterator first,InputIterator last,mpl::true_) + { + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + size_type s=0; + BOOST_TRY{ + for(;first!=last;++first){ + if(this->final_insert_ref_(*first).second)++s; + } + } + BOOST_CATCH(...){ + relocate(position,end()-s,end()); + BOOST_RETHROW; + } + BOOST_CATCH_END + relocate(position,end()-s,end()); + } + + void insert_iter( + iterator position,size_type n,value_param_type x,mpl::false_) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + size_type s=0; + BOOST_TRY{ + while(n--){ + if(push_back(x).second)++s; + } + } + BOOST_CATCH(...){ + relocate(position,end()-s,end()); + BOOST_RETHROW; + } + BOOST_CATCH_END + relocate(position,end()-s,end()); + } + + template + std::pair emplace_front_impl( + BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + return emplace_impl(begin(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + } + + template + std::pair emplace_back_impl( + BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + return emplace_impl(end(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + } + + template + std::pair emplace_impl( + iterator position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; + std::pair p= + this->final_emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + if(p.second&&position.get_node()!=header()){ + relocate(position.get_node(),p.first); + } + return std::pair(make_iterator(p.first),p.second); + } + + ptr_array ptrs; + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ + BOOST_WORKAROUND(__MWERKS__,<=0x3003) +#pragma parse_mfunc_templ reset +#endif +}; + +/* comparison */ + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator==( + const random_access_index& x, + const random_access_index& y) +{ + return x.size()==y.size()&&std::equal(x.begin(),x.end(),y.begin()); +} + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator<( + const random_access_index& x, + const random_access_index& y) +{ + return std::lexicographical_compare(x.begin(),x.end(),y.begin(),y.end()); +} + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator!=( + const random_access_index& x, + const random_access_index& y) +{ + return !(x==y); +} + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator>( + const random_access_index& x, + const random_access_index& y) +{ + return y +bool operator>=( + const random_access_index& x, + const random_access_index& y) +{ + return !(x +bool operator<=( + const random_access_index& x, + const random_access_index& y) +{ + return !(x>y); +} + +/* specialized algorithms */ + +template +void swap( + random_access_index& x, + random_access_index& y) +{ + x.swap(y); +} + +} /* namespace multi_index::detail */ + +/* random access index specifier */ + +template +struct random_access +{ + BOOST_STATIC_ASSERT(detail::is_tag::value); + + template + struct node_class + { + typedef detail::random_access_index_node type; + }; + + template + struct index_class + { + typedef detail::random_access_index< + SuperMeta,typename TagList::type> type; + }; +}; + +} /* namespace multi_index */ + +} /* namespace boost */ + +/* Boost.Foreach compatibility */ + +template +inline boost::mpl::true_* boost_foreach_is_noncopyable( + boost::multi_index::detail::random_access_index*&, + boost_foreach_argument_dependent_lookup_hack) +{ + return 0; +} + +#undef BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT +#undef BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index_fwd.hpp new file mode 100644 index 00000000000..2ea19295426 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index_fwd.hpp @@ -0,0 +1,91 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_RANDOM_ACCESS_INDEX_FWD_HPP +#define BOOST_MULTI_INDEX_RANDOM_ACCESS_INDEX_FWD_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +template +class random_access_index; + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator==( + const random_access_index& x, + const random_access_index& y); + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator<( + const random_access_index& x, + const random_access_index& y); + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator!=( + const random_access_index& x, + const random_access_index& y); + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator>( + const random_access_index& x, + const random_access_index& y); + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator>=( + const random_access_index& x, + const random_access_index& y); + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator<=( + const random_access_index& x, + const random_access_index& y); + +template +void swap( + random_access_index& x, + random_access_index& y); + +} /* namespace multi_index::detail */ + +/* index specifiers */ + +template > +struct random_access; + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index.hpp new file mode 100644 index 00000000000..4b24c4f5937 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index.hpp @@ -0,0 +1,382 @@ +/* Copyright 2003-2017 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_RANKED_INDEX_HPP +#define BOOST_MULTI_INDEX_RANKED_INDEX_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* ranked_index augments a given ordered index to provide rank operations */ + +template +struct ranked_node:OrderedIndexNodeImpl +{ + std::size_t size; +}; + +template +class ranked_index:public OrderedIndexImpl +{ + typedef OrderedIndexImpl super; + +protected: + typedef typename super::node_type node_type; + typedef typename super::node_impl_pointer node_impl_pointer; + +public: + typedef typename super::ctor_args_list ctor_args_list; + typedef typename super::allocator_type allocator_type; + typedef typename super::iterator iterator; + + /* rank operations */ + + iterator nth(std::size_t n)const + { + return this->make_iterator(node_type::from_impl( + ranked_index_nth(n,this->header()->impl()))); + } + + std::size_t rank(iterator position)const + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + + return ranked_index_rank( + position.get_node()->impl(),this->header()->impl()); + } + + template + std::size_t find_rank(const CompatibleKey& x)const + { + return ranked_index_find_rank( + this->root(),this->header(),this->key,x,this->comp_); + } + + template + std::size_t find_rank( + const CompatibleKey& x,const CompatibleCompare& comp)const + { + return ranked_index_find_rank( + this->root(),this->header(),this->key,x,comp); + } + + template + std::size_t lower_bound_rank(const CompatibleKey& x)const + { + return ranked_index_lower_bound_rank( + this->root(),this->header(),this->key,x,this->comp_); + } + + template + std::size_t lower_bound_rank( + const CompatibleKey& x,const CompatibleCompare& comp)const + { + return ranked_index_lower_bound_rank( + this->root(),this->header(),this->key,x,comp); + } + + template + std::size_t upper_bound_rank(const CompatibleKey& x)const + { + return ranked_index_upper_bound_rank( + this->root(),this->header(),this->key,x,this->comp_); + } + + template + std::size_t upper_bound_rank( + const CompatibleKey& x,const CompatibleCompare& comp)const + { + return ranked_index_upper_bound_rank( + this->root(),this->header(),this->key,x,comp); + } + + template + std::pair equal_range_rank( + const CompatibleKey& x)const + { + return ranked_index_equal_range_rank( + this->root(),this->header(),this->key,x,this->comp_); + } + + template + std::pair equal_range_rank( + const CompatibleKey& x,const CompatibleCompare& comp)const + { + return ranked_index_equal_range_rank( + this->root(),this->header(),this->key,x,comp); + } + + template + std::pair + range_rank(LowerBounder lower,UpperBounder upper)const + { + typedef typename mpl::if_< + is_same, + BOOST_DEDUCED_TYPENAME mpl::if_< + is_same, + both_unbounded_tag, + lower_unbounded_tag + >::type, + BOOST_DEDUCED_TYPENAME mpl::if_< + is_same, + upper_unbounded_tag, + none_unbounded_tag + >::type + >::type dispatch; + + return range_rank(lower,upper,dispatch()); + } + +protected: + ranked_index(const ranked_index& x):super(x){}; + + ranked_index(const ranked_index& x,do_not_copy_elements_tag): + super(x,do_not_copy_elements_tag()){}; + + ranked_index( + const ctor_args_list& args_list,const allocator_type& al): + super(args_list,al){} + +private: + template + std::pair + range_rank(LowerBounder lower,UpperBounder upper,none_unbounded_tag)const + { + node_type* y=this->header(); + node_type* z=this->root(); + + if(!z)return std::pair(0,0); + + std::size_t s=z->impl()->size; + + do{ + if(!lower(this->key(z->value()))){ + z=node_type::from_impl(z->right()); + } + else if(!upper(this->key(z->value()))){ + y=z; + s-=ranked_node_size(y->right())+1; + z=node_type::from_impl(z->left()); + } + else{ + return std::pair( + s-z->impl()->size+ + lower_range_rank(node_type::from_impl(z->left()),z,lower), + s-ranked_node_size(z->right())+ + upper_range_rank(node_type::from_impl(z->right()),y,upper)); + } + }while(z); + + return std::pair(s,s); + } + + template + std::pair + range_rank(LowerBounder,UpperBounder upper,lower_unbounded_tag)const + { + return std::pair( + 0, + upper_range_rank(this->root(),this->header(),upper)); + } + + template + std::pair + range_rank(LowerBounder lower,UpperBounder,upper_unbounded_tag)const + { + return std::pair( + lower_range_rank(this->root(),this->header(),lower), + this->size()); + } + + template + std::pair + range_rank(LowerBounder,UpperBounder,both_unbounded_tag)const + { + return std::pair(0,this->size()); + } + + template + std::size_t + lower_range_rank(node_type* top,node_type* y,LowerBounder lower)const + { + if(!top)return 0; + + std::size_t s=top->impl()->size; + + do{ + if(lower(this->key(top->value()))){ + y=top; + s-=ranked_node_size(y->right())+1; + top=node_type::from_impl(top->left()); + } + else top=node_type::from_impl(top->right()); + }while(top); + + return s; + } + + template + std::size_t + upper_range_rank(node_type* top,node_type* y,UpperBounder upper)const + { + if(!top)return 0; + + std::size_t s=top->impl()->size; + + do{ + if(!upper(this->key(top->value()))){ + y=top; + s-=ranked_node_size(y->right())+1; + top=node_type::from_impl(top->left()); + } + else top=node_type::from_impl(top->right()); + }while(top); + + return s; + } +}; + +/* augmenting policy for ordered_index */ + +struct rank_policy +{ + template + struct augmented_node + { + typedef ranked_node type; + }; + + template + struct augmented_interface + { + typedef ranked_index type; + }; + + /* algorithmic stuff */ + + template + static void add(Pointer x,Pointer root) + { + x->size=1; + while(x!=root){ + x=x->parent(); + ++(x->size); + } + } + + template + static void remove(Pointer x,Pointer root) + { + while(x!=root){ + x=x->parent(); + --(x->size); + } + } + + template + static void copy(Pointer x,Pointer y) + { + y->size=x->size; + } + + template + static void rotate_left(Pointer x,Pointer y) /* in: x==y->left() */ + { + y->size=x->size; + x->size=ranked_node_size(x->left())+ranked_node_size(x->right())+1; + } + + template + static void rotate_right(Pointer x,Pointer y) /* in: x==y->right() */ + { + rotate_left(x,y); + } + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) + /* invariant stuff */ + + template + static bool invariant(Pointer x) + { + return x->size==ranked_node_size(x->left())+ranked_node_size(x->right())+1; + } +#endif +}; + +} /* namespace multi_index::detail */ + +/* ranked_index specifiers */ + +template +struct ranked_unique +{ + typedef typename detail::ordered_index_args< + Arg1,Arg2,Arg3> index_args; + typedef typename index_args::tag_list_type::type tag_list_type; + typedef typename index_args::key_from_value_type key_from_value_type; + typedef typename index_args::compare_type compare_type; + + template + struct node_class + { + typedef detail::ordered_index_node type; + }; + + template + struct index_class + { + typedef detail::ordered_index< + key_from_value_type,compare_type, + SuperMeta,tag_list_type,detail::ordered_unique_tag, + detail::rank_policy> type; + }; +}; + +template +struct ranked_non_unique +{ + typedef detail::ordered_index_args< + Arg1,Arg2,Arg3> index_args; + typedef typename index_args::tag_list_type::type tag_list_type; + typedef typename index_args::key_from_value_type key_from_value_type; + typedef typename index_args::compare_type compare_type; + + template + struct node_class + { + typedef detail::ordered_index_node type; + }; + + template + struct index_class + { + typedef detail::ordered_index< + key_from_value_type,compare_type, + SuperMeta,tag_list_type,detail::ordered_non_unique_tag, + detail::rank_policy> type; + }; +}; + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index_fwd.hpp new file mode 100644 index 00000000000..380d3480736 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index_fwd.hpp @@ -0,0 +1,35 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_RANKED_INDEX_FWD_HPP +#define BOOST_MULTI_INDEX_RANKED_INDEX_FWD_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include +#include + +namespace boost{ + +namespace multi_index{ + +/* ranked_index specifiers */ + +template +struct ranked_unique; + +template +struct ranked_non_unique; + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/safe_mode_errors.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/safe_mode_errors.hpp new file mode 100644 index 00000000000..1904706edec --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/safe_mode_errors.hpp @@ -0,0 +1,48 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_SAFE_MODE_ERRORS_HPP +#define BOOST_MULTI_INDEX_SAFE_MODE_ERRORS_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +namespace boost{ + +namespace multi_index{ + +namespace safe_mode{ + +/* Error codes for Boost.MultiIndex safe mode. These go in a separate + * header so that the user can include it when redefining + * BOOST_MULTI_INDEX_SAFE_MODE_ASSERT prior to the inclusion of + * any other header of Boost.MultiIndex. + */ + +enum error_code +{ + invalid_iterator=0, + not_dereferenceable_iterator, + not_incrementable_iterator, + not_decrementable_iterator, + not_owner, + not_same_owner, + invalid_range, + inside_range, + out_of_bounds, + same_container +}; + +} /* namespace multi_index::safe_mode */ + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index.hpp new file mode 100644 index 00000000000..424eebc376d --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index.hpp @@ -0,0 +1,1062 @@ +/* Copyright 2003-2015 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_SEQUENCED_INDEX_HPP +#define BOOST_MULTI_INDEX_SEQUENCED_INDEX_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) +#include +#endif + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +#include +#endif + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) +#define BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF(x) \ + detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ + detail::make_obj_guard(x,&sequenced_index::check_invariant_); \ + BOOST_JOIN(check_invariant_,__LINE__).touch(); +#define BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT \ + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF(*this) +#else +#define BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF(x) +#define BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT +#endif + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +/* sequenced_index adds a layer of sequenced indexing to a given Super */ + +template +class sequenced_index: + BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS SuperMeta::type + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + ,public safe_mode::safe_container< + sequenced_index > +#endif + +{ +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ + BOOST_WORKAROUND(__MWERKS__,<=0x3003) +/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the + * lifetime of const references bound to temporaries --precisely what + * scopeguards are. + */ + +#pragma parse_mfunc_templ off +#endif + + typedef typename SuperMeta::type super; + +protected: + typedef sequenced_index_node< + typename super::node_type> node_type; + +private: + typedef typename node_type::impl_type node_impl_type; + +public: + /* types */ + + typedef typename node_type::value_type value_type; + typedef tuples::null_type ctor_args; + typedef typename super::final_allocator_type allocator_type; + typedef typename allocator_type::reference reference; + typedef typename allocator_type::const_reference const_reference; + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + typedef safe_mode::safe_iterator< + bidir_node_iterator, + sequenced_index> iterator; +#else + typedef bidir_node_iterator iterator; +#endif + + typedef iterator const_iterator; + + typedef std::size_t size_type; + typedef std::ptrdiff_t difference_type; + typedef typename allocator_type::pointer pointer; + typedef typename allocator_type::const_pointer const_pointer; + typedef typename + boost::reverse_iterator reverse_iterator; + typedef typename + boost::reverse_iterator const_reverse_iterator; + typedef TagList tag_list; + +protected: + typedef typename super::final_node_type final_node_type; + typedef tuples::cons< + ctor_args, + typename super::ctor_args_list> ctor_args_list; + typedef typename mpl::push_front< + typename super::index_type_list, + sequenced_index>::type index_type_list; + typedef typename mpl::push_front< + typename super::iterator_type_list, + iterator>::type iterator_type_list; + typedef typename mpl::push_front< + typename super::const_iterator_type_list, + const_iterator>::type const_iterator_type_list; + typedef typename super::copy_map_type copy_map_type; + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + typedef typename super::index_saver_type index_saver_type; + typedef typename super::index_loader_type index_loader_type; +#endif + +private: +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + typedef safe_mode::safe_container< + sequenced_index> safe_super; +#endif + + typedef typename call_traits::param_type value_param_type; + + /* Needed to avoid commas in BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL + * expansion. + */ + + typedef std::pair emplace_return_type; + +public: + + /* construct/copy/destroy + * Default and copy ctors are in the protected section as indices are + * not supposed to be created on their own. No range ctor either. + */ + + sequenced_index& operator=( + const sequenced_index& x) + { + this->final()=x.final(); + return *this; + } + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) + sequenced_index& operator=( + std::initializer_list list) + { + this->final()=list; + return *this; + } +#endif + + template + void assign(InputIterator first,InputIterator last) + { + assign_iter(first,last,mpl::not_ >()); + } + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) + void assign(std::initializer_list list) + { + assign(list.begin(),list.end()); + } +#endif + + void assign(size_type n,value_param_type value) + { + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + clear(); + for(size_type i=0;ifinal().get_allocator(); + } + + /* iterators */ + + iterator begin()BOOST_NOEXCEPT + {return make_iterator(node_type::from_impl(header()->next()));} + const_iterator begin()const BOOST_NOEXCEPT + {return make_iterator(node_type::from_impl(header()->next()));} + iterator + end()BOOST_NOEXCEPT{return make_iterator(header());} + const_iterator + end()const BOOST_NOEXCEPT{return make_iterator(header());} + reverse_iterator + rbegin()BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} + const_reverse_iterator + rbegin()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} + reverse_iterator + rend()BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} + const_reverse_iterator + rend()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} + const_iterator + cbegin()const BOOST_NOEXCEPT{return begin();} + const_iterator + cend()const BOOST_NOEXCEPT{return end();} + const_reverse_iterator + crbegin()const BOOST_NOEXCEPT{return rbegin();} + const_reverse_iterator + crend()const BOOST_NOEXCEPT{return rend();} + + iterator iterator_to(const value_type& x) + { + return make_iterator(node_from_value(&x)); + } + + const_iterator iterator_to(const value_type& x)const + { + return make_iterator(node_from_value(&x)); + } + + /* capacity */ + + bool empty()const BOOST_NOEXCEPT{return this->final_empty_();} + size_type size()const BOOST_NOEXCEPT{return this->final_size_();} + size_type max_size()const BOOST_NOEXCEPT{return this->final_max_size_();} + + void resize(size_type n) + { + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + if(n>size()){ + for(size_type m=n-size();m--;) + this->final_emplace_(BOOST_MULTI_INDEX_NULL_PARAM_PACK); + } + else if(nsize())insert(end(),n-size(),x); + else if(n push_front(const value_type& x) + {return insert(begin(),x);} + std::pair push_front(BOOST_RV_REF(value_type) x) + {return insert(begin(),boost::move(x));} + void pop_front(){erase(begin());} + + BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( + emplace_return_type,emplace_back,emplace_back_impl) + + std::pair push_back(const value_type& x) + {return insert(end(),x);} + std::pair push_back(BOOST_RV_REF(value_type) x) + {return insert(end(),boost::move(x));} + void pop_back(){erase(--end());} + + BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( + emplace_return_type,emplace,emplace_impl,iterator,position) + + std::pair insert(iterator position,const value_type& x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + std::pair p=this->final_insert_(x); + if(p.second&&position.get_node()!=header()){ + relink(position.get_node(),p.first); + } + return std::pair(make_iterator(p.first),p.second); + } + + std::pair insert(iterator position,BOOST_RV_REF(value_type) x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + std::pair p=this->final_insert_rv_(x); + if(p.second&&position.get_node()!=header()){ + relink(position.get_node(),p.first); + } + return std::pair(make_iterator(p.first),p.second); + } + + void insert(iterator position,size_type n,value_param_type x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + for(size_type i=0;i + void insert(iterator position,InputIterator first,InputIterator last) + { + insert_iter(position,first,last,mpl::not_ >()); + } + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) + void insert(iterator position,std::initializer_list list) + { + insert(position,list.begin(),list.end()); + } +#endif + + iterator erase(iterator position) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + this->final_erase_(static_cast(position++.get_node())); + return position; + } + + iterator erase(iterator first,iterator last) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); + BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + while(first!=last){ + first=erase(first); + } + return first; + } + + bool replace(iterator position,const value_type& x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + return this->final_replace_( + x,static_cast(position.get_node())); + } + + bool replace(iterator position,BOOST_RV_REF(value_type) x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + return this->final_replace_rv_( + x,static_cast(position.get_node())); + } + + template + bool modify(iterator position,Modifier mod) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + /* MSVC++ 6.0 optimizer on safe mode code chokes if this + * this is not added. Left it for all compilers as it does no + * harm. + */ + + position.detach(); +#endif + + return this->final_modify_( + mod,static_cast(position.get_node())); + } + + template + bool modify(iterator position,Modifier mod,Rollback back_) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + /* MSVC++ 6.0 optimizer on safe mode code chokes if this + * this is not added. Left it for all compilers as it does no + * harm. + */ + + position.detach(); +#endif + + return this->final_modify_( + mod,back_,static_cast(position.get_node())); + } + + void swap(sequenced_index& x) + { + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF(x); + this->final_swap_(x.final()); + } + + void clear()BOOST_NOEXCEPT + { + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + this->final_clear_(); + } + + /* list operations */ + + void splice(iterator position,sequenced_index& x) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_CHECK_DIFFERENT_CONTAINER(*this,x); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + iterator first=x.begin(),last=x.end(); + while(first!=last){ + if(insert(position,*first).second)first=x.erase(first); + else ++first; + } + } + + void splice(iterator position,sequenced_index& x,iterator i) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(i); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(i); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(i,x); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + if(&x==this){ + if(position!=i)relink(position.get_node(),i.get_node()); + } + else{ + if(insert(position,*i).second){ + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + /* MSVC++ 6.0 optimizer has a hard time with safe mode, and the following + * workaround is needed. Left it for all compilers as it does no + * harm. + */ + i.detach(); + x.erase(x.make_iterator(i.get_node())); +#else + x.erase(i); +#endif + + } + } + } + + void splice( + iterator position,sequenced_index& x, + iterator first,iterator last) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,x); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,x); + BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + if(&x==this){ + BOOST_MULTI_INDEX_CHECK_OUTSIDE_RANGE(position,first,last); + if(position!=last)relink( + position.get_node(),first.get_node(),last.get_node()); + } + else{ + while(first!=last){ + if(insert(position,*first).second)first=x.erase(first); + else ++first; + } + } + } + + void remove(value_param_type value) + { + sequenced_index_remove( + *this, + ::boost::bind(std::equal_to(),::boost::arg<1>(),value)); + } + + template + void remove_if(Predicate pred) + { + sequenced_index_remove(*this,pred); + } + + void unique() + { + sequenced_index_unique(*this,std::equal_to()); + } + + template + void unique(BinaryPredicate binary_pred) + { + sequenced_index_unique(*this,binary_pred); + } + + void merge(sequenced_index& x) + { + sequenced_index_merge(*this,x,std::less()); + } + + template + void merge(sequenced_index& x,Compare comp) + { + sequenced_index_merge(*this,x,comp); + } + + void sort() + { + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + sequenced_index_sort(header(),std::less()); + } + + template + void sort(Compare comp) + { + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + sequenced_index_sort(header(),comp); + } + + void reverse()BOOST_NOEXCEPT + { + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + node_impl_type::reverse(header()->impl()); + } + + /* rearrange operations */ + + void relocate(iterator position,iterator i) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(i); + BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(i); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(i,*this); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + if(position!=i)relink(position.get_node(),i.get_node()); + } + + void relocate(iterator position,iterator first,iterator last) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); + BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); + BOOST_MULTI_INDEX_CHECK_OUTSIDE_RANGE(position,first,last); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + if(position!=last)relink( + position.get_node(),first.get_node(),last.get_node()); + } + + template + void rearrange(InputIterator first) + { + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + node_type* pos=header(); + for(size_type s=size();s--;){ + const value_type& v=*first++; + relink(pos,node_from_value(&v)); + } + } + +BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: + sequenced_index(const ctor_args_list& args_list,const allocator_type& al): + super(args_list.get_tail(),al) + { + empty_initialize(); + } + + sequenced_index(const sequenced_index& x): + super(x) + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + ,safe_super() +#endif + + { + /* the actual copying takes place in subsequent call to copy_() */ + } + + sequenced_index( + const sequenced_index& x,do_not_copy_elements_tag): + super(x,do_not_copy_elements_tag()) + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + ,safe_super() +#endif + + { + empty_initialize(); + } + + ~sequenced_index() + { + /* the container is guaranteed to be empty by now */ + } + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + iterator make_iterator(node_type* node){return iterator(node,this);} + const_iterator make_iterator(node_type* node)const + {return const_iterator(node,const_cast(this));} +#else + iterator make_iterator(node_type* node){return iterator(node);} + const_iterator make_iterator(node_type* node)const + {return const_iterator(node);} +#endif + + void copy_( + const sequenced_index& x,const copy_map_type& map) + { + node_type* org=x.header(); + node_type* cpy=header(); + do{ + node_type* next_org=node_type::from_impl(org->next()); + node_type* next_cpy=map.find(static_cast(next_org)); + cpy->next()=next_cpy->impl(); + next_cpy->prior()=cpy->impl(); + org=next_org; + cpy=next_cpy; + }while(org!=x.header()); + + super::copy_(x,map); + } + + template + final_node_type* insert_( + value_param_type v,final_node_type*& x,Variant variant) + { + final_node_type* res=super::insert_(v,x,variant); + if(res==x)link(static_cast(x)); + return res; + } + + template + final_node_type* insert_( + value_param_type v,node_type* position,final_node_type*& x,Variant variant) + { + final_node_type* res=super::insert_(v,position,x,variant); + if(res==x)link(static_cast(x)); + return res; + } + + void erase_(node_type* x) + { + unlink(x); + super::erase_(x); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + } + + void delete_all_nodes_() + { + for(node_type* x=node_type::from_impl(header()->next());x!=header();){ + node_type* y=node_type::from_impl(x->next()); + this->final_delete_node_(static_cast(x)); + x=y; + } + } + + void clear_() + { + super::clear_(); + empty_initialize(); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super::detach_dereferenceable_iterators(); +#endif + } + + void swap_(sequenced_index& x) + { +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super::swap(x); +#endif + + super::swap_(x); + } + + void swap_elements_(sequenced_index& x) + { +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + safe_super::swap(x); +#endif + + super::swap_elements_(x); + } + + template + bool replace_(value_param_type v,node_type* x,Variant variant) + { + return super::replace_(v,x,variant); + } + + bool modify_(node_type* x) + { + BOOST_TRY{ + if(!super::modify_(x)){ + unlink(x); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + + return false; + } + else return true; + } + BOOST_CATCH(...){ + unlink(x); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + detach_iterators(x); +#endif + + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + bool modify_rollback_(node_type* x) + { + return super::modify_rollback_(x); + } + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + /* serialization */ + + template + void save_( + Archive& ar,const unsigned int version,const index_saver_type& sm)const + { + sm.save(begin(),end(),ar,version); + super::save_(ar,version,sm); + } + + template + void load_( + Archive& ar,const unsigned int version,const index_loader_type& lm) + { + lm.load( + ::boost::bind( + &sequenced_index::rearranger,this,::boost::arg<1>(),::boost::arg<2>()), + ar,version); + super::load_(ar,version,lm); + } +#endif + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) + /* invariant stuff */ + + bool invariant_()const + { + if(size()==0||begin()==end()){ + if(size()!=0||begin()!=end()|| + header()->next()!=header()->impl()|| + header()->prior()!=header()->impl())return false; + } + else{ + size_type s=0; + for(const_iterator it=begin(),it_end=end();it!=it_end;++it,++s){ + if(it.get_node()->next()->prior()!=it.get_node()->impl())return false; + if(it.get_node()->prior()->next()!=it.get_node()->impl())return false; + } + if(s!=size())return false; + } + + return super::invariant_(); + } + + /* This forwarding function eases things for the boost::mem_fn construct + * in BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT. Actually, + * final_check_invariant is already an inherited member function of index. + */ + void check_invariant_()const{this->final_check_invariant_();} +#endif + +private: + node_type* header()const{return this->final_header();} + + void empty_initialize() + { + header()->prior()=header()->next()=header()->impl(); + } + + void link(node_type* x) + { + node_impl_type::link(x->impl(),header()->impl()); + }; + + static void unlink(node_type* x) + { + node_impl_type::unlink(x->impl()); + } + + static void relink(node_type* position,node_type* x) + { + node_impl_type::relink(position->impl(),x->impl()); + } + + static void relink(node_type* position,node_type* first,node_type* last) + { + node_impl_type::relink( + position->impl(),first->impl(),last->impl()); + } + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + void rearranger(node_type* position,node_type *x) + { + if(!position)position=header(); + node_type::increment(position); + if(position!=x)relink(position,x); + } +#endif + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + void detach_iterators(node_type* x) + { + iterator it=make_iterator(x); + safe_mode::detach_equivalent_iterators(it); + } +#endif + + template + void assign_iter(InputIterator first,InputIterator last,mpl::true_) + { + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + clear(); + for(;first!=last;++first)this->final_insert_ref_(*first); + } + + void assign_iter(size_type n,value_param_type value,mpl::false_) + { + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + clear(); + for(size_type i=0;i + void insert_iter( + iterator position,InputIterator first,InputIterator last,mpl::true_) + { + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + for(;first!=last;++first){ + std::pair p= + this->final_insert_ref_(*first); + if(p.second&&position.get_node()!=header()){ + relink(position.get_node(),p.first); + } + } + } + + void insert_iter( + iterator position,size_type n,value_param_type x,mpl::false_) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + for(size_type i=0;i + std::pair emplace_front_impl( + BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + return emplace_impl(begin(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + } + + template + std::pair emplace_back_impl( + BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + return emplace_impl(end(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + } + + template + std::pair emplace_impl( + iterator position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); + BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); + BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; + std::pair p= + this->final_emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + if(p.second&&position.get_node()!=header()){ + relink(position.get_node(),p.first); + } + return std::pair(make_iterator(p.first),p.second); + } + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ + BOOST_WORKAROUND(__MWERKS__,<=0x3003) +#pragma parse_mfunc_templ reset +#endif +}; + +/* comparison */ + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator==( + const sequenced_index& x, + const sequenced_index& y) +{ + return x.size()==y.size()&&std::equal(x.begin(),x.end(),y.begin()); +} + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator<( + const sequenced_index& x, + const sequenced_index& y) +{ + return std::lexicographical_compare(x.begin(),x.end(),y.begin(),y.end()); +} + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator!=( + const sequenced_index& x, + const sequenced_index& y) +{ + return !(x==y); +} + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator>( + const sequenced_index& x, + const sequenced_index& y) +{ + return y +bool operator>=( + const sequenced_index& x, + const sequenced_index& y) +{ + return !(x +bool operator<=( + const sequenced_index& x, + const sequenced_index& y) +{ + return !(x>y); +} + +/* specialized algorithms */ + +template +void swap( + sequenced_index& x, + sequenced_index& y) +{ + x.swap(y); +} + +} /* namespace multi_index::detail */ + +/* sequenced index specifier */ + +template +struct sequenced +{ + BOOST_STATIC_ASSERT(detail::is_tag::value); + + template + struct node_class + { + typedef detail::sequenced_index_node type; + }; + + template + struct index_class + { + typedef detail::sequenced_index type; + }; +}; + +} /* namespace multi_index */ + +} /* namespace boost */ + +/* Boost.Foreach compatibility */ + +template +inline boost::mpl::true_* boost_foreach_is_noncopyable( + boost::multi_index::detail::sequenced_index*&, + boost_foreach_argument_dependent_lookup_hack) +{ + return 0; +} + +#undef BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT +#undef BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index_fwd.hpp new file mode 100644 index 00000000000..a019f2a6d2f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index_fwd.hpp @@ -0,0 +1,91 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_SEQUENCED_INDEX_FWD_HPP +#define BOOST_MULTI_INDEX_SEQUENCED_INDEX_FWD_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include + +namespace boost{ + +namespace multi_index{ + +namespace detail{ + +template +class sequenced_index; + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator==( + const sequenced_index& x, + const sequenced_index& y); + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator<( + const sequenced_index& x, + const sequenced_index& y); + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator!=( + const sequenced_index& x, + const sequenced_index& y); + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator>( + const sequenced_index& x, + const sequenced_index& y); + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator>=( + const sequenced_index& x, + const sequenced_index& y); + +template< + typename SuperMeta1,typename TagList1, + typename SuperMeta2,typename TagList2 +> +bool operator<=( + const sequenced_index& x, + const sequenced_index& y); + +template +void swap( + sequenced_index& x, + sequenced_index& y); + +} /* namespace multi_index::detail */ + +/* index specifiers */ + +template > +struct sequenced; + +} /* namespace multi_index */ + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/tag.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/tag.hpp new file mode 100644 index 00000000000..ce51f8241ee --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index/tag.hpp @@ -0,0 +1,88 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_TAG_HPP +#define BOOST_MULTI_INDEX_TAG_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +/* A wrapper of mpl::vector used to hide MPL from the user. + * tag contains types used as tag names for indices in get() functions. + */ + +/* This user_definable macro limits the number of elements of a tag; + * useful for shortening resulting symbol names (MSVC++ 6.0, for instance, + * has problems coping with very long symbol names.) + */ + +#if !defined(BOOST_MULTI_INDEX_LIMIT_TAG_SIZE) +#define BOOST_MULTI_INDEX_LIMIT_TAG_SIZE BOOST_MPL_LIMIT_VECTOR_SIZE +#endif + +#if BOOST_MULTI_INDEX_LIMIT_TAG_SIZE +struct is_tag +{ + BOOST_STATIC_CONSTANT(bool,value=(is_base_and_derived::value)); +}; + +} /* namespace multi_index::detail */ + +template< + BOOST_PP_ENUM_BINARY_PARAMS( + BOOST_MULTI_INDEX_TAG_SIZE, + typename T, + =mpl::na BOOST_PP_INTERCEPT) +> +struct tag:private detail::tag_marker +{ + /* The mpl::transform pass produces shorter symbols (without + * trailing mpl::na's.) + */ + + typedef typename mpl::transform< + mpl::vector, + mpl::identity + >::type type; + + BOOST_STATIC_ASSERT(detail::no_duplicate_tags::value); +}; + +} /* namespace multi_index */ + +} /* namespace boost */ + +#undef BOOST_MULTI_INDEX_TAG_SIZE + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index_container.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index_container.hpp new file mode 100644 index 00000000000..9993a8dfa10 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index_container.hpp @@ -0,0 +1,1362 @@ +/* Multiply indexed container. + * + * Copyright 2003-2014 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_HPP +#define BOOST_MULTI_INDEX_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) +#include +#endif + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +#include +#include +#include +#include +#include +#include +#include +#endif + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) +#include +#define BOOST_MULTI_INDEX_CHECK_INVARIANT_OF(x) \ + detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ + detail::make_obj_guard(x,&multi_index_container::check_invariant_); \ + BOOST_JOIN(check_invariant_,__LINE__).touch(); +#define BOOST_MULTI_INDEX_CHECK_INVARIANT \ + BOOST_MULTI_INDEX_CHECK_INVARIANT_OF(*this) +#else +#define BOOST_MULTI_INDEX_CHECK_INVARIANT_OF(x) +#define BOOST_MULTI_INDEX_CHECK_INVARIANT +#endif + +namespace boost{ + +namespace multi_index{ + +#if BOOST_WORKAROUND(BOOST_MSVC,BOOST_TESTED_AT(1500)) +#pragma warning(push) +#pragma warning(disable:4522) /* spurious warning on multiple operator=()'s */ +#endif + +template +class multi_index_container: + private ::boost::base_from_member< + typename boost::detail::allocator::rebind_to< + Allocator, + typename detail::multi_index_node_type< + Value,IndexSpecifierList,Allocator>::type + >::type>, + BOOST_MULTI_INDEX_PRIVATE_IF_MEMBER_TEMPLATE_FRIENDS detail::header_holder< + typename boost::detail::allocator::rebind_to< + Allocator, + typename detail::multi_index_node_type< + Value,IndexSpecifierList,Allocator>::type + >::type::pointer, + multi_index_container >, + public detail::multi_index_base_type< + Value,IndexSpecifierList,Allocator>::type +{ +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ + BOOST_WORKAROUND(__MWERKS__,<=0x3003) +/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the + * lifetime of const references bound to temporaries --precisely what + * scopeguards are. + */ + +#pragma parse_mfunc_templ off +#endif + +private: + BOOST_COPYABLE_AND_MOVABLE(multi_index_container) + +#if !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) + template friend class detail::index_base; + template friend struct detail::header_holder; + template friend struct detail::converter; +#endif + + typedef typename detail::multi_index_base_type< + Value,IndexSpecifierList,Allocator>::type super; + typedef typename + boost::detail::allocator::rebind_to< + Allocator, + typename super::node_type + >::type node_allocator; + typedef ::boost::base_from_member< + node_allocator> bfm_allocator; + typedef detail::header_holder< + typename node_allocator::pointer, + multi_index_container> bfm_header; + + +public: + /* All types are inherited from super, a few are explicitly + * brought forward here to save us some typename's. + */ + + typedef typename super::ctor_args_list ctor_args_list; + typedef IndexSpecifierList index_specifier_type_list; + + typedef typename super::index_type_list index_type_list; + + typedef typename super::iterator_type_list iterator_type_list; + typedef typename super::const_iterator_type_list const_iterator_type_list; + typedef typename super::value_type value_type; + typedef typename super::final_allocator_type allocator_type; + typedef typename super::iterator iterator; + typedef typename super::const_iterator const_iterator; + + BOOST_STATIC_ASSERT( + detail::no_duplicate_tags_in_index_list::value); + + /* global project() needs to see this publicly */ + + typedef typename super::node_type node_type; + + /* construct/copy/destroy */ + + explicit multi_index_container( + +#if BOOST_WORKAROUND(__IBMCPP__,<=600) + /* VisualAge seems to have an ETI issue with the default values + * for arguments args_list and al. + */ + + const ctor_args_list& args_list= + typename mpl::identity::type:: + ctor_args_list(), + const allocator_type& al= + typename mpl::identity::type:: + allocator_type()): +#else + const ctor_args_list& args_list=ctor_args_list(), + const allocator_type& al=allocator_type()): +#endif + + bfm_allocator(al), + super(args_list,bfm_allocator::member), + node_count(0) + { + BOOST_MULTI_INDEX_CHECK_INVARIANT; + } + + explicit multi_index_container(const allocator_type& al): + bfm_allocator(al), + super(ctor_args_list(),bfm_allocator::member), + node_count(0) + { + BOOST_MULTI_INDEX_CHECK_INVARIANT; + } + + template + multi_index_container( + InputIterator first,InputIterator last, + +#if BOOST_WORKAROUND(__IBMCPP__,<=600) + /* VisualAge seems to have an ETI issue with the default values + * for arguments args_list and al. + */ + + const ctor_args_list& args_list= + typename mpl::identity::type:: + ctor_args_list(), + const allocator_type& al= + typename mpl::identity::type:: + allocator_type()): +#else + const ctor_args_list& args_list=ctor_args_list(), + const allocator_type& al=allocator_type()): +#endif + + bfm_allocator(al), + super(args_list,bfm_allocator::member), + node_count(0) + { + BOOST_MULTI_INDEX_CHECK_INVARIANT; + BOOST_TRY{ + iterator hint=super::end(); + for(;first!=last;++first){ + hint=super::make_iterator( + insert_ref_(*first,hint.get_node()).first); + ++hint; + } + } + BOOST_CATCH(...){ + clear_(); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) + multi_index_container( + std::initializer_list list, + const ctor_args_list& args_list=ctor_args_list(), + const allocator_type& al=allocator_type()): + bfm_allocator(al), + super(args_list,bfm_allocator::member), + node_count(0) + { + BOOST_MULTI_INDEX_CHECK_INVARIANT; + BOOST_TRY{ + typedef const Value* init_iterator; + + iterator hint=super::end(); + for(init_iterator first=list.begin(),last=list.end(); + first!=last;++first){ + hint=super::make_iterator(insert_(*first,hint.get_node()).first); + ++hint; + } + } + BOOST_CATCH(...){ + clear_(); + BOOST_RETHROW; + } + BOOST_CATCH_END + } +#endif + + multi_index_container( + const multi_index_container& x): + bfm_allocator(x.bfm_allocator::member), + bfm_header(), + super(x), + node_count(0) + { + copy_map_type map(bfm_allocator::member,x.size(),x.header(),header()); + for(const_iterator it=x.begin(),it_end=x.end();it!=it_end;++it){ + map.clone(it.get_node()); + } + super::copy_(x,map); + map.release(); + node_count=x.size(); + + /* Not until this point are the indices required to be consistent, + * hence the position of the invariant checker. + */ + + BOOST_MULTI_INDEX_CHECK_INVARIANT; + } + + multi_index_container(BOOST_RV_REF(multi_index_container) x): + bfm_allocator(x.bfm_allocator::member), + bfm_header(), + super(x,detail::do_not_copy_elements_tag()), + node_count(0) + { + BOOST_MULTI_INDEX_CHECK_INVARIANT; + BOOST_MULTI_INDEX_CHECK_INVARIANT_OF(x); + swap_elements_(x); + } + + ~multi_index_container() + { + delete_all_nodes_(); + } + +#if defined(BOOST_NO_CXX11_RVALUE_REFERENCES) + /* As per http://www.boost.org/doc/html/move/emulation_limitations.html + * #move.emulation_limitations.assignment_operator + */ + + multi_index_container& operator=( + const multi_index_container& x) + { + multi_index_container y(x); + this->swap(y); + return *this; + } +#endif + + multi_index_container& operator=( + BOOST_COPY_ASSIGN_REF(multi_index_container) x) + { + multi_index_container y(x); + this->swap(y); + return *this; + } + + multi_index_container& operator=( + BOOST_RV_REF(multi_index_container) x) + { + this->swap(x); + return *this; + } + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) + multi_index_container& operator=( + std::initializer_list list) + { + BOOST_MULTI_INDEX_CHECK_INVARIANT; + typedef const Value* init_iterator; + + multi_index_container x(*this,detail::do_not_copy_elements_tag()); + iterator hint=x.end(); + for(init_iterator first=list.begin(),last=list.end(); + first!=last;++first){ + hint=x.make_iterator(x.insert_(*first,hint.get_node()).first); + ++hint; + } + x.swap_elements_(*this); + return*this; + } +#endif + + allocator_type get_allocator()const BOOST_NOEXCEPT + { + return allocator_type(bfm_allocator::member); + } + + /* retrieval of indices by number */ + +#if !defined(BOOST_NO_MEMBER_TEMPLATES) + template + struct nth_index + { + BOOST_STATIC_ASSERT(N>=0&&N::type::value); + typedef typename mpl::at_c::type type; + }; + + template + typename nth_index::type& get()BOOST_NOEXCEPT + { + BOOST_STATIC_ASSERT(N>=0&&N::type::value); + return *this; + } + + template + const typename nth_index::type& get()const BOOST_NOEXCEPT + { + BOOST_STATIC_ASSERT(N>=0&&N::type::value); + return *this; + } +#endif + + /* retrieval of indices by tag */ + +#if !defined(BOOST_NO_MEMBER_TEMPLATES) + template + struct index + { + typedef typename mpl::find_if< + index_type_list, + detail::has_tag + >::type iter; + + BOOST_STATIC_CONSTANT( + bool,index_found=!(is_same::type >::value)); + BOOST_STATIC_ASSERT(index_found); + + typedef typename mpl::deref::type type; + }; + + template + typename index::type& get()BOOST_NOEXCEPT + { + return *this; + } + + template + const typename index::type& get()const BOOST_NOEXCEPT + { + return *this; + } +#endif + + /* projection of iterators by number */ + +#if !defined(BOOST_NO_MEMBER_TEMPLATES) + template + struct nth_index_iterator + { + typedef typename nth_index::type::iterator type; + }; + + template + struct nth_index_const_iterator + { + typedef typename nth_index::type::const_iterator type; + }; + + template + typename nth_index_iterator::type project(IteratorType it) + { + typedef typename nth_index::type index_type; + +#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* fails in Sun C++ 5.7 */ + BOOST_STATIC_ASSERT( + (mpl::contains::value)); +#endif + + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); + BOOST_MULTI_INDEX_CHECK_IS_OWNER( + it,static_cast(*this)); + + return index_type::make_iterator(static_cast(it.get_node())); + } + + template + typename nth_index_const_iterator::type project(IteratorType it)const + { + typedef typename nth_index::type index_type; + +#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* fails in Sun C++ 5.7 */ + BOOST_STATIC_ASSERT(( + mpl::contains::value|| + mpl::contains::value)); +#endif + + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); + BOOST_MULTI_INDEX_CHECK_IS_OWNER( + it,static_cast(*this)); + return index_type::make_iterator(static_cast(it.get_node())); + } +#endif + + /* projection of iterators by tag */ + +#if !defined(BOOST_NO_MEMBER_TEMPLATES) + template + struct index_iterator + { + typedef typename index::type::iterator type; + }; + + template + struct index_const_iterator + { + typedef typename index::type::const_iterator type; + }; + + template + typename index_iterator::type project(IteratorType it) + { + typedef typename index::type index_type; + +#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* fails in Sun C++ 5.7 */ + BOOST_STATIC_ASSERT( + (mpl::contains::value)); +#endif + + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); + BOOST_MULTI_INDEX_CHECK_IS_OWNER( + it,static_cast(*this)); + return index_type::make_iterator(static_cast(it.get_node())); + } + + template + typename index_const_iterator::type project(IteratorType it)const + { + typedef typename index::type index_type; + +#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* fails in Sun C++ 5.7 */ + BOOST_STATIC_ASSERT(( + mpl::contains::value|| + mpl::contains::value)); +#endif + + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); + BOOST_MULTI_INDEX_CHECK_IS_OWNER( + it,static_cast(*this)); + return index_type::make_iterator(static_cast(it.get_node())); + } +#endif + +BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: + typedef typename super::copy_map_type copy_map_type; + +#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) + multi_index_container( + const multi_index_container& x, + detail::do_not_copy_elements_tag): + bfm_allocator(x.bfm_allocator::member), + bfm_header(), + super(x,detail::do_not_copy_elements_tag()), + node_count(0) + { + BOOST_MULTI_INDEX_CHECK_INVARIANT; + } +#endif + + node_type* header()const + { + return &*bfm_header::member; + } + + node_type* allocate_node() + { + return &*bfm_allocator::member.allocate(1); + } + + void deallocate_node(node_type* x) + { + typedef typename node_allocator::pointer node_pointer; + bfm_allocator::member.deallocate(static_cast(x),1); + } + + bool empty_()const + { + return node_count==0; + } + + std::size_t size_()const + { + return node_count; + } + + std::size_t max_size_()const + { + return static_cast(-1); + } + + template + std::pair insert_(const Value& v,Variant variant) + { + node_type* x=0; + node_type* res=super::insert_(v,x,variant); + if(res==x){ + ++node_count; + return std::pair(res,true); + } + else{ + return std::pair(res,false); + } + } + + std::pair insert_(const Value& v) + { + return insert_(v,detail::lvalue_tag()); + } + + std::pair insert_rv_(const Value& v) + { + return insert_(v,detail::rvalue_tag()); + } + + template + std::pair insert_ref_(T& t) + { + node_type* x=allocate_node(); + BOOST_TRY{ + new(&x->value()) value_type(t); + BOOST_TRY{ + node_type* res=super::insert_(x->value(),x,detail::emplaced_tag()); + if(res==x){ + ++node_count; + return std::pair(res,true); + } + else{ + boost::detail::allocator::destroy(&x->value()); + deallocate_node(x); + return std::pair(res,false); + } + } + BOOST_CATCH(...){ + boost::detail::allocator::destroy(&x->value()); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + BOOST_CATCH(...){ + deallocate_node(x); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + std::pair insert_ref_(const value_type& x) + { + return insert_(x); + } + + std::pair insert_ref_(value_type& x) + { + return insert_(x); + } + + template + std::pair emplace_( + BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + node_type* x=allocate_node(); + BOOST_TRY{ + detail::vartempl_placement_new( + &x->value(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + BOOST_TRY{ + node_type* res=super::insert_(x->value(),x,detail::emplaced_tag()); + if(res==x){ + ++node_count; + return std::pair(res,true); + } + else{ + boost::detail::allocator::destroy(&x->value()); + deallocate_node(x); + return std::pair(res,false); + } + } + BOOST_CATCH(...){ + boost::detail::allocator::destroy(&x->value()); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + BOOST_CATCH(...){ + deallocate_node(x); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + template + std::pair insert_( + const Value& v,node_type* position,Variant variant) + { + node_type* x=0; + node_type* res=super::insert_(v,position,x,variant); + if(res==x){ + ++node_count; + return std::pair(res,true); + } + else{ + return std::pair(res,false); + } + } + + std::pair insert_(const Value& v,node_type* position) + { + return insert_(v,position,detail::lvalue_tag()); + } + + std::pair insert_rv_(const Value& v,node_type* position) + { + return insert_(v,position,detail::rvalue_tag()); + } + + template + std::pair insert_ref_( + T& t,node_type* position) + { + node_type* x=allocate_node(); + BOOST_TRY{ + new(&x->value()) value_type(t); + BOOST_TRY{ + node_type* res=super::insert_( + x->value(),position,x,detail::emplaced_tag()); + if(res==x){ + ++node_count; + return std::pair(res,true); + } + else{ + boost::detail::allocator::destroy(&x->value()); + deallocate_node(x); + return std::pair(res,false); + } + } + BOOST_CATCH(...){ + boost::detail::allocator::destroy(&x->value()); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + BOOST_CATCH(...){ + deallocate_node(x); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + std::pair insert_ref_( + const value_type& x,node_type* position) + { + return insert_(x,position); + } + + std::pair insert_ref_( + value_type& x,node_type* position) + { + return insert_(x,position); + } + + template + std::pair emplace_hint_( + node_type* position, + BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) + { + node_type* x=allocate_node(); + BOOST_TRY{ + detail::vartempl_placement_new( + &x->value(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); + BOOST_TRY{ + node_type* res=super::insert_( + x->value(),position,x,detail::emplaced_tag()); + if(res==x){ + ++node_count; + return std::pair(res,true); + } + else{ + boost::detail::allocator::destroy(&x->value()); + deallocate_node(x); + return std::pair(res,false); + } + } + BOOST_CATCH(...){ + boost::detail::allocator::destroy(&x->value()); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + BOOST_CATCH(...){ + deallocate_node(x); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + void erase_(node_type* x) + { + --node_count; + super::erase_(x); + deallocate_node(x); + } + + void delete_node_(node_type* x) + { + super::delete_node_(x); + deallocate_node(x); + } + + void delete_all_nodes_() + { + super::delete_all_nodes_(); + } + + void clear_() + { + delete_all_nodes_(); + super::clear_(); + node_count=0; + } + + void swap_(multi_index_container& x) + { + if(bfm_allocator::member!=x.bfm_allocator::member){ + detail::adl_swap(bfm_allocator::member,x.bfm_allocator::member); + } + std::swap(bfm_header::member,x.bfm_header::member); + super::swap_(x); + std::swap(node_count,x.node_count); + } + + void swap_elements_( + multi_index_container& x) + { + std::swap(bfm_header::member,x.bfm_header::member); + super::swap_elements_(x); + std::swap(node_count,x.node_count); + } + + bool replace_(const Value& k,node_type* x) + { + return super::replace_(k,x,detail::lvalue_tag()); + } + + bool replace_rv_(const Value& k,node_type* x) + { + return super::replace_(k,x,detail::rvalue_tag()); + } + + template + bool modify_(Modifier& mod,node_type* x) + { + mod(const_cast(x->value())); + + BOOST_TRY{ + if(!super::modify_(x)){ + deallocate_node(x); + --node_count; + return false; + } + else return true; + } + BOOST_CATCH(...){ + deallocate_node(x); + --node_count; + BOOST_RETHROW; + } + BOOST_CATCH_END + } + + template + bool modify_(Modifier& mod,Rollback& back_,node_type* x) + { + mod(const_cast(x->value())); + + bool b; + BOOST_TRY{ + b=super::modify_rollback_(x); + } + BOOST_CATCH(...){ + BOOST_TRY{ + back_(const_cast(x->value())); + BOOST_RETHROW; + } + BOOST_CATCH(...){ + this->erase_(x); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + BOOST_CATCH_END + + BOOST_TRY{ + if(!b){ + back_(const_cast(x->value())); + return false; + } + else return true; + } + BOOST_CATCH(...){ + this->erase_(x); + BOOST_RETHROW; + } + BOOST_CATCH_END + } + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) + /* serialization */ + + friend class boost::serialization::access; + + BOOST_SERIALIZATION_SPLIT_MEMBER() + + typedef typename super::index_saver_type index_saver_type; + typedef typename super::index_loader_type index_loader_type; + + template + void save(Archive& ar,const unsigned int version)const + { + const serialization::collection_size_type s(size_()); + const detail::serialization_version value_version; + ar< + void load(Archive& ar,const unsigned int version) + { + BOOST_MULTI_INDEX_CHECK_INVARIANT; + + clear_(); + serialization::collection_size_type s; + detail::serialization_version value_version; + if(version<1){ + std::size_t sz; + ar>>serialization::make_nvp("count",sz); + s=static_cast(sz); + } + else{ + ar>>serialization::make_nvp("count",s); + } + if(version<2){ + value_version=0; + } + else{ + ar>>serialization::make_nvp("value_version",value_version); + } + + index_loader_type lm(bfm_allocator::member,s); + + for(std::size_t n=0;n value("item",ar,value_version); + std::pair p=insert_( + value.get(),super::end().get_node()); + if(!p.second)throw_exception( + archive::archive_exception( + archive::archive_exception::other_exception)); + ar.reset_object_address(&p.first->value(),&value.get()); + lm.add(p.first,ar,version); + } + lm.add_track(header(),ar,version); + + super::load_(ar,version,lm); + } +#endif + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) + /* invariant stuff */ + + bool invariant_()const + { + return super::invariant_(); + } + + void check_invariant_()const + { + BOOST_MULTI_INDEX_INVARIANT_ASSERT(invariant_()); + } +#endif + +private: + std::size_t node_count; + +#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ + BOOST_WORKAROUND(__MWERKS__,<=0x3003) +#pragma parse_mfunc_templ reset +#endif +}; + +#if BOOST_WORKAROUND(BOOST_MSVC,BOOST_TESTED_AT(1500)) +#pragma warning(pop) /* C4522 */ +#endif + +/* retrieval of indices by number */ + +template +struct nth_index +{ + BOOST_STATIC_CONSTANT( + int, + M=mpl::size::type::value); + BOOST_STATIC_ASSERT(N>=0&&N::type type; +}; + +template +typename nth_index< + multi_index_container,N>::type& +get( + multi_index_container& m)BOOST_NOEXCEPT +{ + typedef multi_index_container< + Value,IndexSpecifierList,Allocator> multi_index_type; + typedef typename nth_index< + multi_index_container< + Value,IndexSpecifierList,Allocator>, + N + >::type index_type; + + BOOST_STATIC_ASSERT(N>=0&& + N< + mpl::size< + BOOST_DEDUCED_TYPENAME multi_index_type::index_type_list + >::type::value); + + return detail::converter::index(m); +} + +template +const typename nth_index< + multi_index_container,N>::type& +get( + const multi_index_container& m +)BOOST_NOEXCEPT +{ + typedef multi_index_container< + Value,IndexSpecifierList,Allocator> multi_index_type; + typedef typename nth_index< + multi_index_container< + Value,IndexSpecifierList,Allocator>, + N + >::type index_type; + + BOOST_STATIC_ASSERT(N>=0&& + N< + mpl::size< + BOOST_DEDUCED_TYPENAME multi_index_type::index_type_list + >::type::value); + + return detail::converter::index(m); +} + +/* retrieval of indices by tag */ + +template +struct index +{ + typedef typename MultiIndexContainer::index_type_list index_type_list; + + typedef typename mpl::find_if< + index_type_list, + detail::has_tag + >::type iter; + + BOOST_STATIC_CONSTANT( + bool,index_found=!(is_same::type >::value)); + BOOST_STATIC_ASSERT(index_found); + + typedef typename mpl::deref::type type; +}; + +template< + typename Tag,typename Value,typename IndexSpecifierList,typename Allocator +> +typename ::boost::multi_index::index< + multi_index_container,Tag>::type& +get( + multi_index_container& m)BOOST_NOEXCEPT +{ + typedef multi_index_container< + Value,IndexSpecifierList,Allocator> multi_index_type; + typedef typename ::boost::multi_index::index< + multi_index_container< + Value,IndexSpecifierList,Allocator>, + Tag + >::type index_type; + + return detail::converter::index(m); +} + +template< + typename Tag,typename Value,typename IndexSpecifierList,typename Allocator +> +const typename ::boost::multi_index::index< + multi_index_container,Tag>::type& +get( + const multi_index_container& m +)BOOST_NOEXCEPT +{ + typedef multi_index_container< + Value,IndexSpecifierList,Allocator> multi_index_type; + typedef typename ::boost::multi_index::index< + multi_index_container< + Value,IndexSpecifierList,Allocator>, + Tag + >::type index_type; + + return detail::converter::index(m); +} + +/* projection of iterators by number */ + +template +struct nth_index_iterator +{ + typedef typename nth_index::type::iterator type; +}; + +template +struct nth_index_const_iterator +{ + typedef typename nth_index::type::const_iterator type; +}; + +template< + int N,typename IteratorType, + typename Value,typename IndexSpecifierList,typename Allocator> +typename nth_index_iterator< + multi_index_container,N>::type +project( + multi_index_container& m, + IteratorType it) +{ + typedef multi_index_container< + Value,IndexSpecifierList,Allocator> multi_index_type; + typedef typename nth_index::type index_type; + +#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* Sun C++ 5.7 fails */ + BOOST_STATIC_ASSERT(( + mpl::contains< + BOOST_DEDUCED_TYPENAME multi_index_type::iterator_type_list, + IteratorType>::value)); +#endif + + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + typedef detail::converter< + multi_index_type, + BOOST_DEDUCED_TYPENAME IteratorType::container_type> converter; + BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,converter::index(m)); +#endif + + return detail::converter::iterator( + m,static_cast(it.get_node())); +} + +template< + int N,typename IteratorType, + typename Value,typename IndexSpecifierList,typename Allocator> +typename nth_index_const_iterator< + multi_index_container,N>::type +project( + const multi_index_container& m, + IteratorType it) +{ + typedef multi_index_container< + Value,IndexSpecifierList,Allocator> multi_index_type; + typedef typename nth_index::type index_type; + +#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* Sun C++ 5.7 fails */ + BOOST_STATIC_ASSERT(( + mpl::contains< + BOOST_DEDUCED_TYPENAME multi_index_type::iterator_type_list, + IteratorType>::value|| + mpl::contains< + BOOST_DEDUCED_TYPENAME multi_index_type::const_iterator_type_list, + IteratorType>::value)); +#endif + + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + typedef detail::converter< + multi_index_type, + BOOST_DEDUCED_TYPENAME IteratorType::container_type> converter; + BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,converter::index(m)); +#endif + + return detail::converter::const_iterator( + m,static_cast(it.get_node())); +} + +/* projection of iterators by tag */ + +template +struct index_iterator +{ + typedef typename ::boost::multi_index::index< + MultiIndexContainer,Tag>::type::iterator type; +}; + +template +struct index_const_iterator +{ + typedef typename ::boost::multi_index::index< + MultiIndexContainer,Tag>::type::const_iterator type; +}; + +template< + typename Tag,typename IteratorType, + typename Value,typename IndexSpecifierList,typename Allocator> +typename index_iterator< + multi_index_container,Tag>::type +project( + multi_index_container& m, + IteratorType it) +{ + typedef multi_index_container< + Value,IndexSpecifierList,Allocator> multi_index_type; + typedef typename ::boost::multi_index::index< + multi_index_type,Tag>::type index_type; + +#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* Sun C++ 5.7 fails */ + BOOST_STATIC_ASSERT(( + mpl::contains< + BOOST_DEDUCED_TYPENAME multi_index_type::iterator_type_list, + IteratorType>::value)); +#endif + + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + typedef detail::converter< + multi_index_type, + BOOST_DEDUCED_TYPENAME IteratorType::container_type> converter; + BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,converter::index(m)); +#endif + + return detail::converter::iterator( + m,static_cast(it.get_node())); +} + +template< + typename Tag,typename IteratorType, + typename Value,typename IndexSpecifierList,typename Allocator> +typename index_const_iterator< + multi_index_container,Tag>::type +project( + const multi_index_container& m, + IteratorType it) +{ + typedef multi_index_container< + Value,IndexSpecifierList,Allocator> multi_index_type; + typedef typename ::boost::multi_index::index< + multi_index_type,Tag>::type index_type; + +#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* Sun C++ 5.7 fails */ + BOOST_STATIC_ASSERT(( + mpl::contains< + BOOST_DEDUCED_TYPENAME multi_index_type::iterator_type_list, + IteratorType>::value|| + mpl::contains< + BOOST_DEDUCED_TYPENAME multi_index_type::const_iterator_type_list, + IteratorType>::value)); +#endif + + BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); + +#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) + typedef detail::converter< + multi_index_type, + BOOST_DEDUCED_TYPENAME IteratorType::container_type> converter; + BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,converter::index(m)); +#endif + + return detail::converter::const_iterator( + m,static_cast(it.get_node())); +} + +/* Comparison. Simple forward to first index. */ + +template< + typename Value1,typename IndexSpecifierList1,typename Allocator1, + typename Value2,typename IndexSpecifierList2,typename Allocator2 +> +bool operator==( + const multi_index_container& x, + const multi_index_container& y) +{ + return get<0>(x)==get<0>(y); +} + +template< + typename Value1,typename IndexSpecifierList1,typename Allocator1, + typename Value2,typename IndexSpecifierList2,typename Allocator2 +> +bool operator<( + const multi_index_container& x, + const multi_index_container& y) +{ + return get<0>(x)(y); +} + +template< + typename Value1,typename IndexSpecifierList1,typename Allocator1, + typename Value2,typename IndexSpecifierList2,typename Allocator2 +> +bool operator!=( + const multi_index_container& x, + const multi_index_container& y) +{ + return get<0>(x)!=get<0>(y); +} + +template< + typename Value1,typename IndexSpecifierList1,typename Allocator1, + typename Value2,typename IndexSpecifierList2,typename Allocator2 +> +bool operator>( + const multi_index_container& x, + const multi_index_container& y) +{ + return get<0>(x)>get<0>(y); +} + +template< + typename Value1,typename IndexSpecifierList1,typename Allocator1, + typename Value2,typename IndexSpecifierList2,typename Allocator2 +> +bool operator>=( + const multi_index_container& x, + const multi_index_container& y) +{ + return get<0>(x)>=get<0>(y); +} + +template< + typename Value1,typename IndexSpecifierList1,typename Allocator1, + typename Value2,typename IndexSpecifierList2,typename Allocator2 +> +bool operator<=( + const multi_index_container& x, + const multi_index_container& y) +{ + return get<0>(x)<=get<0>(y); +} + +/* specialized algorithms */ + +template +void swap( + multi_index_container& x, + multi_index_container& y) +{ + x.swap(y); +} + +} /* namespace multi_index */ + +#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) +/* class version = 1 : we now serialize the size through + * boost::serialization::collection_size_type. + * class version = 2 : proper use of {save|load}_construct_data. + */ + +namespace serialization { +template +struct version< + boost::multi_index_container +> +{ + BOOST_STATIC_CONSTANT(int,value=2); +}; +} /* namespace serialization */ +#endif + +/* Associated global functions are promoted to namespace boost, except + * comparison operators and swap, which are meant to be Koenig looked-up. + */ + +using multi_index::get; +using multi_index::project; + +} /* namespace boost */ + +#undef BOOST_MULTI_INDEX_CHECK_INVARIANT +#undef BOOST_MULTI_INDEX_CHECK_INVARIANT_OF + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index_container_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index_container_fwd.hpp new file mode 100644 index 00000000000..b35acad407a --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/multi_index_container_fwd.hpp @@ -0,0 +1,121 @@ +/* Copyright 2003-2013 Joaquin M Lopez Munoz. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * http://www.boost.org/LICENSE_1_0.txt) + * + * See http://www.boost.org/libs/multi_index for library home page. + */ + +#ifndef BOOST_MULTI_INDEX_FWD_HPP +#define BOOST_MULTI_INDEX_FWD_HPP + +#if defined(_MSC_VER) +#pragma once +#endif + +#include /* keep it first to prevent nasty warns in MSVC */ +#include +#include +#include +#include + +namespace boost{ + +namespace multi_index{ + +/* Default value for IndexSpecifierList specifies a container + * equivalent to std::set. + */ + +template< + typename Value, + typename IndexSpecifierList=indexed_by > >, + typename Allocator=std::allocator > +class multi_index_container; + +template +struct nth_index; + +template +struct index; + +template +struct nth_index_iterator; + +template +struct nth_index_const_iterator; + +template +struct index_iterator; + +template +struct index_const_iterator; + +/* get and project functions not fwd declared due to problems + * with dependent typenames + */ + +template< + typename Value1,typename IndexSpecifierList1,typename Allocator1, + typename Value2,typename IndexSpecifierList2,typename Allocator2 +> +bool operator==( + const multi_index_container& x, + const multi_index_container& y); + +template< + typename Value1,typename IndexSpecifierList1,typename Allocator1, + typename Value2,typename IndexSpecifierList2,typename Allocator2 +> +bool operator<( + const multi_index_container& x, + const multi_index_container& y); + +template< + typename Value1,typename IndexSpecifierList1,typename Allocator1, + typename Value2,typename IndexSpecifierList2,typename Allocator2 +> +bool operator!=( + const multi_index_container& x, + const multi_index_container& y); + +template< + typename Value1,typename IndexSpecifierList1,typename Allocator1, + typename Value2,typename IndexSpecifierList2,typename Allocator2 +> +bool operator>( + const multi_index_container& x, + const multi_index_container& y); + +template< + typename Value1,typename IndexSpecifierList1,typename Allocator1, + typename Value2,typename IndexSpecifierList2,typename Allocator2 +> +bool operator>=( + const multi_index_container& x, + const multi_index_container& y); + +template< + typename Value1,typename IndexSpecifierList1,typename Allocator1, + typename Value2,typename IndexSpecifierList2,typename Allocator2 +> +bool operator<=( + const multi_index_container& x, + const multi_index_container& y); + +template +void swap( + multi_index_container& x, + multi_index_container& y); + +} /* namespace multi_index */ + +/* multi_index_container, being the main type of this library, is promoted to + * namespace boost. + */ + +using multi_index::multi_index_container; + +} /* namespace boost */ + +#endif diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/access.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/access.hpp new file mode 100644 index 00000000000..f6581accc91 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/access.hpp @@ -0,0 +1,145 @@ +#ifndef BOOST_SERIALIZATION_ACCESS_HPP +#define BOOST_SERIALIZATION_ACCESS_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// access.hpp: interface for serialization system. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +namespace boost { + +namespace archive { +namespace detail { + template + class iserializer; + template + class oserializer; +} // namespace detail +} // namespace archive + +namespace serialization { + +// forward declarations +template +inline void serialize_adl(Archive &, T &, const unsigned int); +namespace detail { + template + struct member_saver; + template + struct member_loader; +} // namespace detail + +// use an "accessor class so that we can use: +// "friend class boost::serialization::access;" +// in any serialized class to permit clean, safe access to private class members +// by the serialization system + +class access { +public: + // grant access to "real" serialization defaults +#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +public: +#else + template + friend struct detail::member_saver; + template + friend struct detail::member_loader; + template + friend class archive::detail::iserializer; + template + friend class archive::detail::oserializer; + template + friend inline void serialize( + Archive & ar, + T & t, + const unsigned int file_version + ); + template + friend inline void save_construct_data( + Archive & ar, + const T * t, + const unsigned int file_version + ); + template + friend inline void load_construct_data( + Archive & ar, + T * t, + const unsigned int file_version + ); +#endif + + // pass calls to users's class implementation + template + static void member_save( + Archive & ar, + //const T & t, + T & t, + const unsigned int file_version + ){ + t.save(ar, file_version); + } + template + static void member_load( + Archive & ar, + T & t, + const unsigned int file_version + ){ + t.load(ar, file_version); + } + template + static void serialize( + Archive & ar, + T & t, + const unsigned int file_version + ){ + // note: if you get a compile time error here with a + // message something like: + // cannot convert parameter 1 from to + // a likely possible cause is that the class T contains a + // serialize function - but that serialize function isn't + // a template and corresponds to a file type different than + // the class Archive. To resolve this, don't include an + // archive type other than that for which the serialization + // function is defined!!! + t.serialize(ar, file_version); + } + template + static void destroy( const T * t) // const appropriate here? + { + // the const business is an MSVC 6.0 hack that should be + // benign on everything else + delete const_cast(t); + } + template + static void construct(T * t){ + // default is inplace invocation of default constructor + // Note the :: before the placement new. Required if the + // class doesn't have a class-specific placement new defined. + ::new(t)T; + } + template + static T & cast_reference(U & u){ + return static_cast(u); + } + template + static T * cast_pointer(U * u){ + return static_cast(u); + } +}; + +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_ACCESS_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_map.hpp new file mode 100644 index 00000000000..ccf806b1813 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_map.hpp @@ -0,0 +1,85 @@ +#ifndef BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_MAP_HPP +#define BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_MAP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) && (_MSC_VER >= 1020) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// serialization/unordered_map.hpp: +// serialization for stl unordered_map templates + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// (C) Copyright 2014 Jim Bell +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include +#include +#include + +namespace boost { +namespace serialization { +namespace stl { + +// map input +template +struct archive_input_unordered_map +{ + inline void operator()( + Archive &ar, + Container &s, + const unsigned int v + ){ + typedef typename Container::value_type type; + detail::stack_construct t(ar, v); + ar >> boost::serialization::make_nvp("item", t.reference()); + std::pair result = + s.insert(boost::move(t.reference())); + // note: the following presumes that the map::value_type was NOT tracked + // in the archive. This is the usual case, but here there is no way + // to determine that. + if(result.second){ + ar.reset_object_address( + & (result.first->second), + & t.reference().second + ); + } + } +}; + +// multimap input +template +struct archive_input_unordered_multimap +{ + inline void operator()( + Archive &ar, + Container &s, + const unsigned int v + ){ + typedef typename Container::value_type type; + detail::stack_construct t(ar, v); + ar >> boost::serialization::make_nvp("item", t.reference()); + typename Container::const_iterator result = + s.insert(t.reference()); + // note: the following presumes that the map::value_type was NOT tracked + // in the archive. This is the usual case, but here there is no way + // to determine that. + ar.reset_object_address( + & result->second, + & t.reference() + ); + } +}; + +} // stl +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_set.hpp new file mode 100644 index 00000000000..7f0003cc6a4 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_set.hpp @@ -0,0 +1,72 @@ +#ifndef BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_SET_HPP +#define BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_SET_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) && (_MSC_VER >= 1020) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// archive_input_unordered_set.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// (C) Copyright 2014 Jim Bell +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +namespace stl { + +// unordered_set input +template +struct archive_input_unordered_set +{ + inline void operator()( + Archive &ar, + Container &s, + const unsigned int v + ){ + typedef typename Container::value_type type; + detail::stack_construct t(ar, v); + // borland fails silently w/o full namespace + ar >> boost::serialization::make_nvp("item", t.reference()); + std::pair result = + s.insert(boost::move(t.reference())); + if(result.second) + ar.reset_object_address(& (* result.first), & t.reference()); + } +}; + +// unordered_multiset input +template +struct archive_input_unordered_multiset +{ + inline void operator()( + Archive &ar, + Container &s, + const unsigned int v + ){ + typedef typename Container::value_type type; + detail::stack_construct t(ar, v); + ar >> boost::serialization::make_nvp("item", t.reference()); + typename Container::const_iterator result = + s.insert(boost::move(t.reference())); + ar.reset_object_address(& (* result), & t.reference()); + } +}; + +} // stl +} // serialization +} // boost + +#endif // BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/array.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/array.hpp new file mode 100644 index 00000000000..612d1a61985 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/array.hpp @@ -0,0 +1,48 @@ +#ifndef BOOST_SERIALIZATION_ARRAY_HPP +#define BOOST_SERIALIZATION_ARRAY_HPP + +// (C) Copyright 2005 Matthias Troyer and Dave Abrahams +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// for serialization of . If not supported by the standard +// library - this file becomes empty. This is to avoid breaking backward +// compatibiliy for applications which used this header to support +// serialization of native arrays. Code to serialize native arrays is +// now always include by default. RR + +#include // msvc 6.0 needs this for warning suppression + +#if defined(BOOST_NO_STDC_NAMESPACE) + +#include +#include // std::size_t +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include + +#ifndef BOOST_NO_CXX11_HDR_ARRAY + +#include +#include + +namespace boost { namespace serialization { + +template +void serialize(Archive& ar, std::array& a, const unsigned int /* version */) +{ + ar & boost::serialization::make_nvp( + "elems", + *static_cast(static_cast(a.data())) + ); + +} +} } // end namespace boost::serialization + +#endif // BOOST_NO_CXX11_HDR_ARRAY + +#endif //BOOST_SERIALIZATION_ARRAY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/array_optimization.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/array_optimization.hpp new file mode 100644 index 00000000000..40dffba871a --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/array_optimization.hpp @@ -0,0 +1,37 @@ +#ifndef BOOST_SERIALIZATION_ARRAY_OPTIMIZATON_HPP +#define BOOST_SERIALIZATION_ARRAY_OPTIMIZATON_HPP + +// (C) Copyright 2005 Matthias Troyer and Dave Abrahams +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +#include // msvc 6.0 needs this for warning suppression + +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include +#include +#include + +namespace boost { namespace serialization { + +template +struct use_array_optimization : boost::mpl::always {}; + +} } // end namespace boost::serialization + +#define BOOST_SERIALIZATION_USE_ARRAY_OPTIMIZATION(Archive) \ +namespace boost { namespace serialization { \ +template <> struct use_array_optimization { \ + template \ + struct apply : boost::mpl::apply1::type \ + >::type {}; \ +}; }} + +#endif //BOOST_SERIALIZATION_ARRAY_OPTIMIZATON_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/array_wrapper.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/array_wrapper.hpp new file mode 100644 index 00000000000..adf436e15b4 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/array_wrapper.hpp @@ -0,0 +1,121 @@ +#ifndef BOOST_SERIALIZATION_ARRAY_WRAPPER_HPP +#define BOOST_SERIALIZATION_ARRAY_WRAPPER_HPP + +// (C) Copyright 2005 Matthias Troyer and Dave Abrahams +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +//#include + +#include // msvc 6.0 needs this for warning suppression + +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace boost { namespace serialization { + +template +class array_wrapper : + public wrapper_traits > +{ +private: + array_wrapper & operator=(const array_wrapper & rhs); + // note: I would like to make the copy constructor private but this breaks + // make_array. So I make make_array a friend + template + friend const boost::serialization::array_wrapper make_array(Tx * t, S s); +public: + + array_wrapper(const array_wrapper & rhs) : + m_t(rhs.m_t), + m_element_count(rhs.m_element_count) + {} +public: + array_wrapper(T * t, std::size_t s) : + m_t(t), + m_element_count(s) + {} + + // default implementation + template + void serialize_optimized(Archive &ar, const unsigned int, mpl::false_ ) const + { + // default implemention does the loop + std::size_t c = count(); + T * t = address(); + while(0 < c--) + ar & boost::serialization::make_nvp("item", *t++); + } + + // optimized implementation + template + void serialize_optimized(Archive &ar, const unsigned int version, mpl::true_ ) + { + boost::serialization::split_member(ar, *this, version); + } + + // default implementation + template + void save(Archive &ar, const unsigned int version) const + { + ar.save_array(*this,version); + } + + // default implementation + template + void load(Archive &ar, const unsigned int version) + { + ar.load_array(*this,version); + } + + // default implementation + template + void serialize(Archive &ar, const unsigned int version) + { + typedef typename + boost::serialization::use_array_optimization::template apply< + typename remove_const< T >::type + >::type use_optimized; + serialize_optimized(ar,version,use_optimized()); + } + + T * address() const + { + return m_t; + } + + std::size_t count() const + { + return m_element_count; + } + +private: + T * const m_t; + const std::size_t m_element_count; +}; + +template +inline +const array_wrapper< T > make_array(T* t, S s){ + const array_wrapper< T > a(t, s); + return a; +} + +} } // end namespace boost::serialization + + +#endif //BOOST_SERIALIZATION_ARRAY_WRAPPER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/assume_abstract.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/assume_abstract.hpp new file mode 100644 index 00000000000..632f9312f5f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/assume_abstract.hpp @@ -0,0 +1,60 @@ +#ifndef BOOST_SERIALIZATION_ASSUME_ABSTRACT_HPP +#define BOOST_SERIALIZATION_ASSUME_ABSTRACT_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// assume_abstract_class.hpp: + +// (C) Copyright 2008 Robert Ramey +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// this is useful for compilers which don't support the boost::is_abstract + +#include +#include + +#ifndef BOOST_NO_IS_ABSTRACT + +// if there is an intrinsic is_abstract defined, we don't have to do anything +#define BOOST_SERIALIZATION_ASSUME_ABSTRACT(T) + +// but forward to the "official" is_abstract +namespace boost { +namespace serialization { + template + struct is_abstract : boost::is_abstract< T > {} ; +} // namespace serialization +} // namespace boost + +#else +// we have to "make" one + +namespace boost { +namespace serialization { + template + struct is_abstract : boost::false_type {}; +} // namespace serialization +} // namespace boost + +// define a macro to make explicit designation of this more transparent +#define BOOST_SERIALIZATION_ASSUME_ABSTRACT(T) \ +namespace boost { \ +namespace serialization { \ +template<> \ +struct is_abstract< T > : boost::true_type {}; \ +template<> \ +struct is_abstract< const T > : boost::true_type {}; \ +}} \ +/**/ + +#endif // BOOST_NO_IS_ABSTRACT + +#endif //BOOST_SERIALIZATION_ASSUME_ABSTRACT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/base_object.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/base_object.hpp new file mode 100644 index 00000000000..1a82cecd4b5 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/base_object.hpp @@ -0,0 +1,100 @@ +#ifndef BOOST_SERIALIZATION_BASE_OBJECT_HPP +#define BOOST_SERIALIZATION_BASE_OBJECT_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// base_object.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// if no archive headers have been included this is a no op +// this is to permit BOOST_EXPORT etc to be included in a +// file declaration header + +#include +#include + +#include +#include +#include + +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +namespace detail +{ + // get the base type for a given derived type + // preserving the const-ness + template + struct base_cast + { + typedef typename + mpl::if_< + is_const, + const B, + B + >::type type; + BOOST_STATIC_ASSERT(is_const::value == is_const::value); + }; + + // only register void casts if the types are polymorphic + template + struct base_register + { + struct polymorphic { + static void const * invoke(){ + Base const * const b = 0; + Derived const * const d = 0; + return & void_cast_register(d, b); + } + }; + struct non_polymorphic { + static void const * invoke(){ + return 0; + } + }; + static void const * invoke(){ + typedef typename mpl::eval_if< + is_polymorphic, + mpl::identity, + mpl::identity + >::type type; + return type::invoke(); + } + }; + +} // namespace detail +template +typename detail::base_cast::type & +base_object(Derived &d) +{ + BOOST_STATIC_ASSERT(( is_base_and_derived::value)); + BOOST_STATIC_ASSERT(! is_pointer::value); + typedef typename detail::base_cast::type type; + detail::base_register::invoke(); + return access::cast_reference(d); +} + +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_BASE_OBJECT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/binary_object.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/binary_object.hpp new file mode 100644 index 00000000000..5c9038e5a9f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/binary_object.hpp @@ -0,0 +1,79 @@ +#ifndef BOOST_SERIALIZATION_BINARY_OBJECT_HPP +#define BOOST_SERIALIZATION_BINARY_OBJECT_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// nvp.hpp: interface for serialization system. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include // std::size_t +#include +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include +#include +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +struct binary_object : + public wrapper_traits > +{ + void const * m_t; + std::size_t m_size; + template + void save(Archive & ar, const unsigned int /* file_version */) const { + ar.save_binary(m_t, m_size); + } + template + void load(Archive & ar, const unsigned int /* file_version */) const { + ar.load_binary(const_cast(m_t), m_size); + } + BOOST_SERIALIZATION_SPLIT_MEMBER() + binary_object & operator=(const binary_object & rhs) { + m_t = rhs.m_t; + m_size = rhs.m_size; + return *this; + } + binary_object(const void * const t, std::size_t size) : + m_t(t), + m_size(size) + {} + binary_object(const binary_object & rhs) : + m_t(rhs.m_t), + m_size(rhs.m_size) + {} +}; + +// just a little helper to support the convention that all serialization +// wrappers follow the naming convention make_xxxxx +inline +const binary_object +make_binary_object(const void * t, std::size_t size){ + return binary_object(t, size); +} + +} // namespace serialization +} // boost + +#endif // BOOST_SERIALIZATION_BINARY_OBJECT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/bitset.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/bitset.hpp new file mode 100644 index 00000000000..78f9bd74336 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/bitset.hpp @@ -0,0 +1,75 @@ +/*! + * \file bitset.hpp + * \brief Provides Boost.Serialization support for std::bitset + * \author Brian Ravnsgaard Riis + * \author Kenneth Riddile + * \date 16.09.2004, updated 04.03.2009 + * \copyright 2004 Brian Ravnsgaard Riis + * \license Boost Software License 1.0 + */ +#ifndef BOOST_SERIALIZATION_BITSET_HPP +#define BOOST_SERIALIZATION_BITSET_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +#include +#include // size_t + +#include +#include +#include +#include + +namespace boost{ +namespace serialization{ + +template +inline void save( + Archive & ar, + std::bitset const & t, + const unsigned int /* version */ +){ + const std::string bits = t.template to_string< + std::string::value_type, + std::string::traits_type, + std::string::allocator_type + >(); + ar << BOOST_SERIALIZATION_NVP( bits ); +} + +template +inline void load( + Archive & ar, + std::bitset & t, + const unsigned int /* version */ +){ + std::string bits; + ar >> BOOST_SERIALIZATION_NVP( bits ); + t = std::bitset(bits); +} + +template +inline void serialize( + Archive & ar, + std::bitset & t, + const unsigned int version +){ + boost::serialization::split_free( ar, t, version ); +} + +// don't track bitsets since that would trigger tracking +// all over the program - which probably would be a surprise. +// also, tracking would be hard to implement since, we're +// serialization a representation of the data rather than +// the data itself. +template +struct tracking_level > + : mpl::int_ {} ; + +} //serialization +} //boost + +#endif // BOOST_SERIALIZATION_BITSET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/boost_array.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/boost_array.hpp new file mode 100644 index 00000000000..d564ff15de0 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/boost_array.hpp @@ -0,0 +1,33 @@ +#ifndef BOOST_SERIALIZATION_ARRAY_HPP +#define BOOST_SERIALIZATION_ARRAY_HPP + +// (C) Copyright 2005 Matthias Troyer and Dave Abrahams +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +//#include + +#include // msvc 6.0 needs this for warning suppression + +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +#include +#include + +namespace boost { namespace serialization { +// implement serialization for boost::array +template +void serialize(Archive& ar, boost::array& a, const unsigned int /* version */) +{ + ar & boost::serialization::make_nvp("elems", a.elems); +} + +} } // end namespace boost::serialization + + +#endif //BOOST_SERIALIZATION_ARRAY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_map.hpp new file mode 100644 index 00000000000..8913b31f9e6 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_map.hpp @@ -0,0 +1,154 @@ +#ifndef BOOST_SERIALIZATION_UNORDERED_MAP_HPP +#define BOOST_SERIALIZATION_UNORDERED_MAP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) && (_MSC_VER >= 1020) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// serialization/unordered_map.hpp: +// serialization for stl unordered_map templates + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// (C) Copyright 2014 Jim Bell +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include + +#include +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void save( + Archive & ar, + const boost::unordered_map &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::save_unordered_collection< + Archive, + boost::unordered_map + >(ar, t); +} + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void load( + Archive & ar, + boost::unordered_map &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::load_unordered_collection< + Archive, + boost::unordered_map, + boost::serialization::stl::archive_input_unordered_map< + Archive, + boost::unordered_map + > + >(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void serialize( + Archive & ar, + boost::unordered_map &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +// unordered_multimap +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void save( + Archive & ar, + const boost::unordered_multimap &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::save_unordered_collection< + Archive, + boost::unordered_multimap + >(ar, t); +} + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void load( + Archive & ar, + boost::unordered_multimap< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::load_unordered_collection< + Archive, + boost::unordered_multimap, + boost::serialization::stl::archive_input_unordered_multimap< + Archive, + boost::unordered_multimap + > + >(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void serialize( + Archive & ar, + boost::unordered_multimap &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_UNORDERED_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_set.hpp new file mode 100644 index 00000000000..307c7819cbd --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_set.hpp @@ -0,0 +1,150 @@ +#ifndef BOOST_SERIALIZATION_BOOST_UNORDERED_SET_HPP +#define BOOST_SERIALIZATION_BOOST_UNORDERED_SET_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) && (_MSC_VER >= 1020) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// unordered_set.hpp: serialization for boost unordered_set templates + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// (C) Copyright 2014 Jim Bell +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include + +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void save( + Archive & ar, + const boost::unordered_set &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::save_unordered_collection< + Archive, + boost::unordered_set + >(ar, t); +} + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void load( + Archive & ar, + boost::unordered_set &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::load_unordered_collection< + Archive, + boost::unordered_set, + boost::serialization::stl::archive_input_unordered_set< + Archive, + boost::unordered_set + > + >(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void serialize( + Archive & ar, + boost::unordered_set &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +// unordered_multiset +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void save( + Archive & ar, + const boost::unordered_multiset &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::save_unordered_collection< + Archive, + boost::unordered_multiset + >(ar, t); +} + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void load( + Archive & ar, + boost::unordered_multiset &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::load_unordered_collection< + Archive, + boost::unordered_multiset, + boost::serialization::stl::archive_input_unordered_multiset< + Archive, + boost::unordered_multiset + > + >(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void serialize( + Archive & ar, + boost::unordered_multiset &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_BOOST_UNORDERED_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/collection_size_type.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/collection_size_type.hpp new file mode 100644 index 00000000000..2dd8fa72584 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/collection_size_type.hpp @@ -0,0 +1,62 @@ +#ifndef BOOST_SERIALIZATION_COLLECTION_SIZE_TYPE_HPP +#define BOOST_SERIALIZATION_COLLECTION_SIZE_TYPE_HPP + +// (C) Copyright 2005 Matthias Troyer +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +#include // size_t +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +//BOOST_STRONG_TYPEDEF(std::size_t, collection_size_type) + +class collection_size_type { +private: + typedef std::size_t base_type; + base_type t; +public: + collection_size_type(): t(0) {}; + explicit collection_size_type(const std::size_t & t_) : + t(t_) + {} + collection_size_type(const collection_size_type & t_) : + t(t_.t) + {} + collection_size_type & operator=(const collection_size_type & rhs){ + t = rhs.t; + return *this; + } + collection_size_type & operator=(const unsigned int & rhs){ + t = rhs; + return *this; + } + // used for text output + operator base_type () const { + return t; + } + // used for text input + operator base_type & () { + return t; + } + bool operator==(const collection_size_type & rhs) const { + return t == rhs.t; + } + bool operator<(const collection_size_type & rhs) const { + return t < rhs.t; + } +}; + + +} } // end namespace boost::serialization + +BOOST_CLASS_IMPLEMENTATION(collection_size_type, primitive_type) +BOOST_IS_BITWISE_SERIALIZABLE(collection_size_type) + +#endif //BOOST_SERIALIZATION_COLLECTION_SIZE_TYPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/collection_traits.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/collection_traits.hpp new file mode 100644 index 00000000000..3ec9401eff0 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/collection_traits.hpp @@ -0,0 +1,79 @@ +#ifndef BOOST_SERIALIZATION_COLLECTION_TRAITS_HPP +#define BOOST_SERIALIZATION_COLLECTION_TRAITS_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// collection_traits.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// This header assigns a level implemenation trait to a collection type +// for all primitives. It is needed so that archives which are meant to be +// portable don't write class information in the archive. Since, not all +// compiles recognize the same set of primitive types, the possibility +// exists for archives to be non-portable if class information for primitive +// types is included. This is addressed by the following macros. +#include +//#include +#include + +#include +#include +#include // ULONG_MAX +#include + +#define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(T, C) \ +template<> \ +struct implementation_level< C < T > > { \ + typedef mpl::integral_c_tag tag; \ + typedef mpl::int_ type; \ + BOOST_STATIC_CONSTANT(int, value = object_serializable); \ +}; \ +/**/ + +#if defined(BOOST_NO_CWCHAR) || defined(BOOST_NO_INTRINSIC_WCHAR_T) + #define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_WCHAR(C) +#else + #define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_WCHAR(C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(wchar_t, C) \ + /**/ +#endif + +#if defined(BOOST_HAS_LONG_LONG) + #define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_INT64(C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(boost::long_long_type, C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(boost::ulong_long_type, C) \ + /**/ +#else + #define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_INT64(C) +#endif + +#define BOOST_SERIALIZATION_COLLECTION_TRAITS(C) \ + namespace boost { namespace serialization { \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(bool, C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(char, C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(signed char, C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(unsigned char, C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(signed int, C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(unsigned int, C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(signed long, C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(unsigned long, C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(float, C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(double, C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(unsigned short, C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(signed short, C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_INT64(C) \ + BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_WCHAR(C) \ + } } \ + /**/ + +#endif // BOOST_SERIALIZATION_COLLECTION_TRAITS diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/collections_load_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/collections_load_imp.hpp new file mode 100644 index 00000000000..e042c0c130d --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/collections_load_imp.hpp @@ -0,0 +1,106 @@ +#ifndef BOOST_SERIALIZATION_COLLECTIONS_LOAD_IMP_HPP +#define BOOST_SERIALIZATION_COLLECTIONS_LOAD_IMP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +#if defined(_MSC_VER) && (_MSC_VER <= 1020) +# pragma warning (disable : 4786) // too long name, harmless warning +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// collections_load_imp.hpp: serialization for loading stl collections + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// helper function templates for serialization of collections + +#include +#include // size_t +#include // msvc 6.0 needs this for warning suppression +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace boost{ +namespace serialization { +namespace stl { + +////////////////////////////////////////////////////////////////////// +// implementation of serialization for STL containers +// + +template< + class Archive, + class T +> +typename boost::enable_if< + typename detail::is_default_constructible< + typename T::value_type + >, + void +>::type +collection_load_impl( + Archive & ar, + T & t, + collection_size_type count, + item_version_type /*item_version*/ +){ + t.resize(count); + typename T::iterator hint; + hint = t.begin(); + while(count-- > 0){ + ar >> boost::serialization::make_nvp("item", *hint++); + } +} + +template< + class Archive, + class T +> +typename boost::disable_if< + typename detail::is_default_constructible< + typename T::value_type + >, + void +>::type +collection_load_impl( + Archive & ar, + T & t, + collection_size_type count, + item_version_type item_version +){ + t.clear(); + while(count-- > 0){ + detail::stack_construct u(ar, item_version); + ar >> boost::serialization::make_nvp("item", u.reference()); + t.push_back(boost::move(u.reference())); + ar.reset_object_address(& t.back() , & u.reference()); + } +} + +} // namespace stl +} // namespace serialization +} // namespace boost + +#endif //BOOST_SERIALIZATION_COLLECTIONS_LOAD_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/collections_save_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/collections_save_imp.hpp new file mode 100644 index 00000000000..f3cabfcf3f5 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/collections_save_imp.hpp @@ -0,0 +1,82 @@ +#ifndef BOOST_SERIALIZATION_COLLECTIONS_SAVE_IMP_HPP +#define BOOST_SERIALIZATION_COLLECTIONS_SAVE_IMP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// collections_save_imp.hpp: serialization for stl collections + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// helper function templates for serialization of collections + +#include +#include +#include +#include +#include +#include + +namespace boost{ +namespace serialization { +namespace stl { + +////////////////////////////////////////////////////////////////////// +// implementation of serialization for STL containers +// + +template +inline void save_collection( + Archive & ar, + const Container &s, + collection_size_type count) +{ + ar << BOOST_SERIALIZATION_NVP(count); + // record number of elements + const item_version_type item_version( + version::value + ); + #if 0 + boost::archive::library_version_type library_version( + ar.get_library_version() + ); + if(boost::archive::library_version_type(3) < library_version){ + ar << BOOST_SERIALIZATION_NVP(item_version); + } + #else + ar << BOOST_SERIALIZATION_NVP(item_version); + #endif + + typename Container::const_iterator it = s.begin(); + while(count-- > 0){ + // note borland emits a no-op without the explicit namespace + boost::serialization::save_construct_data_adl( + ar, + &(*it), + item_version + ); + ar << boost::serialization::make_nvp("item", *it++); + } +} + +template +inline void save_collection(Archive & ar, const Container &s) +{ + // record number of elements + collection_size_type count(s.size()); + save_collection(ar, s, count); +} + +} // namespace stl +} // namespace serialization +} // namespace boost + +#endif //BOOST_SERIALIZATION_COLLECTIONS_SAVE_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/complex.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/complex.hpp new file mode 100644 index 00000000000..b4ef44cf973 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/complex.hpp @@ -0,0 +1,81 @@ +#ifndef BOOST_SERIALIZATION_COMPLEX_HPP +#define BOOST_SERIALIZATION_COMPLEX_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// serialization/utility.hpp: +// serialization for stl utility templates + +// (C) Copyright 2007 Matthias Troyer . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include + +#include +#include +#include + +namespace boost { +namespace serialization { + +template +inline void serialize( + Archive & ar, + std::complex< T > & t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +template +inline void save( + Archive & ar, + std::complex< T > const & t, + const unsigned int /* file_version */ +){ + const T re = t.real(); + const T im = t.imag(); + ar << boost::serialization::make_nvp("real", re); + ar << boost::serialization::make_nvp("imag", im); +} + +template +inline void load( + Archive & ar, + std::complex< T >& t, + const unsigned int /* file_version */ +){ + T re; + T im; + ar >> boost::serialization::make_nvp("real", re); + ar >> boost::serialization::make_nvp("imag", im); + t = std::complex< T >(re,im); +} + +// specialization of serialization traits for complex +template +struct is_bitwise_serializable > + : public is_bitwise_serializable< T > {}; + +template +struct implementation_level > + : mpl::int_ {} ; + +// treat complex just like builtin arithmetic types for tracking +template +struct tracking_level > + : mpl::int_ {} ; + +} // serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_COMPLEX_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/config.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/config.hpp new file mode 100644 index 00000000000..ea8cb9239ed --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/config.hpp @@ -0,0 +1,74 @@ +#ifndef BOOST_SERIALIZATION_CONFIG_HPP +#define BOOST_SERIALIZATION_CONFIG_HPP + +// config.hpp ---------------------------------------------// + +// (c) Copyright Robert Ramey 2004 +// Use, modification, and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See library home page at http://www.boost.org/libs/serialization + +//----------------------------------------------------------------------------// + +// This header implements separate compilation features as described in +// http://www.boost.org/more/separate_compilation.html + +#include +#include + +// note: this version incorporates the related code into the the +// the same library as BOOST_ARCHIVE. This could change some day in the +// future + +// if BOOST_SERIALIZATION_DECL is defined undefine it now: +#ifdef BOOST_SERIALIZATION_DECL + #undef BOOST_SERIALIZATION_DECL +#endif + +// we need to import/export our code only if the user has specifically +// asked for it by defining either BOOST_ALL_DYN_LINK if they want all boost +// libraries to be dynamically linked, or BOOST_SERIALIZATION_DYN_LINK +// if they want just this one to be dynamically liked: +#if defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) + #if !defined(BOOST_DYN_LINK) + #define BOOST_DYN_LINK + #endif + // export if this is our own source, otherwise import: + #if defined(BOOST_SERIALIZATION_SOURCE) + #define BOOST_SERIALIZATION_DECL BOOST_SYMBOL_EXPORT + #else + #define BOOST_SERIALIZATION_DECL BOOST_SYMBOL_IMPORT + #endif // defined(BOOST_SERIALIZATION_SOURCE) +#endif // defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) + +// if BOOST_SERIALIZATION_DECL isn't defined yet define it now: +#ifndef BOOST_SERIALIZATION_DECL + #define BOOST_SERIALIZATION_DECL +#endif + +// enable automatic library variant selection ------------------------------// + +#if !defined(BOOST_ALL_NO_LIB) && !defined(BOOST_SERIALIZATION_NO_LIB) \ +&& !defined(BOOST_ARCHIVE_SOURCE) && !defined(BOOST_WARCHIVE_SOURCE) \ +&& !defined(BOOST_SERIALIZATION_SOURCE) + // + // Set the name of our library, this will get undef'ed by auto_link.hpp + // once it's done with it: + // + #define BOOST_LIB_NAME boost_serialization + // + // If we're importing code from a dll, then tell auto_link.hpp about it: + // + #if defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) + # define BOOST_DYN_LINK + #endif + // + // And include the header that does the work: + // + #include + +#endif + +#endif // BOOST_SERIALIZATION_CONFIG_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/deque.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/deque.hpp new file mode 100644 index 00000000000..bba81364ce2 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/deque.hpp @@ -0,0 +1,80 @@ +#ifndef BOOST_SERIALIZATION_DEQUE_HPP +#define BOOST_SERIALIZATION_DEQUE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// deque.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include + +#include + +#include +#include +#include + +namespace boost { +namespace serialization { + +template +inline void save( + Archive & ar, + const std::deque &t, + const unsigned int /* file_version */ +){ + boost::serialization::stl::save_collection< + Archive, std::deque + >(ar, t); +} + +template +inline void load( + Archive & ar, + std::deque &t, + const unsigned int /* file_version */ +){ + const boost::archive::library_version_type library_version( + ar.get_library_version() + ); + // retrieve number of elements + item_version_type item_version(0); + collection_size_type count; + ar >> BOOST_SERIALIZATION_NVP(count); + if(boost::archive::library_version_type(3) < library_version){ + ar >> BOOST_SERIALIZATION_NVP(item_version); + } + stl::collection_load_impl(ar, t, count, item_version); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template +inline void serialize( + Archive & ar, + std::deque &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +#include + +BOOST_SERIALIZATION_COLLECTION_TRAITS(std::deque) + +#endif // BOOST_SERIALIZATION_DEQUE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/is_default_constructible.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/is_default_constructible.hpp new file mode 100644 index 00000000000..4d20b13bf3e --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/detail/is_default_constructible.hpp @@ -0,0 +1,54 @@ +#ifndef BOOST_SERIALIZATION_DETAIL_IS_DEFAULT_CONSTRUCTIBLE_HPP +#define BOOST_SERIALIZATION_DETAIL_IS_DEFAULT_CONSTRUCTIBLE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// is_default_constructible.hpp: serialization for loading stl collections +// +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#if ! defined(BOOST_NO_CXX11_HDR_TYPE_TRAITS) + #include + namespace boost{ + namespace serialization { + namespace detail { + + template + struct is_default_constructible : public std::is_default_constructible {}; + + } // detail + } // serializaition + } // boost +#else + // we don't have standard library support for is_default_constructible + // so we fake it by using boost::has_trivial_construtor. But this is not + // actually correct because it's possible that a default constructor + // to be non trivial. So when using this, make sure you're not using your + // own definition of of T() but are using the actual default one! + #include + namespace boost{ + namespace serialization { + namespace detail { + + template + struct is_default_constructible : public boost::has_trivial_constructor {}; + + } // detail + } // serializaition + } // boost + +#endif + + +#endif // BOOST_SERIALIZATION_DETAIL_IS_DEFAULT_CONSTRUCTIBLE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_count_132.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_count_132.hpp new file mode 100644 index 00000000000..a5872557cf2 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_count_132.hpp @@ -0,0 +1,551 @@ +#ifndef BOOST_DETAIL_SHARED_COUNT_132_HPP_INCLUDED +#define BOOST_DETAIL_SHARED_COUNT_132_HPP_INCLUDED + +// MS compatible compilers support #pragma once + +#if defined(_MSC_VER) +# pragma once +#endif + +// +// detail/shared_count.hpp +// +// Copyright (c) 2001, 2002, 2003 Peter Dimov and Multi Media Ltd. +// +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) +// + +#include + +#if defined(BOOST_SP_USE_STD_ALLOCATOR) && defined(BOOST_SP_USE_QUICK_ALLOCATOR) +# error BOOST_SP_USE_STD_ALLOCATOR and BOOST_SP_USE_QUICK_ALLOCATOR are incompatible. +#endif + +#include +#include +#include + +#if defined(BOOST_SP_USE_QUICK_ALLOCATOR) +#include +#endif + +#include // std::auto_ptr, std::allocator +#include // std::less +#include // std::exception +#include // std::bad_alloc +#include // std::type_info in get_deleter +#include // std::size_t + +#include // msvc 6.0 needs this for warning suppression +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif + +namespace boost_132 { + +// Debug hooks + +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + +void sp_scalar_constructor_hook(void * px, std::size_t size, void * pn); +void sp_array_constructor_hook(void * px); +void sp_scalar_destructor_hook(void * px, std::size_t size, void * pn); +void sp_array_destructor_hook(void * px); + +#endif + + +// The standard library that comes with Borland C++ 5.5.1 +// defines std::exception and its members as having C calling +// convention (-pc). When the definition of bad_weak_ptr +// is compiled with -ps, the compiler issues an error. +// Hence, the temporary #pragma option -pc below. The version +// check is deliberately conservative. + +class bad_weak_ptr: public std::exception +{ +public: + + virtual char const * what() const throw() + { + return "boost::bad_weak_ptr"; + } +}; + +namespace detail{ + +class sp_counted_base +{ +//private: + + typedef boost::detail::lightweight_mutex mutex_type; + +public: + + sp_counted_base(): use_count_(1), weak_count_(1) + { + } + + virtual ~sp_counted_base() // nothrow + { + } + + // dispose() is called when use_count_ drops to zero, to release + // the resources managed by *this. + + virtual void dispose() = 0; // nothrow + + // destruct() is called when weak_count_ drops to zero. + + virtual void destruct() // nothrow + { + delete this; + } + + virtual void * get_deleter(std::type_info const & ti) = 0; + + void add_ref_copy() + { +#if defined(BOOST_HAS_THREADS) + mutex_type::scoped_lock lock(mtx_); +#endif + ++use_count_; + } + + void add_ref_lock() + { +#if defined(BOOST_HAS_THREADS) + mutex_type::scoped_lock lock(mtx_); +#endif + if(use_count_ == 0) boost::serialization::throw_exception(bad_weak_ptr()); + ++use_count_; + } + + void release() // nothrow + { + { +#if defined(BOOST_HAS_THREADS) + mutex_type::scoped_lock lock(mtx_); +#endif + long new_use_count = --use_count_; + + if(new_use_count != 0) return; + } + + dispose(); + weak_release(); + } + + void weak_add_ref() // nothrow + { +#if defined(BOOST_HAS_THREADS) + mutex_type::scoped_lock lock(mtx_); +#endif + ++weak_count_; + } + + void weak_release() // nothrow + { + long new_weak_count; + + { +#if defined(BOOST_HAS_THREADS) + mutex_type::scoped_lock lock(mtx_); +#endif + new_weak_count = --weak_count_; + } + + if(new_weak_count == 0) + { + destruct(); + } + } + + long use_count() const // nothrow + { +#if defined(BOOST_HAS_THREADS) + mutex_type::scoped_lock lock(mtx_); +#endif + return use_count_; + } + +//private: +public: + sp_counted_base(sp_counted_base const &); + sp_counted_base & operator= (sp_counted_base const &); + + long use_count_; // #shared + long weak_count_; // #weak + (#shared != 0) + +#if defined(BOOST_HAS_THREADS) || defined(BOOST_LWM_WIN32) + mutable mutex_type mtx_; +#endif +}; + +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + +template void cbi_call_constructor_hook(sp_counted_base * pn, T * px, boost::checked_deleter< T > const &) +{ + boost::sp_scalar_constructor_hook(px, sizeof(T), pn); +} + +template void cbi_call_constructor_hook(sp_counted_base *, T * px, boost::checked_array_deleter< T > const &) +{ + boost::sp_array_constructor_hook(px); +} + +template void cbi_call_constructor_hook(sp_counted_base *, P const &, D const &, long) +{ +} + +template void cbi_call_destructor_hook(sp_counted_base * pn, T * px, boost::checked_deleter< T > const &) +{ + boost::sp_scalar_destructor_hook(px, sizeof(T), pn); +} + +template void cbi_call_destructor_hook(sp_counted_base *, T * px, boost::checked_array_deleter< T > const &) +{ + boost::sp_array_destructor_hook(px); +} + +template void cbi_call_destructor_hook(sp_counted_base *, P const &, D const &, long) +{ +} + +#endif + +// +// Borland's Codeguard trips up over the -Vx- option here: +// +#ifdef __CODEGUARD__ +# pragma option push -Vx- +#endif + +template class sp_counted_base_impl: public sp_counted_base +{ +//private: +public: + P ptr; // copy constructor must not throw + D del; // copy constructor must not throw + + sp_counted_base_impl(sp_counted_base_impl const &); + sp_counted_base_impl & operator= (sp_counted_base_impl const &); + + typedef sp_counted_base_impl this_type; + +public: + + // pre: initial_use_count <= initial_weak_count, d(p) must not throw + + sp_counted_base_impl(P p, D d): ptr(p), del(d) + { +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + detail::cbi_call_constructor_hook(this, p, d, 0); +#endif + } + + virtual void dispose() // nothrow + { +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + detail::cbi_call_destructor_hook(this, ptr, del, 0); +#endif + del(ptr); + } + + virtual void * get_deleter(std::type_info const & ti) + { + return ti == typeid(D)? &del: 0; + } + +#if defined(BOOST_SP_USE_STD_ALLOCATOR) + + void * operator new(std::size_t) + { + return std::allocator().allocate(1, static_cast(0)); + } + + void operator delete(void * p) + { + std::allocator().deallocate(static_cast(p), 1); + } + +#endif + +#if defined(BOOST_SP_USE_QUICK_ALLOCATOR) + + void * operator new(std::size_t) + { + return boost::detail::quick_allocator::alloc(); + } + + void operator delete(void * p) + { + boost::detail::quick_allocator::dealloc(p); + } + +#endif +}; + +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + +int const shared_count_id = 0x2C35F101; +int const weak_count_id = 0x298C38A4; + +#endif + +class weak_count; + +class shared_count +{ +//private: +public: + sp_counted_base * pi_; + +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + int id_; +#endif + + friend class weak_count; + +public: + + shared_count(): pi_(0) // nothrow +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + , id_(shared_count_id) +#endif + { + } + + template shared_count(P p, D d): pi_(0) +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + , id_(shared_count_id) +#endif + { +#ifndef BOOST_NO_EXCEPTIONS + + try + { + pi_ = new sp_counted_base_impl(p, d); + } + catch(...) + { + d(p); // delete p + throw; + } + +#else + + pi_ = new sp_counted_base_impl(p, d); + + if(pi_ == 0) + { + d(p); // delete p + boost::serialization::throw_exception(std::bad_alloc()); + } + +#endif + } + +#ifndef BOOST_NO_AUTO_PTR + + // auto_ptr is special cased to provide the strong guarantee + + template + explicit shared_count(std::auto_ptr & r): pi_( + new sp_counted_base_impl< + Y *, + boost::checked_deleter + >(r.get(), boost::checked_deleter())) +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + , id_(shared_count_id) +#endif + { + r.release(); + } + +#endif + + ~shared_count() // nothrow + { + if(pi_ != 0) pi_->release(); +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + id_ = 0; +#endif + } + + shared_count(shared_count const & r): pi_(r.pi_) // nothrow +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + , id_(shared_count_id) +#endif + { + if(pi_ != 0) pi_->add_ref_copy(); + } + + explicit shared_count(weak_count const & r); // throws bad_weak_ptr when r.use_count() == 0 + + shared_count & operator= (shared_count const & r) // nothrow + { + sp_counted_base * tmp = r.pi_; + + if(tmp != pi_) + { + if(tmp != 0) tmp->add_ref_copy(); + if(pi_ != 0) pi_->release(); + pi_ = tmp; + } + + return *this; + } + + void swap(shared_count & r) // nothrow + { + sp_counted_base * tmp = r.pi_; + r.pi_ = pi_; + pi_ = tmp; + } + + long use_count() const // nothrow + { + return pi_ != 0? pi_->use_count(): 0; + } + + bool unique() const // nothrow + { + return use_count() == 1; + } + + friend inline bool operator==(shared_count const & a, shared_count const & b) + { + return a.pi_ == b.pi_; + } + + friend inline bool operator<(shared_count const & a, shared_count const & b) + { + return std::less()(a.pi_, b.pi_); + } + + void * get_deleter(std::type_info const & ti) const + { + return pi_? pi_->get_deleter(ti): 0; + } +}; + +#ifdef __CODEGUARD__ +# pragma option pop +#endif + + +class weak_count +{ +private: + + sp_counted_base * pi_; + +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + int id_; +#endif + + friend class shared_count; + +public: + + weak_count(): pi_(0) // nothrow +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + , id_(weak_count_id) +#endif + { + } + + weak_count(shared_count const & r): pi_(r.pi_) // nothrow +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + , id_(shared_count_id) +#endif + { + if(pi_ != 0) pi_->weak_add_ref(); + } + + weak_count(weak_count const & r): pi_(r.pi_) // nothrow +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + , id_(shared_count_id) +#endif + { + if(pi_ != 0) pi_->weak_add_ref(); + } + + ~weak_count() // nothrow + { + if(pi_ != 0) pi_->weak_release(); +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + id_ = 0; +#endif + } + + weak_count & operator= (shared_count const & r) // nothrow + { + sp_counted_base * tmp = r.pi_; + if(tmp != 0) tmp->weak_add_ref(); + if(pi_ != 0) pi_->weak_release(); + pi_ = tmp; + + return *this; + } + + weak_count & operator= (weak_count const & r) // nothrow + { + sp_counted_base * tmp = r.pi_; + if(tmp != 0) tmp->weak_add_ref(); + if(pi_ != 0) pi_->weak_release(); + pi_ = tmp; + + return *this; + } + + void swap(weak_count & r) // nothrow + { + sp_counted_base * tmp = r.pi_; + r.pi_ = pi_; + pi_ = tmp; + } + + long use_count() const // nothrow + { + return pi_ != 0? pi_->use_count(): 0; + } + + friend inline bool operator==(weak_count const & a, weak_count const & b) + { + return a.pi_ == b.pi_; + } + + friend inline bool operator<(weak_count const & a, weak_count const & b) + { + return std::less()(a.pi_, b.pi_); + } +}; + +inline shared_count::shared_count(weak_count const & r): pi_(r.pi_) +#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) + , id_(shared_count_id) +#endif +{ + if(pi_ != 0) + { + pi_->add_ref_lock(); + } + else + { + boost::serialization::throw_exception(bad_weak_ptr()); + } +} + +} // namespace detail + +} // namespace boost + +BOOST_SERIALIZATION_ASSUME_ABSTRACT(boost_132::detail::sp_counted_base) + +#endif // #ifndef BOOST_DETAIL_SHARED_COUNT_HPP_INCLUDED diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_132.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_132.hpp new file mode 100644 index 00000000000..ee98b7b9449 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_132.hpp @@ -0,0 +1,443 @@ +#ifndef BOOST_SHARED_PTR_132_HPP_INCLUDED +#define BOOST_SHARED_PTR_132_HPP_INCLUDED + +// +// shared_ptr.hpp +// +// (C) Copyright Greg Colvin and Beman Dawes 1998, 1999. +// Copyright (c) 2001, 2002, 2003 Peter Dimov +// +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) +// +// See http://www.boost.org/libs/smart_ptr/shared_ptr.htm for documentation. +// + +#include // for broken compiler workarounds + +#if defined(BOOST_NO_MEMBER_TEMPLATES) && !defined(BOOST_MSVC6_MEMBER_TEMPLATES) +#include +#else + +#include +#include +#include +#include + +#include +#include + +#include // for std::auto_ptr +#include // for std::swap +#include // for std::less +#include // for std::bad_cast +#include // for std::basic_ostream + +#ifdef BOOST_MSVC // moved here to work around VC++ compiler crash +# pragma warning(push) +# pragma warning(disable:4284) // odd return type for operator-> +#endif + +namespace boost_132 { + +template class weak_ptr; +template class enable_shared_from_this; + +namespace detail +{ + +struct static_cast_tag {}; +struct const_cast_tag {}; +struct dynamic_cast_tag {}; +struct polymorphic_cast_tag {}; + +template struct shared_ptr_traits +{ + typedef T & reference; +}; + +template<> struct shared_ptr_traits +{ + typedef void reference; +}; + +#if !defined(BOOST_NO_CV_VOID_SPECIALIZATIONS) + +template<> struct shared_ptr_traits +{ + typedef void reference; +}; + +template<> struct shared_ptr_traits +{ + typedef void reference; +}; + +template<> struct shared_ptr_traits +{ + typedef void reference; +}; + +#endif + +// enable_shared_from_this support + +template void sp_enable_shared_from_this( shared_count const & pn, enable_shared_from_this< T > const * pe, Y const * px ) +{ + if(pe != 0) pe->_internal_weak_this._internal_assign(const_cast(px), pn); +} + +inline void sp_enable_shared_from_this( shared_count const & /*pn*/, ... ) +{ +} + +} // namespace detail + + +// +// shared_ptr +// +// An enhanced relative of scoped_ptr with reference counted copy semantics. +// The object pointed to is deleted when the last shared_ptr pointing to it +// is destroyed or reset. +// + +template class shared_ptr +{ +private: + // Borland 5.5.1 specific workaround + typedef shared_ptr< T > this_type; + +public: + + typedef T element_type; + typedef T value_type; + typedef T * pointer; + typedef typename detail::shared_ptr_traits< T >::reference reference; + + shared_ptr(): px(0), pn() // never throws in 1.30+ + { + } + + template + explicit shared_ptr(Y * p): px(p), pn(p, boost::checked_deleter()) // Y must be complete + { + detail::sp_enable_shared_from_this( pn, p, p ); + } + + // + // Requirements: D's copy constructor must not throw + // + // shared_ptr will release p by calling d(p) + // + + template shared_ptr(Y * p, D d): px(p), pn(p, d) + { + detail::sp_enable_shared_from_this( pn, p, p ); + } + +// generated copy constructor, assignment, destructor are fine... + +// except that Borland C++ has a bug, and g++ with -Wsynth warns +#if defined(__GNUC__) + shared_ptr & operator=(shared_ptr const & r) // never throws + { + px = r.px; + pn = r.pn; // shared_count::op= doesn't throw + return *this; + } +#endif + + template + explicit shared_ptr(weak_ptr const & r): pn(r.pn) // may throw + { + // it is now safe to copy r.px, as pn(r.pn) did not throw + px = r.px; + } + + template + shared_ptr(shared_ptr const & r): px(r.px), pn(r.pn) // never throws + { + } + + template + shared_ptr(shared_ptr const & r, detail::static_cast_tag): px(static_cast(r.px)), pn(r.pn) + { + } + + template + shared_ptr(shared_ptr const & r, detail::const_cast_tag): px(const_cast(r.px)), pn(r.pn) + { + } + + template + shared_ptr(shared_ptr const & r, detail::dynamic_cast_tag): px(dynamic_cast(r.px)), pn(r.pn) + { + if(px == 0) // need to allocate new counter -- the cast failed + { + pn = detail::shared_count(); + } + } + + template + shared_ptr(shared_ptr const & r, detail::polymorphic_cast_tag): px(dynamic_cast(r.px)), pn(r.pn) + { + if(px == 0) + { + boost::serialization::throw_exception(std::bad_cast()); + } + } + +#ifndef BOOST_NO_AUTO_PTR + + template + explicit shared_ptr(std::auto_ptr & r): px(r.get()), pn() + { + Y * tmp = r.get(); + pn = detail::shared_count(r); + detail::sp_enable_shared_from_this( pn, tmp, tmp ); + } + +#endif + +#if !defined(BOOST_MSVC) || (BOOST_MSVC > 1200) + + template + shared_ptr & operator=(shared_ptr const & r) // never throws + { + px = r.px; + pn = r.pn; // shared_count::op= doesn't throw + return *this; + } + +#endif + +#ifndef BOOST_NO_AUTO_PTR + + template + shared_ptr & operator=(std::auto_ptr & r) + { + this_type(r).swap(*this); + return *this; + } + +#endif + + void reset() // never throws in 1.30+ + { + this_type().swap(*this); + } + + template void reset(Y * p) // Y must be complete + { + BOOST_ASSERT(p == 0 || p != px); // catch self-reset errors + this_type(p).swap(*this); + } + + template void reset(Y * p, D d) + { + this_type(p, d).swap(*this); + } + + reference operator* () const // never throws + { + BOOST_ASSERT(px != 0); + return *px; + } + + T * operator-> () const // never throws + { + BOOST_ASSERT(px != 0); + return px; + } + + T * get() const // never throws + { + return px; + } + + // implicit conversion to "bool" + +#if defined(__SUNPRO_CC) && BOOST_WORKAROUND(__SUNPRO_CC, <= 0x530) + + operator bool () const + { + return px != 0; + } + +#elif defined(__MWERKS__) && BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3003)) + typedef T * (this_type::*unspecified_bool_type)() const; + + operator unspecified_bool_type() const // never throws + { + return px == 0? 0: &this_type::get; + } + +#else + + typedef T * this_type::*unspecified_bool_type; + + operator unspecified_bool_type() const // never throws + { + return px == 0? 0: &this_type::px; + } + +#endif + + // operator! is redundant, but some compilers need it + + bool operator! () const // never throws + { + return px == 0; + } + + bool unique() const // never throws + { + return pn.unique(); + } + + long use_count() const // never throws + { + return pn.use_count(); + } + + void swap(shared_ptr< T > & other) // never throws + { + std::swap(px, other.px); + pn.swap(other.pn); + } + + template bool _internal_less(shared_ptr const & rhs) const + { + return pn < rhs.pn; + } + + void * _internal_get_deleter(std::type_info const & ti) const + { + return pn.get_deleter(ti); + } + +// Tasteless as this may seem, making all members public allows member templates +// to work in the absence of member template friends. (Matthew Langston) + +#ifndef BOOST_NO_MEMBER_TEMPLATE_FRIENDS + +private: + + template friend class shared_ptr; + template friend class weak_ptr; + + +#endif +public: // for serialization + T * px; // contained pointer + detail::shared_count pn; // reference counter + +}; // shared_ptr + +template inline bool operator==(shared_ptr< T > const & a, shared_ptr const & b) +{ + return a.get() == b.get(); +} + +template inline bool operator!=(shared_ptr< T > const & a, shared_ptr const & b) +{ + return a.get() != b.get(); +} + +template inline bool operator<(shared_ptr< T > const & a, shared_ptr const & b) +{ + return a._internal_less(b); +} + +template inline void swap(shared_ptr< T > & a, shared_ptr< T > & b) +{ + a.swap(b); +} + +template shared_ptr< T > static_pointer_cast(shared_ptr const & r) +{ + return shared_ptr< T >(r, detail::static_cast_tag()); +} + +template shared_ptr< T > const_pointer_cast(shared_ptr const & r) +{ + return shared_ptr< T >(r, detail::const_cast_tag()); +} + +template shared_ptr< T > dynamic_pointer_cast(shared_ptr const & r) +{ + return shared_ptr< T >(r, detail::dynamic_cast_tag()); +} + +// shared_*_cast names are deprecated. Use *_pointer_cast instead. + +template shared_ptr< T > shared_static_cast(shared_ptr const & r) +{ + return shared_ptr< T >(r, detail::static_cast_tag()); +} + +template shared_ptr< T > shared_dynamic_cast(shared_ptr const & r) +{ + return shared_ptr< T >(r, detail::dynamic_cast_tag()); +} + +template shared_ptr< T > shared_polymorphic_cast(shared_ptr const & r) +{ + return shared_ptr< T >(r, detail::polymorphic_cast_tag()); +} + +template shared_ptr< T > shared_polymorphic_downcast(shared_ptr const & r) +{ + BOOST_ASSERT(dynamic_cast(r.get()) == r.get()); + return shared_static_cast< T >(r); +} + +// get_pointer() enables boost::mem_fn to recognize shared_ptr + +template inline T * get_pointer(shared_ptr< T > const & p) +{ + return p.get(); +} + +// operator<< + + +template std::basic_ostream & operator<< (std::basic_ostream & os, shared_ptr const & p) +{ + os << p.get(); + return os; +} + +// get_deleter (experimental) + +#if defined(__EDG_VERSION__) && (__EDG_VERSION__ <= 238) + +// g++ 2.9x doesn't allow static_cast(void *) +// apparently EDG 2.38 also doesn't accept it + +template D * get_deleter(shared_ptr< T > const & p) +{ + void const * q = p._internal_get_deleter(typeid(D)); + return const_cast(static_cast(q)); +} + +#else + +template D * get_deleter(shared_ptr< T > const & p) +{ + return static_cast(p._internal_get_deleter(typeid(D))); +} + +#endif + +} // namespace boost + +#ifdef BOOST_MSVC +# pragma warning(pop) +#endif + +#endif // #if defined(BOOST_NO_MEMBER_TEMPLATES) && !defined(BOOST_MSVC6_MEMBER_TEMPLATES) + +#endif // #ifndef BOOST_SHARED_PTR_132_HPP_INCLUDED diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_nmt_132.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_nmt_132.hpp new file mode 100644 index 00000000000..490e7ddd3d0 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_nmt_132.hpp @@ -0,0 +1,182 @@ +#ifndef BOOST_DETAIL_SHARED_PTR_NMT_132_HPP_INCLUDED +#define BOOST_DETAIL_SHARED_PTR_NMT_132_HPP_INCLUDED + +// +// detail/shared_ptr_nmt.hpp - shared_ptr.hpp without member templates +// +// (C) Copyright Greg Colvin and Beman Dawes 1998, 1999. +// Copyright (c) 2001, 2002 Peter Dimov +// +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) +// +// See http://www.boost.org/libs/smart_ptr/shared_ptr.htm for documentation. +// + +#include +#include +#include +#include + +#ifndef BOOST_NO_AUTO_PTR +# include // for std::auto_ptr +#endif + +#include // for std::swap +#include // for std::less +#include // for std::bad_alloc + +namespace boost +{ + +template class shared_ptr +{ +private: + + typedef detail::atomic_count count_type; + +public: + + typedef T element_type; + typedef T value_type; + + explicit shared_ptr(T * p = 0): px(p) + { +#ifndef BOOST_NO_EXCEPTIONS + + try // prevent leak if new throws + { + pn = new count_type(1); + } + catch(...) + { + boost::checked_delete(p); + throw; + } + +#else + + pn = new count_type(1); + + if(pn == 0) + { + boost::checked_delete(p); + boost::serialization::throw_exception(std::bad_alloc()); + } + +#endif + } + + ~shared_ptr() + { + if(--*pn == 0) + { + boost::checked_delete(px); + delete pn; + } + } + + shared_ptr(shared_ptr const & r): px(r.px) // never throws + { + pn = r.pn; + ++*pn; + } + + shared_ptr & operator=(shared_ptr const & r) + { + shared_ptr(r).swap(*this); + return *this; + } + +#ifndef BOOST_NO_AUTO_PTR + + explicit shared_ptr(std::auto_ptr< T > & r) + { + pn = new count_type(1); // may throw + px = r.release(); // fix: moved here to stop leak if new throws + } + + shared_ptr & operator=(std::auto_ptr< T > & r) + { + shared_ptr(r).swap(*this); + return *this; + } + +#endif + + void reset(T * p = 0) + { + BOOST_ASSERT(p == 0 || p != px); + shared_ptr(p).swap(*this); + } + + T & operator*() const // never throws + { + BOOST_ASSERT(px != 0); + return *px; + } + + T * operator->() const // never throws + { + BOOST_ASSERT(px != 0); + return px; + } + + T * get() const // never throws + { + return px; + } + + long use_count() const // never throws + { + return *pn; + } + + bool unique() const // never throws + { + return *pn == 1; + } + + void swap(shared_ptr< T > & other) // never throws + { + std::swap(px, other.px); + std::swap(pn, other.pn); + } + +private: + + T * px; // contained pointer + count_type * pn; // ptr to reference counter +}; + +template inline bool operator==(shared_ptr< T > const & a, shared_ptr const & b) +{ + return a.get() == b.get(); +} + +template inline bool operator!=(shared_ptr< T > const & a, shared_ptr const & b) +{ + return a.get() != b.get(); +} + +template inline bool operator<(shared_ptr< T > const & a, shared_ptr< T > const & b) +{ + return std::less()(a.get(), b.get()); +} + +template void swap(shared_ptr< T > & a, shared_ptr< T > & b) +{ + a.swap(b); +} + +// get_pointer() enables boost::mem_fn to recognize shared_ptr + +template inline T * get_pointer(shared_ptr< T > const & p) +{ + return p.get(); +} + +} // namespace boost + +#endif // #ifndef BOOST_DETAIL_SHARED_PTR_NMT_132_HPP_INCLUDED diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/stack_constructor.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/stack_constructor.hpp new file mode 100644 index 00000000000..ae14832c6db --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/detail/stack_constructor.hpp @@ -0,0 +1,66 @@ +#ifndef BOOST_SERIALIZATION_DETAIL_STACK_CONSTRUCTOR_HPP +#define BOOST_SERIALIZATION_DETAIL_STACK_CONSTRUCTOR_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// stack_constructor.hpp: serialization for loading stl collections + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include + +namespace boost{ +namespace serialization { +namespace detail { + +// reserve space on stack for an object of type T without actually +// construction such an object +template +struct stack_allocate +{ + T * address() { + return static_cast(storage_.address()); + } + T & reference() { + return * address(); + } +private: + typedef typename boost::aligned_storage< + sizeof(T), + boost::alignment_of::value + > type; + type storage_; +}; + +// construct element on the stack +template +struct stack_construct : public stack_allocate +{ + stack_construct(Archive & ar, const unsigned int version){ + // note borland emits a no-op without the explicit namespace + boost::serialization::load_construct_data_adl( + ar, + this->address(), + version + ); + } + ~stack_construct(){ + this->address()->~T(); // undo load_construct_data above + } +}; + +} // detail +} // serializaition +} // boost + +#endif // BOOST_SERIALIZATION_DETAIL_STACH_CONSTRUCTOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/ephemeral.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/ephemeral.hpp new file mode 100644 index 00000000000..3a422c30a35 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/ephemeral.hpp @@ -0,0 +1,72 @@ +#ifndef BOOST_SERIALIZATION_EPHEMERAL_HPP +#define BOOST_SERIALIZATION_EPHEMERAL_HPP + +// MS compatible compilers support +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// ephemeral_object.hpp: interface for serialization system. + +// (C) Copyright 2007 Matthias Troyer. +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +template +struct ephemeral_object : + public wrapper_traits > +{ + explicit ephemeral_object(T& t) : + val(t) + {} + + T & value() const { + return val; + } + + const T & const_value() const { + return val; + } + + template + void serialize(Archive &ar, const unsigned int) const + { + ar & val; + } + +private: + T & val; +}; + +template +inline +const ephemeral_object ephemeral(const char * name, T & t){ + return ephemeral_object(name, t); +} + +} // seralization +} // boost + +#endif // BOOST_SERIALIZATION_EPHEMERAL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/export.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/export.hpp new file mode 100644 index 00000000000..9eef440df42 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/export.hpp @@ -0,0 +1,225 @@ +#ifndef BOOST_SERIALIZATION_EXPORT_HPP +#define BOOST_SERIALIZATION_EXPORT_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// export.hpp: set traits of classes to be serialized + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// (C) Copyright 2006 David Abrahams - http://www.boost.org. +// implementation of class export functionality. This is an alternative to +// "forward declaration" method to provoke instantiation of derived classes +// that are to be serialized through pointers. + +#include +#include // NULL + +#include +#include +#include +#include + +#include +#include +#include +#include + +#include // for guid_defined only +#include +#include +#include +#include + +#include + +#include + +namespace boost { +namespace archive { +namespace detail { + +class basic_pointer_iserializer; +class basic_pointer_oserializer; + +template +class pointer_iserializer; +template +class pointer_oserializer; + +template +struct export_impl +{ + static const basic_pointer_iserializer & + enable_load(mpl::true_){ + return boost::serialization::singleton< + pointer_iserializer + >::get_const_instance(); + } + + static const basic_pointer_oserializer & + enable_save(mpl::true_){ + return boost::serialization::singleton< + pointer_oserializer + >::get_const_instance(); + } + inline static void enable_load(mpl::false_) {} + inline static void enable_save(mpl::false_) {} +}; + +// On many platforms, naming a specialization of this template is +// enough to cause its argument to be instantiated. +template +struct instantiate_function {}; + +template +struct ptr_serialization_support +{ +# if defined(BOOST_MSVC) || defined(__SUNPRO_CC) + virtual BOOST_DLLEXPORT void instantiate() BOOST_USED; +# else + static BOOST_DLLEXPORT void instantiate() BOOST_USED; + typedef instantiate_function< + &ptr_serialization_support::instantiate + > x; +# endif +}; + +template +BOOST_DLLEXPORT void +ptr_serialization_support::instantiate() +{ + export_impl::enable_save( + typename Archive::is_saving() + ); + + export_impl::enable_load( + typename Archive::is_loading() + ); +} + +// Note INTENTIONAL usage of anonymous namespace in header. +// This was made this way so that export.hpp could be included +// in other headers. This is still under study. + +namespace extra_detail { + +template +struct guid_initializer +{ + void export_guid(mpl::false_) const { + // generates the statically-initialized objects whose constructors + // register the information allowing serialization of T objects + // through pointers to their base classes. + instantiate_ptr_serialization((T*)0, 0, adl_tag()); + } + void export_guid(mpl::true_) const { + } + guid_initializer const & export_guid() const { + BOOST_STATIC_WARNING(boost::is_polymorphic< T >::value); + // note: exporting an abstract base class will have no effect + // and cannot be used to instantitiate serialization code + // (one might be using this in a DLL to instantiate code) + //BOOST_STATIC_WARNING(! boost::serialization::is_abstract< T >::value); + export_guid(boost::serialization::is_abstract< T >()); + return *this; + } +}; + +template +struct init_guid; + +} // anonymous +} // namespace detail +} // namespace archive +} // namespace boost + +#define BOOST_CLASS_EXPORT_IMPLEMENT(T) \ + namespace boost { \ + namespace archive { \ + namespace detail { \ + namespace extra_detail { \ + template<> \ + struct init_guid< T > { \ + static guid_initializer< T > const & g; \ + }; \ + guid_initializer< T > const & init_guid< T >::g = \ + ::boost::serialization::singleton< \ + guid_initializer< T > \ + >::get_mutable_instance().export_guid(); \ + }}}} \ +/**/ + +#define BOOST_CLASS_EXPORT_KEY2(T, K) \ +namespace boost { \ +namespace serialization { \ +template<> \ +struct guid_defined< T > : boost::mpl::true_ {}; \ +template<> \ +inline const char * guid< T >(){ \ + return K; \ +} \ +} /* serialization */ \ +} /* boost */ \ +/**/ + +#define BOOST_CLASS_EXPORT_KEY(T) \ + BOOST_CLASS_EXPORT_KEY2(T, BOOST_PP_STRINGIZE(T)) \ +/**/ + +#define BOOST_CLASS_EXPORT_GUID(T, K) \ +BOOST_CLASS_EXPORT_KEY2(T, K) \ +BOOST_CLASS_EXPORT_IMPLEMENT(T) \ +/**/ + +#if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) + +// CodeWarrior fails to construct static members of class templates +// when they are instantiated from within templates, so on that +// compiler we ask users to specifically register base/derived class +// relationships for exported classes. On all other compilers, use of +// this macro is entirely optional. +# define BOOST_SERIALIZATION_MWERKS_BASE_AND_DERIVED(Base,Derived) \ +namespace { \ + static int BOOST_PP_CAT(boost_serialization_mwerks_init_, __LINE__) = \ + (::boost::archive::detail::instantiate_ptr_serialization((Derived*)0,0), 3); \ + static int BOOST_PP_CAT(boost_serialization_mwerks_init2_, __LINE__) = ( \ + ::boost::serialization::void_cast_register((Derived*)0,(Base*)0) \ + , 3); \ +} + +#else + +# define BOOST_SERIALIZATION_MWERKS_BASE_AND_DERIVED(Base,Derived) + +#endif + +// check for unnecessary export. T isn't polymorphic so there is no +// need to export it. +#define BOOST_CLASS_EXPORT_CHECK(T) \ + BOOST_STATIC_WARNING( \ + boost::is_polymorphic::value \ + ); \ + /**/ + +// the default exportable class identifier is the class name +// the default list of archives types for which code id generated +// are the originally included with this serialization system +#define BOOST_CLASS_EXPORT(T) \ + BOOST_CLASS_EXPORT_GUID( \ + T, \ + BOOST_PP_STRINGIZE(T) \ + ) \ + /**/ + +#endif // BOOST_SERIALIZATION_EXPORT_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info.hpp new file mode 100644 index 00000000000..bb2a190d465 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info.hpp @@ -0,0 +1,116 @@ +#ifndef BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_HPP +#define BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// extended_type_info.hpp: interface for portable version of type_info + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// for now, extended type info is part of the serialization libraries +// this could change in the future. +#include +#include +#include // NULL +#include +#include +#include + +#include +#include // must be the last header +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4251 4231 4660 4275) +#endif + +#define BOOST_SERIALIZATION_MAX_KEY_SIZE 128 + +namespace boost { +namespace serialization { + +namespace void_cast_detail{ + class void_caster; +} + +class BOOST_SYMBOL_VISIBLE extended_type_info : + private boost::noncopyable +{ +private: + friend class boost::serialization::void_cast_detail::void_caster; + + // used to uniquely identify the type of class derived from this one + // so that different derivations of this class can be simultaneously + // included in implementation of sets and maps. + const unsigned int m_type_info_key; + virtual bool is_less_than(const extended_type_info & /*rhs*/) const = 0; + virtual bool is_equal(const extended_type_info & /*rhs*/) const = 0; + const char * m_key; + +protected: + BOOST_SERIALIZATION_DECL void key_unregister() const; + BOOST_SERIALIZATION_DECL void key_register() const; + // this class can't be used as is. It's just the + // common functionality for all type_info replacement + // systems. Hence, make these protected + BOOST_SERIALIZATION_DECL extended_type_info( + const unsigned int type_info_key, + const char * key + ); + virtual BOOST_SERIALIZATION_DECL ~extended_type_info(); +public: + const char * get_key() const { + return m_key; + } + virtual const char * get_debug_info() const = 0; + BOOST_SERIALIZATION_DECL bool operator<(const extended_type_info &rhs) const; + BOOST_SERIALIZATION_DECL bool operator==(const extended_type_info &rhs) const; + bool operator!=(const extended_type_info &rhs) const { + return !(operator==(rhs)); + } + // note explicit "export" of static function to work around + // gcc 4.5 mingw error + static BOOST_SERIALIZATION_DECL const extended_type_info * + find(const char *key); + // for plugins + virtual void * construct(unsigned int /*count*/ = 0, ...) const = 0; + virtual void destroy(void const * const /*p*/) const = 0; +}; + +template +struct guid_defined : boost::mpl::false_ {}; + +namespace ext { + template + struct guid_impl + { + static inline const char * call() + { + return NULL; + } + }; +} + +template +inline const char * guid(){ + return ext::guid_impl::call(); +} + +} // namespace serialization +} // namespace boost + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_no_rtti.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_no_rtti.hpp new file mode 100644 index 00000000000..aaa8b44459b --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_no_rtti.hpp @@ -0,0 +1,182 @@ +#ifndef BOOST_EXTENDED_TYPE_INFO_NO_RTTI_HPP +#define BOOST_EXTENDED_TYPE_INFO_NO_RTTI_HPP + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +// extended_type_info_no_rtti.hpp: implementation for version that depends +// on runtime typing (rtti - typeid) but uses a user specified string +// as the portable class identifier. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. +#include + +#include +#include + +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +// hijack serialization access +#include + +#include // must be the last header +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4251 4231 4660 4275 4511 4512) +#endif + +namespace boost { +namespace serialization { +/////////////////////////////////////////////////////////////////////// +// define a special type_info that doesn't depend on rtti which is not +// available in all situations. + +namespace no_rtti_system { + +// common base class to share type_info_key. This is used to +// identify the method used to keep track of the extended type +class BOOST_SYMBOL_VISIBLE extended_type_info_no_rtti_0 : + public extended_type_info +{ +protected: + BOOST_SERIALIZATION_DECL extended_type_info_no_rtti_0(const char * key); + BOOST_SERIALIZATION_DECL ~extended_type_info_no_rtti_0(); +public: + virtual BOOST_SERIALIZATION_DECL bool + is_less_than(const boost::serialization::extended_type_info &rhs) const ; + virtual BOOST_SERIALIZATION_DECL bool + is_equal(const boost::serialization::extended_type_info &rhs) const ; +}; + +} // no_rtti_system + +template +class extended_type_info_no_rtti : + public no_rtti_system::extended_type_info_no_rtti_0, + public singleton > +{ + template + struct action { + struct defined { + static const char * invoke(){ + return guid< T >(); + } + }; + struct undefined { + // if your program traps here - you failed to + // export a guid for this type. the no_rtti + // system requires export for types serialized + // as pointers. + BOOST_STATIC_ASSERT(0 == sizeof(T)); + static const char * invoke(); + }; + static const char * invoke(){ + typedef + typename boost::mpl::if_c< + tf, + defined, + undefined + >::type type; + return type::invoke(); + } + }; +public: + extended_type_info_no_rtti() : + no_rtti_system::extended_type_info_no_rtti_0(get_key()) + { + key_register(); + } + ~extended_type_info_no_rtti(){ + key_unregister(); + } + const extended_type_info * + get_derived_extended_type_info(const T & t) const { + // find the type that corresponds to the most derived type. + // this implementation doesn't depend on typeid() but assumes + // that the specified type has a function of the following signature. + // A common implemention of such a function is to define as a virtual + // function. So if the is not a polymporphic type it's likely an error + BOOST_STATIC_WARNING(boost::is_polymorphic< T >::value); + const char * derived_key = t.get_key(); + BOOST_ASSERT(NULL != derived_key); + return boost::serialization::extended_type_info::find(derived_key); + } + const char * get_key() const{ + return action::value >::invoke(); + } + virtual const char * get_debug_info() const{ + return action::value >::invoke(); + } + virtual void * construct(unsigned int count, ...) const{ + // count up the arguments + std::va_list ap; + va_start(ap, count); + switch(count){ + case 0: + return factory::type, 0>(ap); + case 1: + return factory::type, 1>(ap); + case 2: + return factory::type, 2>(ap); + case 3: + return factory::type, 3>(ap); + case 4: + return factory::type, 4>(ap); + default: + BOOST_ASSERT(false); // too many arguments + // throw exception here? + return NULL; + } + } + virtual void destroy(void const * const p) const{ + boost::serialization::access::destroy( + static_cast(p) + ); + //delete static_cast(p) ; + } +}; + +} // namespace serialization +} // namespace boost + +/////////////////////////////////////////////////////////////////////////////// +// If no other implementation has been designated as default, +// use this one. To use this implementation as the default, specify it +// before any of the other headers. + +#ifndef BOOST_SERIALIZATION_DEFAULT_TYPE_INFO + #define BOOST_SERIALIZATION_DEFAULT_TYPE_INFO + namespace boost { + namespace serialization { + template + struct extended_type_info_impl { + typedef typename + boost::serialization::extended_type_info_no_rtti< T > type; + }; + } // namespace serialization + } // namespace boost +#endif + +#ifdef BOOST_MSVC +# pragma warning(pop) +#endif +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_EXTENDED_TYPE_INFO_NO_RTTI_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_typeid.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_typeid.hpp new file mode 100644 index 00000000000..8ee591b3169 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_typeid.hpp @@ -0,0 +1,167 @@ +#ifndef BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_TYPEID_HPP +#define BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_TYPEID_HPP + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +// extended_type_info_typeid.hpp: implementation for version that depends +// on runtime typing (rtti - typeid) but uses a user specified string +// as the portable class identifier. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include +#include + +#include +#include +#include +#include + +#include +#include +#include +#include + +// hijack serialization access +#include + +#include + +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4251 4231 4660 4275 4511 4512) +#endif + +namespace boost { +namespace serialization { +namespace typeid_system { + +class BOOST_SYMBOL_VISIBLE extended_type_info_typeid_0 : + public extended_type_info +{ + virtual const char * get_debug_info() const { + if(static_cast(0) == m_ti) + return static_cast(0); + return m_ti->name(); + } +protected: + const std::type_info * m_ti; + BOOST_SERIALIZATION_DECL extended_type_info_typeid_0(const char * key); + BOOST_SERIALIZATION_DECL ~extended_type_info_typeid_0(); + BOOST_SERIALIZATION_DECL void type_register(const std::type_info & ti); + BOOST_SERIALIZATION_DECL void type_unregister(); + BOOST_SERIALIZATION_DECL const extended_type_info * + get_extended_type_info(const std::type_info & ti) const; +public: + virtual BOOST_SERIALIZATION_DECL bool + is_less_than(const extended_type_info &rhs) const; + virtual BOOST_SERIALIZATION_DECL bool + is_equal(const extended_type_info &rhs) const; + const std::type_info & get_typeid() const { + return *m_ti; + } +}; + +} // typeid_system + +template +class extended_type_info_typeid : + public typeid_system::extended_type_info_typeid_0, + public singleton > +{ +public: + extended_type_info_typeid() : + typeid_system::extended_type_info_typeid_0( + boost::serialization::guid< T >() + ) + { + type_register(typeid(T)); + key_register(); + } + ~extended_type_info_typeid(){ + key_unregister(); + type_unregister(); + } + // get the eti record for the true type of this record + // relying upon standard type info implemenation (rtti) + const extended_type_info * + get_derived_extended_type_info(const T & t) const { + // note: this implementation - based on usage of typeid (rtti) + // only does something if the class has at least one virtual function. + BOOST_STATIC_WARNING(boost::is_polymorphic< T >::value); + return + typeid_system::extended_type_info_typeid_0::get_extended_type_info( + typeid(t) + ); + } + const char * get_key() const { + return boost::serialization::guid< T >(); + } + virtual void * construct(unsigned int count, ...) const{ + // count up the arguments + std::va_list ap; + va_start(ap, count); + switch(count){ + case 0: + return factory::type, 0>(ap); + case 1: + return factory::type, 1>(ap); + case 2: + return factory::type, 2>(ap); + case 3: + return factory::type, 3>(ap); + case 4: + return factory::type, 4>(ap); + default: + BOOST_ASSERT(false); // too many arguments + // throw exception here? + return NULL; + } + } + virtual void destroy(void const * const p) const { + boost::serialization::access::destroy( + static_cast(p) + ); + //delete static_cast(p); + } +}; + +} // namespace serialization +} // namespace boost + +/////////////////////////////////////////////////////////////////////////////// +// If no other implementation has been designated as default, +// use this one. To use this implementation as the default, specify it +// before any of the other headers. +#ifndef BOOST_SERIALIZATION_DEFAULT_TYPE_INFO + #define BOOST_SERIALIZATION_DEFAULT_TYPE_INFO + namespace boost { + namespace serialization { + template + struct extended_type_info_impl { + typedef typename + boost::serialization::extended_type_info_typeid< T > type; + }; + } // namespace serialization + } // namespace boost +#endif + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_TYPEID_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/factory.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/factory.hpp new file mode 100644 index 00000000000..2db7e7e36c3 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/factory.hpp @@ -0,0 +1,102 @@ +#ifndef BOOST_SERIALIZATION_FACTORY_HPP +#define BOOST_SERIALIZATION_FACTORY_HPP + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +// factory.hpp: create an instance from an extended_type_info instance. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // valist +#include // NULL + +#include +#include +#include + +namespace std{ + #if defined(__LIBCOMO__) + using ::va_list; + #endif +} // namespace std + +namespace boost { +namespace serialization { + +// default implementation does nothing. +template +T * factory(std::va_list){ + BOOST_ASSERT(false); + // throw exception here? + return NULL; +} + +} // namespace serialization +} // namespace boost + +#define BOOST_SERIALIZATION_FACTORY(N, T, A0, A1, A2, A3) \ +namespace boost { \ +namespace serialization { \ + template<> \ + T * factory(std::va_list ap){ \ + BOOST_PP_IF(BOOST_PP_GREATER(N, 0) \ + , A0 a0 = va_arg(ap, A0);, BOOST_PP_EMPTY()) \ + BOOST_PP_IF(BOOST_PP_GREATER(N, 1) \ + , A1 a1 = va_arg(ap, A1);, BOOST_PP_EMPTY()) \ + BOOST_PP_IF(BOOST_PP_GREATER(N, 2) \ + , A2 a2 = va_arg(ap, A2);, BOOST_PP_EMPTY()) \ + BOOST_PP_IF(BOOST_PP_GREATER(N, 3) \ + , A3 a3 = va_arg(ap, A3);, BOOST_PP_EMPTY()) \ + return new T( \ + BOOST_PP_IF(BOOST_PP_GREATER(N, 0) \ + , a0, BOOST_PP_EMPTY()) \ + BOOST_PP_IF(BOOST_PP_GREATER(N, 1)) \ + , BOOST_PP_COMMA, BOOST_PP_EMPTY)() \ + BOOST_PP_IF(BOOST_PP_GREATER(N, 1) \ + , a1, BOOST_PP_EMPTY()) \ + BOOST_PP_IF(BOOST_PP_GREATER(N, 2)) \ + , BOOST_PP_COMMA, BOOST_PP_EMPTY)() \ + BOOST_PP_IF(BOOST_PP_GREATER(N, 2) \ + , a2, BOOST_PP_EMPTY()) \ + BOOST_PP_IF(BOOST_PP_GREATER(N, 3)) \ + , BOOST_PP_COMMA, BOOST_PP_EMPTY)() \ + BOOST_PP_IF(BOOST_PP_GREATER(N, 3) \ + , a3, BOOST_PP_EMPTY()) \ + ); \ + } \ +} \ +} /**/ + +#define BOOST_SERIALIZATION_FACTORY_4(T, A0, A1, A2, A3) \ + BOOST_SERIALIZATION_FACTORY(4, T, A0, A1, A2, A3) + +#define BOOST_SERIALIZATION_FACTORY_3(T, A0, A1, A2) \ + BOOST_SERIALIZATION_FACTORY(3, T, A0, A1, A2, 0) + +#define BOOST_SERIALIZATION_FACTORY_2(T, A0, A1) \ + BOOST_SERIALIZATION_FACTORY(2, T, A0, A1, 0, 0) + +#define BOOST_SERIALIZATION_FACTORY_1(T, A0) \ + BOOST_SERIALIZATION_FACTORY(1, T, A0, 0, 0, 0) + +#define BOOST_SERIALIZATION_FACTORY_0(T) \ +namespace boost { \ +namespace serialization { \ + template<> \ + T * factory(std::va_list){ \ + return new T(); \ + } \ +} \ +} \ +/**/ + +#endif // BOOST_SERIALIZATION_FACTORY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/force_include.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/force_include.hpp new file mode 100644 index 00000000000..55ab79d0d58 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/force_include.hpp @@ -0,0 +1,55 @@ +#ifndef BOOST_SERIALIZATION_FORCE_INCLUDE_HPP +#define BOOST_SERIALIZATION_FORCE_INCLUDE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// force_include.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +// the following help macro is to guarentee that certain coded +// is not removed by over-eager linker optimiser. In certain cases +// we create static objects must be created but are actually never +// referenced - creation has a side-effect such as global registration +// which is important to us. We make an effort to refer these objects +// so that a smart linker won't remove them as being unreferenced. +// In microsoft compilers, inlining the code that does the referring +// means the code gets lost and the static object is not included +// in the library and hence never registered. This manifests itself +// in an ungraceful crash at runtime when (and only when) built in +// release mode. + +#if defined(BOOST_HAS_DECLSPEC) && !defined(__COMO__) +# define BOOST_DLLEXPORT __declspec(dllexport) +#elif ! defined(_WIN32) && ! defined(_WIN64) +# if defined(__MWERKS__) +# define BOOST_DLLEXPORT __declspec(dllexport) +# elif defined(__GNUC__) && (__GNUC__ >= 3) +# define BOOST_USED __attribute__ ((__used__)) +# elif defined(__IBMCPP__) && (__IBMCPP__ >= 1110) +# define BOOST_USED __attribute__ ((__used__)) +# elif defined(__INTEL_COMPILER) && (BOOST_INTEL_CXX_VERSION >= 800) +# define BOOST_USED __attribute__ ((__used__)) +# endif +#endif + +#ifndef BOOST_USED +# define BOOST_USED +#endif + +#ifndef BOOST_DLLEXPORT +# define BOOST_DLLEXPORT +#endif + +#endif // BOOST_SERIALIZATION_FORCE_INCLUDE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/forward_list.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/forward_list.hpp new file mode 100644 index 00000000000..b8a3c20a6ea --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/forward_list.hpp @@ -0,0 +1,124 @@ +#ifndef BOOST_SERIALIZATION_FORWARD_LIST_HPP +#define BOOST_SERIALIZATION_FORWARD_LIST_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// forward_list.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include // distance + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +template +inline void save( + Archive & ar, + const std::forward_list &t, + const unsigned int /*file_version*/ +){ + const collection_size_type count(std::distance(t.cbegin(), t.cend())); + boost::serialization::stl::save_collection< + Archive, + std::forward_list + >(ar, t, count); +} + +namespace stl { + +template< + class Archive, + class T, + class Allocator +> +typename boost::disable_if< + typename detail::is_default_constructible< + typename std::forward_list::value_type + >, + void +>::type +collection_load_impl( + Archive & ar, + std::forward_list &t, + collection_size_type count, + item_version_type item_version +){ + t.clear(); + boost::serialization::detail::stack_construct u(ar, item_version); + ar >> boost::serialization::make_nvp("item", u.reference()); + t.push_front(boost::move(u.reference())); + typename std::forward_list::iterator last; + last = t.begin(); + ar.reset_object_address(&(*t.begin()) , & u.reference()); + while(--count > 0){ + detail::stack_construct u(ar, item_version); + ar >> boost::serialization::make_nvp("item", u.reference()); + last = t.insert_after(last, boost::move(u.reference())); + ar.reset_object_address(&(*last) , & u.reference()); + } +} + +} // stl + +template +inline void load( + Archive & ar, + std::forward_list &t, + const unsigned int /*file_version*/ +){ + const boost::archive::library_version_type library_version( + ar.get_library_version() + ); + // retrieve number of elements + item_version_type item_version(0); + collection_size_type count; + ar >> BOOST_SERIALIZATION_NVP(count); + if(boost::archive::library_version_type(3) < library_version){ + ar >> BOOST_SERIALIZATION_NVP(item_version); + } + stl::collection_load_impl(ar, t, count, item_version); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template +inline void serialize( + Archive & ar, + std::forward_list &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // serialization +} // namespace boost + +#include + +BOOST_SERIALIZATION_COLLECTION_TRAITS(std::forward_list) + +#endif // BOOST_SERIALIZATION_FORWARD_LIST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_load_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_load_imp.hpp new file mode 100644 index 00000000000..88def8f1aa4 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_load_imp.hpp @@ -0,0 +1,77 @@ +#ifndef BOOST_SERIALIZATION_HASH_COLLECTIONS_LOAD_IMP_HPP +#define BOOST_SERIALIZATION_HASH_COLLECTIONS_LOAD_IMP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +# pragma warning (disable : 4786) // too long name, harmless warning +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// hash_collections_load_imp.hpp: serialization for loading stl collections + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// helper function templates for serialization of hashed collections +#include +#include +#include +#include +#include + +namespace boost{ +namespace serialization { +namespace stl { + +////////////////////////////////////////////////////////////////////// +// implementation of serialization for STL containers +// +template +inline void load_hash_collection(Archive & ar, Container &s) +{ + collection_size_type count; + collection_size_type bucket_count; + boost::serialization::item_version_type item_version(0); + boost::archive::library_version_type library_version( + ar.get_library_version() + ); + // retrieve number of elements + if(boost::archive::library_version_type(6) != library_version){ + ar >> BOOST_SERIALIZATION_NVP(count); + ar >> BOOST_SERIALIZATION_NVP(bucket_count); + } + else{ + // note: fixup for error in version 6. collection size was + // changed to size_t BUT for hashed collections it was implemented + // as an unsigned int. This should be a problem only on win64 machines + // but I'll leave it for everyone just in case. + unsigned int c; + unsigned int bc; + ar >> BOOST_SERIALIZATION_NVP(c); + count = c; + ar >> BOOST_SERIALIZATION_NVP(bc); + bucket_count = bc; + } + if(boost::archive::library_version_type(3) < library_version){ + ar >> BOOST_SERIALIZATION_NVP(item_version); + } + s.clear(); + #if ! defined(__MWERKS__) + s.resize(bucket_count); + #endif + InputFunction ifunc; + while(count-- > 0){ + ifunc(ar, s, item_version); + } +} + +} // namespace stl +} // namespace serialization +} // namespace boost + +#endif //BOOST_SERIALIZATION_HASH_COLLECTIONS_LOAD_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_save_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_save_imp.hpp new file mode 100644 index 00000000000..65dfe83f16e --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_save_imp.hpp @@ -0,0 +1,97 @@ +#ifndef BOOST_SERIALIZATION_HASH_COLLECTIONS_SAVE_IMP_HPP +#define BOOST_SERIALIZATION_HASH_COLLECTIONS_SAVE_IMP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// hash_collections_save_imp.hpp: serialization for stl collections + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// helper function templates for serialization of collections + +#include +#include +#include +#include +#include +#include + +namespace boost{ +namespace serialization { +namespace stl { + +////////////////////////////////////////////////////////////////////// +// implementation of serialization for STL containers +// + +template +inline void save_hash_collection(Archive & ar, const Container &s) +{ + collection_size_type count(s.size()); + const collection_size_type bucket_count(s.bucket_count()); + const item_version_type item_version( + version::value + ); + + #if 0 + /* should only be necessary to create archives of previous versions + * which is not currently supported. So for now comment this out + */ + boost::archive::library_version_type library_version( + ar.get_library_version() + ); + // retrieve number of elements + if(boost::archive::library_version_type(6) != library_version){ + ar << BOOST_SERIALIZATION_NVP(count); + ar << BOOST_SERIALIZATION_NVP(bucket_count); + } + else{ + // note: fixup for error in version 6. collection size was + // changed to size_t BUT for hashed collections it was implemented + // as an unsigned int. This should be a problem only on win64 machines + // but I'll leave it for everyone just in case. + const unsigned int c = count; + const unsigned int bc = bucket_count; + ar << BOOST_SERIALIZATION_NVP(c); + ar << BOOST_SERIALIZATION_NVP(bc); + } + if(boost::archive::library_version_type(3) < library_version){ + // record number of elements + // make sure the target type is registered so we can retrieve + // the version when we load + ar << BOOST_SERIALIZATION_NVP(item_version); + } + #else + ar << BOOST_SERIALIZATION_NVP(count); + ar << BOOST_SERIALIZATION_NVP(bucket_count); + ar << BOOST_SERIALIZATION_NVP(item_version); + #endif + + typename Container::const_iterator it = s.begin(); + while(count-- > 0){ + // note borland emits a no-op without the explicit namespace + boost::serialization::save_construct_data_adl( + ar, + &(*it), + boost::serialization::version< + typename Container::value_type + >::value + ); + ar << boost::serialization::make_nvp("item", *it++); + } +} + +} // namespace stl +} // namespace serialization +} // namespace boost + +#endif //BOOST_SERIALIZATION_HASH_COLLECTIONS_SAVE_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/hash_map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/hash_map.hpp new file mode 100644 index 00000000000..22626db6838 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/hash_map.hpp @@ -0,0 +1,232 @@ +#ifndef BOOST_SERIALIZATION_HASH_MAP_HPP +#define BOOST_SERIALIZATION_HASH_MAP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// serialization/hash_map.hpp: +// serialization for stl hash_map templates + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifdef BOOST_HAS_HASH +#include BOOST_HASH_MAP_HEADER + +#include +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +namespace stl { + +// map input +template +struct archive_input_hash_map +{ + inline void operator()( + Archive &ar, + Container &s, + const unsigned int v + ){ + typedef typename Container::value_type type; + detail::stack_construct t(ar, v); + // borland fails silently w/o full namespace + ar >> boost::serialization::make_nvp("item", t.reference()); + std::pair result = + s.insert(boost::move(t.reference())); + // note: the following presumes that the map::value_type was NOT tracked + // in the archive. This is the usual case, but here there is no way + // to determine that. + if(result.second){ + ar.reset_object_address( + & (result.first->second), + & t.reference().second + ); + } + } +}; + +// multimap input +template +struct archive_input_hash_multimap +{ + inline void operator()( + Archive &ar, + Container &s, + const unsigned int v + ){ + typedef typename Container::value_type type; + detail::stack_construct t(ar, v); + // borland fails silently w/o full namespace + ar >> boost::serialization::make_nvp("item", t.reference()); + typename Container::const_iterator result + = s.insert(boost::move(t.reference())); + // note: the following presumes that the map::value_type was NOT tracked + // in the archive. This is the usual case, but here there is no way + // to determine that. + ar.reset_object_address( + & result->second, + & t.reference() + ); + } +}; + +} // stl + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void save( + Archive & ar, + const BOOST_STD_EXTENSION_NAMESPACE::hash_map< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int file_version +){ + boost::serialization::stl::save_hash_collection< + Archive, + BOOST_STD_EXTENSION_NAMESPACE::hash_map< + Key, HashFcn, EqualKey, Allocator + > + >(ar, t); +} + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void load( + Archive & ar, + BOOST_STD_EXTENSION_NAMESPACE::hash_map< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int file_version +){ + boost::serialization::stl::load_hash_collection< + Archive, + BOOST_STD_EXTENSION_NAMESPACE::hash_map< + Key, HashFcn, EqualKey, Allocator + >, + boost::serialization::stl::archive_input_hash_map< + Archive, + BOOST_STD_EXTENSION_NAMESPACE::hash_map< + Key, HashFcn, EqualKey, Allocator + > + > + >(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void serialize( + Archive & ar, + BOOST_STD_EXTENSION_NAMESPACE::hash_map< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +// hash_multimap +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void save( + Archive & ar, + const BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int file_version +){ + boost::serialization::stl::save_hash_collection< + Archive, + BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< + Key, HashFcn, EqualKey, Allocator + > + >(ar, t); +} + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void load( + Archive & ar, + BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int file_version +){ + boost::serialization::stl::load_hash_collection< + Archive, + BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< + Key, HashFcn, EqualKey, Allocator + >, + boost::serialization::stl::archive_input_hash_multimap< + Archive, + BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< + Key, HashFcn, EqualKey, Allocator + > + > + >(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void serialize( + Archive & ar, + BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +#endif // BOOST_HAS_HASH +#endif // BOOST_SERIALIZATION_HASH_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/hash_set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/hash_set.hpp new file mode 100644 index 00000000000..0c72c18457e --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/hash_set.hpp @@ -0,0 +1,222 @@ +#ifndef BOOST_SERIALIZATION_HASH_SET_HPP +#define BOOST_SERIALIZATION_HASH_SET_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// hash_set.hpp: serialization for stl hash_set templates + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifdef BOOST_HAS_HASH +#include BOOST_HASH_SET_HEADER + +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +namespace stl { + +// hash_set input +template +struct archive_input_hash_set +{ + inline void operator()( + Archive &ar, + Container &s, + const unsigned int v + ){ + typedef typename Container::value_type type; + detail::stack_construct t(ar, v); + // borland fails silently w/o full namespace + ar >> boost::serialization::make_nvp("item", t.reference()); + std::pair result = + s.insert(boost::move(t.reference())); + if(result.second) + ar.reset_object_address(& (* result.first), & t.reference()); + } +}; + +// hash_multiset input +template +struct archive_input_hash_multiset +{ + inline void operator()( + Archive &ar, + Container &s, + const unsigned int v + ){ + typedef typename Container::value_type type; + detail::stack_construct t(ar, v); + // borland fails silently w/o full namespace + ar >> boost::serialization::make_nvp("item", t.reference()); + typename Container::const_iterator result + = s.insert(boost::move(t.reference())); + ar.reset_object_address(& (* result), & t.reference()); + } +}; + +} // stl + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void save( + Archive & ar, + const BOOST_STD_EXTENSION_NAMESPACE::hash_set< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int file_version +){ + boost::serialization::stl::save_hash_collection< + Archive, + BOOST_STD_EXTENSION_NAMESPACE::hash_set< + Key, HashFcn, EqualKey, Allocator + > + >(ar, t); +} + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void load( + Archive & ar, + BOOST_STD_EXTENSION_NAMESPACE::hash_set< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int file_version +){ + boost::serialization::stl::load_hash_collection< + Archive, + BOOST_STD_EXTENSION_NAMESPACE::hash_set< + Key, HashFcn, EqualKey, Allocator + >, + boost::serialization::stl::archive_input_hash_set< + Archive, + BOOST_STD_EXTENSION_NAMESPACE::hash_set< + Key, HashFcn, EqualKey, Allocator + > + > + >(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void serialize( + Archive & ar, + BOOST_STD_EXTENSION_NAMESPACE::hash_set< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +// hash_multiset +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void save( + Archive & ar, + const BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int file_version +){ + boost::serialization::stl::save_hash_collection< + Archive, + BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< + Key, HashFcn, EqualKey, Allocator + > + >(ar, t); +} + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void load( + Archive & ar, + BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int file_version +){ + boost::serialization::stl::load_hash_collection< + Archive, + BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< + Key, HashFcn, EqualKey, Allocator + >, + boost::serialization::stl::archive_input_hash_multiset< + Archive, + BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< + Key, HashFcn, EqualKey, Allocator + > + > + >(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void serialize( + Archive & ar, + BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< + Key, HashFcn, EqualKey, Allocator + > & t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +#include + +BOOST_SERIALIZATION_COLLECTION_TRAITS(BOOST_STD_EXTENSION_NAMESPACE::hash_set) +BOOST_SERIALIZATION_COLLECTION_TRAITS(BOOST_STD_EXTENSION_NAMESPACE::hash_multiset) + +#endif // BOOST_HAS_HASH +#endif // BOOST_SERIALIZATION_HASH_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/is_bitwise_serializable.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/is_bitwise_serializable.hpp new file mode 100644 index 00000000000..7e24a2cb6d8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/is_bitwise_serializable.hpp @@ -0,0 +1,46 @@ +// (C) Copyright 2007 Matthias Troyer + +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// Authors: Matthias Troyer + +/** @file is_bitwise_serializable.hpp + * + * This header provides a traits class for determining whether a class + * can be serialized (in a non-portable way) just by copying the bits. + */ + + +#ifndef BOOST_SERIALIZATION_IS_BITWISE_SERIALIZABLE_HPP +#define BOOST_SERIALIZATION_IS_BITWISE_SERIALIZABLE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +#include +#include + +namespace boost { +namespace serialization { + template + struct is_bitwise_serializable + : public is_arithmetic< T > + {}; +} // namespace serialization +} // namespace boost + + +// define a macro to make explicit designation of this more transparent +#define BOOST_IS_BITWISE_SERIALIZABLE(T) \ +namespace boost { \ +namespace serialization { \ +template<> \ +struct is_bitwise_serializable< T > : mpl::true_ {}; \ +}} \ +/**/ + +#endif //BOOST_SERIALIZATION_IS_BITWISE_SERIALIZABLE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/item_version_type.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/item_version_type.hpp new file mode 100644 index 00000000000..f3e5adac6f8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/item_version_type.hpp @@ -0,0 +1,68 @@ +#ifndef BOOST_SERIALIZATION_ITEM_VERSION_TYPE_HPP +#define BOOST_SERIALIZATION_ITEM_VERSION_TYPE_HPP + +// (C) Copyright 2010 Robert Ramey +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +#include // uint_least8_t +#include +#include +#include + +// fixes broken example build on x86_64-linux-gnu-gcc-4.6.0 +#include + +namespace boost { +namespace serialization { + +#if defined(_MSC_VER) +#pragma warning( push ) +#pragma warning( disable : 4244 4267 ) +#endif + +class item_version_type { +private: + typedef unsigned int base_type; + base_type t; +public: + // should be private - but MPI fails if it's not!!! + item_version_type(): t(0) {}; + explicit item_version_type(const unsigned int t_) : t(t_){ + BOOST_ASSERT(t_ <= boost::integer_traits::const_max); + } + item_version_type(const item_version_type & t_) : + t(t_.t) + {} + item_version_type & operator=(item_version_type rhs){ + t = rhs.t; + return *this; + } + // used for text output + operator base_type () const { + return t; + } + // used for text input + operator base_type & () { + return t; + } + bool operator==(const item_version_type & rhs) const { + return t == rhs.t; + } + bool operator<(const item_version_type & rhs) const { + return t < rhs.t; + } +}; + +#if defined(_MSC_VER) +#pragma warning( pop ) +#endif + +} } // end namespace boost::serialization + +BOOST_IS_BITWISE_SERIALIZABLE(item_version_type) + +BOOST_CLASS_IMPLEMENTATION(item_version_type, primitive_type) + +#endif //BOOST_SERIALIZATION_ITEM_VERSION_TYPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/level.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/level.hpp new file mode 100644 index 00000000000..f6a84d10422 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/level.hpp @@ -0,0 +1,116 @@ +#ifndef BOOST_SERIALIZATION_LEVEL_HPP +#define BOOST_SERIALIZATION_LEVEL_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// level.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include + +namespace boost { +namespace serialization { + +struct basic_traits; + +// default serialization implementation level +template +struct implementation_level_impl { + template + struct traits_class_level { + typedef typename U::level type; + }; + + typedef mpl::integral_c_tag tag; + // note: at least one compiler complained w/o the full qualification + // on basic traits below + typedef + typename mpl::eval_if< + is_base_and_derived, + traits_class_level< T >, + //else + typename mpl::eval_if< + is_fundamental< T >, + mpl::int_, + //else + typename mpl::eval_if< + is_class< T >, + mpl::int_, + //else + typename mpl::eval_if< + is_array< T >, + mpl::int_, + //else + typename mpl::eval_if< + is_enum< T >, + mpl::int_, + //else + mpl::int_ + > + > + > + > + >::type type; + // vc 7.1 doesn't like enums here + BOOST_STATIC_CONSTANT(int, value = type::value); +}; + +template +struct implementation_level : + public implementation_level_impl +{ +}; + +template +inline bool operator>=(implementation_level< T > t, enum level_type l) +{ + return t.value >= (int)l; +} + +} // namespace serialization +} // namespace boost + +// specify the level of serialization implementation for the class +// require that class info saved when versioning is used +#define BOOST_CLASS_IMPLEMENTATION(T, E) \ + namespace boost { \ + namespace serialization { \ + template <> \ + struct implementation_level_impl< const T > \ + { \ + typedef mpl::integral_c_tag tag; \ + typedef mpl::int_< E > type; \ + BOOST_STATIC_CONSTANT( \ + int, \ + value = implementation_level_impl::type::value \ + ); \ + }; \ + } \ + } + /**/ + +#endif // BOOST_SERIALIZATION_LEVEL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/level_enum.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/level_enum.hpp new file mode 100644 index 00000000000..baf64e04f31 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/level_enum.hpp @@ -0,0 +1,55 @@ +#ifndef BOOST_SERIALIZATION_LEVEL_ENUM_HPP +#define BOOST_SERIALIZATION_LEVEL_ENUM_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// level_enum.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +namespace boost { +namespace serialization { + +// for each class used in the program, specify which level +// of serialization should be implemented + +// names for each level +enum level_type +{ + // Don't serialize this type. An attempt to do so should + // invoke a compile time assertion. + not_serializable = 0, + // write/read this type directly to the archive. In this case + // serialization code won't be called. This is the default + // case for fundamental types. It presumes a member function or + // template in the archive class that can handle this type. + // there is no runtime overhead associated reading/writing + // instances of this level + primitive_type = 1, + // Serialize the objects of this type using the objects "serialize" + // function or template. This permits values to be written/read + // to/from archives but includes no class or version information. + object_serializable = 2, + /////////////////////////////////////////////////////////////////// + // once an object is serialized at one of the above levels, the + // corresponding archives cannot be read if the implementation level + // for the archive object is changed. + /////////////////////////////////////////////////////////////////// + // Add class information to the archive. Class information includes + // implementation level, class version and class name if available + object_class_info = 3 +}; + +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_LEVEL_ENUM_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/list.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/list.hpp new file mode 100644 index 00000000000..5fdc114d7ed --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/list.hpp @@ -0,0 +1,85 @@ +#ifndef BOOST_SERIALIZATION_LIST_HPP +#define BOOST_SERIALIZATION_LIST_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// list.hpp: serialization for stl list templates + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include + +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +template +inline void save( + Archive & ar, + const std::list &t, + const unsigned int /* file_version */ +){ + boost::serialization::stl::save_collection< + Archive, + std::list + >(ar, t); +} + +template +inline void load( + Archive & ar, + std::list &t, + const unsigned int /* file_version */ +){ + const boost::archive::library_version_type library_version( + ar.get_library_version() + ); + // retrieve number of elements + item_version_type item_version(0); + collection_size_type count; + ar >> BOOST_SERIALIZATION_NVP(count); + if(boost::archive::library_version_type(3) < library_version){ + ar >> BOOST_SERIALIZATION_NVP(item_version); + } + stl::collection_load_impl(ar, t, count, item_version); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template +inline void serialize( + Archive & ar, + std::list & t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // serialization +} // namespace boost + +#include + +BOOST_SERIALIZATION_COLLECTION_TRAITS(std::list) + +#endif // BOOST_SERIALIZATION_LIST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/map.hpp new file mode 100644 index 00000000000..9209864c8cf --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/map.hpp @@ -0,0 +1,139 @@ +#ifndef BOOST_SERIALIZATION_MAP_HPP +#define BOOST_SERIALIZATION_MAP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// serialization/map.hpp: +// serialization for stl map templates + +// (C) Copyright 2002-2014 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include + +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// implementation of serialization for map and mult-map STL containers + +template +inline void load_map_collection(Archive & ar, Container &s) +{ + s.clear(); + const boost::archive::library_version_type library_version( + ar.get_library_version() + ); + // retrieve number of elements + item_version_type item_version(0); + collection_size_type count; + ar >> BOOST_SERIALIZATION_NVP(count); + if(boost::archive::library_version_type(3) < library_version){ + ar >> BOOST_SERIALIZATION_NVP(item_version); + } + typename Container::iterator hint; + hint = s.begin(); + while(count-- > 0){ + typedef typename Container::value_type type; + detail::stack_construct t(ar, item_version); + ar >> boost::serialization::make_nvp("item", t.reference()); + typename Container::iterator result = + s.insert(hint, boost::move(t.reference())); + ar.reset_object_address(& (result->second), & t.reference().second); + hint = result; + ++hint; + } +} + +// map +template +inline void save( + Archive & ar, + const std::map &t, + const unsigned int /* file_version */ +){ + boost::serialization::stl::save_collection< + Archive, + std::map + >(ar, t); +} + +template +inline void load( + Archive & ar, + std::map &t, + const unsigned int /* file_version */ +){ + load_map_collection(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template +inline void serialize( + Archive & ar, + std::map &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +// multimap +template +inline void save( + Archive & ar, + const std::multimap &t, + const unsigned int /* file_version */ +){ + boost::serialization::stl::save_collection< + Archive, + std::multimap + >(ar, t); +} + +template +inline void load( + Archive & ar, + std::multimap &t, + const unsigned int /* file_version */ +){ + load_map_collection(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template +inline void serialize( + Archive & ar, + std::multimap &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/nvp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/nvp.hpp new file mode 100644 index 00000000000..4e2297b3cc9 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/nvp.hpp @@ -0,0 +1,123 @@ +#ifndef BOOST_SERIALIZATION_NVP_HPP +#define BOOST_SERIALIZATION_NVP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// nvp.hpp: interface for serialization system. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +template +struct nvp : + public std::pair, + public wrapper_traits > +{ +//private: + nvp(const nvp & rhs) : + std::pair(rhs.first, rhs.second) + {} +public: + explicit nvp(const char * name_, T & t) : + // note: added _ to suppress useless gcc warning + std::pair(name_, & t) + {} + + const char * name() const { + return this->first; + } + T & value() const { + return *(this->second); + } + + const T & const_value() const { + return *(this->second); + } + + template + void save( + Archive & ar, + const unsigned int /* file_version */ + ) const { + ar.operator<<(const_value()); + } + template + void load( + Archive & ar, + const unsigned int /* file_version */ + ){ + ar.operator>>(value()); + } + BOOST_SERIALIZATION_SPLIT_MEMBER() +}; + +template +inline +const nvp< T > make_nvp(const char * name, T & t){ + return nvp< T >(name, t); +} + +// to maintain efficiency and portability, we want to assign +// specific serialization traits to all instances of this wrappers. +// we can't strait forward method below as it depends upon +// Partial Template Specialization and doing so would mean that wrappers +// wouldn't be treated the same on different platforms. This would +// break archive portability. Leave this here as reminder not to use it !!! + +template +struct implementation_level > +{ + typedef mpl::integral_c_tag tag; + typedef mpl::int_ type; + BOOST_STATIC_CONSTANT(int, value = implementation_level::type::value); +}; + +// nvp objects are generally created on the stack and are never tracked +template +struct tracking_level > +{ + typedef mpl::integral_c_tag tag; + typedef mpl::int_ type; + BOOST_STATIC_CONSTANT(int, value = tracking_level::type::value); +}; + +} // seralization +} // boost + +#include + +#define BOOST_SERIALIZATION_NVP(name) \ + boost::serialization::make_nvp(BOOST_PP_STRINGIZE(name), name) +/**/ + +#define BOOST_SERIALIZATION_BASE_OBJECT_NVP(name) \ + boost::serialization::make_nvp( \ + BOOST_PP_STRINGIZE(name), \ + boost::serialization::base_object(*this) \ + ) +/**/ + +#endif // BOOST_SERIALIZATION_NVP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/optional.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/optional.hpp new file mode 100644 index 00000000000..d6ff830a8c3 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/optional.hpp @@ -0,0 +1,107 @@ +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 + +// (C) Copyright 2002-4 Pavel Vozenilek . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// Provides non-intrusive serialization for boost::optional. + +#ifndef BOOST_SERIALIZATION_OPTIONAL_HPP_ +#define BOOST_SERIALIZATION_OPTIONAL_HPP_ + +#if defined(_MSC_VER) +# pragma once +#endif + +#include + +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +// function specializations must be defined in the appropriate +// namespace - boost::serialization +namespace boost { +namespace serialization { + +template +void save( + Archive & ar, + const boost::optional< T > & t, + const unsigned int /*version*/ +){ + // It is an inherent limitation to the serialization of optional.hpp + // that the underlying type must be either a pointer or must have a + // default constructor. It's possible that this could change sometime + // in the future, but for now, one will have to work around it. This can + // be done by serialization the optional as optional + #if ! defined(BOOST_NO_CXX11_HDR_TYPE_TRAITS) + BOOST_STATIC_ASSERT( + boost::serialization::detail::is_default_constructible::value + || boost::is_pointer::value + ); + #endif + const bool tflag = t.is_initialized(); + ar << boost::serialization::make_nvp("initialized", tflag); + if (tflag){ + ar << boost::serialization::make_nvp("value", *t); + } +} + +template +void load( + Archive & ar, + boost::optional< T > & t, + const unsigned int version +){ + bool tflag; + ar >> boost::serialization::make_nvp("initialized", tflag); + if(! tflag){ + t.reset(); + return; + } + + if(0 == version){ + boost::serialization::item_version_type item_version(0); + boost::archive::library_version_type library_version( + ar.get_library_version() + ); + if(boost::archive::library_version_type(3) < library_version){ + ar >> BOOST_SERIALIZATION_NVP(item_version); + } + } + if(! t.is_initialized()) + t = T(); + ar >> boost::serialization::make_nvp("value", *t); +} + +template +void serialize( + Archive & ar, + boost::optional< T > & t, + const unsigned int version +){ + boost::serialization::split_free(ar, t, version); +} + +template +struct version > { + BOOST_STATIC_CONSTANT(int, value = 1); +}; + +} // serialization +} // boost + +#endif // BOOST_SERIALIZATION_OPTIONAL_HPP_ diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/priority_queue.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/priority_queue.hpp new file mode 100644 index 00000000000..5b08ffd1e82 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/priority_queue.hpp @@ -0,0 +1,76 @@ +#ifndef BOOST_SERIALIZATION_PRIORITY_QUEUE_HPP +#define BOOST_SERIALIZATION_PRIORITY_QUEUE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) && (_MSC_VER >= 1020) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// priority_queue.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include +#include + +// function specializations must be defined in the appropriate +// namespace - boost::serialization +#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) +#define STD _STLP_STD +#else +#define STD std +#endif + +namespace boost { +namespace serialization { +namespace detail{ + +template +struct priority_queue_save : public STD::priority_queue { + template + void operator()(Archive & ar, const unsigned int file_version) const { + save(ar, STD::priority_queue::c, file_version); + } +}; +template +struct priority_queue_load : public STD::priority_queue { + template + void operator()(Archive & ar, const unsigned int file_version) { + load(ar, STD::priority_queue::c, file_version); + } +}; + +} // detail + +template +inline void serialize( + Archive & ar, + std::priority_queue< T, Container, Compare> & t, + const unsigned int file_version +){ + typedef typename mpl::eval_if< + typename Archive::is_saving, + mpl::identity >, + mpl::identity > + >::type typex; + static_cast(t)(ar, file_version); +} + +} // namespace serialization +} // namespace boost + +#include + +BOOST_SERIALIZATION_COLLECTION_TRAITS(STD::priority_queue) + +#undef STD + +#endif // BOOST_SERIALIZATION_PRIORITY_QUEUE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/queue.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/queue.hpp new file mode 100644 index 00000000000..b22745215d9 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/queue.hpp @@ -0,0 +1,76 @@ +#ifndef BOOST_SERIALIZATION_QUEUE_HPP +#define BOOST_SERIALIZATION_QUEUE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) && (_MSC_VER >= 1020) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// queue.hpp + +// (C) Copyright 2014 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include +#include + +// function specializations must be defined in the appropriate +// namespace - boost::serialization +#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) +#define STD _STLP_STD +#else +#define STD std +#endif + +namespace boost { +namespace serialization { +namespace detail { + +template +struct queue_save : public STD::queue { + template + void operator()(Archive & ar, const unsigned int file_version) const { + save(ar, STD::queue::c, file_version); + } +}; +template +struct queue_load : public STD::queue { + template + void operator()(Archive & ar, const unsigned int file_version) { + load(ar, STD::queue::c, file_version); + } +}; + +} // detail + +template +inline void serialize( + Archive & ar, + std::queue< T, C> & t, + const unsigned int file_version +){ + typedef typename mpl::eval_if< + typename Archive::is_saving, + mpl::identity >, + mpl::identity > + >::type typex; + static_cast(t)(ar, file_version); +} + +} // namespace serialization +} // namespace boost + +#include + +BOOST_SERIALIZATION_COLLECTION_TRAITS(STD::queue) + +#undef STD + +#endif // BOOST_SERIALIZATION_QUEUE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/scoped_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/scoped_ptr.hpp new file mode 100644 index 00000000000..0d11f8436e0 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/scoped_ptr.hpp @@ -0,0 +1,58 @@ +#ifndef BOOST_SERIALIZATION_SCOPED_PTR_HPP_VP_2003_10_30 +#define BOOST_SERIALIZATION_SCOPED_PTR_HPP_VP_2003_10_30 + +#if defined(_MSC_VER) +# pragma once +#endif + +// Copyright (c) 2003 Vladimir Prus. +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// Provides non-intrusive serialization for boost::scoped_ptr +// Does not allow to serialize scoped_ptr's to builtin types. + +#include + +#include +#include +#include + +namespace boost { +namespace serialization { + + template + void save( + Archive & ar, + const boost::scoped_ptr< T > & t, + const unsigned int /* version */ + ){ + T* r = t.get(); + ar << boost::serialization::make_nvp("scoped_ptr", r); + } + + template + void load( + Archive & ar, + boost::scoped_ptr< T > & t, + const unsigned int /* version */ + ){ + T* r; + ar >> boost::serialization::make_nvp("scoped_ptr", r); + t.reset(r); + } + + template + void serialize( + Archive& ar, + boost::scoped_ptr< T >& t, + const unsigned int version + ){ + boost::serialization::split_free(ar, t, version); + } + +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_SCOPED_PTR_HPP_VP_2003_10_30 diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/serialization.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/serialization.hpp new file mode 100644 index 00000000000..a4d04723c75 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/serialization.hpp @@ -0,0 +1,154 @@ +#ifndef BOOST_SERIALIZATION_SERIALIZATION_HPP +#define BOOST_SERIALIZATION_SERIALIZATION_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +#if defined(_MSC_VER) +# pragma warning (disable : 4675) // suppress ADL warning +#endif + +#include +#include + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// serialization.hpp: interface for serialization system. + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +////////////////////////////////////////////////////////////////////// +// public interface to serialization. + +///////////////////////////////////////////////////////////////////////////// +// layer 0 - intrusive verison +// declared and implemented for each user defined class to be serialized +// +// template +// serialize(Archive &ar, const unsigned int file_version){ +// ar & base_object(*this) & member1 & member2 ... ; +// } + +///////////////////////////////////////////////////////////////////////////// +// layer 1 - layer that routes member access through the access class. +// this is what permits us to grant access to private class member functions +// by specifying friend class boost::serialization::access + +#include + +///////////////////////////////////////////////////////////////////////////// +// layer 2 - default implementation of non-intrusive serialization. +// +// note the usage of function overloading to compensate that C++ does not +// currently support Partial Template Specialization for function templates +// We have declared the version number as "const unsigned long". +// Overriding templates for specific data types should declare the version +// number as "const unsigned int". Template matching will first be applied +// to functions with the same version types - that is the overloads. +// If there is no declared function prototype that matches, the second argument +// will be converted to "const unsigned long" and a match will be made with +// one of the default template functions below. + +namespace boost { +namespace serialization { + +BOOST_STRONG_TYPEDEF(unsigned int, version_type) + +// default implementation - call the member function "serialize" +template +inline void serialize( + Archive & ar, T & t, const unsigned int file_version +){ + access::serialize(ar, t, static_cast(file_version)); +} + +// save data required for construction +template +inline void save_construct_data( + Archive & /*ar*/, + const T * /*t*/, + const unsigned int /*file_version */ +){ + // default is to save no data because default constructor + // requires no arguments. +} + +// load data required for construction and invoke constructor in place +template +inline void load_construct_data( + Archive & /*ar*/, + T * t, + const unsigned int /*file_version*/ +){ + // default just uses the default constructor. going + // through access permits usage of otherwise private default + // constructor + access::construct(t); +} + +///////////////////////////////////////////////////////////////////////////// +// layer 3 - move call into serialization namespace so that ADL will function +// in the manner we desire. +// +// on compilers which don't implement ADL. only the current namespace +// i.e. boost::serialization will be searched. +// +// on compilers which DO implement ADL +// serialize overrides can be in any of the following +// +// 1) same namepace as Archive +// 2) same namespace as T +// 3) boost::serialization +// +// Due to Martin Ecker + +template +inline void serialize_adl( + Archive & ar, + T & t, + const unsigned int file_version +){ + // note usage of function overloading to delay final resolution + // until the point of instantiation. This works around the two-phase + // lookup "feature" which inhibits redefintion of a default function + // template implementation. Due to Robert Ramey + // + // Note that this trick generates problems for compiles which don't support + // PFTO, suppress it here. As far as we know, there are no compilers + // which fail to support PFTO while supporting two-phase lookup. + const version_type v(file_version); + serialize(ar, t, v); +} + +template +inline void save_construct_data_adl( + Archive & ar, + const T * t, + const unsigned int file_version +){ + // see above + const version_type v(file_version); + save_construct_data(ar, t, v); +} + +template +inline void load_construct_data_adl( + Archive & ar, + T * t, + const unsigned int file_version +){ + // see above comment + const version_type v(file_version); + load_construct_data(ar, t, v); +} + +} // namespace serialization +} // namespace boost + +#endif //BOOST_SERIALIZATION_SERIALIZATION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/set.hpp new file mode 100644 index 00000000000..643906c5aac --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/set.hpp @@ -0,0 +1,137 @@ +#ifndef BOOST_SERIALIZATION_SET_HPP +#define BOOST_SERIALIZATION_SET_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// set.hpp: serialization for stl set templates + +// (C) Copyright 2002-2014 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include + +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +namespace boost { +namespace serialization { + +template +inline void load_set_collection(Archive & ar, Container &s) +{ + s.clear(); + const boost::archive::library_version_type library_version( + ar.get_library_version() + ); + // retrieve number of elements + item_version_type item_version(0); + collection_size_type count; + ar >> BOOST_SERIALIZATION_NVP(count); + if(boost::archive::library_version_type(3) < library_version){ + ar >> BOOST_SERIALIZATION_NVP(item_version); + } + typename Container::iterator hint; + hint = s.begin(); + while(count-- > 0){ + typedef typename Container::value_type type; + detail::stack_construct t(ar, item_version); + // borland fails silently w/o full namespace + ar >> boost::serialization::make_nvp("item", t.reference()); + typename Container::iterator result = + s.insert(hint, boost::move(t.reference())); + ar.reset_object_address(& (* result), & t.reference()); + hint = result; + } +} + +template +inline void save( + Archive & ar, + const std::set &t, + const unsigned int /* file_version */ +){ + boost::serialization::stl::save_collection< + Archive, std::set + >(ar, t); +} + +template +inline void load( + Archive & ar, + std::set &t, + const unsigned int /* file_version */ +){ + load_set_collection(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template +inline void serialize( + Archive & ar, + std::set & t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +// multiset +template +inline void save( + Archive & ar, + const std::multiset &t, + const unsigned int /* file_version */ +){ + boost::serialization::stl::save_collection< + Archive, + std::multiset + >(ar, t); +} + +template +inline void load( + Archive & ar, + std::multiset &t, + const unsigned int /* file_version */ +){ + load_set_collection(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template +inline void serialize( + Archive & ar, + std::multiset & t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +#include + +BOOST_SERIALIZATION_COLLECTION_TRAITS(std::set) +BOOST_SERIALIZATION_COLLECTION_TRAITS(std::multiset) + +#endif // BOOST_SERIALIZATION_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr.hpp new file mode 100644 index 00000000000..0d4c5ae6056 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr.hpp @@ -0,0 +1,281 @@ +#ifndef BOOST_SERIALIZATION_SHARED_PTR_HPP +#define BOOST_SERIALIZATION_SHARED_PTR_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// shared_ptr.hpp: serialization for boost shared pointer + +// (C) Copyright 2004 Robert Ramey and Martin Ecker +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // NULL +#include + +#include +#include +#include + +#include +#include + +#include +#include +#include +#include +#include + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// boost:: shared_ptr serialization traits +// version 1 to distinguish from boost 1.32 version. Note: we can only do this +// for a template when the compiler supports partial template specialization + +#ifndef BOOST_NO_TEMPLATE_PARTIAL_SPECIALIZATION + namespace boost { + namespace serialization{ + template + struct version< ::boost::shared_ptr< T > > { + typedef mpl::integral_c_tag tag; + #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3206)) + typedef typename mpl::int_<1> type; + #else + typedef mpl::int_<1> type; + #endif + BOOST_STATIC_CONSTANT(int, value = type::value); + }; + // don't track shared pointers + template + struct tracking_level< ::boost::shared_ptr< T > > { + typedef mpl::integral_c_tag tag; + #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3206)) + typedef typename mpl::int_< ::boost::serialization::track_never> type; + #else + typedef mpl::int_< ::boost::serialization::track_never> type; + #endif + BOOST_STATIC_CONSTANT(int, value = type::value); + }; + }} + #define BOOST_SERIALIZATION_SHARED_PTR(T) +#else + // define macro to let users of these compilers do this + #define BOOST_SERIALIZATION_SHARED_PTR(T) \ + BOOST_CLASS_VERSION( \ + ::boost::shared_ptr< T >, \ + 1 \ + ) \ + BOOST_CLASS_TRACKING( \ + ::boost::shared_ptr< T >, \ + ::boost::serialization::track_never \ + ) \ + /**/ +#endif + +namespace boost { +namespace serialization{ + +struct null_deleter { + void operator()(void const *) const {} +}; + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// serialization for boost::shared_ptr + +// Using a constant means that all shared pointers are held in the same set. +// Thus we detect handle multiple pointers to the same value instances +// in the archive. +void * const shared_ptr_helper_id = 0; + +template +inline void save( + Archive & ar, + const boost::shared_ptr< T > &t, + const unsigned int /* file_version */ +){ + // The most common cause of trapping here would be serializing + // something like shared_ptr. This occurs because int + // is never tracked by default. Wrap int in a trackable type + BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); + const T * t_ptr = t.get(); + ar << boost::serialization::make_nvp("px", t_ptr); +} + +#ifdef BOOST_SERIALIZATION_SHARED_PTR_132_HPP +template +inline void load( + Archive & ar, + boost::shared_ptr< T > &t, + const unsigned int file_version +){ + // something like shared_ptr. This occurs because int + // is never tracked by default. Wrap int in a trackable type + BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); + T* r; + if(file_version < 1){ + ar.register_type(static_cast< + boost_132::detail::sp_counted_base_impl * + >(NULL)); + boost_132::shared_ptr< T > sp; + ar >> boost::serialization::make_nvp("px", sp.px); + ar >> boost::serialization::make_nvp("pn", sp.pn); + // got to keep the sps around so the sp.pns don't disappear + boost::serialization::shared_ptr_helper & h = + ar.template get_helper< shared_ptr_helper >( + shared_ptr_helper_id + ); + h.append(sp); + r = sp.get(); + } + else{ + ar >> boost::serialization::make_nvp("px", r); + } + shared_ptr_helper & h = + ar.template get_helper >( + shared_ptr_helper_id + ); + h.reset(t,r); +} +#else + +template +inline void load( + Archive & ar, + boost::shared_ptr< T > &t, + const unsigned int /*file_version*/ +){ + // The most common cause of trapping here would be serializing + // something like shared_ptr. This occurs because int + // is never tracked by default. Wrap int in a trackable type + BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); + T* r; + ar >> boost::serialization::make_nvp("px", r); + + boost::serialization::shared_ptr_helper & h = + ar.template get_helper >( + shared_ptr_helper_id + ); + h.reset(t,r); +} +#endif + +template +inline void serialize( + Archive & ar, + boost::shared_ptr< T > &t, + const unsigned int file_version +){ + // correct shared_ptr serialization depends upon object tracking + // being used. + BOOST_STATIC_ASSERT( + boost::serialization::tracking_level< T >::value + != boost::serialization::track_never + ); + boost::serialization::split_free(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// std::shared_ptr serialization traits +// version 1 to distinguish from boost 1.32 version. Note: we can only do this +// for a template when the compiler supports partial template specialization + +#ifndef BOOST_NO_CXX11_SMART_PTR +#include + +// note: we presume that any compiler/library which supports C++11 +// std::pointers also supports template partial specialization +// trap here if such presumption were to turn out to wrong!!! +#ifdef BOOST_NO_TEMPLATE_PARTIAL_SPECIALIZATION + BOOST_STATIC_ASSERT(false); +#endif + +namespace boost { +namespace serialization{ + template + struct version< ::std::shared_ptr< T > > { + typedef mpl::integral_c_tag tag; + typedef mpl::int_<1> type; + BOOST_STATIC_CONSTANT(int, value = type::value); + }; + // don't track shared pointers + template + struct tracking_level< ::std::shared_ptr< T > > { + typedef mpl::integral_c_tag tag; + typedef mpl::int_< ::boost::serialization::track_never> type; + BOOST_STATIC_CONSTANT(int, value = type::value); + }; +}} +// the following just keeps older programs from breaking +#define BOOST_SERIALIZATION_SHARED_PTR(T) + +namespace boost { +namespace serialization{ + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// serialization for std::shared_ptr + +template +inline void save( + Archive & ar, + const std::shared_ptr< T > &t, + const unsigned int /* file_version */ +){ + // The most common cause of trapping here would be serializing + // something like shared_ptr. This occurs because int + // is never tracked by default. Wrap int in a trackable type + BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); + const T * t_ptr = t.get(); + ar << boost::serialization::make_nvp("px", t_ptr); +} + +template +inline void load( + Archive & ar, + std::shared_ptr< T > &t, + const unsigned int /*file_version*/ +){ + // The most common cause of trapping here would be serializing + // something like shared_ptr. This occurs because int + // is never tracked by default. Wrap int in a trackable type + BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); + T* r; + ar >> boost::serialization::make_nvp("px", r); + //void (* const id)(Archive &, std::shared_ptr< T > &, const unsigned int) = & load; + boost::serialization::shared_ptr_helper & h = + ar.template get_helper< + shared_ptr_helper + >( + shared_ptr_helper_id + ); + h.reset(t,r); +} + +template +inline void serialize( + Archive & ar, + std::shared_ptr< T > &t, + const unsigned int file_version +){ + // correct shared_ptr serialization depends upon object tracking + // being used. + BOOST_STATIC_ASSERT( + boost::serialization::tracking_level< T >::value + != boost::serialization::track_never + ); + boost::serialization::split_free(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +#endif // BOOST_NO_CXX11_SMART_PTR + +#endif // BOOST_SERIALIZATION_SHARED_PTR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_132.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_132.hpp new file mode 100644 index 00000000000..3dfaba4d69a --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_132.hpp @@ -0,0 +1,222 @@ +#ifndef BOOST_SERIALIZATION_SHARED_PTR_132_HPP +#define BOOST_SERIALIZATION_SHARED_PTR_132_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// shared_ptr.hpp: serialization for boost shared pointer + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// note: totally unadvised hack to gain access to private variables +// in shared_ptr and shared_count. Unfortunately its the only way to +// do this without changing shared_ptr and shared_count +// the best we can do is to detect a conflict here +#include + +#include +#include // NULL + +#include +#include +#include +#include +#include + +// mark base class as an (uncreatable) base class +#include + +///////////////////////////////////////////////////////////// +// Maintain a couple of lists of loaded shared pointers of the old previous +// version (1.32) + +namespace boost_132 { +namespace serialization { +namespace detail { + +struct null_deleter { + void operator()(void const *) const {} +}; + +} // namespace detail +} // namespace serialization +} // namespace boost_132 + +///////////////////////////////////////////////////////////// +// sp_counted_base_impl serialization + +namespace boost { +namespace serialization { + +template +inline void serialize( + Archive & /* ar */, + boost_132::detail::sp_counted_base_impl & /* t */, + const unsigned int /*file_version*/ +){ + // register the relationship between each derived class + // its polymorphic base + boost::serialization::void_cast_register< + boost_132::detail::sp_counted_base_impl, + boost_132::detail::sp_counted_base + >( + static_cast *>(NULL), + static_cast(NULL) + ); +} + +template +inline void save_construct_data( + Archive & ar, + const + boost_132::detail::sp_counted_base_impl *t, + const unsigned int /* file_version */ +){ + // variables used for construction + ar << boost::serialization::make_nvp("ptr", t->ptr); +} + +template +inline void load_construct_data( + Archive & ar, + boost_132::detail::sp_counted_base_impl * t, + const unsigned int /* file_version */ +){ + P ptr_; + ar >> boost::serialization::make_nvp("ptr", ptr_); + // ::new(t)boost_132::detail::sp_counted_base_impl(ptr_, D()); + // placement + // note: the original ::new... above is replaced by the one here. This one + // creates all new objects with a null_deleter so that after the archive + // is finished loading and the shared_ptrs are destroyed - the underlying + // raw pointers are NOT deleted. This is necessary as they are used by the + // new system as well. + ::new(t)boost_132::detail::sp_counted_base_impl< + P, + boost_132::serialization::detail::null_deleter + >( + ptr_, boost_132::serialization::detail::null_deleter() + ); // placement new + // compensate for that fact that a new shared count always is + // initialized with one. the add_ref_copy below will increment it + // every time its serialized so without this adjustment + // the use and weak counts will be off by one. + t->use_count_ = 0; +} + +} // serialization +} // namespace boost + +///////////////////////////////////////////////////////////// +// shared_count serialization + +namespace boost { +namespace serialization { + +template +inline void save( + Archive & ar, + const boost_132::detail::shared_count &t, + const unsigned int /* file_version */ +){ + ar << boost::serialization::make_nvp("pi", t.pi_); +} + +template +inline void load( + Archive & ar, + boost_132::detail::shared_count &t, + const unsigned int /* file_version */ +){ + ar >> boost::serialization::make_nvp("pi", t.pi_); + if(NULL != t.pi_) + t.pi_->add_ref_copy(); +} + +} // serialization +} // namespace boost + +BOOST_SERIALIZATION_SPLIT_FREE(boost_132::detail::shared_count) + +///////////////////////////////////////////////////////////// +// implement serialization for shared_ptr< T > + +namespace boost { +namespace serialization { + +template +inline void save( + Archive & ar, + const boost_132::shared_ptr< T > &t, + const unsigned int /* file_version */ +){ + // only the raw pointer has to be saved + // the ref count is maintained automatically as shared pointers are loaded + ar.register_type(static_cast< + boost_132::detail::sp_counted_base_impl > * + >(NULL)); + ar << boost::serialization::make_nvp("px", t.px); + ar << boost::serialization::make_nvp("pn", t.pn); +} + +template +inline void load( + Archive & ar, + boost_132::shared_ptr< T > &t, + const unsigned int /* file_version */ +){ + // only the raw pointer has to be saved + // the ref count is maintained automatically as shared pointers are loaded + ar.register_type(static_cast< + boost_132::detail::sp_counted_base_impl > * + >(NULL)); + ar >> boost::serialization::make_nvp("px", t.px); + ar >> boost::serialization::make_nvp("pn", t.pn); +} + +template +inline void serialize( + Archive & ar, + boost_132::shared_ptr< T > &t, + const unsigned int file_version +){ + // correct shared_ptr serialization depends upon object tracking + // being used. + BOOST_STATIC_ASSERT( + boost::serialization::tracking_level< T >::value + != boost::serialization::track_never + ); + boost::serialization::split_free(ar, t, file_version); +} + +} // serialization +} // namespace boost + +// note: change below uses null_deleter +// This macro is used to export GUIDS for shared pointers to allow +// the serialization system to export them properly. David Tonge +#define BOOST_SHARED_POINTER_EXPORT_GUID(T, K) \ + typedef boost_132::detail::sp_counted_base_impl< \ + T *, \ + boost::checked_deleter< T > \ + > __shared_ptr_ ## T; \ + BOOST_CLASS_EXPORT_GUID(__shared_ptr_ ## T, "__shared_ptr_" K) \ + BOOST_CLASS_EXPORT_GUID(T, K) \ + /**/ + +#define BOOST_SHARED_POINTER_EXPORT(T) \ + BOOST_SHARED_POINTER_EXPORT_GUID( \ + T, \ + BOOST_PP_STRINGIZE(T) \ + ) \ + /**/ + +#endif // BOOST_SERIALIZATION_SHARED_PTR_132_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_helper.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_helper.hpp new file mode 100644 index 00000000000..37c34d6b2c4 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_helper.hpp @@ -0,0 +1,209 @@ +#ifndef BOOST_SERIALIZATION_SHARED_PTR_HELPER_HPP +#define BOOST_SERIALIZATION_SHARED_PTR_HELPER_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// shared_ptr_helper.hpp: serialization for boost shared pointern + +// (C) Copyright 2004-2009 Robert Ramey, Martin Ecker and Takatoshi Kondo +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include +#include // NULL + +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace boost_132 { + template class shared_ptr; +} +namespace boost { +namespace serialization { + +#ifndef BOOST_NO_MEMBER_TEMPLATE_FRIENDS +template class SPT > +void load( + Archive & ar, + SPT< class U > &t, + const unsigned int file_version +); +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// a common class for holding various types of shared pointers + +template class SPT> +class shared_ptr_helper { + typedef std::map< + const void *, // address of object + SPT // address shared ptr to single instance + > object_shared_pointer_map; + + // list of shared_pointers create accessable by raw pointer. This + // is used to "match up" shared pointers loaded at different + // points in the archive. Note, we delay construction until + // it is actually used since this is by default included as + // a "mix-in" even if shared_ptr isn't used. + object_shared_pointer_map * m_o_sp; + + struct null_deleter { + void operator()(void const *) const {} + }; + +#if defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) \ +|| defined(BOOST_MSVC) \ +|| defined(__SUNPRO_CC) +public: +#else + template + friend void boost::serialization::load( + Archive & ar, + SPT< U > &t, + const unsigned int file_version + ); +#endif + + #ifdef BOOST_SERIALIZATION_SHARED_PTR_132_HPP + // list of loaded pointers. This is used to be sure that the pointers + // stay around long enough to be "matched" with other pointers loaded + // by the same archive. These are created with a "null_deleter" so that + // when this list is destroyed - the underlaying raw pointers are not + // destroyed. This has to be done because the pointers are also held by + // new system which is disjoint from this set. This is implemented + // by a change in load_construct_data below. It makes this file suitable + // only for loading pointers into a 1.33 or later boost system. + std::list > * m_pointers_132; + void + append(const boost_132::shared_ptr & t){ + if(NULL == m_pointers_132) + m_pointers_132 = new std::list >; + m_pointers_132->push_back(t); + } + #endif + + struct non_polymorphic { + template + static const boost::serialization::extended_type_info * + get_object_type(U & ){ + return & boost::serialization::singleton< + typename + boost::serialization::type_info_implementation< U >::type + >::get_const_instance(); + } + }; + struct polymorphic { + template + static const boost::serialization::extended_type_info * + get_object_type(U & u){ + return boost::serialization::singleton< + typename + boost::serialization::type_info_implementation< U >::type + >::get_const_instance().get_derived_extended_type_info(u); + } + }; + +public: + template + void reset(SPT< T > & s, T * t){ + if(NULL == t){ + s.reset(); + return; + } + const boost::serialization::extended_type_info * this_type + = & boost::serialization::type_info_implementation< T >::type + ::get_const_instance(); + + // get pointer to the most derived object's eti. This is effectively + // the object type identifer + typedef typename mpl::if_< + is_polymorphic< T >, + polymorphic, + non_polymorphic + >::type type; + + const boost::serialization::extended_type_info * true_type + = type::get_object_type(*t); + + // note:if this exception is thrown, be sure that derived pointern + // is either registered or exported. + if(NULL == true_type) + boost::serialization::throw_exception( + boost::archive::archive_exception( + boost::archive::archive_exception::unregistered_class, + this_type->get_debug_info() + ) + ); + // get void pointer to the most derived type + // this uniquely identifies the object referred to + // oid = "object identifier" + const void * oid = void_downcast( + *true_type, + *this_type, + t + ); + if(NULL == oid) + boost::serialization::throw_exception( + boost::archive::archive_exception( + boost::archive::archive_exception::unregistered_cast, + true_type->get_debug_info(), + this_type->get_debug_info() + ) + ); + + // make tracking array if necessary + if(NULL == m_o_sp) + m_o_sp = new object_shared_pointer_map; + + typename object_shared_pointer_map::iterator i = m_o_sp->find(oid); + + // if it's a new object + if(i == m_o_sp->end()){ + s.reset(t); + std::pair result; + result = m_o_sp->insert(std::make_pair(oid, s)); + BOOST_ASSERT(result.second); + } + // if the object has already been seen + else{ + s = SPT(i->second, t); + } + } + + shared_ptr_helper() : + m_o_sp(NULL) + #ifdef BOOST_SERIALIZATION_SHARED_PTR_132_HPP + , m_pointers_132(NULL) + #endif + {} + virtual ~shared_ptr_helper(){ + if(NULL != m_o_sp) + delete m_o_sp; + #ifdef BOOST_SERIALIZATION_SHARED_PTR_132_HPP + if(NULL != m_pointers_132) + delete m_pointers_132; + #endif + } +}; + +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_SHARED_PTR_HELPER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/singleton.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/singleton.hpp new file mode 100644 index 00000000000..b50afedbb92 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/singleton.hpp @@ -0,0 +1,166 @@ +#ifndef BOOST_SERIALIZATION_SINGLETON_HPP +#define BOOST_SERIALIZATION_SINGLETON_HPP + +/////////1/////////2///////// 3/////////4/////////5/////////6/////////7/////////8 +// singleton.hpp +// +// Copyright David Abrahams 2006. Original version +// +// Copyright Robert Ramey 2007. Changes made to permit +// application throughout the serialization library. +// +// Distributed under the Boost +// Software License, Version 1.0. (See accompanying +// file LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +// +// The intention here is to define a template which will convert +// any class into a singleton with the following features: +// +// a) initialized before first use. +// b) thread-safe for const access to the class +// c) non-locking +// +// In order to do this, +// a) Initialize dynamically when used. +// b) Require that all singletons be initialized before main +// is called or any entry point into the shared library is invoked. +// This guarentees no race condition for initialization. +// In debug mode, we assert that no non-const functions are called +// after main is invoked. +// + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +#include +#include +#include +#include + +#include +#include +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4511 4512) +#endif + +namespace boost { +namespace serialization { + +////////////////////////////////////////////////////////////////////// +// Provides a dynamically-initialized (singleton) instance of T in a +// way that avoids LNK1179 on vc6. See http://tinyurl.com/ljdp8 or +// http://lists.boost.org/Archives/boost/2006/05/105286.php for +// details. +// + +// singletons created by this code are guarenteed to be unique +// within the executable or shared library which creates them. +// This is sufficient and in fact ideal for the serialization library. +// The singleton is created when the module is loaded and destroyed +// when the module is unloaded. + +// This base class has two functions. + +// First it provides a module handle for each singleton indicating +// the executable or shared library in which it was created. This +// turns out to be necessary and sufficient to implement the tables +// used by serialization library. + +// Second, it provides a mechanism to detect when a non-const function +// is called after initialization. + +// make a singleton to lock/unlock all singletons for alteration. +// The intent is that all singletons created/used by this code +// are to be initialized before main is called. A test program +// can lock all the singletons when main is entereed. This any +// attempt to retieve a mutable instances while locked will +// generate a assertion if compiled for debug. + +// note usage of BOOST_DLLEXPORT. These functions are in danger of +// being eliminated by the optimizer when building an application in +// release mode. Usage of the macro is meant to signal the compiler/linker +// to avoid dropping these functions which seem to be unreferenced. +// This usage is not related to autolinking. + +class BOOST_SYMBOL_VISIBLE singleton_module : + public boost::noncopyable +{ +private: + BOOST_SERIALIZATION_DECL BOOST_DLLEXPORT static bool & get_lock() BOOST_USED; +public: + BOOST_DLLEXPORT static void lock(){ + get_lock() = true; + } + BOOST_DLLEXPORT static void unlock(){ + get_lock() = false; + } + BOOST_DLLEXPORT static bool is_locked(){ + return get_lock(); + } +}; + +template +class singleton : public singleton_module +{ +private: + static T & m_instance; + // include this to provoke instantiation at pre-execution time + static void use(T const *) {} + static T & get_instance() { + // use a wrapper so that types T with protected constructors + // can be used + class singleton_wrapper : public T {}; + static singleton_wrapper t; + // refer to instance, causing it to be instantiated (and + // initialized at startup on working compilers) + BOOST_ASSERT(! is_destroyed()); + // note that the following is absolutely essential. + // commenting out this statement will cause compilers to fail to + // construct the instance at pre-execution time. This would prevent + // our usage/implementation of "locking" and introduce uncertainty into + // the sequence of object initializaition. + use(& m_instance); + return static_cast(t); + } + static bool & get_is_destroyed(){ + static bool is_destroyed; + return is_destroyed; + } + +public: + BOOST_DLLEXPORT static T & get_mutable_instance(){ + BOOST_ASSERT(! is_locked()); + return get_instance(); + } + BOOST_DLLEXPORT static const T & get_const_instance(){ + return get_instance(); + } + BOOST_DLLEXPORT static bool is_destroyed(){ + return get_is_destroyed(); + } + BOOST_DLLEXPORT singleton(){ + get_is_destroyed() = false; + } + BOOST_DLLEXPORT ~singleton() { + get_is_destroyed() = true; + } +}; + +template +T & singleton< T >::m_instance = singleton< T >::get_instance(); + +} // namespace serialization +} // namespace boost + +#include // pops abi_suffix.hpp pragmas + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +#endif // BOOST_SERIALIZATION_SINGLETON_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/slist.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/slist.hpp new file mode 100644 index 00000000000..d9b971bc4f1 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/slist.hpp @@ -0,0 +1,145 @@ +#ifndef BOOST_SERIALIZATION_SLIST_HPP +#define BOOST_SERIALIZATION_SLIST_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// slist.hpp + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#ifdef BOOST_HAS_SLIST +#include BOOST_SLIST_HEADER + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +template +inline void save( + Archive & ar, + const BOOST_STD_EXTENSION_NAMESPACE::slist &t, + const unsigned int file_version +){ + boost::serialization::stl::save_collection< + Archive, + BOOST_STD_EXTENSION_NAMESPACE::slist + >(ar, t); +} + +namespace stl { + +template< + class Archive, + class T, + class Allocator +> +typename boost::disable_if< + typename detail::is_default_constructible< + typename BOOST_STD_EXTENSION_NAMESPACE::slist::value_type + >, + void +>::type +collection_load_impl( + Archive & ar, + BOOST_STD_EXTENSION_NAMESPACE::slist &t, + collection_size_type count, + item_version_type item_version +){ + t.clear(); + boost::serialization::detail::stack_construct u(ar, item_version); + ar >> boost::serialization::make_nvp("item", u.reference()); + t.push_front(boost::move(u.reference())); + typename BOOST_STD_EXTENSION_NAMESPACE::slist::iterator last; + last = t.begin(); + ar.reset_object_address(&(*t.begin()) , & u.reference()); + while(--count > 0){ + detail::stack_construct u(ar, item_version); + ar >> boost::serialization::make_nvp("item", u.reference()); + last = t.insert_after(last, boost::move(u.reference())); + ar.reset_object_address(&(*last) , & u.reference()); + } +} + +} // stl + +template +inline void load( + Archive & ar, + BOOST_STD_EXTENSION_NAMESPACE::slist &t, + const unsigned int file_version +){ + const boost::archive::library_version_type library_version( + ar.get_library_version() + ); + // retrieve number of elements + item_version_type item_version(0); + collection_size_type count; + ar >> BOOST_SERIALIZATION_NVP(count); + if(boost::archive::library_version_type(3) < library_version){ + ar >> BOOST_SERIALIZATION_NVP(item_version); + } + if(detail::is_default_constructible()){ + t.resize(count); + typename BOOST_STD_EXTENSION_NAMESPACE::slist::iterator hint; + hint = t.begin(); + while(count-- > 0){ + ar >> boost::serialization::make_nvp("item", *hint++); + } + } + else{ + t.clear(); + boost::serialization::detail::stack_construct u(ar, item_version); + ar >> boost::serialization::make_nvp("item", u.reference()); + t.push_front(boost::move(u.reference())); + typename BOOST_STD_EXTENSION_NAMESPACE::slist::iterator last; + last = t.begin(); + ar.reset_object_address(&(*t.begin()) , & u.reference()); + while(--count > 0){ + detail::stack_construct u(ar, item_version); + ar >> boost::serialization::make_nvp("item", u.reference()); + last = t.insert_after(last, boost::move(u.reference())); + ar.reset_object_address(&(*last) , & u.reference()); + } + } +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template +inline void serialize( + Archive & ar, + BOOST_STD_EXTENSION_NAMESPACE::slist &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // serialization +} // namespace boost + +#include + +BOOST_SERIALIZATION_COLLECTION_TRAITS(BOOST_STD_EXTENSION_NAMESPACE::slist) + +#endif // BOOST_HAS_SLIST +#endif // BOOST_SERIALIZATION_SLIST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/smart_cast.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/smart_cast.hpp new file mode 100644 index 00000000000..563f36aa20b --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/smart_cast.hpp @@ -0,0 +1,275 @@ +#ifndef BOOST_SERIALIZATION_SMART_CAST_HPP +#define BOOST_SERIALIZATION_SMART_CAST_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// smart_cast.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org/libs/serialization for updates, documentation, and revision history. + +// casting of pointers and references. + +// In casting between different C++ classes, there are a number of +// rules that have to be kept in mind in deciding whether to use +// static_cast or dynamic_cast. + +// a) dynamic casting can only be applied when one of the types is polymorphic +// Otherwise static_cast must be used. +// b) only dynamic casting can do runtime error checking +// use of static_cast is generally un checked even when compiled for debug +// c) static_cast would be considered faster than dynamic_cast. + +// If casting is applied to a template parameter, there is no apriori way +// to know which of the two casting methods will be permitted or convenient. + +// smart_cast uses C++ type_traits, and program debug mode to select the +// most convenient cast to use. + +#include +#include +#include // NULL + +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include + +namespace boost { +namespace serialization { +namespace smart_cast_impl { + + template + struct reference { + + struct polymorphic { + + struct linear { + template + static T cast(U & u){ + return static_cast< T >(u); + } + }; + + struct cross { + template + static T cast(U & u){ + return dynamic_cast< T >(u); + } + }; + + template + static T cast(U & u){ + // if we're in debug mode + #if ! defined(NDEBUG) \ + || defined(__MWERKS__) + // do a checked dynamic cast + return cross::cast(u); + #else + // borland 5.51 chokes here so we can't use it + // note: if remove_reference isn't function for these types + // cross casting will be selected this will work but will + // not be the most efficient method. This will conflict with + // the original smart_cast motivation. + typedef typename mpl::eval_if< + typename mpl::and_< + mpl::not_::type, + U + > >, + mpl::not_::type + > > + >, + // borland chokes w/o full qualification here + mpl::identity, + mpl::identity + >::type typex; + // typex works around gcc 2.95 issue + return typex::cast(u); + #endif + } + }; + + struct non_polymorphic { + template + static T cast(U & u){ + return static_cast< T >(u); + } + }; + template + static T cast(U & u){ + typedef typename mpl::eval_if< + boost::is_polymorphic, + mpl::identity, + mpl::identity + >::type typex; + return typex::cast(u); + } + }; + + template + struct pointer { + + struct polymorphic { + // unfortunately, this below fails to work for virtual base + // classes. need has_virtual_base to do this. + // Subject for further study + #if 0 + struct linear { + template + static T cast(U * u){ + return static_cast< T >(u); + } + }; + + struct cross { + template + static T cast(U * u){ + T tmp = dynamic_cast< T >(u); + #ifndef NDEBUG + if ( tmp == 0 ) throw_exception(std::bad_cast()); + #endif + return tmp; + } + }; + + template + static T cast(U * u){ + typedef + typename mpl::eval_if< + typename mpl::and_< + mpl::not_::type, + U + > >, + mpl::not_::type + > > + >, + // borland chokes w/o full qualification here + mpl::identity, + mpl::identity + >::type typex; + return typex::cast(u); + } + #else + template + static T cast(U * u){ + T tmp = dynamic_cast< T >(u); + #ifndef NDEBUG + if ( tmp == 0 ) throw_exception(std::bad_cast()); + #endif + return tmp; + } + #endif + }; + + struct non_polymorphic { + template + static T cast(U * u){ + return static_cast< T >(u); + } + }; + + template + static T cast(U * u){ + typedef typename mpl::eval_if< + boost::is_polymorphic, + mpl::identity, + mpl::identity + >::type typex; + return typex::cast(u); + } + + }; + + template + struct void_pointer { + template + static TPtr cast(UPtr uptr){ + return static_cast(uptr); + } + }; + + template + struct error { + // if we get here, its because we are using one argument in the + // cast on a system which doesn't support partial template + // specialization + template + static T cast(U){ + BOOST_STATIC_ASSERT(sizeof(T)==0); + return * static_cast(NULL); + } + }; + +} // smart_cast_impl + +// this implements: +// smart_cast(Source * s) +// smart_cast(s) +// note that it will fail with +// smart_cast(s) +template +T smart_cast(U u) { + typedef + typename mpl::eval_if< + typename mpl::or_< + boost::is_same, + boost::is_same, + boost::is_same, + boost::is_same + >, + mpl::identity >, + // else + typename mpl::eval_if, + mpl::identity >, + // else + typename mpl::eval_if, + mpl::identity >, + // else + mpl::identity + > + > + > + >::type typex; + return typex::cast(u); +} + +// this implements: +// smart_cast_reference(Source & s) +template +T smart_cast_reference(U & u) { + return smart_cast_impl::reference< T >::cast(u); +} + +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_SMART_CAST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/split_free.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/split_free.hpp new file mode 100644 index 00000000000..85e2f590fe4 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/split_free.hpp @@ -0,0 +1,93 @@ +#ifndef BOOST_SERIALIZATION_SPLIT_FREE_HPP +#define BOOST_SERIALIZATION_SPLIT_FREE_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// split_free.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include +#include + +namespace boost { +namespace archive { + namespace detail { + template class interface_oarchive; + template class interface_iarchive; + } // namespace detail +} // namespace archive + +namespace serialization { + +//namespace detail { +template +struct free_saver { + static void invoke( + Archive & ar, + const T & t, + const unsigned int file_version + ){ + // use function overload (version_type) to workaround + // two-phase lookup issue + const version_type v(file_version); + save(ar, t, v); + } +}; +template +struct free_loader { + static void invoke( + Archive & ar, + T & t, + const unsigned int file_version + ){ + // use function overload (version_type) to workaround + // two-phase lookup issue + const version_type v(file_version); + load(ar, t, v); + } +}; +//} // namespace detail + +template +inline void split_free( + Archive & ar, + T & t, + const unsigned int file_version +){ + typedef typename mpl::eval_if< + typename Archive::is_saving, + mpl::identity >, + mpl::identity > + >::type typex; + typex::invoke(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +#define BOOST_SERIALIZATION_SPLIT_FREE(T) \ +namespace boost { namespace serialization { \ +template \ +inline void serialize( \ + Archive & ar, \ + T & t, \ + const unsigned int file_version \ +){ \ + split_free(ar, t, file_version); \ +} \ +}} +/**/ + +#endif // BOOST_SERIALIZATION_SPLIT_FREE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/split_member.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/split_member.hpp new file mode 100644 index 00000000000..5f32520559e --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/split_member.hpp @@ -0,0 +1,86 @@ +#ifndef BOOST_SERIALIZATION_SPLIT_MEMBER_HPP +#define BOOST_SERIALIZATION_SPLIT_MEMBER_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// split_member.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include + +#include + +namespace boost { +namespace archive { + namespace detail { + template class interface_oarchive; + template class interface_iarchive; + } // namespace detail +} // namespace archive + +namespace serialization { +namespace detail { + + template + struct member_saver { + static void invoke( + Archive & ar, + const T & t, + const unsigned int file_version + ){ + access::member_save(ar, t, file_version); + } + }; + + template + struct member_loader { + static void invoke( + Archive & ar, + T & t, + const unsigned int file_version + ){ + access::member_load(ar, t, file_version); + } + }; + +} // detail + +template +inline void split_member( + Archive & ar, T & t, const unsigned int file_version +){ + typedef typename mpl::eval_if< + typename Archive::is_saving, + mpl::identity >, + mpl::identity > + >::type typex; + typex::invoke(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +// split member function serialize funcition into save/load +#define BOOST_SERIALIZATION_SPLIT_MEMBER() \ +template \ +void serialize( \ + Archive &ar, \ + const unsigned int file_version \ +){ \ + boost::serialization::split_member(ar, *this, file_version); \ +} \ +/**/ + +#endif // BOOST_SERIALIZATION_SPLIT_MEMBER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/stack.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/stack.hpp new file mode 100644 index 00000000000..96f90fe8767 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/stack.hpp @@ -0,0 +1,76 @@ +#ifndef BOOST_SERIALIZATION_STACK_HPP +#define BOOST_SERIALIZATION_STACK_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) && (_MSC_VER >= 1020) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// stack.hpp + +// (C) Copyright 2014 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include +#include + +// function specializations must be defined in the appropriate +// namespace - boost::serialization +#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) +#define STD _STLP_STD +#else +#define STD std +#endif + +namespace boost { +namespace serialization { +namespace detail{ + +template +struct stack_save : public STD::stack { + template + void operator()(Archive & ar, const unsigned int file_version) const { + save(ar, STD::stack::c, file_version); + } +}; +template +struct stack_load : public STD::stack { + template + void operator()(Archive & ar, const unsigned int file_version) { + load(ar, STD::stack::c, file_version); + } +}; + +} // detail + +template +inline void serialize( + Archive & ar, + std::stack< T, C> & t, + const unsigned int file_version +){ + typedef typename mpl::eval_if< + typename Archive::is_saving, + mpl::identity >, + mpl::identity > + >::type typex; + static_cast(t)(ar, file_version); +} + +} // namespace serialization +} // namespace boost + +#include + +BOOST_SERIALIZATION_COLLECTION_TRAITS(STD::stack) + +#undef STD + +#endif // BOOST_SERIALIZATION_DEQUE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/state_saver.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/state_saver.hpp new file mode 100644 index 00000000000..248b8d91556 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/state_saver.hpp @@ -0,0 +1,96 @@ +#ifndef BOOST_SERIALIZATION_STATE_SAVER_HPP +#define BOOST_SERIALIZATION_STATE_SAVER_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// state_saver.hpp: + +// (C) Copyright 2003-4 Pavel Vozenilek and Robert Ramey - http://www.rrsd.com. +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org/libs/serialization for updates, documentation, and revision history. + +// Inspired by Daryle Walker's iostate_saver concept. This saves the original +// value of a variable when a state_saver is constructed and restores +// upon destruction. Useful for being sure that state is restored to +// variables upon exit from scope. + + +#include +#ifndef BOOST_NO_EXCEPTIONS + #include +#endif + +#include +#include +#include +#include + +#include +#include + +namespace boost { +namespace serialization { + +template +// T requirements: +// - POD or object semantic (cannot be reference, function, ...) +// - copy constructor +// - operator = (no-throw one preferred) +class state_saver : private boost::noncopyable +{ +private: + const T previous_value; + T & previous_ref; + + struct restore { + static void invoke(T & previous_ref, const T & previous_value){ + previous_ref = previous_value; // won't throw + } + }; + + struct restore_with_exception { + static void invoke(T & previous_ref, const T & previous_value){ + BOOST_TRY{ + previous_ref = previous_value; + } + BOOST_CATCH(::std::exception &) { + // we must ignore it - we are in destructor + } + BOOST_CATCH_END + } + }; + +public: + state_saver( + T & object + ) : + previous_value(object), + previous_ref(object) + {} + + ~state_saver() { + #ifndef BOOST_NO_EXCEPTIONS + typedef typename mpl::eval_if< + has_nothrow_copy< T >, + mpl::identity, + mpl::identity + >::type typex; + typex::invoke(previous_ref, previous_value); + #else + previous_ref = previous_value; + #endif + } + +}; // state_saver<> + +} // serialization +} // boost + +#endif //BOOST_SERIALIZATION_STATE_SAVER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/static_warning.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/static_warning.hpp new file mode 100644 index 00000000000..1d9238fc4d9 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/static_warning.hpp @@ -0,0 +1,103 @@ +#ifndef BOOST_SERIALIZATION_STATIC_WARNING_HPP +#define BOOST_SERIALIZATION_STATIC_WARNING_HPP + +// (C) Copyright Robert Ramey 2003. Jonathan Turkanis 2004. +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org/libs/static_assert for documentation. + +/* + Revision history: + 15 June 2003 - Initial version. + 31 March 2004 - improved diagnostic messages and portability + (Jonathan Turkanis) + 03 April 2004 - works on VC6 at class and namespace scope + - ported to DigitalMars + - static warnings disabled by default; when enabled, + uses pragmas to enable required compiler warnings + on MSVC, Intel, Metrowerks and Borland 5.x. + (Jonathan Turkanis) + 30 May 2004 - tweaked for msvc 7.1 and gcc 3.3 + - static warnings ENabled by default; when enabled, + (Robert Ramey) +*/ + +#include + +// +// Implementation +// Makes use of the following warnings: +// 1. GCC prior to 3.3: division by zero. +// 2. BCC 6.0 preview: unreferenced local variable. +// 3. DigitalMars: returning address of local automatic variable. +// 4. VC6: class previously seen as struct (as in 'boost/mpl/print.hpp') +// 5. All others: deletion of pointer to incomplete type. +// +// The trick is to find code which produces warnings containing the name of +// a structure or variable. Details, with same numbering as above: +// 1. static_warning_impl::value is zero iff B is false, so diving an int +// by this value generates a warning iff B is false. +// 2. static_warning_impl::type has a constructor iff B is true, so an +// unreferenced variable of this type generates a warning iff B is false. +// 3. static_warning_impl::type overloads operator& to return a dynamically +// allocated int pointer only is B is true, so returning the address of an +// automatic variable of this type generates a warning iff B is fasle. +// 4. static_warning_impl::STATIC_WARNING is decalred as a struct iff B is +// false. +// 5. static_warning_impl::type is incomplete iff B is false, so deleting a +// pointer to this type generates a warning iff B is false. +// + +//------------------Enable selected warnings----------------------------------// + +// Enable the warnings relied on by BOOST_STATIC_WARNING, where possible. + +// 6. replaced implementation with one which depends solely on +// mpl::print<>. The previous one was found to fail for functions +// under recent versions of gcc and intel compilers - Robert Ramey + +#include +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +template +struct BOOST_SERIALIZATION_STATIC_WARNING_LINE{}; + +template +struct static_warning_test{ + typename boost::mpl::eval_if_c< + B, + boost::mpl::true_, + typename boost::mpl::identity< + boost::mpl::print< + BOOST_SERIALIZATION_STATIC_WARNING_LINE + > + > + >::type type; +}; + +template +struct BOOST_SERIALIZATION_SS {}; + +} // serialization +} // boost + +#define BOOST_SERIALIZATION_BSW(B, L) \ + typedef boost::serialization::BOOST_SERIALIZATION_SS< \ + sizeof( boost::serialization::static_warning_test< B, L > ) \ + > BOOST_JOIN(STATIC_WARNING_LINE, L) BOOST_ATTRIBUTE_UNUSED; +#define BOOST_STATIC_WARNING(B) BOOST_SERIALIZATION_BSW(B, __LINE__) + +#endif // BOOST_SERIALIZATION_STATIC_WARNING_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/string.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/string.hpp new file mode 100644 index 00000000000..76e695d4f3c --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/string.hpp @@ -0,0 +1,30 @@ +#ifndef BOOST_SERIALIZATION_STRING_HPP +#define BOOST_SERIALIZATION_STRING_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// serialization/string.hpp: +// serialization for stl string templates + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include + +BOOST_CLASS_IMPLEMENTATION(std::string, boost::serialization::primitive_type) +#ifndef BOOST_NO_STD_WSTRING +BOOST_CLASS_IMPLEMENTATION(std::wstring, boost::serialization::primitive_type) +#endif + +#endif // BOOST_SERIALIZATION_STRING_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/strong_typedef.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/strong_typedef.hpp new file mode 100644 index 00000000000..fdd1b24c9cb --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/strong_typedef.hpp @@ -0,0 +1,50 @@ +#ifndef BOOST_SERIALIZATION_STRONG_TYPEDEF_HPP +#define BOOST_SERIALIZATION_STRONG_TYPEDEF_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// strong_typedef.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// (C) Copyright 2016 Ashish Sadanandan +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org/libs/serialization for updates, documentation, and revision history. + +// macro used to implement a strong typedef. strong typedef +// guarentees that two types are distinguised even though the +// share the same underlying implementation. typedef does not create +// a new type. BOOST_STRONG_TYPEDEF(T, D) creates a new type named D +// that operates as a type T. + +#include +#include +#include +#include +#include + +#define BOOST_STRONG_TYPEDEF(T, D) \ +struct D \ + : boost::totally_ordered1< D \ + , boost::totally_ordered2< D, T \ + > > \ +{ \ + T t; \ + explicit D(const T& t_) BOOST_NOEXCEPT_IF(boost::has_nothrow_copy_constructor::value) : t(t_) {} \ + D() BOOST_NOEXCEPT_IF(boost::has_nothrow_default_constructor::value) : t() {} \ + D(const D & t_) BOOST_NOEXCEPT_IF(boost::has_nothrow_copy_constructor::value) : t(t_.t) {} \ + D& operator=(const D& rhs) BOOST_NOEXCEPT_IF(boost::has_nothrow_assign::value) {t = rhs.t; return *this;} \ + D& operator=(const T& rhs) BOOST_NOEXCEPT_IF(boost::has_nothrow_assign::value) {t = rhs; return *this;} \ + operator const T&() const {return t;} \ + operator T&() {return t;} \ + bool operator==(const D& rhs) const {return t == rhs.t;} \ + bool operator<(const D& rhs) const {return t < rhs.t;} \ +}; + +#endif // BOOST_SERIALIZATION_STRONG_TYPEDEF_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/throw_exception.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/throw_exception.hpp new file mode 100644 index 00000000000..b67618adc92 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/throw_exception.hpp @@ -0,0 +1,44 @@ +#ifndef BOOST_SERIALIZATION_THROW_EXCEPTION_HPP_INCLUDED +#define BOOST_SERIALIZATION_THROW_EXCEPTION_HPP_INCLUDED + +// MS compatible compilers support #pragma once + +#if defined(_MSC_VER) +# pragma once +#endif + +// boost/throw_exception.hpp +// +// Copyright (c) 2002 Peter Dimov and Multi Media Ltd. +// +// Distributed under the Boost Software License, Version 1.0. (See +// accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +#include + +#ifndef BOOST_NO_EXCEPTIONS +#include +#endif + +namespace boost { +namespace serialization { + +#ifdef BOOST_NO_EXCEPTIONS + +inline void throw_exception(std::exception const & e) { + ::boost::throw_exception(e); +} + +#else + +template inline void throw_exception(E const & e){ + throw e; +} + +#endif + +} // namespace serialization +} // namespace boost + +#endif // #ifndef BOOST_SERIALIZATION_THROW_EXCEPTION_HPP_INCLUDED diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/tracking.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/tracking.hpp new file mode 100644 index 00000000000..d5c79b8409d --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/tracking.hpp @@ -0,0 +1,118 @@ +#ifndef BOOST_SERIALIZATION_TRACKING_HPP +#define BOOST_SERIALIZATION_TRACKING_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// tracking.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +struct basic_traits; + +// default tracking level +template +struct tracking_level_impl { + template + struct traits_class_tracking { + typedef typename U::tracking type; + }; + typedef mpl::integral_c_tag tag; + // note: at least one compiler complained w/o the full qualification + // on basic traits below + typedef + typename mpl::eval_if< + is_base_and_derived, + traits_class_tracking< T >, + //else + typename mpl::eval_if< + is_pointer< T >, + // pointers are not tracked by default + mpl::int_, + //else + typename mpl::eval_if< + // for primitives + typename mpl::equal_to< + implementation_level< T >, + mpl::int_ + >, + // is never + mpl::int_, + // otherwise its selective + mpl::int_ + > > >::type type; + BOOST_STATIC_CONSTANT(int, value = type::value); +}; + +template +struct tracking_level : + public tracking_level_impl +{ +}; + +template +inline bool operator>=(tracking_level< T > t, enum tracking_type l) +{ + return t.value >= (int)l; +} + +} // namespace serialization +} // namespace boost + + +// The STATIC_ASSERT is prevents one from setting tracking for a primitive type. +// This almost HAS to be an error. Doing this will effect serialization of all +// char's in your program which is almost certainly what you don't want to do. +// If you want to track all instances of a given primitive type, You'll have to +// wrap it in your own type so its not a primitive anymore. Then it will compile +// without problem. +#define BOOST_CLASS_TRACKING(T, E) \ +namespace boost { \ +namespace serialization { \ +template<> \ +struct tracking_level< T > \ +{ \ + typedef mpl::integral_c_tag tag; \ + typedef mpl::int_< E> type; \ + BOOST_STATIC_CONSTANT( \ + int, \ + value = tracking_level::type::value \ + ); \ + /* tracking for a class */ \ + BOOST_STATIC_ASSERT(( \ + mpl::greater< \ + /* that is a prmitive */ \ + implementation_level< T >, \ + mpl::int_ \ + >::value \ + )); \ +}; \ +}} + +#endif // BOOST_SERIALIZATION_TRACKING_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/tracking_enum.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/tracking_enum.hpp new file mode 100644 index 00000000000..278051e1baf --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/tracking_enum.hpp @@ -0,0 +1,41 @@ +#ifndef BOOST_SERIALIZATION_TRACKING_ENUM_HPP +#define BOOST_SERIALIZATION_TRACKING_ENUM_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// tracking_enum.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +namespace boost { +namespace serialization { + +// addresses of serialized objects may be tracked to avoid saving/loading +// redundant copies. This header defines a class trait that can be used +// to specify when objects should be tracked + +// names for each tracking level +enum tracking_type +{ + // never track this type + track_never = 0, + // track objects of this type if the object is serialized through a + // pointer. + track_selectively = 1, + // always track this type + track_always = 2 +}; + +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_TRACKING_ENUM_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/traits.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/traits.hpp new file mode 100644 index 00000000000..9e114fdd3df --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/traits.hpp @@ -0,0 +1,65 @@ +#ifndef BOOST_SERIALIZATION_TRAITS_HPP +#define BOOST_SERIALIZATION_TRAITS_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// traits.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// This header is used to apply serialization traits to templates. The +// standard system can't be used for platforms which don't support +// Partial Templlate Specialization. + +// The motivation for this is the Name-Value Pair (NVP) template. +// it has to work the same on all platforms in order for archives +// to be portable accross platforms. + +#include +#include + +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +// common base class used to detect appended traits class +struct basic_traits {}; + +template +struct extended_type_info_impl; + +template< + class T, + int Level, + int Tracking, + unsigned int Version = 0, + class ETII = extended_type_info_impl< T >, + class Wrapper = mpl::false_ +> +struct traits : public basic_traits { + BOOST_STATIC_ASSERT(Version == 0 || Level >= object_class_info); + BOOST_STATIC_ASSERT(Tracking == track_never || Level >= object_serializable); + typedef typename mpl::int_ level; + typedef typename mpl::int_ tracking; + typedef typename mpl::int_ version; + typedef ETII type_info_implementation; + typedef Wrapper is_wrapper; +}; + +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_TRAITS_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/type_info_implementation.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/type_info_implementation.hpp new file mode 100644 index 00000000000..24637a8dbb3 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/type_info_implementation.hpp @@ -0,0 +1,73 @@ +#ifndef BOOST_SERIALIZATION_TYPE_INFO_IMPLEMENTATION_HPP +#define BOOST_SERIALIZATION_TYPE_INFO_IMPLEMENTATION_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// type_info_implementation.hpp: interface for portable version of type_info + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + + +#include +#include + +#include +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +// note that T and const T are folded into const T so that +// there is only one table entry per type +template +struct type_info_implementation { + template + struct traits_class_typeinfo_implementation { + typedef typename U::type_info_implementation::type type; + }; + // note: at least one compiler complained w/o the full qualification + // on basic traits below + typedef + typename mpl::eval_if< + is_base_and_derived, + traits_class_typeinfo_implementation< T >, + //else + mpl::identity< + typename extended_type_info_impl< T >::type + > + >::type type; +}; + +} // namespace serialization +} // namespace boost + +// define a macro to assign a particular derivation of extended_type_info +// to a specified a class. +#define BOOST_CLASS_TYPE_INFO(T, ETI) \ +namespace boost { \ +namespace serialization { \ +template<> \ +struct type_info_implementation< T > { \ + typedef ETI type; \ +}; \ +template<> \ +struct type_info_implementation< const T > { \ + typedef ETI type; \ +}; \ +} \ +} \ +/**/ + +#endif /// BOOST_SERIALIZATION_TYPE_INFO_IMPLEMENTATION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unique_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unique_ptr.hpp new file mode 100644 index 00000000000..8d8703ef4f7 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/unique_ptr.hpp @@ -0,0 +1,68 @@ +#ifndef BOOST_SERIALIZATION_UNIQUE_PTR_HPP +#define BOOST_SERIALIZATION_UNIQUE_PTR_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// unique_ptr.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. +#include +#include +#include + +namespace boost { +namespace serialization { + +///////////////////////////////////////////////////////////// +// implement serialization for unique_ptr< T > +// note: this must be added to the boost namespace in order to +// be called by the library +template +inline void save( + Archive & ar, + const std::unique_ptr< T > &t, + const unsigned int /*file_version*/ +){ + // only the raw pointer has to be saved + // the ref count is rebuilt automatically on load + const T * const tx = t.get(); + ar << BOOST_SERIALIZATION_NVP(tx); +} + +template +inline void load( + Archive & ar, + std::unique_ptr< T > &t, + const unsigned int /*file_version*/ +){ + T *tx; + ar >> BOOST_SERIALIZATION_NVP(tx); + // note that the reset automagically maintains the reference count + t.reset(tx); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template +inline void serialize( + Archive & ar, + std::unique_ptr< T > &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + + +#endif // BOOST_SERIALIZATION_UNIQUE_PTR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_load_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_load_imp.hpp new file mode 100644 index 00000000000..d56a423d180 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_load_imp.hpp @@ -0,0 +1,73 @@ +#ifndef BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_LOAD_IMP_HPP +#define BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_LOAD_IMP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) && (_MSC_VER >= 1020) +# pragma once +# pragma warning (disable : 4786) // too long name, harmless warning +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// unordered_collections_load_imp.hpp: serialization for loading stl collections + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// (C) Copyright 2014 Jim Bell +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// helper function templates for serialization of collections + +#include +#include // size_t +#include // msvc 6.0 needs this for warning suppression +#if defined(BOOST_NO_STDC_NAMESPACE) +namespace std{ + using ::size_t; +} // namespace std +#endif +#include + +#include +#include +#include +#include +#include + +namespace boost{ +namespace serialization { +namespace stl { + +////////////////////////////////////////////////////////////////////// +// implementation of serialization for STL containers +// +template +inline void load_unordered_collection(Archive & ar, Container &s) +{ + collection_size_type count; + collection_size_type bucket_count; + boost::serialization::item_version_type item_version(0); + boost::archive::library_version_type library_version( + ar.get_library_version() + ); + // retrieve number of elements + ar >> BOOST_SERIALIZATION_NVP(count); + ar >> BOOST_SERIALIZATION_NVP(bucket_count); + if(boost::archive::library_version_type(3) < library_version){ + ar >> BOOST_SERIALIZATION_NVP(item_version); + } + s.clear(); + s.rehash(bucket_count); + InputFunction ifunc; + while(count-- > 0){ + ifunc(ar, s, item_version); + } +} + +} // namespace stl +} // namespace serialization +} // namespace boost + +#endif //BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_LOAD_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_save_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_save_imp.hpp new file mode 100644 index 00000000000..56746ebeaa3 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_save_imp.hpp @@ -0,0 +1,86 @@ +#ifndef BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_SAVE_IMP_HPP +#define BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_SAVE_IMP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) && (_MSC_VER >= 1020) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// hash_collections_save_imp.hpp: serialization for stl collections + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// (C) Copyright 2014 Jim Bell +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +// helper function templates for serialization of collections + +#include +#include +#include +#include +#include +#include + +namespace boost{ +namespace serialization { +namespace stl { + +////////////////////////////////////////////////////////////////////// +// implementation of serialization for STL containers +// + +template +inline void save_unordered_collection(Archive & ar, const Container &s) +{ + collection_size_type count(s.size()); + const collection_size_type bucket_count(s.bucket_count()); + const item_version_type item_version( + version::value + ); + + #if 0 + /* should only be necessary to create archives of previous versions + * which is not currently supported. So for now comment this out + */ + boost::archive::library_version_type library_version( + ar.get_library_version() + ); + // retrieve number of elements + ar << BOOST_SERIALIZATION_NVP(count); + ar << BOOST_SERIALIZATION_NVP(bucket_count); + if(boost::archive::library_version_type(3) < library_version){ + // record number of elements + // make sure the target type is registered so we can retrieve + // the version when we load + ar << BOOST_SERIALIZATION_NVP(item_version); + } + #else + ar << BOOST_SERIALIZATION_NVP(count); + ar << BOOST_SERIALIZATION_NVP(bucket_count); + ar << BOOST_SERIALIZATION_NVP(item_version); + #endif + + typename Container::const_iterator it = s.begin(); + while(count-- > 0){ + // note borland emits a no-op without the explicit namespace + boost::serialization::save_construct_data_adl( + ar, + &(*it), + boost::serialization::version< + typename Container::value_type + >::value + ); + ar << boost::serialization::make_nvp("item", *it++); + } +} + +} // namespace stl +} // namespace serialization +} // namespace boost + +#endif //BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_SAVE_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_map.hpp new file mode 100644 index 00000000000..4fdbddd7b65 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_map.hpp @@ -0,0 +1,160 @@ +#ifndef BOOST_SERIALIZATION_UNORDERED_MAP_HPP +#define BOOST_SERIALIZATION_UNORDERED_MAP_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) && (_MSC_VER >= 1020) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// serialization/unordered_map.hpp: +// serialization for stl unordered_map templates + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// (C) Copyright 2014 Jim Bell +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include + +#include +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void save( + Archive & ar, + const std::unordered_map &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::save_unordered_collection< + Archive, + std::unordered_map + >(ar, t); +} + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void load( + Archive & ar, + std::unordered_map &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::load_unordered_collection< + Archive, + std::unordered_map, + boost::serialization::stl::archive_input_unordered_map< + Archive, + std::unordered_map + > + >(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void serialize( + Archive & ar, + std::unordered_map &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +// unordered_multimap +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void save( + Archive & ar, + const std::unordered_multimap< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::save_unordered_collection< + Archive, + std::unordered_multimap + >(ar, t); +} + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void load( + Archive & ar, + std::unordered_multimap< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::load_unordered_collection< + Archive, + std::unordered_multimap< + Key, HashFcn, EqualKey, Allocator + >, + boost::serialization::stl::archive_input_unordered_multimap< + Archive, + std::unordered_multimap + > + >(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void serialize( + Archive & ar, + std::unordered_multimap< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_UNORDERED_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_set.hpp new file mode 100644 index 00000000000..adfee609cbe --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_set.hpp @@ -0,0 +1,162 @@ +#ifndef BOOST_SERIALIZATION_UNORDERED_SET_HPP +#define BOOST_SERIALIZATION_UNORDERED_SET_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) && (_MSC_VER >= 1020) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// unordered_set.hpp: serialization for stl unordered_set templates + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// (C) Copyright 2014 Jim Bell +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include + +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void save( + Archive & ar, + const std::unordered_set< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::save_unordered_collection< + Archive, + std::unordered_set + >(ar, t); +} + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void load( + Archive & ar, + std::unordered_set< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::load_unordered_collection< + Archive, + std::unordered_set, + stl::archive_input_unordered_set< + Archive, + std::unordered_set< + Key, HashFcn, EqualKey, Allocator + > + > + >(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void serialize( + Archive & ar, + std::unordered_set< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int file_version +){ + split_free(ar, t, file_version); +} + +// unordered_multiset +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void save( + Archive & ar, + const std::unordered_multiset< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int /*file_version*/ +){ + stl::save_unordered_collection< + Archive, + std::unordered_multiset + >(ar, t); +} + +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void load( + Archive & ar, + std::unordered_multiset< + Key, HashFcn, EqualKey, Allocator + > &t, + const unsigned int /*file_version*/ +){ + boost::serialization::stl::load_unordered_collection< + Archive, + std::unordered_multiset, + boost::serialization::stl::archive_input_unordered_multiset< + Archive, + std::unordered_multiset + > + >(ar, t); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template< + class Archive, + class Key, + class HashFcn, + class EqualKey, + class Allocator +> +inline void serialize( + Archive & ar, + std::unordered_multiset &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_UNORDERED_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/utility.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/utility.hpp new file mode 100644 index 00000000000..4867a4a12d2 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/utility.hpp @@ -0,0 +1,56 @@ +#ifndef BOOST_SERIALIZATION_UTILITY_HPP +#define BOOST_SERIALIZATION_UTILITY_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// serialization/utility.hpp: +// serialization for stl utility templates + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include + +#include +#include +#include +#include + +namespace boost { +namespace serialization { + +// pair +template +inline void serialize( + Archive & ar, + std::pair & p, + const unsigned int /* file_version */ +){ + // note: we remove any const-ness on the first argument. The reason is that + // for stl maps, the type saved is pair::type typef; + ar & boost::serialization::make_nvp("first", const_cast(p.first)); + ar & boost::serialization::make_nvp("second", p.second); +} + +/// specialization of is_bitwise_serializable for pairs +template +struct is_bitwise_serializable > + : public mpl::and_,is_bitwise_serializable > +{ +}; + +} // serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_UTILITY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/valarray.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/valarray.hpp new file mode 100644 index 00000000000..9eece5c1737 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/valarray.hpp @@ -0,0 +1,86 @@ +#ifndef BOOST_SERIALIZATION_VALARAY_HPP +#define BOOST_SERIALIZATION_VALARAY_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// valarray.hpp: serialization for stl vector templates + +// (C) Copyright 2005 Matthias Troyer . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include + +#include +#include +#include +#include +#include + +// function specializations must be defined in the appropriate +// namespace - boost::serialization +#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) +#define STD _STLP_STD +#else +#define STD std +#endif + +namespace boost { +namespace serialization { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// valarray< T > + +template +void save( Archive & ar, const STD::valarray &t, const unsigned int /*file_version*/ ) +{ + const collection_size_type count(t.size()); + ar << BOOST_SERIALIZATION_NVP(count); + if (t.size()){ + // explict template arguments to pass intel C++ compiler + ar << serialization::make_array( + static_cast(&t[0]), + count + ); + } +} + +template +void load( Archive & ar, STD::valarray &t, const unsigned int /*file_version*/ ) +{ + collection_size_type count; + ar >> BOOST_SERIALIZATION_NVP(count); + t.resize(count); + if (t.size()){ + // explict template arguments to pass intel C++ compiler + ar >> serialization::make_array( + static_cast(&t[0]), + count + ); + } +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template +inline void serialize( Archive & ar, STD::valarray & t, const unsigned int file_version) +{ + boost::serialization::split_free(ar, t, file_version); +} + +} } // end namespace boost::serialization + +#include + +BOOST_SERIALIZATION_COLLECTION_TRAITS(STD::valarray) +#undef STD + +#endif // BOOST_SERIALIZATION_VALARAY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/variant.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/variant.hpp new file mode 100644 index 00000000000..dce6f3d49e7 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/variant.hpp @@ -0,0 +1,158 @@ +#ifndef BOOST_SERIALIZATION_VARIANT_HPP +#define BOOST_SERIALIZATION_VARIANT_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// variant.hpp - non-intrusive serialization of variant types +// +// copyright (c) 2005 +// troy d. straszheim +// http://www.resophonic.com +// +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) +// +// See http://www.boost.org for updates, documentation, and revision history. +// +// thanks to Robert Ramey, Peter Dimov, and Richard Crossley. +// + +#include +#include +#include +#include +#include +#include + +#include + +#include + +#include + +#include +#include +#include + +namespace boost { +namespace serialization { + +template +struct variant_save_visitor : + boost::static_visitor<> +{ + variant_save_visitor(Archive& ar) : + m_ar(ar) + {} + template + void operator()(T const & value) const + { + m_ar << BOOST_SERIALIZATION_NVP(value); + } +private: + Archive & m_ar; +}; + +template +void save( + Archive & ar, + boost::variant const & v, + unsigned int /*version*/ +){ + int which = v.which(); + ar << BOOST_SERIALIZATION_NVP(which); + variant_save_visitor visitor(ar); + v.apply_visitor(visitor); +} + +template +struct variant_impl { + + struct load_null { + template + static void invoke( + Archive & /*ar*/, + int /*which*/, + V & /*v*/, + const unsigned int /*version*/ + ){} + }; + + struct load_impl { + template + static void invoke( + Archive & ar, + int which, + V & v, + const unsigned int version + ){ + if(which == 0){ + // note: A non-intrusive implementation (such as this one) + // necessary has to copy the value. This wouldn't be necessary + // with an implementation that de-serialized to the address of the + // aligned storage included in the variant. + typedef typename mpl::front::type head_type; + head_type value; + ar >> BOOST_SERIALIZATION_NVP(value); + v = value; + ar.reset_object_address(& boost::get(v), & value); + return; + } + typedef typename mpl::pop_front::type type; + variant_impl::load(ar, which - 1, v, version); + } + }; + + template + static void load( + Archive & ar, + int which, + V & v, + const unsigned int version + ){ + typedef typename mpl::eval_if, + mpl::identity, + mpl::identity + >::type typex; + typex::invoke(ar, which, v, version); + } + +}; + +template +void load( + Archive & ar, + boost::variant& v, + const unsigned int version +){ + int which; + typedef typename boost::variant::types types; + ar >> BOOST_SERIALIZATION_NVP(which); + if(which >= mpl::size::value) + // this might happen if a type was removed from the list of variant types + boost::serialization::throw_exception( + boost::archive::archive_exception( + boost::archive::archive_exception::unsupported_version + ) + ); + variant_impl::load(ar, which, v, version); +} + +template +inline void serialize( + Archive & ar, + boost::variant & v, + const unsigned int file_version +){ + split_free(ar,v,file_version); +} + +} // namespace serialization +} // namespace boost + +#endif //BOOST_SERIALIZATION_VARIANT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/vector.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/vector.hpp new file mode 100644 index 00000000000..9a114c00e20 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/vector.hpp @@ -0,0 +1,233 @@ +#ifndef BOOST_SERIALIZATION_VECTOR_HPP +#define BOOST_SERIALIZATION_VECTOR_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// vector.hpp: serialization for stl vector templates + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// fast array serialization (C) Copyright 2005 Matthias Troyer +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include + +#include +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +// default is being compatible with version 1.34.1 files, not 1.35 files +#ifndef BOOST_SERIALIZATION_VECTOR_VERSIONED +#define BOOST_SERIALIZATION_VECTOR_VERSIONED(V) (V==4 || V==5) +#endif + +// function specializations must be defined in the appropriate +// namespace - boost::serialization +#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) +#define STD _STLP_STD +#else +#define STD std +#endif + +namespace boost { +namespace serialization { + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// vector< T > + +// the default versions + +template +inline void save( + Archive & ar, + const std::vector &t, + const unsigned int /* file_version */, + mpl::false_ +){ + boost::serialization::stl::save_collection >( + ar, t + ); +} + +template +inline void load( + Archive & ar, + std::vector &t, + const unsigned int /* file_version */, + mpl::false_ +){ + const boost::archive::library_version_type library_version( + ar.get_library_version() + ); + // retrieve number of elements + item_version_type item_version(0); + collection_size_type count; + ar >> BOOST_SERIALIZATION_NVP(count); + if(boost::archive::library_version_type(3) < library_version){ + ar >> BOOST_SERIALIZATION_NVP(item_version); + } + t.reserve(count); + stl::collection_load_impl(ar, t, count, item_version); +} + +// the optimized versions + +template +inline void save( + Archive & ar, + const std::vector &t, + const unsigned int /* file_version */, + mpl::true_ +){ + const collection_size_type count(t.size()); + ar << BOOST_SERIALIZATION_NVP(count); + if (!t.empty()) + // explict template arguments to pass intel C++ compiler + ar << serialization::make_array( + static_cast(&t[0]), + count + ); +} + +template +inline void load( + Archive & ar, + std::vector &t, + const unsigned int /* file_version */, + mpl::true_ +){ + collection_size_type count(t.size()); + ar >> BOOST_SERIALIZATION_NVP(count); + t.resize(count); + unsigned int item_version=0; + if(BOOST_SERIALIZATION_VECTOR_VERSIONED(ar.get_library_version())) { + ar >> BOOST_SERIALIZATION_NVP(item_version); + } + if (!t.empty()) + // explict template arguments to pass intel C++ compiler + ar >> serialization::make_array( + static_cast(&t[0]), + count + ); + } + +// dispatch to either default or optimized versions + +template +inline void save( + Archive & ar, + const std::vector &t, + const unsigned int file_version +){ + typedef typename + boost::serialization::use_array_optimization::template apply< + typename remove_const::type + >::type use_optimized; + save(ar,t,file_version, use_optimized()); +} + +template +inline void load( + Archive & ar, + std::vector &t, + const unsigned int file_version +){ +#ifdef BOOST_SERIALIZATION_VECTOR_135_HPP + if (ar.get_library_version()==boost::archive::library_version_type(5)) + { + load(ar,t,file_version, boost::is_arithmetic()); + return; + } +#endif + typedef typename + boost::serialization::use_array_optimization::template apply< + typename remove_const::type + >::type use_optimized; + load(ar,t,file_version, use_optimized()); +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template +inline void serialize( + Archive & ar, + std::vector & t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// vector +template +inline void save( + Archive & ar, + const std::vector &t, + const unsigned int /* file_version */ +){ + // record number of elements + collection_size_type count (t.size()); + ar << BOOST_SERIALIZATION_NVP(count); + std::vector::const_iterator it = t.begin(); + while(count-- > 0){ + bool tb = *it++; + ar << boost::serialization::make_nvp("item", tb); + } +} + +template +inline void load( + Archive & ar, + std::vector &t, + const unsigned int /* file_version */ +){ + // retrieve number of elements + collection_size_type count; + ar >> BOOST_SERIALIZATION_NVP(count); + t.resize(count); + for(collection_size_type i = collection_size_type(0); i < count; ++i){ + bool b; + ar >> boost::serialization::make_nvp("item", b); + t[i] = b; + } +} + +// split non-intrusive serialization function member into separate +// non intrusive save/load member functions +template +inline void serialize( + Archive & ar, + std::vector & t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // serialization +} // namespace boost + +#include + +BOOST_SERIALIZATION_COLLECTION_TRAITS(std::vector) +#undef STD + +#endif // BOOST_SERIALIZATION_VECTOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/vector_135.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/vector_135.hpp new file mode 100644 index 00000000000..fd1a7393d1b --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/vector_135.hpp @@ -0,0 +1,26 @@ +////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// vector_135.hpp: serialization for stl vector templates for compatibility +// with release 1.35, which had a bug + +// (C) Copyright 2008 Matthias Troyer +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + + +#ifndef BOOST_SERIALIZATION_VECTOR_135_HPP +#define BOOST_SERIALIZATION_VECTOR_135_HPP + +#ifdef BOOST_SERIALIZATION_VECTOR_VERSIONED +#if BOOST_SERIALIZATION_VECTOR_VERSION != 4 +#error "Boost.Serialization cannot be compatible with both 1.35 and 1.36-1.40 files" +#endif +#else +#define BOOST_SERIALIZATION_VECTOR_VERSIONED(V) (V>4) +#endif + +#include + +#endif // BOOST_SERIALIZATION_VECTOR_135_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/version.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/version.hpp new file mode 100644 index 00000000000..21a74d73daa --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/version.hpp @@ -0,0 +1,107 @@ +#ifndef BOOST_SERIALIZATION_VERSION_HPP +#define BOOST_SERIALIZATION_VERSION_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// version.hpp: + +// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include +#include +#include +#include +#include + +#include + +namespace boost { +namespace serialization { + +struct basic_traits; + +// default version number is 0. Override with higher version +// when class definition changes. +template +struct version +{ + template + struct traits_class_version { + typedef typename U::version type; + }; + + typedef mpl::integral_c_tag tag; + // note: at least one compiler complained w/o the full qualification + // on basic traits below + typedef + typename mpl::eval_if< + is_base_and_derived, + traits_class_version< T >, + mpl::int_<0> + >::type type; + BOOST_STATIC_CONSTANT(int, value = version::type::value); +}; + +#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION +template +const int version::value; +#endif + +} // namespace serialization +} // namespace boost + +/* note: at first it seemed that this would be a good place to trap + * as an error an attempt to set a version # for a class which doesn't + * save its class information (including version #) in the archive. + * However, this imposes a requirement that the version be set after + * the implemention level which would be pretty confusing. If this + * is to be done, do this check in the input or output operators when + * ALL the serialization traits are available. Included the implementation + * here with this comment as a reminder not to do this! + */ +//#include +//#include + +#include +#include + +// specify the current version number for the class +// version numbers limited to 8 bits !!! +#define BOOST_CLASS_VERSION(T, N) \ +namespace boost { \ +namespace serialization { \ +template<> \ +struct version \ +{ \ + typedef mpl::int_ type; \ + typedef mpl::integral_c_tag tag; \ + BOOST_STATIC_CONSTANT(int, value = version::type::value); \ + BOOST_MPL_ASSERT(( \ + boost::mpl::less< \ + boost::mpl::int_, \ + boost::mpl::int_<256> \ + > \ + )); \ + /* \ + BOOST_MPL_ASSERT(( \ + mpl::equal_to< \ + :implementation_level, \ + mpl::int_ \ + >::value \ + )); \ + */ \ +}; \ +} \ +} + +#endif // BOOST_SERIALIZATION_VERSION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/void_cast.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/void_cast.hpp new file mode 100644 index 00000000000..f1b38286115 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/void_cast.hpp @@ -0,0 +1,298 @@ +#ifndef BOOST_SERIALIZATION_VOID_CAST_HPP +#define BOOST_SERIALIZATION_VOID_CAST_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// void_cast.hpp: interface for run-time casting of void pointers. + +// (C) Copyright 2002-2009 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) +// gennadiy.rozental@tfn.com + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // for ptrdiff_t +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include +#include // must be the last header + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4251 4231 4660 4275) +#endif + +namespace boost { +namespace serialization { + +class extended_type_info; + +// Given a void *, assume that it really points to an instance of one type +// and alter it so that it would point to an instance of a related type. +// Return the altered pointer. If there exists no sequence of casts that +// can transform from_type to to_type, return a NULL. + +BOOST_SERIALIZATION_DECL void const * +void_upcast( + extended_type_info const & derived, + extended_type_info const & base, + void const * const t +); + +inline void * +void_upcast( + extended_type_info const & derived, + extended_type_info const & base, + void * const t +){ + return const_cast(void_upcast( + derived, + base, + const_cast(t) + )); +} + +BOOST_SERIALIZATION_DECL void const * +void_downcast( + extended_type_info const & derived, + extended_type_info const & base, + void const * const t +); + +inline void * +void_downcast( + extended_type_info const & derived, + extended_type_info const & base, + void * const t +){ + return const_cast(void_downcast( + derived, + base, + const_cast(t) + )); +} + +namespace void_cast_detail { + +class BOOST_SYMBOL_VISIBLE void_caster : + private boost::noncopyable +{ + friend + BOOST_SERIALIZATION_DECL void const * + boost::serialization::void_upcast( + extended_type_info const & derived, + extended_type_info const & base, + void const * const + ); + friend + BOOST_SERIALIZATION_DECL void const * + boost::serialization::void_downcast( + extended_type_info const & derived, + extended_type_info const & base, + void const * const + ); +protected: + BOOST_SERIALIZATION_DECL void recursive_register(bool includes_virtual_base = false) const; + BOOST_SERIALIZATION_DECL void recursive_unregister() const; + virtual bool has_virtual_base() const = 0; +public: + // Data members + const extended_type_info * m_derived; + const extended_type_info * m_base; + /*const*/ std::ptrdiff_t m_difference; + void_caster const * const m_parent; + + // note that void_casters are keyed on value of + // member extended type info records - NOT their + // addresses. This is necessary in order for the + // void cast operations to work across dll and exe + // module boundries. + bool operator<(const void_caster & rhs) const; + + const void_caster & operator*(){ + return *this; + } + // each derived class must re-implement these; + virtual void const * upcast(void const * const t) const = 0; + virtual void const * downcast(void const * const t) const = 0; + // Constructor + void_caster( + extended_type_info const * derived, + extended_type_info const * base, + std::ptrdiff_t difference = 0, + void_caster const * const parent = 0 + ) : + m_derived(derived), + m_base(base), + m_difference(difference), + m_parent(parent) + {} + virtual ~void_caster(){} +}; + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable : 4251 4231 4660 4275 4511 4512) +#endif + +template +class BOOST_SYMBOL_VISIBLE void_caster_primitive : + public void_caster +{ + virtual void const * downcast(void const * const t) const { + const Derived * d = + boost::serialization::smart_cast( + static_cast(t) + ); + return d; + } + virtual void const * upcast(void const * const t) const { + const Base * b = + boost::serialization::smart_cast( + static_cast(t) + ); + return b; + } + virtual bool has_virtual_base() const { + return false; + } +public: + void_caster_primitive(); + virtual ~void_caster_primitive(); +}; + +template +void_caster_primitive::void_caster_primitive() : + void_caster( + & type_info_implementation::type::get_const_instance(), + & type_info_implementation::type::get_const_instance(), + // note:I wanted to displace from 0 here, but at least one compiler + // treated 0 by not shifting it at all. + reinterpret_cast( + static_cast( + reinterpret_cast(8) + ) + ) - 8 + ) +{ + recursive_register(); +} + +template +void_caster_primitive::~void_caster_primitive(){ + recursive_unregister(); +} + +template +class BOOST_SYMBOL_VISIBLE void_caster_virtual_base : + public void_caster +{ + virtual bool has_virtual_base() const { + return true; + } +public: + virtual void const * downcast(void const * const t) const { + const Derived * d = + dynamic_cast( + static_cast(t) + ); + return d; + } + virtual void const * upcast(void const * const t) const { + const Base * b = + dynamic_cast( + static_cast(t) + ); + return b; + } + void_caster_virtual_base(); + virtual ~void_caster_virtual_base(); +}; + +#ifdef BOOST_MSVC +#pragma warning(pop) +#endif + +template +void_caster_virtual_base::void_caster_virtual_base() : + void_caster( + & (type_info_implementation::type::get_const_instance()), + & (type_info_implementation::type::get_const_instance()) + ) +{ + recursive_register(true); +} + +template +void_caster_virtual_base::~void_caster_virtual_base(){ + recursive_unregister(); +} + +template +struct BOOST_SYMBOL_VISIBLE void_caster_base : + public void_caster +{ + typedef + typename mpl::eval_if, + mpl::identity< + void_cast_detail::void_caster_virtual_base + > + ,// else + mpl::identity< + void_cast_detail::void_caster_primitive + > + >::type type; +}; + +} // void_cast_detail + +template +BOOST_DLLEXPORT +inline const void_cast_detail::void_caster & void_cast_register( + Derived const * /* dnull = NULL */, + Base const * /* bnull = NULL */ +){ + typedef + typename mpl::eval_if, + mpl::identity< + void_cast_detail::void_caster_virtual_base + > + ,// else + mpl::identity< + void_cast_detail::void_caster_primitive + > + >::type typex; + return singleton::get_const_instance(); +} + +template +class BOOST_SYMBOL_VISIBLE void_caster : + public void_cast_detail::void_caster_base::type +{ +}; + +} // namespace serialization +} // namespace boost + +#ifdef BOOST_MSVC +# pragma warning(pop) +#endif + +#include // pops abi_suffix.hpp pragmas + +#endif // BOOST_SERIALIZATION_VOID_CAST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/void_cast_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/void_cast_fwd.hpp new file mode 100644 index 00000000000..def61d52bb7 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/void_cast_fwd.hpp @@ -0,0 +1,37 @@ +#ifndef BOOST_SERIALIZATION_VOID_CAST_FWD_HPP +#define BOOST_SERIALIZATION_VOID_CAST_FWD_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// void_cast_fwd.hpp: interface for run-time casting of void pointers. + +// (C) Copyright 2005 Robert Ramey - http://www.rrsd.com . +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) +// gennadiy.rozental@tfn.com + +// See http://www.boost.org for updates, documentation, and revision history. + +#include // NULL +#include + +namespace boost { +namespace serialization { +namespace void_cast_detail{ +class void_caster; +} // namespace void_cast_detail +template +BOOST_DLLEXPORT +inline const void_cast_detail::void_caster & void_cast_register( + const Derived * dnull = NULL, + const Base * bnull = NULL +) BOOST_USED; +} // namespace serialization +} // namespace boost + +#endif // BOOST_SERIALIZATION_VOID_CAST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/weak_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/weak_ptr.hpp new file mode 100644 index 00000000000..6952d24cb37 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/weak_ptr.hpp @@ -0,0 +1,99 @@ +#ifndef BOOST_SERIALIZATION_WEAK_PTR_HPP +#define BOOST_SERIALIZATION_WEAK_PTR_HPP + +// MS compatible compilers support #pragma once +#if defined(_MSC_VER) +# pragma once +#endif + +/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 +// weak_ptr.hpp: serialization for boost weak pointer + +// (C) Copyright 2004 Robert Ramey and Martin Ecker +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +// See http://www.boost.org for updates, documentation, and revision history. + +#include +#include + +namespace boost { +namespace serialization{ + +template +inline void save( + Archive & ar, + const boost::weak_ptr< T > &t, + const unsigned int /* file_version */ +){ + const boost::shared_ptr< T > sp = t.lock(); + ar << boost::serialization::make_nvp("weak_ptr", sp); +} + +template +inline void load( + Archive & ar, + boost::weak_ptr< T > &t, + const unsigned int /* file_version */ +){ + boost::shared_ptr< T > sp; + ar >> boost::serialization::make_nvp("weak_ptr", sp); + t = sp; +} + +template +inline void serialize( + Archive & ar, + boost::weak_ptr< T > &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +#ifndef BOOST_NO_CXX11_SMART_PTR +#include + +namespace boost { +namespace serialization{ + +template +inline void save( + Archive & ar, + const std::weak_ptr< T > &t, + const unsigned int /* file_version */ +){ + const std::shared_ptr< T > sp = t.lock(); + ar << boost::serialization::make_nvp("weak_ptr", sp); +} + +template +inline void load( + Archive & ar, + std::weak_ptr< T > &t, + const unsigned int /* file_version */ +){ + std::shared_ptr< T > sp; + ar >> boost::serialization::make_nvp("weak_ptr", sp); + t = sp; +} + +template +inline void serialize( + Archive & ar, + std::weak_ptr< T > &t, + const unsigned int file_version +){ + boost::serialization::split_free(ar, t, file_version); +} + +} // namespace serialization +} // namespace boost + +#endif // BOOST_NO_CXX11_SMART_PTR + +#endif // BOOST_SERIALIZATION_WEAK_PTR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/wrapper.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/wrapper.hpp new file mode 100644 index 00000000000..60d7910b17a --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/serialization/wrapper.hpp @@ -0,0 +1,60 @@ +#ifndef BOOST_SERIALIZATION_WRAPPER_HPP +#define BOOST_SERIALIZATION_WRAPPER_HPP + +// (C) Copyright 2005-2006 Matthias Troyer +// Use, modification and distribution is subject to the Boost Software +// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at +// http://www.boost.org/LICENSE_1_0.txt) + +#include +#include +#include +#include + +namespace boost { namespace serialization { + +/// the base class for serialization wrappers +/// +/// wrappers need to be treated differently at various places in the serialization library, +/// e.g. saving of non-const wrappers has to be possible. Since partial specialization +// is not supported by all compilers, we derive all wrappers from wrapper_traits. + +template< + class T, + int Level = object_serializable, + int Tracking = track_never, + unsigned int Version = 0, + class ETII = extended_type_info_impl< T > +> +struct wrapper_traits : + public traits +{}; + +template +struct is_wrapper_impl : + boost::mpl::eval_if< + boost::is_base_and_derived, + boost::mpl::true_, + boost::mpl::false_ + >::type +{}; + +template +struct is_wrapper { + typedef typename is_wrapper_impl::type type; +}; + +} // serialization +} // boost + +// A macro to define that a class is a wrapper +#define BOOST_CLASS_IS_WRAPPER(T) \ +namespace boost { \ +namespace serialization { \ +template<> \ +struct is_wrapper_impl : boost::mpl::true_ {}; \ +} \ +} \ +/**/ + +#endif //BOOST_SERIALIZATION_WRAPPER_HPP From 1b248be8c9c7109e26205a26d594b1d79657c4b1 Mon Sep 17 00:00:00 2001 From: robot-metrika-test Date: Thu, 30 Nov 2017 20:25:34 +0300 Subject: [PATCH 017/509] Auto version update to [54318] --- dbms/cmake/version.cmake | 4 ++-- debian/changelog | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index be5d4cf6005..6751aebb320 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -1,6 +1,6 @@ # This strings autochanged from release_lib.sh: -set(VERSION_DESCRIBE v1.1.54310-testing) -set(VERSION_REVISION 54310) +set(VERSION_DESCRIBE v1.1.54318-testing) +set(VERSION_REVISION 54318) # end of autochange set (VERSION_MAJOR 1) diff --git a/debian/changelog b/debian/changelog index cff5905660c..3530f559dc3 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (1.1.54310) unstable; urgency=low +clickhouse (1.1.54318) unstable; urgency=low * Modified source code - -- Wed, 01 Nov 2017 08:03:17 +0300 + -- Thu, 30 Nov 2017 20:25:34 +0300 From f24bedd2b8e98d44aa8a5b49c22891cbf78e0e51 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 12 May 2019 14:36:02 +0300 Subject: [PATCH 018/509] Added constraints to parsers --- dbms/src/Parsers/ASTConstraintDeclaration.h | 38 ++++++++++++++ dbms/src/Parsers/ASTCreateQuery.cpp | 12 +++++ dbms/src/Parsers/ASTCreateQuery.h | 1 + dbms/src/Parsers/ParserCreateQuery.cpp | 55 ++++++++++++++++++--- dbms/src/Parsers/ParserCreateQuery.h | 22 +++++++-- 5 files changed, 118 insertions(+), 10 deletions(-) create mode 100644 dbms/src/Parsers/ASTConstraintDeclaration.h diff --git a/dbms/src/Parsers/ASTConstraintDeclaration.h b/dbms/src/Parsers/ASTConstraintDeclaration.h new file mode 100644 index 00000000000..cfa4f8f98ab --- /dev/null +++ b/dbms/src/Parsers/ASTConstraintDeclaration.h @@ -0,0 +1,38 @@ +#pragma once + +#include + +namespace DB +{ + +/** name CHECK logical_expr + */ +class ASTConstraintDeclaration : public IAST { +public: + String name; + IAST *expr; + + String getID(char) const override { return "Constraint"; } + + ASTPtr clone() const override { + auto res = std::make_shared(); + + res->name = name; + + if (expr) + res->set(res->expr, expr->clone()); + + return res; + } + + void formatImpl(const FormatSettings &s, FormatState &state, FormatStateStacked frame) const override { + frame.need_parens = false; + std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' '); + + s.ostr << s.nl_or_ws << indent_str; + s.ostr << backQuoteIfNeed(name); + s.ostr << (s.hilite ? hilite_keyword : "") << " CHECK " << (s.hilite ? hilite_none : ""); + expr->formatImpl(s, state, frame); + } +}; +} diff --git a/dbms/src/Parsers/ASTCreateQuery.cpp b/dbms/src/Parsers/ASTCreateQuery.cpp index e99c543f5ec..b60eceb5167 100644 --- a/dbms/src/Parsers/ASTCreateQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuery.cpp @@ -128,6 +128,8 @@ ASTPtr ASTColumns::clone() const res->set(res->columns, columns->clone()); if (indices) res->set(res->indices, indices->clone()); + if (constraints) + res->set(res->constraints, constraints->clone()); return res; } @@ -156,6 +158,16 @@ void ASTColumns::formatImpl(const FormatSettings & s, FormatState & state, Forma list.children.push_back(elem); } } + if (constraints) + { + for (const auto & constraint : constraints->children) + { + auto elem = std::make_shared(); + elem->prefix = "CONSTRAINT"; + elem->set(elem->elem, constraint->clone()); + list.children.push_back(elem); + } + } if (!list.children.empty()) list.formatImpl(s, state, frame); diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 2755e1a3d78..b6948c19146 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -36,6 +36,7 @@ class ASTColumns : public IAST public: ASTExpressionList * columns = nullptr; ASTExpressionList * indices = nullptr; + ASTExpressionList * constraints = nullptr; String getID(char) const override { return "Columns definition"; } diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index fd6665a5a2c..9f584dbbf8c 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB @@ -137,12 +138,41 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return true; } +bool ParserConstraintDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_check("CHECK"); -bool ParserColumnAndIndexDeclaraion::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) + ParserIdentifier name_p; + ParserLogicalOrExpression expression_p; + + ASTPtr name; + ASTPtr expr; + + if (!name_p.parse(pos, name, expected)) + return false; + + if (!s_check.ignore(pos, expected)) + return false; + + if (!expression_p.parse(pos, expr, expected)) + return false; + + auto constraint = std::make_shared(); + constraint->name = name->as().name; + constraint->set(constraint->expr, expr); + node = constraint; + + return true; +} + + +bool ParserTablePropertyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_index("INDEX"); + ParserKeyword s_constraint("CONSTRAINT"); ParserIndexDeclaration index_p; + ParserConstraintDeclaration constraint_p; ParserColumnDeclaration column_p; ASTPtr new_node = nullptr; @@ -152,6 +182,11 @@ bool ParserColumnAndIndexDeclaraion::parseImpl(Pos & pos, ASTPtr & node, Expecte if (!index_p.parse(pos, new_node, expected)) return false; } + else if (s_constraint.ignore(pos, expected)) + { + if (!constraint_p.parse(pos, new_node, expected)) + return false; + } else { if (!column_p.parse(pos, new_node, expected)) @@ -168,16 +203,18 @@ bool ParserIndexDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & .parse(pos, node, expected); } - -bool ParserColumnsOrIndicesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr list; - if (!ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) + if (!ParserList( + std::make_unique(), + std::make_unique(TokenType::Comma), false) .parse(pos, list, expected)) return false; ASTPtr columns = std::make_shared(); ASTPtr indices = std::make_shared(); + ASTPtr constraints = std::make_shared(); for (const auto & elem : list->children) { @@ -185,6 +222,8 @@ bool ParserColumnsOrIndicesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, columns->children.push_back(elem); else if (elem->as()) indices->children.push_back(elem); + else if (elem->as()) + constraints->children.push_back(elem); else return false; } @@ -195,6 +234,8 @@ bool ParserColumnsOrIndicesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, res->set(res->columns, columns); if (!indices->children.empty()) res->set(res->indices, indices); + if (!constraints->children.empty()) + res->set(res->constraints, constraints); node = res; @@ -317,7 +358,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserToken s_rparen(TokenType::ClosingRoundBracket); ParserStorage storage_p; ParserIdentifier name_p; - ParserColumnsOrIndicesDeclarationList columns_or_indices_p; + ParserTablePropertiesDeclarationList table_properties_p; ParserSelectWithUnionQuery select_p; ASTPtr database; @@ -391,7 +432,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// List of columns. if (s_lparen.ignore(pos, expected)) { - if (!columns_or_indices_p.parse(pos, columns_list, expected)) + if (!table_properties_p.parse(pos, columns_list, expected)) return false; if (!s_rparen.ignore(pos, expected)) @@ -498,7 +539,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// Optional - a list of columns can be specified. It must fully comply with SELECT. if (s_lparen.ignore(pos, expected)) { - if (!columns_or_indices_p.parse(pos, columns_list, expected)) + if (!table_properties_p.parse(pos, columns_list, expected)) return false; if (!s_rparen.ignore(pos, expected)) diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index bd3c8f671f0..42583d8dd19 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -244,11 +244,20 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +class ParserConstraintDeclaration : public IParserBase +{ +public: + ParserConstraintDeclaration() {} -class ParserColumnAndIndexDeclaraion : public IParserBase +protected: + const char * getName() const override { return "constraint declaration"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +class ParserTablePropertyDeclaration : public IParserBase { protected: - const char * getName() const override { return "column or index declaration"; } + const char * getName() const override { return "table propery (column, index, constraint) declaration"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; @@ -260,8 +269,15 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +class ParserConstraintDeclarationList : public IParserBase +{ +protected: + const char * getName() const override { return "constraint declaration list"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; -class ParserColumnsOrIndicesDeclarationList : public IParserBase + +class ParserTablePropertiesDeclarationList : public IParserBase { protected: const char * getName() const override { return "columns or indices declaration list"; } From 502d86bd022bc53f8f4f9452cd4d1201e849aa4c Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Fri, 17 May 2019 07:08:03 +0300 Subject: [PATCH 019/509] Added constraints to InterpreterCreateQuery and storages --- .../Interpreters/InterpreterCreateQuery.cpp | 23 +++++++++++ .../src/Interpreters/InterpreterCreateQuery.h | 2 + dbms/src/Storages/ConstraintsDescription.cpp | 39 +++++++++++++++++++ dbms/src/Storages/ConstraintsDescription.h | 23 +++++++++++ dbms/src/Storages/ITableDeclaration.cpp | 5 +++ dbms/src/Storages/ITableDeclaration.h | 5 +++ dbms/src/Storages/StorageMergeTree.h | 3 ++ 7 files changed, 100 insertions(+) create mode 100644 dbms/src/Storages/ConstraintsDescription.cpp create mode 100644 dbms/src/Storages/ConstraintsDescription.h diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 4bc35f1e378..e4e5a8e7f83 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -252,6 +252,16 @@ ASTPtr InterpreterCreateQuery::formatIndices(const IndicesDescription & indices) return res; } +ASTPtr InterpreterCreateQuery::formatConstraints(const ConstraintsDescription & constraints) +{ + auto res = std::make_shared(); + + for (const auto & constraint : constraints.constraints) + res->children.push_back(constraint->clone()); + + return res; +} + ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpressionList & columns_ast, const Context & context) { /// First, deduce implicit types. @@ -370,6 +380,8 @@ ColumnsDescription InterpreterCreateQuery::setColumns( { ColumnsDescription columns; IndicesDescription indices; + ConstraintsDescription constraints; + if (create.columns_list) { @@ -379,11 +391,16 @@ ColumnsDescription InterpreterCreateQuery::setColumns( for (const auto & index : create.columns_list->indices->children) indices.indices.push_back( std::dynamic_pointer_cast(index->clone())); + if (create.columns_list->constraints) + for (const auto & constraint : create.columns_list->constraints->children) + constraints.constraints.push_back( + std::dynamic_pointer_cast(constraint->clone())); } else if (!create.as_table.empty()) { columns = as_storage->getColumns(); indices = as_storage->getIndicesDescription(); + constraints = as_storage->getConstraintsDescription(); } else if (create.select) { @@ -395,6 +412,7 @@ ColumnsDescription InterpreterCreateQuery::setColumns( /// Even if query has list of columns, canonicalize it (unfold Nested columns). ASTPtr new_columns = formatColumns(columns); ASTPtr new_indices = formatIndices(indices); + ASTPtr new_constraints = formatConstraints(constraints); if (!create.columns_list) { @@ -412,6 +430,11 @@ ColumnsDescription InterpreterCreateQuery::setColumns( else if (new_indices) create.columns_list->set(create.columns_list->indices, new_indices); + if (new_constraints && create.columns_list->constraints) + create.columns_list->replace(create.columns_list->constraints, new_constraints); + else if (new_constraints) + create.columns_list->set(create.columns_list->constraints, new_constraints); + /// Check for duplicates std::set all_columns; for (const auto & column : columns) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.h b/dbms/src/Interpreters/InterpreterCreateQuery.h index 2f124e7df9b..a7886f644ad 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.h +++ b/dbms/src/Interpreters/InterpreterCreateQuery.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -31,6 +32,7 @@ public: static ASTPtr formatColumns(const ColumnsDescription & columns); static ASTPtr formatIndices(const IndicesDescription & indices); + static ASTPtr formatConstraints(const ConstraintsDescription & constraints); void setDatabaseLoadingThreadpool(ThreadPool & thread_pool_) { diff --git a/dbms/src/Storages/ConstraintsDescription.cpp b/dbms/src/Storages/ConstraintsDescription.cpp new file mode 100644 index 00000000000..042ee06ff59 --- /dev/null +++ b/dbms/src/Storages/ConstraintsDescription.cpp @@ -0,0 +1,39 @@ +#include + +#include +#include +#include +#include + + +namespace DB +{ + +String ConstraintsDescription::toString() const +{ + if (constraints.empty()) + return {}; + + ASTExpressionList list; + for (const auto & constraint : constraints) + list.children.push_back(constraint); + + return serializeAST(list, true); +} + +ConstraintsDescription ConstraintsDescription::parse(const String & str) +{ + if (str.empty()) + return {}; + + ConstraintsDescription res; + ParserConstraintDeclarationList parser; + ASTPtr list = parseQuery(parser, str, 0); + + for (const auto & constraint : list->children) + res.constraints.push_back(std::dynamic_pointer_cast(constraint)); + + return res; +} + +} diff --git a/dbms/src/Storages/ConstraintsDescription.h b/dbms/src/Storages/ConstraintsDescription.h new file mode 100644 index 00000000000..c2954d94428 --- /dev/null +++ b/dbms/src/Storages/ConstraintsDescription.h @@ -0,0 +1,23 @@ +#pragma once + +#include + + +namespace DB +{ + +using ConstraintsASTs = std::vector>; + +struct ConstraintsDescription +{ + ConstraintsASTs constraints; + + ConstraintsDescription() = default; + + bool empty() const { return constraints.empty(); } + String toString() const; + + static ConstraintsDescription parse(const String & str); +}; + +} diff --git a/dbms/src/Storages/ITableDeclaration.cpp b/dbms/src/Storages/ITableDeclaration.cpp index c9385c24cbe..b24072f0de8 100644 --- a/dbms/src/Storages/ITableDeclaration.cpp +++ b/dbms/src/Storages/ITableDeclaration.cpp @@ -36,6 +36,11 @@ void ITableDeclaration::setIndicesDescription(IndicesDescription indices_) indices = std::move(indices_); } +void ITableDeclaration::setConstraintsDescription(ConstraintsDescription constraints_) +{ + constraints = std::move(constraints_); +} + bool ITableDeclaration::hasColumn(const String & column_name) const { diff --git a/dbms/src/Storages/ITableDeclaration.h b/dbms/src/Storages/ITableDeclaration.h index e2ac3b1d6c4..e1e6d88a68f 100644 --- a/dbms/src/Storages/ITableDeclaration.h +++ b/dbms/src/Storages/ITableDeclaration.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -19,6 +20,9 @@ public: virtual const IndicesDescription & getIndicesDescription() const { return indices; } virtual void setIndicesDescription(IndicesDescription indices_); + virtual const ConstraintsDescription & getConstraintsDescription() const { return constraints; } + virtual void setConstraintsDescription(ConstraintsDescription constraints_); + /// NOTE: These methods should include virtual columns, but should NOT include ALIAS columns /// (they are treated separately). virtual NameAndTypePair getColumn(const String & column_name) const; @@ -57,6 +61,7 @@ public: private: ColumnsDescription columns; IndicesDescription indices; + ConstraintsDescription constraints; }; } diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index c5ea2d8c3a0..0b4b00b7cb7 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -46,6 +46,9 @@ public: virtual const IndicesDescription & getIndicesDescription() const override { return data.getIndicesDescription(); } virtual void setIndicesDescription(IndicesDescription indices_) override { data.setIndicesDescription(std::move(indices_)); } + virtual const ConstraintsDescription & getConstraintsDescription() const override { return data.getConstraintsDescription(); } + virtual void setConstraintsDescription(ConstraintsDescription constraints_) override { data.setConstraintsDescription(constraints_; )} + NameAndTypePair getColumn(const String & column_name) const override { return data.getColumn(column_name); } bool hasColumn(const String & column_name) const override { return data.hasColumn(column_name); } From 2f8864a6ea96f124af79914fd351fe998d47b538 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Fri, 17 May 2019 07:14:13 +0300 Subject: [PATCH 020/509] Some style fixes --- dbms/src/Parsers/ASTConstraintDeclaration.h | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/dbms/src/Parsers/ASTConstraintDeclaration.h b/dbms/src/Parsers/ASTConstraintDeclaration.h index cfa4f8f98ab..d72358be498 100644 --- a/dbms/src/Parsers/ASTConstraintDeclaration.h +++ b/dbms/src/Parsers/ASTConstraintDeclaration.h @@ -7,14 +7,16 @@ namespace DB /** name CHECK logical_expr */ -class ASTConstraintDeclaration : public IAST { +class ASTConstraintDeclaration : public IAST +{ public: String name; IAST *expr; String getID(char) const override { return "Constraint"; } - ASTPtr clone() const override { + ASTPtr clone() const override + { auto res = std::make_shared(); res->name = name; @@ -25,7 +27,8 @@ public: return res; } - void formatImpl(const FormatSettings &s, FormatState &state, FormatStateStacked frame) const override { + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override + { frame.need_parens = false; std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' '); From 72d5a4634c10d4d1e846867c22da3061fe20417e Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Fri, 17 May 2019 07:27:09 +0300 Subject: [PATCH 021/509] Syntax fix --- dbms/src/Storages/StorageMergeTree.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 0b4b00b7cb7..184db1a858b 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -47,7 +47,7 @@ public: virtual void setIndicesDescription(IndicesDescription indices_) override { data.setIndicesDescription(std::move(indices_)); } virtual const ConstraintsDescription & getConstraintsDescription() const override { return data.getConstraintsDescription(); } - virtual void setConstraintsDescription(ConstraintsDescription constraints_) override { data.setConstraintsDescription(constraints_; )} + virtual void setConstraintsDescription(ConstraintsDescription constraints_) override { data.setConstraintsDescription(constraints_); } NameAndTypePair getColumn(const String & column_name) const override { return data.getColumn(column_name); } bool hasColumn(const String & column_name) const override { return data.hasColumn(column_name); } From 07abed6d31b2a024527f34601c457de40bdcc220 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Fri, 17 May 2019 08:05:19 +0300 Subject: [PATCH 022/509] Renamed constraints getter and setter --- dbms/src/Interpreters/InterpreterCreateQuery.cpp | 2 +- dbms/src/Storages/ITableDeclaration.cpp | 2 +- dbms/src/Storages/ITableDeclaration.h | 4 ++-- dbms/src/Storages/StorageMergeTree.h | 4 ++-- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index e9027f473e5..15968d58ac5 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -400,7 +400,7 @@ ColumnsDescription InterpreterCreateQuery::setColumns( { columns = as_storage->getColumns(); indices = as_storage->getIndices(); - constraints = as_storage->getConstraintsDescription(); + constraints = as_storage->getConstraints(); } else if (create.select) { diff --git a/dbms/src/Storages/ITableDeclaration.cpp b/dbms/src/Storages/ITableDeclaration.cpp index cb9ba9daf9a..47d24300452 100644 --- a/dbms/src/Storages/ITableDeclaration.cpp +++ b/dbms/src/Storages/ITableDeclaration.cpp @@ -46,7 +46,7 @@ void ITableDeclaration::setIndices(IndicesDescription indices_) indices = std::move(indices_); } -void ITableDeclaration::setConstraintsDescription(ConstraintsDescription constraints_) +void ITableDeclaration::setConstraints(ConstraintsDescription constraints_) { constraints = std::move(constraints_); } diff --git a/dbms/src/Storages/ITableDeclaration.h b/dbms/src/Storages/ITableDeclaration.h index 6d50f8acc0f..52db660b19b 100644 --- a/dbms/src/Storages/ITableDeclaration.h +++ b/dbms/src/Storages/ITableDeclaration.h @@ -20,8 +20,8 @@ public: const IndicesDescription & getIndices() const; void setIndices(IndicesDescription indices_); - virtual const ConstraintsDescription & getConstraintsDescription() const { return constraints; } - virtual void setConstraintsDescription(ConstraintsDescription constraints_); + virtual const ConstraintsDescription & getConstraints() const { return constraints; } + virtual void setConstraints(ConstraintsDescription constraints_); /// NOTE: These methods should include virtual columns, but should NOT include ALIAS columns /// (they are treated separately). diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 2c42bbd35c5..18386d78b8c 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -32,8 +32,8 @@ public: std::string getDatabaseName() const override { return database_name; } bool supportsIndexForIn() const override { return true; } - virtual const ConstraintsDescription & getConstraintsDescription() const override { return data.getConstraintsDescription(); } - virtual void setConstraintsDescription(ConstraintsDescription constraints_) override { data.setConstraintsDescription(constraints_); } + virtual const ConstraintsDescription & getConstraints() const override { return data.getConstraints(); } + virtual void setConstraints(ConstraintsDescription constraints_) override { data.setConstraints(constraints_); } BlockInputStreams read( const Names & column_names, From 7919a62198ce97493f1f151914ea69765423037c Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Fri, 17 May 2019 09:21:29 +0300 Subject: [PATCH 023/509] Removed constraints getter and setter from StorageMergeTree --- dbms/src/Storages/StorageMergeTree.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 18386d78b8c..b5156ce7137 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -32,8 +32,6 @@ public: std::string getDatabaseName() const override { return database_name; } bool supportsIndexForIn() const override { return true; } - virtual const ConstraintsDescription & getConstraints() const override { return data.getConstraints(); } - virtual void setConstraints(ConstraintsDescription constraints_) override { data.setConstraints(constraints_); } BlockInputStreams read( const Names & column_names, From 1f5715b985125d8820d06c5a23a98d290d278cdb Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sat, 18 May 2019 08:16:33 +0300 Subject: [PATCH 024/509] Removed double whitespace --- dbms/src/Parsers/ParserCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 9f584dbbf8c..76150b95b07 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -143,7 +143,7 @@ bool ParserConstraintDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & ParserKeyword s_check("CHECK"); ParserIdentifier name_p; - ParserLogicalOrExpression expression_p; + ParserLogicalOrExpression expression_p; ASTPtr name; ASTPtr expr; From ce9389660853759cb7566c7bf2a932894ccd06de Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sat, 18 May 2019 11:05:52 +0300 Subject: [PATCH 025/509] Added constraints description to MergeTree and related storages --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 ++ dbms/src/Storages/MergeTree/MergeTreeData.h | 1 + .../Storages/MergeTree/registerStorageMergeTree.cpp | 10 +++++++--- dbms/src/Storages/StorageMergeTree.cpp | 3 ++- dbms/src/Storages/StorageMergeTree.h | 1 + dbms/src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- dbms/src/Storages/StorageReplicatedMergeTree.h | 1 + 7 files changed, 16 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index ea51159d9ba..4f09547a578 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -93,6 +93,7 @@ MergeTreeData::MergeTreeData( const String & database_, const String & table_, const String & full_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, + const ConstraintsDescription & constraints_, Context & context_, const String & date_column_name, const ASTPtr & partition_by_ast_, @@ -121,6 +122,7 @@ MergeTreeData::MergeTreeData( data_parts_by_state_and_info(data_parts_indexes.get()) { setPrimaryKeyIndicesAndColumns(order_by_ast_, primary_key_ast_, columns_, indices_); + setConstraints(constraints_); /// NOTE: using the same columns list as is read when performing actual merges. merging_params.check(getColumns().getAllPhysical()); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index fecddb28540..294588721ae 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -332,6 +332,7 @@ public: const String & full_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, + const ConstraintsDescription & constraints_, Context & context_, const String & date_column_name, const ASTPtr & partition_by_ast_, diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index b23a2eedc0e..b255f16c327 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -574,6 +574,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) ASTPtr sample_by_ast; ASTPtr ttl_table_ast; IndicesDescription indices_description; + ConstraintsDescription constraints_description; MergeTreeSettings storage_settings = args.context.getMergeTreeSettings(); if (is_extended_storage_def) @@ -602,7 +603,10 @@ static StoragePtr create(const StorageFactory::Arguments & args) indices_description.indices.push_back( std::dynamic_pointer_cast(index->clone())); - + if (args.query.columns_list && args.query.columns_list->constraints) + for (const auto & constraint : args.query.columns_list->constraints->children) + constraints_description.constraints.push_back( + std::dynamic_pointer_cast(constraint->clone())); storage_settings.loadFromQuery(*args.storage_def); } else @@ -639,14 +643,14 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) return StorageReplicatedMergeTree::create( zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name, - args.columns, indices_description, + args.columns, indices_description, constraints_description, args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast, sample_by_ast, ttl_table_ast, merging_params, storage_settings, args.has_force_restore_data_flag); else return StorageMergeTree::create( args.data_path, args.database_name, args.table_name, args.columns, indices_description, - args.attach, args.context, date_column_name, partition_by_ast, order_by_ast, + constraints_description, args.attach, args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast, sample_by_ast, ttl_table_ast, merging_params, storage_settings, args.has_force_restore_data_flag); } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 8feb2d1fe81..12e2ffc77af 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -53,6 +53,7 @@ StorageMergeTree::StorageMergeTree( const String & table_name_, const ColumnsDescription & columns_, const IndicesDescription & indices_, + const ConstraintsDescription & constraints_, bool attach, Context & context_, const String & date_column_name, @@ -66,7 +67,7 @@ StorageMergeTree::StorageMergeTree( bool has_force_restore_data_flag) : MergeTreeData(database_name_, table_name_, path_ + escapeForFileName(table_name_) + '/', - columns_, indices_, + columns_, indices_, constraints_, context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, sample_by_ast_, ttl_table_ast_, merging_params_, settings_, false, attach), diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index b5156ce7137..74e0d85bfb8 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -139,6 +139,7 @@ protected: const String & table_name_, const ColumnsDescription & columns_, const IndicesDescription & indices_, + const ConstraintsDescription & constraints_, bool attach, Context & context_, const String & date_column_name, diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index daa1b0d10e2..8d467418f12 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -199,6 +199,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const String & table_name_, const ColumnsDescription & columns_, const IndicesDescription & indices_, + const ConstraintsDescription & constraints_, Context & context_, const String & date_column_name, const ASTPtr & partition_by_ast_, @@ -211,7 +212,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( bool has_force_restore_data_flag) : MergeTreeData(database_name_, table_name_, path_ + escapeForFileName(table_name_) + '/', - columns_, indices_, + columns_, indices_, constraints_, context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, sample_by_ast_, ttl_table_ast_, merging_params_, settings_, true, attach, diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index eba0511e15e..cd7d043a54a 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -513,6 +513,7 @@ protected: const String & path_, const String & database_name_, const String & name_, const ColumnsDescription & columns_, const IndicesDescription & indices_, + const ConstraintsDescription & constraints_, Context & context_, const String & date_column_name, const ASTPtr & partition_by_ast_, From 300ec160f40fffb48bf160fe82a11781819ba30e Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 19 May 2019 08:27:00 +0300 Subject: [PATCH 026/509] Constraints MVP --- .../CheckConstraintsBlockOutputStream.cpp | 43 +++++++++++++++++ .../CheckConstraintsBlockOutputStream.h | 48 +++++++++++++++++++ .../Interpreters/InterpreterInsertQuery.cpp | 4 ++ dbms/src/Storages/ConstraintsDescription.h | 14 +++++- 4 files changed, 108 insertions(+), 1 deletion(-) create mode 100644 dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp create mode 100644 dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp new file mode 100644 index 00000000000..99f9f9bc90d --- /dev/null +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -0,0 +1,43 @@ +#include + + +namespace DB +{ + +void CheckConstraintsBlockOutputStream::write(const Block & block) +{ + for (auto & constraint_expr: expressions) + if (!checkConstraintOnBlock(block, constraint_expr)) + throw Exception("Some constraints are not satisfied", ErrorCodes::QUERY_WAS_CANCELLED); + output->write(block); +} + +void CheckConstraintsBlockOutputStream::flush() +{ + output->flush(); +} + +void CheckConstraintsBlockOutputStream::writePrefix() +{ + output->writePrefix(); +} + +void CheckConstraintsBlockOutputStream::writeSuffix() +{ + output->writeSuffix(); +} + +bool CheckConstraintsBlockOutputStream::checkConstraintOnBlock(const Block & block, const ExpressionActionsPtr & constraint) +{ + Block res = block; + constraint->execute(res); + assert(block.columns() == res.columns() - 1); + ColumnWithTypeAndName res_column = res.safeGetByPosition(res.columns() - 1); + size_t column_size = res_column.column->size(); + for (size_t i = 0; i < column_size; ++i) + if (!res_column.column->getBool(i)) + return false; + return true; +} + +} diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h new file mode 100644 index 00000000000..623eccc8172 --- /dev/null +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CONSTRAINTS_ARE_NOT_SATISFIED; +} + +class CheckConstraintsBlockOutputStream : public IBlockOutputStream +{ +public: + CheckConstraintsBlockOutputStream( + const BlockOutputStreamPtr & output_, + const Block & header_, + const ConstraintsDescription & constraints_, + const Context & context_) + : output(output_), + header(header_), + constraints(constraints_), + expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())), + context(context_) + { } + + Block getHeader() const override { return header; } + void write(const Block & block) override; + + void flush() override; + + void writePrefix() override; + void writeSuffix() override; + + bool checkConstraintOnBlock(const Block & block, const ExpressionActionsPtr & constraint); + +private: + BlockOutputStreamPtr output; + Block header; + const ConstraintsDescription constraints; + const ConstraintsExpressions expressions; + const Context & context; +}; +} diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index e4391f52247..fa6df1599ea 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -5,6 +5,7 @@ #include #include +#include #include #include #include @@ -117,6 +118,9 @@ BlockIO InterpreterInsertQuery::execute() out = std::make_shared( out, query_sample_block, table->getSampleBlock(), table->getColumns().getDefaults(), context); + out = std::make_shared( + out, query_sample_block, table->getConstraints(), context); + auto out_wrapper = std::make_shared(out); out_wrapper->setProcessListElement(context.getProcessListElement()); out = std::move(out_wrapper); diff --git a/dbms/src/Storages/ConstraintsDescription.h b/dbms/src/Storages/ConstraintsDescription.h index c2954d94428..fbb0f5167fc 100644 --- a/dbms/src/Storages/ConstraintsDescription.h +++ b/dbms/src/Storages/ConstraintsDescription.h @@ -1,12 +1,13 @@ #pragma once #include - +#include namespace DB { using ConstraintsASTs = std::vector>; +using ConstraintsExpressions = std::vector; struct ConstraintsDescription { @@ -18,6 +19,17 @@ struct ConstraintsDescription String toString() const; static ConstraintsDescription parse(const String & str); + + ConstraintsExpressions getExpressions(const Context & context, const NamesAndTypesList & source_columns_) const { + ConstraintsExpressions res; + res.reserve(constraints.size()); + for (const auto & constraint : constraints) { + ASTPtr expr = constraint->expr->clone(); + auto syntax_result = SyntaxAnalyzer(context).analyze(expr, source_columns_); + res.push_back(ExpressionAnalyzer(constraint->expr->clone(), syntax_result, context).getActions(false)); + } + return res; + } }; } From 60a5b94ba47ee4050dbb5d38822ceb2d15a479f2 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 19 May 2019 08:44:31 +0300 Subject: [PATCH 027/509] Style fix --- dbms/src/Storages/ConstraintsDescription.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/ConstraintsDescription.h b/dbms/src/Storages/ConstraintsDescription.h index fbb0f5167fc..0f565379204 100644 --- a/dbms/src/Storages/ConstraintsDescription.h +++ b/dbms/src/Storages/ConstraintsDescription.h @@ -20,10 +20,12 @@ struct ConstraintsDescription static ConstraintsDescription parse(const String & str); - ConstraintsExpressions getExpressions(const Context & context, const NamesAndTypesList & source_columns_) const { + ConstraintsExpressions getExpressions(const Context & context, const NamesAndTypesList & source_columns_) const + { ConstraintsExpressions res; res.reserve(constraints.size()); - for (const auto & constraint : constraints) { + for (const auto & constraint : constraints) + { ASTPtr expr = constraint->expr->clone(); auto syntax_result = SyntaxAnalyzer(context).analyze(expr, source_columns_); res.push_back(ExpressionAnalyzer(constraint->expr->clone(), syntax_result, context).getActions(false)); From 19d099f90a147fc385721a525a39249bfb682304 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 19 May 2019 09:03:18 +0300 Subject: [PATCH 028/509] Removed ITableDeclaration --- dbms/src/Storages/ITableDeclaration.cpp | 0 dbms/src/Storages/ITableDeclaration.h | 0 2 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 dbms/src/Storages/ITableDeclaration.cpp delete mode 100644 dbms/src/Storages/ITableDeclaration.h diff --git a/dbms/src/Storages/ITableDeclaration.cpp b/dbms/src/Storages/ITableDeclaration.cpp deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/src/Storages/ITableDeclaration.h b/dbms/src/Storages/ITableDeclaration.h deleted file mode 100644 index e69de29bb2d..00000000000 From 773849a43d9ab25df77c38e00de5d6b914c0b0f2 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 19 May 2019 09:08:25 +0300 Subject: [PATCH 029/509] Added constraints getter and setter to IStorage --- dbms/src/Storages/IStorage.cpp | 10 ++++++++++ dbms/src/Storages/IStorage.h | 5 +++++ 2 files changed, 15 insertions(+) diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 06320cc1f30..7c19fd94aea 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -47,6 +47,16 @@ void IStorage::setIndices(IndicesDescription indices_) indices = std::move(indices_); } +const ConstraintsDescription & IStorage::getConstraints() const +{ + return constraints; +} + +void IStorage::setConstraints(ConstraintsDescription constraints_) +{ + constraints = std::move(constraints_); +} + NameAndTypePair IStorage::getColumn(const String & column_name) const { /// By default, we assume that there are no virtual columns in the storage. diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index f18592ebce5..b01244ba111 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -88,6 +89,9 @@ public: /// thread-unsafe part. lockStructure must be acquired const IndicesDescription & getIndices() const; void setIndices(IndicesDescription indices_); + const ConstraintsDescription & getConstraints() const; + void setConstraints(ConstraintsDescription constraints_); + /// NOTE: these methods should include virtual columns, /// but should NOT include ALIAS columns (they are treated separately). virtual NameAndTypePair getColumn(const String & column_name) const; @@ -115,6 +119,7 @@ public: /// thread-unsafe part. lockStructure must be acquired private: ColumnsDescription columns; IndicesDescription indices; + ConstraintsDescription constraints; public: /// Acquire this lock if you need the table structure to remain constant during the execution of From d1492fc05d534034d1a0ed1322a9e86f1020cc62 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 19 May 2019 10:17:06 +0300 Subject: [PATCH 030/509] Removed context from CheckConstraintsBlockOutputStream --- dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h | 7 ++----- dbms/src/Interpreters/InterpreterInsertQuery.cpp | 3 +-- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h index 623eccc8172..e1e15f8e454 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h @@ -19,13 +19,11 @@ public: CheckConstraintsBlockOutputStream( const BlockOutputStreamPtr & output_, const Block & header_, - const ConstraintsDescription & constraints_, - const Context & context_) + const ConstraintsDescription & constraints_) : output(output_), header(header_), constraints(constraints_), - expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())), - context(context_) + expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())) { } Block getHeader() const override { return header; } @@ -43,6 +41,5 @@ private: Block header; const ConstraintsDescription constraints; const ConstraintsExpressions expressions; - const Context & context; }; } diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index fa6df1599ea..d47dd978b3a 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -118,8 +118,7 @@ BlockIO InterpreterInsertQuery::execute() out = std::make_shared( out, query_sample_block, table->getSampleBlock(), table->getColumns().getDefaults(), context); - out = std::make_shared( - out, query_sample_block, table->getConstraints(), context); + out = std::make_shared(out, query_sample_block, table->getConstraints()); auto out_wrapper = std::make_shared(out); out_wrapper->setProcessListElement(context.getProcessListElement()); From 9e6625441c936a4fef236c34cefa0b0654645364 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 19 May 2019 10:19:44 +0300 Subject: [PATCH 031/509] Returned context to CheckConstraintsBlockOutputStream constructor --- dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h | 3 ++- dbms/src/Interpreters/InterpreterInsertQuery.cpp | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h index e1e15f8e454..16b240eb758 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h @@ -19,7 +19,8 @@ public: CheckConstraintsBlockOutputStream( const BlockOutputStreamPtr & output_, const Block & header_, - const ConstraintsDescription & constraints_) + const ConstraintsDescription & constraints_, + const Context & context_) : output(output_), header(header_), constraints(constraints_), diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index d47dd978b3a..fa6df1599ea 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -118,7 +118,8 @@ BlockIO InterpreterInsertQuery::execute() out = std::make_shared( out, query_sample_block, table->getSampleBlock(), table->getColumns().getDefaults(), context); - out = std::make_shared(out, query_sample_block, table->getConstraints()); + out = std::make_shared( + out, query_sample_block, table->getConstraints(), context); auto out_wrapper = std::make_shared(out); out_wrapper->setProcessListElement(context.getProcessListElement()); From c926cf65627dfbbc724b8b469c254399e6fb8486 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sat, 25 May 2019 17:07:45 +0300 Subject: [PATCH 032/509] Minor review fixes --- dbms/src/Parsers/ASTConstraintDeclaration.cpp | 28 +++++++++++++++++++ dbms/src/Parsers/ASTConstraintDeclaration.h | 25 ++--------------- dbms/src/Parsers/ParserCreateQuery.cpp | 6 ++++ dbms/src/Parsers/ParserCreateQuery.h | 3 -- dbms/src/Storages/ConstraintsDescription.cpp | 15 ++++++++++ dbms/src/Storages/ConstraintsDescription.h | 13 +-------- 6 files changed, 53 insertions(+), 37 deletions(-) create mode 100644 dbms/src/Parsers/ASTConstraintDeclaration.cpp diff --git a/dbms/src/Parsers/ASTConstraintDeclaration.cpp b/dbms/src/Parsers/ASTConstraintDeclaration.cpp new file mode 100644 index 00000000000..834ac81891b --- /dev/null +++ b/dbms/src/Parsers/ASTConstraintDeclaration.cpp @@ -0,0 +1,28 @@ +#include + +namespace DB { + +ASTPtr ASTConstraintDeclaration::clone() const +{ + auto res = std::make_shared(); + + res->name = name; + + if (expr) + res->set(res->expr, expr->clone()); + + return res; +} + +void ASTConstraintDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const +{ + frame.need_parens = false; + std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' '); + + s.ostr << s.nl_or_ws << indent_str; + s.ostr << backQuoteIfNeed(name); + s.ostr << (s.hilite ? hilite_keyword : "") << " CHECK " << (s.hilite ? hilite_none : ""); + expr->formatImpl(s, state, frame); +} + +} \ No newline at end of file diff --git a/dbms/src/Parsers/ASTConstraintDeclaration.h b/dbms/src/Parsers/ASTConstraintDeclaration.h index d72358be498..3a8ad75f54b 100644 --- a/dbms/src/Parsers/ASTConstraintDeclaration.h +++ b/dbms/src/Parsers/ASTConstraintDeclaration.h @@ -11,31 +11,12 @@ class ASTConstraintDeclaration : public IAST { public: String name; - IAST *expr; + IAST * expr; String getID(char) const override { return "Constraint"; } - ASTPtr clone() const override - { - auto res = std::make_shared(); + ASTPtr clone() const override; - res->name = name; - - if (expr) - res->set(res->expr, expr->clone()); - - return res; - } - - void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override - { - frame.need_parens = false; - std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' '); - - s.ostr << s.nl_or_ws << indent_str; - s.ostr << backQuoteIfNeed(name); - s.ostr << (s.hilite ? hilite_keyword : "") << " CHECK " << (s.hilite ? hilite_none : ""); - expr->formatImpl(s, state, frame); - } + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; }; } diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 76150b95b07..c828cd0d780 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -203,6 +203,12 @@ bool ParserIndexDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & .parse(pos, node, expected); } +bool ParserConstraintDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) + .parse(pos, node, expected); +} + bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr list; diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 42583d8dd19..bc921773605 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -246,9 +246,6 @@ protected: class ParserConstraintDeclaration : public IParserBase { -public: - ParserConstraintDeclaration() {} - protected: const char * getName() const override { return "constraint declaration"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; diff --git a/dbms/src/Storages/ConstraintsDescription.cpp b/dbms/src/Storages/ConstraintsDescription.cpp index 042ee06ff59..ad0cd76733a 100644 --- a/dbms/src/Storages/ConstraintsDescription.cpp +++ b/dbms/src/Storages/ConstraintsDescription.cpp @@ -36,4 +36,19 @@ ConstraintsDescription ConstraintsDescription::parse(const String & str) return res; } +ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::Context & context, + const DB::NamesAndTypesList & source_columns_) const +{ + ConstraintsExpressions res; + res.reserve(constraints.size()); + for (const auto & constraint : constraints) + { + // SyntaxAnalyzer::analyze has query as non-const argument so to avoid accidental query changes we clone it + ASTPtr expr = constraint->expr->clone(); + auto syntax_result = SyntaxAnalyzer(context).analyze(expr, source_columns_); + res.push_back(ExpressionAnalyzer(constraint->expr->clone(), syntax_result, context).getActions(false)); + } + return res; +} + } diff --git a/dbms/src/Storages/ConstraintsDescription.h b/dbms/src/Storages/ConstraintsDescription.h index 0f565379204..3ced0e8ddc9 100644 --- a/dbms/src/Storages/ConstraintsDescription.h +++ b/dbms/src/Storages/ConstraintsDescription.h @@ -20,18 +20,7 @@ struct ConstraintsDescription static ConstraintsDescription parse(const String & str); - ConstraintsExpressions getExpressions(const Context & context, const NamesAndTypesList & source_columns_) const - { - ConstraintsExpressions res; - res.reserve(constraints.size()); - for (const auto & constraint : constraints) - { - ASTPtr expr = constraint->expr->clone(); - auto syntax_result = SyntaxAnalyzer(context).analyze(expr, source_columns_); - res.push_back(ExpressionAnalyzer(constraint->expr->clone(), syntax_result, context).getActions(false)); - } - return res; - } + ConstraintsExpressions getExpressions(const Context & context, const NamesAndTypesList & source_columns_) const; }; } From 94db808cd7f73c9ad5ec3dfcd2f348061c38b8e3 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sat, 25 May 2019 18:54:49 +0300 Subject: [PATCH 033/509] Basic test for constraints parsers and failures --- .../0_stateless/00951_basic_constraints.sh | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00951_basic_constraints.sh diff --git a/dbms/tests/queries/0_stateless/00951_basic_constraints.sh b/dbms/tests/queries/0_stateless/00951_basic_constraints.sh new file mode 100644 index 00000000000..43bf274a82b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00951_basic_constraints.sh @@ -0,0 +1,45 @@ +exec 2>&1 + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_constraints;" + +$CLICKHOUSE_CLIENT --query="CREATE TABLE test_constraints +( + a UInt32, + b UInt32, + CONSTRAINT b_constraint CHECK b > 0 +) +ENGINE = MergeTree ORDER BY (a);" + +# This one must succeed +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" +$CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" + +# This one must throw and exception +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (3, 4), (1, 0);" +$CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" + +$CLICKHOUSE_CLIENT --query="DROP TABLE test_constraints;" + +# Test two constraints on one table +$CLICKHOUSE_CLIENT --query="CREATE TABLE test_constraints +( + a UInt32, + b UInt32, + CONSTRAINT b_constraint CHECK b > 10, + CONSTRAINT a_constraint CHECK a < 10 +) +ENGINE = MergeTree ORDER BY (a);" + +# This one must throw an exception +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" +$CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" + +# This one must throw an exception +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (5, 16), (10, 11);" +$CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" + +# This one must succeed +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (7, 18), (0, 11);" +$CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" + +$CLICKHOUSE_CLIENT --query="DROP TABLE test_constraints;" \ No newline at end of file From a3535c69b23aff266dcbdc8ed856024efff7fa69 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sat, 25 May 2019 18:57:35 +0300 Subject: [PATCH 034/509] Minor style fix --- dbms/src/Parsers/ASTConstraintDeclaration.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/src/Parsers/ASTConstraintDeclaration.cpp b/dbms/src/Parsers/ASTConstraintDeclaration.cpp index 834ac81891b..a1b063fc44a 100644 --- a/dbms/src/Parsers/ASTConstraintDeclaration.cpp +++ b/dbms/src/Parsers/ASTConstraintDeclaration.cpp @@ -1,6 +1,7 @@ #include -namespace DB { +namespace DB +{ ASTPtr ASTConstraintDeclaration::clone() const { @@ -25,4 +26,4 @@ void ASTConstraintDeclaration::formatImpl(const FormatSettings & s, FormatState expr->formatImpl(s, state, frame); } -} \ No newline at end of file +} From c185f5741c116f6038979e56f220e8f2b632ab2a Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sat, 25 May 2019 21:13:43 +0300 Subject: [PATCH 035/509] Fixed test --- dbms/tests/queries/0_stateless/00951_basic_constraints.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00951_basic_constraints.sh b/dbms/tests/queries/0_stateless/00951_basic_constraints.sh index 43bf274a82b..cf622293688 100644 --- a/dbms/tests/queries/0_stateless/00951_basic_constraints.sh +++ b/dbms/tests/queries/0_stateless/00951_basic_constraints.sh @@ -1,3 +1,5 @@ +#!/usr/bin/env bash + exec 2>&1 $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_constraints;" From b0e3315ac7855bae85dbb8adc21b826e971f6ce0 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 26 May 2019 08:32:22 +0300 Subject: [PATCH 036/509] Added tests reference --- .../0_stateless/00951_basic_constraints.reference | 10 ++++++++++ 1 file changed, 10 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00951_basic_constraints.reference diff --git a/dbms/tests/queries/0_stateless/00951_basic_constraints.reference b/dbms/tests/queries/0_stateless/00951_basic_constraints.reference new file mode 100644 index 00000000000..28bf65f8d48 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00951_basic_constraints.reference @@ -0,0 +1,10 @@ +1 2 +Received exception from server (version 19.8.1): +Code: 394. DB::Exception: Received from localhost:9001, ::1. DB::Exception: Some constraints are not satisfied. +1 2 +Received exception from server (version 19.8.1): +Code: 394. DB::Exception: Received from localhost:9001, ::1. DB::Exception: Some constraints are not satisfied. +Received exception from server (version 19.8.1): +Code: 394. DB::Exception: Received from localhost:9001, ::1. DB::Exception: Some constraints are not satisfied. +0 11 +7 18 From f87bb846586a848d5c24a793d8474783008f4cd8 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 26 May 2019 11:43:45 +0300 Subject: [PATCH 037/509] Added curdir to 00951 test --- dbms/tests/queries/0_stateless/00951_basic_constraints.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00951_basic_constraints.sh b/dbms/tests/queries/0_stateless/00951_basic_constraints.sh index cf622293688..49bf6771ab4 100644 --- a/dbms/tests/queries/0_stateless/00951_basic_constraints.sh +++ b/dbms/tests/queries/0_stateless/00951_basic_constraints.sh @@ -1,5 +1,8 @@ #!/usr/bin/env bash +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + exec 2>&1 $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_constraints;" From 94e7521beae08dafcc700bf8c9991f786a6b39ec Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 26 May 2019 12:37:35 +0300 Subject: [PATCH 038/509] Changed number of tests --- ...ic_constraints.reference => 00952_basic_constraints.reference} | 0 .../{00951_basic_constraints.sh => 00952_basic_constraints.sh} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{00951_basic_constraints.reference => 00952_basic_constraints.reference} (100%) rename dbms/tests/queries/0_stateless/{00951_basic_constraints.sh => 00952_basic_constraints.sh} (100%) diff --git a/dbms/tests/queries/0_stateless/00951_basic_constraints.reference b/dbms/tests/queries/0_stateless/00952_basic_constraints.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00951_basic_constraints.reference rename to dbms/tests/queries/0_stateless/00952_basic_constraints.reference diff --git a/dbms/tests/queries/0_stateless/00951_basic_constraints.sh b/dbms/tests/queries/0_stateless/00952_basic_constraints.sh similarity index 100% rename from dbms/tests/queries/0_stateless/00951_basic_constraints.sh rename to dbms/tests/queries/0_stateless/00952_basic_constraints.sh From af50b1ff5cef7bea4c35d25067c914174acc7b89 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 26 May 2019 14:08:37 +0300 Subject: [PATCH 039/509] Made 00952 test executable --- dbms/tests/queries/0_stateless/00952_basic_constraints.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 dbms/tests/queries/0_stateless/00952_basic_constraints.sh diff --git a/dbms/tests/queries/0_stateless/00952_basic_constraints.sh b/dbms/tests/queries/0_stateless/00952_basic_constraints.sh old mode 100644 new mode 100755 From 9dd07bcc23aab9a0e498631e80505da9336f7e5f Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sun, 26 May 2019 18:03:30 -0400 Subject: [PATCH 040/509] * Work in progress to redo live view implementation to be less invasive --- dbms/src/DataStreams/BlocksBlockInputStream.h | 51 +++ .../DataStreams/LiveViewBlockInputStream.h | 209 +++++++++ .../Interpreters/InterpreterWatchQuery.cpp | 102 +++++ dbms/src/Interpreters/InterpreterWatchQuery.h | 50 +++ dbms/src/Parsers/ASTWatchQuery.h | 59 +++ dbms/src/Parsers/ParserWatchQuery.cpp | 77 ++++ dbms/src/Parsers/ParserWatchQuery.h | 30 ++ dbms/src/Storages/LiveViewCommands.h | 65 +++ dbms/src/Storages/ProxyStorage.h | 64 +++ dbms/src/Storages/StorageLiveView.cpp | 405 ++++++++++++++++++ dbms/src/Storages/StorageLiveView.h | 294 +++++++++++++ 11 files changed, 1406 insertions(+) create mode 100644 dbms/src/DataStreams/BlocksBlockInputStream.h create mode 100644 dbms/src/DataStreams/LiveViewBlockInputStream.h create mode 100644 dbms/src/Interpreters/InterpreterWatchQuery.cpp create mode 100644 dbms/src/Interpreters/InterpreterWatchQuery.h create mode 100644 dbms/src/Parsers/ASTWatchQuery.h create mode 100644 dbms/src/Parsers/ParserWatchQuery.cpp create mode 100644 dbms/src/Parsers/ParserWatchQuery.h create mode 100644 dbms/src/Storages/LiveViewCommands.h create mode 100644 dbms/src/Storages/ProxyStorage.h create mode 100644 dbms/src/Storages/StorageLiveView.cpp create mode 100644 dbms/src/Storages/StorageLiveView.h diff --git a/dbms/src/DataStreams/BlocksBlockInputStream.h b/dbms/src/DataStreams/BlocksBlockInputStream.h new file mode 100644 index 00000000000..ad0d37da622 --- /dev/null +++ b/dbms/src/DataStreams/BlocksBlockInputStream.h @@ -0,0 +1,51 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include + + +namespace DB +{ + +/** A stream of blocks from a shared vector of blocks + */ +class BlocksBlockInputStream : public IBlockInputStream +{ +public: + /// Acquires shared ownership of the blocks vector + BlocksBlockInputStream(std::shared_ptr blocks_ptr_, Block header) + : blocks_ptr(blocks_ptr_), it((*blocks_ptr_)->begin()), end((*blocks_ptr_)->end()), header(header) {} + + String getName() const override { return "Blocks"; } + + Block getHeader() const override { return header; } + +protected: + Block readImpl() override + { + if (it == end) + return Block(); + + Block res = *it; + ++it; + return res; + } + +private: + std::shared_ptr blocks_ptr; + Blocks::iterator it; + const Blocks::iterator end; + Block header; +}; + +} diff --git a/dbms/src/DataStreams/LiveViewBlockInputStream.h b/dbms/src/DataStreams/LiveViewBlockInputStream.h new file mode 100644 index 00000000000..3bb0a52efae --- /dev/null +++ b/dbms/src/DataStreams/LiveViewBlockInputStream.h @@ -0,0 +1,209 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** + */ + +class LiveViewBlockInputStream : public IBlockInputStream +{ + +using NonBlockingResult = std::pair; + +public: + ~LiveViewBlockInputStream() override + { + /// Start storage no users thread + /// if we are the last active user + if (!storage.is_dropped && blocks_ptr.use_count() < 3) + storage.startNoUsersThread(); + } + /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update + /// and LIMIT 0 just returns data without waiting for any updates + LiveViewBlockInputStream(StorageLiveView & storage_, std::shared_ptr blocks_ptr_, std::shared_ptr active_ptr_, Poco::Condition & condition_, Poco::FastMutex & mutex_, + int64_t length_, const UInt64 & heartbeat_delay_) + : storage(storage_), blocks_ptr(blocks_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_delay(heartbeat_delay_), blocks_hash("") + { + /// grab active pointer + active = active_ptr.lock(); + } + + String getName() const override { return "LiveViewBlockInputStream"; } + + void cancel(bool kill) override + { + if (isCancelled() || storage.is_dropped) + return; + IBlockInputStream::cancel(kill); + Poco::FastMutex::ScopedLock lock(mutex); + condition.broadcast(); + } + + Block getHeader() const override { return storage.getHeader(); } + + void refresh() + { + if (active && blocks && it == end) + it = blocks->begin(); + } + + void suspend() + { + active.reset(); + } + + void resume() + { + active = active_ptr.lock(); + { + if (!blocks || blocks.get() != (*blocks_ptr).get()) + blocks = (*blocks_ptr); + } + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + + NonBlockingResult tryRead() + { + return tryRead_(false); + } + +protected: + Block readImpl() override + { + /// try reading + return tryRead_(true).first; + } + + /** tryRead method attempts to read a block in either blocking + * or non-blocking mode. If blocking is set to false + * then method return empty block with flag set to false + * to indicate that method would block to get the next block. + */ + NonBlockingResult tryRead_(bool blocking) + { + Block res; + + if (length == 0) + { + return { Block(), true }; + } + /// If blocks were never assigned get blocks + if (!blocks) + { + Poco::FastMutex::ScopedLock lock(mutex); + if (!active) + return { Block(), false }; + blocks = (*blocks_ptr); + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + + if (isCancelled() || storage.is_dropped) + { + return { Block(), true }; + } + + if (it == end) + { + { + Poco::FastMutex::ScopedLock lock(mutex); + if (!active) + return { Block(), false }; + /// If we are done iterating over our blocks + /// and there are new blocks availble then get them + if (blocks.get() != (*blocks_ptr).get()) + { + blocks = (*blocks_ptr); + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + /// No new blocks available wait for new ones + else + { + if (!blocking) + { + return { Block(), false }; + } + while (true) + { + bool signaled = condition.tryWait(mutex, std::max((UInt64)0, heartbeat_delay - ((UInt64)timestamp.epochMicroseconds() - last_event_timestamp)) / 1000); + + if (isCancelled() || storage.is_dropped) + { + return { Block(), true }; + } + if (signaled) + { + break; + } + else + { + //hashmap["blocks"] = blocks_hash; + last_event_timestamp = (UInt64)timestamp.epochMicroseconds(); + //heartbeat(Heartbeat(last_event_timestamp, std::move(hashmap))); + } + } + } + } + return tryRead_(blocking); + } + + res = *it; + + ++it; + + if (it == end) + { + if (length > 0) + --length; + } + + last_event_timestamp = (UInt64)timestamp.epochMicroseconds(); + return { res, true }; + } + +private: + StorageLiveView & storage; + std::shared_ptr blocks_ptr; + std::weak_ptr active_ptr; + std::shared_ptr active; + BlocksPtr blocks; + Blocks::iterator it; + Blocks::iterator end; + Blocks::iterator begin; + Poco::Condition & condition; + Poco::FastMutex & mutex; + /// Length specifies number of updates to send, default -1 (no limit) + int64_t length; + UInt64 heartbeat_delay; + String blocks_hash; + UInt64 last_event_timestamp{0}; + Poco::Timestamp timestamp; +}; + +} diff --git a/dbms/src/Interpreters/InterpreterWatchQuery.cpp b/dbms/src/Interpreters/InterpreterWatchQuery.cpp new file mode 100644 index 00000000000..4672a42a304 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterWatchQuery.cpp @@ -0,0 +1,102 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_STORAGE; + extern const int TOO_MANY_COLUMNS; +} + +BlockInputStreamPtr InterpreterWatchQuery::executeImpl() +{ + return std::make_shared(Block()); +} + +BlockIO InterpreterWatchQuery::execute() +{ + BlockIO res; + const ASTWatchQuery & query = typeid_cast(*query_ptr); + String database; + String table; + /// Get database + if (!query.database.empty()) + database = query.database; + else + database = context.getCurrentDatabase(); + + /// Get table + table = query.table; + + /// Get storage + storage = context.tryGetTable(database, table); + + /// List of columns to read to execute the query. + Names required_columns = storage->getColumns().getNamesOfPhysical(); + + /// Get context settings for this query + const Settings & settings = context.getSettingsRef(); + + /// Limitation on the number of columns to read. + if (settings.max_columns_to_read && required_columns.size() > settings.max_columns_to_read) + throw Exception("Limit for number of columns to read exceeded. " + "Requested: " + std::to_string(required_columns.size()) + + ", maximum: " + settings.max_columns_to_read.toString(), + ErrorCodes::TOO_MANY_COLUMNS); + + size_t max_block_size = settings.max_block_size; + size_t max_streams = 1; + + /// Define query info + SelectQueryInfo query_info; + query_info.query = query_ptr; + + /// From stage + QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; + QueryProcessingStage::Enum to_stage = QueryProcessingStage::Complete; + + /// Watch storage + streams = storage->watch(required_columns, query_info, context, from_stage, max_block_size, max_streams); + + /// Constraints on the result, the quota on the result, and also callback for progress. + if (IBlockInputStream * stream = dynamic_cast(streams[0].get())) + { + /// Constraints apply only to the final result. + if (to_stage == QueryProcessingStage::Complete) + { + IBlockInputStream::LocalLimits limits; + limits.mode = IBlockInputStream::LIMITS_CURRENT; + limits.size_limits.max_rows = settings.max_result_rows; + limits.size_limits.max_bytes = settings.max_result_bytes; + limits.size_limits.overflow_mode = settings.result_overflow_mode; + + stream->setLimits(limits); + stream->setQuota(context.getQuota()); + } + } + + res.in = streams[0]; + + return res; +} + + +} diff --git a/dbms/src/Interpreters/InterpreterWatchQuery.h b/dbms/src/Interpreters/InterpreterWatchQuery.h new file mode 100644 index 00000000000..9315ee1f889 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterWatchQuery.h @@ -0,0 +1,50 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class IAST; +using ASTPtr = std::shared_ptr; +using StoragePtr = std::shared_ptr; + +class InterpreterWatchQuery : public IInterpreter +{ +public: + InterpreterWatchQuery(const ASTPtr & query_ptr_, Context & context_) + : query_ptr(query_ptr_), context(context_) {} + + BlockIO execute() override; + +private: + ASTPtr query_ptr; + Context & context; + + BlockInputStreamPtr executeImpl(); + /// Table from where to read data, if not subquery. + StoragePtr storage; + /// Streams of read data + BlockInputStreams streams; +}; + + +} diff --git a/dbms/src/Parsers/ASTWatchQuery.h b/dbms/src/Parsers/ASTWatchQuery.h new file mode 100644 index 00000000000..7e75d62a629 --- /dev/null +++ b/dbms/src/Parsers/ASTWatchQuery.h @@ -0,0 +1,59 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include + + +namespace DB +{ + +class ASTWatchQuery : public ASTQueryWithTableAndOutput +{ + +public: + ASTPtr limit_length; + bool is_watch_events; + + ASTWatchQuery() = default; + String getID(char) const override { return "WatchQuery_" + database + "_" + table; }; + + ASTPtr clone() const override + { + std::shared_ptr res = std::make_shared(*this); + res->children.clear(); + cloneOutputOptions(*res); + return res; + } + +protected: + void formatQueryImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override + { + std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' '); + + s.ostr << (s.hilite ? hilite_keyword : "") << "WATCH" << " " << (s.hilite ? hilite_none : "") + << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); + + if (is_watch_events) + { + s.ostr << " " << (s.hilite ? hilite_keyword : "") << "EVENTS" << (s.hilite ? hilite_none : ""); + } + + if (limit_length) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "LIMIT " << (s.hilite ? hilite_none : ""); + limit_length->formatImpl(s, state, frame); + } + } +}; + +} diff --git a/dbms/src/Parsers/ParserWatchQuery.cpp b/dbms/src/Parsers/ParserWatchQuery.cpp new file mode 100644 index 00000000000..c29320c5a56 --- /dev/null +++ b/dbms/src/Parsers/ParserWatchQuery.cpp @@ -0,0 +1,77 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +bool ParserWatchQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_watch("WATCH"); + ParserToken s_dot(TokenType::Dot); + ParserIdentifier name_p; + ParserKeyword s_events("EVENTS"); + ParserKeyword s_limit("LIMIT"); + + ASTPtr database; + ASTPtr table; + auto query = std::make_shared(); + + if (!s_watch.ignore(pos, expected)) + { + return false; + } + + if (!name_p.parse(pos, table, expected)) + return false; + + if (s_dot.ignore(pos, expected)) + { + database = table; + if (!name_p.parse(pos, table, expected)) + return false; + } + + /// EVENTS + if (s_events.ignore(pos, expected)) + { + query->is_watch_events = true; + } + + /// LIMIT length + if (s_limit.ignore(pos, expected)) + { + ParserNumber num; + + if (!num.parse(pos, query->limit_length, expected)) + return false; + } + + if (database) + getIdentifierName(database, query->database); + + if (table) + getIdentifierName(table, query->table); + + node = query; + + return true; +} + + +} diff --git a/dbms/src/Parsers/ParserWatchQuery.h b/dbms/src/Parsers/ParserWatchQuery.h new file mode 100644 index 00000000000..330f0a432df --- /dev/null +++ b/dbms/src/Parsers/ParserWatchQuery.h @@ -0,0 +1,30 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include + + +namespace DB +{ + +/** Query like this: + * WATCH [db.]table EVENTS + */ +class ParserWatchQuery : public IParserBase +{ +protected: + const char * getName() const { return "WATCH query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); +}; + +} diff --git a/dbms/src/Storages/LiveViewCommands.h b/dbms/src/Storages/LiveViewCommands.h new file mode 100644 index 00000000000..35015a7e5aa --- /dev/null +++ b/dbms/src/Storages/LiveViewCommands.h @@ -0,0 +1,65 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ + +#pragma once + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_STORAGE; +} + +struct LiveViewCommand +{ + enum Type + { + REFRESH + }; + + Type type; + + ASTPtr values; + + static LiveViewCommand refresh(const ASTPtr & values) + { + LiveViewCommand res; + res.type = REFRESH; + res.values = values; + return res; + } + + static std::optional parse(ASTAlterCommand * command) + { + if (command->type == ASTAlterCommand::LIVE_VIEW_REFRESH) + return refresh(command->values); + return {}; + } +}; + + +class LiveViewCommands : public std::vector +{ +public: + void validate(const IStorage & table) + { + if (!empty() && !dynamic_cast(&table)) + throw Exception("Wrong storage type. Must be StorageLiveView", DB::ErrorCodes::UNKNOWN_STORAGE); + } +}; + +} diff --git a/dbms/src/Storages/ProxyStorage.h b/dbms/src/Storages/ProxyStorage.h new file mode 100644 index 00000000000..a4196b7d6f2 --- /dev/null +++ b/dbms/src/Storages/ProxyStorage.h @@ -0,0 +1,64 @@ +#pragma once + +#include + +namespace DB +{ + +class ProxyStorage : public IStorage +{ +public: + ProxyStorage(StoragePtr storage, BlockInputStreams streams, QueryProcessingStage::Enum to_stage) + : storage(std::move(storage)), streams(std::move(streams)), to_stage(to_stage) {} + +public: + std::string getName() const override { return "ProxyStorage(" + storage->getName() + ")"; } + std::string getTableName() const override { return storage->getTableName(); } + + bool isRemote() const override { return storage->isRemote(); } + bool supportsSampling() const override { return storage->supportsSampling(); } + bool supportsFinal() const override { return storage->supportsFinal(); } + bool supportsPrewhere() const override { return storage->supportsPrewhere(); } + bool supportsReplication() const override { return storage->supportsReplication(); } + bool supportsDeduplication() const override { return storage->supportsDeduplication(); } + + QueryProcessingStage::Enum getQueryProcessingStage(const Context & /*context*/) const override { return to_stage; } + + BlockInputStreams read( + const Names & /*column_names*/, + const SelectQueryInfo & /*query_info*/, + const Context & /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + unsigned /*num_streams*/) override + { + return streams; + } + + bool supportsIndexForIn() const override { return storage->supportsIndexForIn(); } + bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override { return storage->mayBenefitFromIndexForIn(left_in_operand, query_context); } + ASTPtr getPartitionKeyAST() const override { return storage->getPartitionKeyAST(); } + ASTPtr getSortingKeyAST() const override { return storage->getSortingKeyAST(); } + ASTPtr getPrimaryKeyAST() const override { return storage->getPrimaryKeyAST(); } + ASTPtr getSamplingKeyAST() const override { return storage->getSamplingKeyAST(); } + Names getColumnsRequiredForPartitionKey() const override { return storage->getColumnsRequiredForPartitionKey(); } + Names getColumnsRequiredForSortingKey() const override { return storage->getColumnsRequiredForSortingKey(); } + Names getColumnsRequiredForPrimaryKey() const override { return storage->getColumnsRequiredForPrimaryKey(); } + Names getColumnsRequiredForSampling() const override { return storage->getColumnsRequiredForSampling(); } + Names getColumnsRequiredForFinal() const override { return storage->getColumnsRequiredForFinal(); } + + NameAndTypePair getColumn(const String & column_name) const override { return storage->getColumn(column_name); } + bool hasColumn(const String & column_name) const override { return storage->hasColumn(column_name); } + static StoragePtr createProxyStorage(StoragePtr storage, BlockInputStreams streams, QueryProcessingStage::Enum to_stage) + { + return std::make_shared(std::move(storage), std::move(streams), to_stage); + } +private: + StoragePtr storage; + BlockInputStreams streams; + QueryProcessingStage::Enum to_stage; +}; + + + +} diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp new file mode 100644 index 00000000000..5a4e225e52c --- /dev/null +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -0,0 +1,405 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_QUERY; + extern const int TABLE_WAS_NOT_DROPPED; + extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW; +} + +static void extractDependentTable(ASTSelectQuery & query, String & select_database_name, String & select_table_name) +{ + auto db_and_table = getDatabaseAndTable(query, 0); + ASTPtr subquery = extractTableExpression(query, 0); + + if (!db_and_table && !subquery) + return; + + if (db_and_table) + { + select_table_name = db_and_table->table; + + if (db_and_table->database.empty()) + { + db_and_table->database = select_database_name; + AddDefaultDatabaseVisitor visitor(select_database_name); + visitor.visit(query); + } + else + select_database_name = db_and_table->database; + } + else if (auto * ast_select = subquery->as()) + { + if (ast_select->list_of_selects->children.size() != 1) + throw Exception("UNION is not supported for LIVE VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW); + + auto & inner_query = ast_select->list_of_selects->children.at(0); + + extractDependentTable(inner_query->as(), select_database_name, select_table_name); + } + else + throw Exception("Logical error while creating StorageLiveView." + " Could not retrieve table name from select query.", + DB::ErrorCodes::LOGICAL_ERROR); +} + +static void checkAllowedQueries(const ASTSelectQuery & query) +{ + if (query.prewhere() || query.final() || query.sample_size()) + throw Exception("LIVE VIEW cannot have PREWHERE, SAMPLE or FINAL.", DB::ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW); + + ASTPtr subquery = extractTableExpression(query, 0); + if (!subquery) + return; + + if (const auto * ast_select = subquery->as()) + { + if (ast_select->list_of_selects->children.size() != 1) + throw Exception("UNION is not supported for LIVE VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW); + + const auto & inner_query = ast_select->list_of_selects->children.at(0); + + checkAllowedQueries(inner_query->as()); + } +} + +StorageLiveView::StorageLiveView( + const String & table_name_, + const String & database_name_, + Context & local_context, + const ASTCreateQuery & query, + const ColumnsDescription & columns) + : IStorage(columns), table_name(table_name_), + database_name(database_name_), global_context(local_context.getGlobalContext()) +{ + if (!query.select) + throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); + + /// Default value, if only table name exist in the query + select_database_name = local_context.getCurrentDatabase(); + if (query.select->list_of_selects->children.size() != 1) + throw Exception("UNION is not supported for LIVE VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW); + + inner_query = query.select->list_of_selects->children.at(0); + + ASTSelectQuery & select_query = typeid_cast(*inner_query); + extractDependentTable(select_query, select_database_name, select_table_name); + + /// If the table is not specified - use the table `system.one` + if (select_table_name.empty()) + { + select_database_name = "system"; + select_table_name = "one"; + } + + global_context.addDependency( + DatabaseAndTableName(select_database_name, select_table_name), + DatabaseAndTableName(database_name, table_name)); + + is_temporary = query.temporary; + + blocks_ptr = std::make_shared(); + active_ptr = std::make_shared(true); +} + +Block StorageLiveView::getHeader() const +{ + if (!sample_block) + { + auto storage = global_context.getTable(select_database_name, select_table_name); + sample_block = InterpreterSelectQuery(inner_query, global_context, storage).getSampleBlock(); + } + + return sample_block; +} + +bool StorageLiveView::getNewBlocks() +{ + Block block; + SipHash hash; + UInt128 key; + BlocksPtr new_blocks = std::make_shared(); + BlocksPtr new_mergeable_blocks = std::make_shared(); + + InterpreterSelectQuery interpreter(inner_query->clone(), global_context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); + auto mergeable_stream = std::make_shared(interpreter.execute().in); + + while (Block block = mergeable_stream->read()) + new_mergeable_blocks->push_back(block); + + mergeable_blocks = std::make_shared>(); + mergeable_blocks->push_back(new_mergeable_blocks); + BlockInputStreamPtr from = std::make_shared(std::make_shared(new_mergeable_blocks), mergeable_stream->getHeader()); + auto proxy_storage = ProxyStorage::createProxyStorage(global_context.getTable(select_database_name, select_table_name), {from}, QueryProcessingStage::WithMergeableState); + InterpreterSelectQuery select(inner_query->clone(), global_context, proxy_storage, SelectQueryOptions(QueryProcessingStage::Complete)); + BlockInputStreamPtr data = std::make_shared(select.execute().in); + while (Block block = data->read()) + { + block.updateHash(hash); + new_blocks->push_back(block); + } + + hash.get128(key.low, key.high); + + /// Update blocks only if hash keys do not match + /// NOTE: hash could be different for the same result + /// if blocks are not in the same order + bool updated = false; + { + if (hash_key != key.toHexString()) + { + if (new_blocks->empty()) + { + new_blocks->push_back(getHeader()); + } + (*blocks_ptr) = new_blocks; + hash_key = key.toHexString(); + updated = true; + } + } + return updated; +} + +void StorageLiveView::checkTableCanBeDropped() const +{ + Dependencies dependencies = global_context.getDependencies(database_name, table_name); + if (!dependencies.empty()) + { + DatabaseAndTableName database_and_table_name = dependencies.front(); + throw Exception("Table has dependency " + database_and_table_name.first + "." + database_and_table_name.second, ErrorCodes::TABLE_WAS_NOT_DROPPED); + } +} + +void StorageLiveView::noUsersThread() +{ + if (shutdown_called) + return; + + bool drop_table = false; + + { + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); + while (1) + { + if (!noUsersThreadWakeUp && !noUsersThreadCondition.tryWait(noUsersThreadMutex, global_context.getSettingsRef().temporary_live_view_timeout.totalSeconds() * 1000)) + { + noUsersThreadWakeUp = false; + if (shutdown_called) + return; + if (hasUsers()) + return; + if (!global_context.getDependencies(database_name, table_name).empty()) + continue; + drop_table = true; + } + break; + } + } + + if (drop_table) + { + if (global_context.tryGetTable(database_name, table_name)) + { + try + { + /// We create and execute `drop` query for this table + auto drop_query = std::make_shared(); + drop_query->database = database_name; + drop_query->table = table_name; + ASTPtr ast_drop_query = drop_query; + InterpreterDropQuery drop_interpreter(ast_drop_query, global_context); + drop_interpreter.execute(); + } + catch (...) + { + } + } + } +} + +void StorageLiveView::startNoUsersThread() +{ + bool expected = false; + if (!startnousersthread_called.compare_exchange_strong(expected, true)) + return; + + if (is_dropped) + return; + + if (is_temporary) + { + if (no_users_thread.joinable()) + { + { + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); + noUsersThreadWakeUp = true; + noUsersThreadCondition.signal(); + } + no_users_thread.join(); + } + { + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); + noUsersThreadWakeUp = false; + } + if (!is_dropped) + no_users_thread = std::thread(&StorageLiveView::noUsersThread, this); + } + startnousersthread_called = false; +} + +void StorageLiveView::startup() +{ + startNoUsersThread(); +} + +void StorageLiveView::shutdown() +{ + bool expected = false; + if (!shutdown_called.compare_exchange_strong(expected, true)) + return; + + if (no_users_thread.joinable()) + { + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); + noUsersThreadWakeUp = true; + noUsersThreadCondition.signal(); + /// Must detach the no users thread + /// as we can't join it as it will result + /// in a deadlock + no_users_thread.detach(); + } +} + +StorageLiveView::~StorageLiveView() +{ + shutdown(); +} + +void StorageLiveView::drop() +{ + global_context.removeDependency( + DatabaseAndTableName(select_database_name, select_table_name), + DatabaseAndTableName(database_name, table_name)); + Poco::FastMutex::ScopedLock lock(mutex); + is_dropped = true; + condition.broadcast(); +} + +void StorageLiveView::refresh(const Context & context) +{ + auto alter_lock = lockAlterIntention(context.getCurrentQueryId()); + { + Poco::FastMutex::ScopedLock lock(mutex); + if (getNewBlocks()) + condition.broadcast(); + } +} + +BlockInputStreams StorageLiveView::read( + const Names & /*column_names*/, + const SelectQueryInfo & /*query_info*/, + const Context & /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) +{ + /// add user to the blocks_ptr + std::shared_ptr stream_blocks_ptr = blocks_ptr; + { + Poco::FastMutex::ScopedLock lock(mutex); + if (!(*blocks_ptr)) + { + if (getNewBlocks()) + condition.broadcast(); + } + } + return { std::make_shared(stream_blocks_ptr, getHeader()) }; +} + +BlockInputStreams StorageLiveView::watch( + const Names & /*column_names*/, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t /*max_block_size*/, + const unsigned /*num_streams*/) +{ + ASTWatchQuery & query = typeid_cast(*query_info.query); + + /// By default infinite stream of updates + int64_t length = -2; + + if (query.limit_length) + length = (int64_t)safeGet(typeid_cast(*query.limit_length).value); + + auto reader = std::make_shared(*this, blocks_ptr, active_ptr, condition, mutex, length, context.getSettingsRef().heartbeat_delay); + + if (no_users_thread.joinable()) + { + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); + noUsersThreadWakeUp = true; + noUsersThreadCondition.signal(); + } + + { + Poco::FastMutex::ScopedLock lock(mutex); + if (!(*blocks_ptr)) + { + if (getNewBlocks()) + condition.broadcast(); + } + } + + processed_stage = QueryProcessingStage::Complete; + + return { reader }; +} + +BlockOutputStreamPtr StorageLiveView::write(const ASTPtr & /*query*/, const Context & /*context*/) +{ + return std::make_shared(*this); +} + +void registerStorageLiveView(StorageFactory & factory) +{ + factory.registerStorage("LiveView", [](const StorageFactory::Arguments & args) + { + return StorageLiveView::create(args.table_name, args.database_name, args.local_context, args.query, args.columns); + }); +} + +} diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h new file mode 100644 index 00000000000..cb90e2b3e71 --- /dev/null +++ b/dbms/src/Storages/StorageLiveView.h @@ -0,0 +1,294 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class IAST; +using ASTPtr = std::shared_ptr; + +class StorageLiveView : public ext::shared_ptr_helper, public IStorage +{ +friend struct ext::shared_ptr_helper; +friend class LiveViewBlockOutputStream; + +public: + ~StorageLiveView() override; + String getName() const override { return "LiveView"; } + String getTableName() const override { return table_name; } + String getDatabaseName() const { return database_name; } + String getSelectDatabaseName() const { return select_database_name; } + String getSelectTableName() const { return select_table_name; } + + // const NamesAndTypesList & getColumnsListImpl() const override { return *columns; } + ASTPtr getInnerQuery() const { return inner_query->clone(); }; + + /// It is passed inside the query and solved at its level. + bool supportsSampling() const override { return true; } + bool supportsFinal() const override { return true; } + + /// Mutex for the blocks and ready condition + Poco::FastMutex mutex; + /// New blocks ready condition to broadcast to readers + /// that new blocks are available + Poco::Condition condition; + + bool isTemporary() { return is_temporary; } + + /// Check if we have any readers + /// must be called with mutex locked + bool hasUsers() + { + return blocks_ptr.use_count() > 1; + } + + /// Check we we have any active readers + /// must be called with mutex locked + bool hasActiveUsers() + { + return active_ptr.use_count() > 1; + } + /// Background thread for temporary tables + /// which drops this table if there are no users + void startNoUsersThread(); + Poco::FastMutex noUsersThreadMutex; + bool noUsersThreadWakeUp{false}; + Poco::Condition noUsersThreadCondition; + + String getBlocksHashKey() + { + return hash_key; + } + + /// Reset blocks + /// must be called with mutex locked + void reset() + { + (*blocks_ptr).reset(); + mergeable_blocks.reset(); + hash_key = ""; + } + + void checkTableCanBeDropped() const override; + void drop() override; + void startup() override; + void shutdown() override; + + void refresh(const Context & context); + + BlockOutputStreamPtr write( + const ASTPtr &, + const Context &) override; + + BlockInputStreams read( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + BlockInputStreams watch( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + std::shared_ptr getBlocksPtr() { return blocks_ptr; } + BlocksPtrs getMergeableBlocks() { return mergeable_blocks; } + std::shared_ptr getActivePtr() { return active_ptr; } + + /// Read new data blocks that store query result + bool getNewBlocks(); + + Block getHeader() const; + + static void writeIntoLiveView(StorageLiveView & live_view, + const Block & block, + const Context & context, + BlockOutputStreamPtr & output) + { + /// Check if live view has any readers if not + /// just reset blocks to empty and do nothing else + /// When first reader comes the blocks will be read. + { + Poco::FastMutex::ScopedLock lock(live_view.mutex); + if (!live_view.hasActiveUsers()) + { + live_view.reset(); + return; + } + } + + SipHash hash; + UInt128 key; + BlockInputStreams from; + BlocksPtr blocks = std::make_shared(); + BlocksPtrs mergeable_blocks; + BlocksPtr new_mergeable_blocks = std::make_shared(); + + { + auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); + BlockInputStreams streams = {std::make_shared(block)}; + auto proxy_storage = std::make_shared(parent_storage, std::move(streams), QueryProcessingStage::FetchColumns); + InterpreterSelectQuery select_block(live_view.getInnerQuery(), context, proxy_storage, + QueryProcessingStage::WithMergeableState); + auto data_mergeable_stream = std::make_shared(select_block.execute().in); + while (Block this_block = data_mergeable_stream->read()) + new_mergeable_blocks->push_back(this_block); + } + + if (new_mergeable_blocks->empty()) + return; + + { + Poco::FastMutex::ScopedLock lock(live_view.mutex); + + mergeable_blocks = live_view.getMergeableBlocks(); + if (!mergeable_blocks || mergeable_blocks->size() >= 64) + { + mergeable_blocks = std::make_shared>(); + BlocksPtr base_mergeable_blocks = std::make_shared(); + InterpreterSelectQuery interpreter(live_view.getInnerQuery(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names{}); + auto view_mergeable_stream = std::make_shared(interpreter.execute().in); + while (Block this_block = view_mergeable_stream->read()) + base_mergeable_blocks->push_back(this_block); + mergeable_blocks->push_back(base_mergeable_blocks); + } + + /// Need make new mergeable block structure match the other mergeable blocks + if (!mergeable_blocks->front()->empty() && !new_mergeable_blocks->empty()) + { + auto sample_block = mergeable_blocks->front()->front(); + auto sample_new_block = new_mergeable_blocks->front(); + for (auto col : sample_new_block) + { + for (auto & new_block : *new_mergeable_blocks) + { + if (!sample_block.has(col.name)) + new_block.erase(col.name); + } + } + } + + mergeable_blocks->push_back(new_mergeable_blocks); + + /// Create from blocks streams + for (auto & blocks : *mergeable_blocks) + { + auto sample_block = mergeable_blocks->front()->front().cloneEmpty(); + BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks), sample_block); + from.push_back(std::move(stream)); + } + } + + auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); + auto proxy_storage = std::make_shared(parent_storage, std::move(from), QueryProcessingStage::WithMergeableState); + InterpreterSelectQuery select(live_view.getInnerQuery(), context, proxy_storage, QueryProcessingStage::Complete); + BlockInputStreamPtr data = std::make_shared(select.execute().in); + while (Block this_block = data->read()) + { + this_block.updateHash(hash); + blocks->push_back(this_block); + } + /// get hash key + hash.get128(key.low, key.high); + /// Update blocks only if hash keys do not match + /// NOTE: hash could be different for the same result + /// if blocks are not in the same order + if (live_view.getBlocksHashKey() != key.toHexString()) + { + auto sample_block = blocks->front().cloneEmpty(); + BlockInputStreamPtr new_data = std::make_shared(std::make_shared(blocks), sample_block); + { + Poco::FastMutex::ScopedLock lock(live_view.mutex); + copyData(*new_data, *output); + } + } + } + +private: + String select_database_name; + String select_table_name; + String table_name; + String database_name; + ASTPtr inner_query; + Context & global_context; + bool is_temporary {false}; + mutable Block sample_block; + + /// Active users + std::shared_ptr active_ptr; + /// Current data blocks that store query result + std::shared_ptr blocks_ptr; + BlocksPtr new_blocks; + BlocksPtrs mergeable_blocks; + + /// Current blocks hash key + String hash_key; + String new_hash_key; + + void noUsersThread(); + std::thread no_users_thread; + std::atomic shutdown_called{false}; + std::atomic startnousersthread_called{false}; + + StorageLiveView( + const String & table_name_, + const String & database_name_, + Context & local_context, + const ASTCreateQuery & query, + const ColumnsDescription & columns + ); +}; + +class LiveViewBlockOutputStream : public IBlockOutputStream +{ +public: + explicit LiveViewBlockOutputStream(StorageLiveView & storage_) : storage(storage_) {} + + void write(const Block & block) override + { + if (!new_blocks) + new_blocks = std::make_shared(); + + new_blocks->push_back(block); + // FIXME: do I need to calculate block hash? + (*storage.blocks_ptr) = new_blocks; + new_blocks.reset(); + storage.condition.broadcast(); + } + + Block getHeader() const override { return storage.getHeader(); } + +private: + BlocksPtr new_blocks; + String new_hash_key; + StorageLiveView & storage; +}; + +} From 1aaab0745969277b9e9fab643d10832a571ff469 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Mon, 27 May 2019 02:57:18 +0300 Subject: [PATCH 041/509] Fixed exception handling in 00952 test --- .../CheckConstraintsBlockOutputStream.cpp | 2 +- .../0_stateless/00952_basic_constraints.reference | 9 +++------ .../queries/0_stateless/00952_basic_constraints.sh | 14 +++++++++----- 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 99f9f9bc90d..1cc271e5578 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -8,7 +8,7 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) { for (auto & constraint_expr: expressions) if (!checkConstraintOnBlock(block, constraint_expr)) - throw Exception("Some constraints are not satisfied", ErrorCodes::QUERY_WAS_CANCELLED); + throw Exception{"Some constraints are not satisfied", ErrorCodes::QUERY_WAS_CANCELLED}; output->write(block); } diff --git a/dbms/tests/queries/0_stateless/00952_basic_constraints.reference b/dbms/tests/queries/0_stateless/00952_basic_constraints.reference index 28bf65f8d48..1bede18351d 100644 --- a/dbms/tests/queries/0_stateless/00952_basic_constraints.reference +++ b/dbms/tests/queries/0_stateless/00952_basic_constraints.reference @@ -1,10 +1,7 @@ 1 2 -Received exception from server (version 19.8.1): -Code: 394. DB::Exception: Received from localhost:9001, ::1. DB::Exception: Some constraints are not satisfied. +Exception ok 1 2 -Received exception from server (version 19.8.1): -Code: 394. DB::Exception: Received from localhost:9001, ::1. DB::Exception: Some constraints are not satisfied. -Received exception from server (version 19.8.1): -Code: 394. DB::Exception: Received from localhost:9001, ::1. DB::Exception: Some constraints are not satisfied. +Exception ok +Exception ok 0 11 7 18 diff --git a/dbms/tests/queries/0_stateless/00952_basic_constraints.sh b/dbms/tests/queries/0_stateless/00952_basic_constraints.sh index 49bf6771ab4..b214982da5e 100755 --- a/dbms/tests/queries/0_stateless/00952_basic_constraints.sh +++ b/dbms/tests/queries/0_stateless/00952_basic_constraints.sh @@ -3,8 +3,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -exec 2>&1 - $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_constraints;" $CLICKHOUSE_CLIENT --query="CREATE TABLE test_constraints @@ -20,7 +18,9 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must throw and exception -$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (3, 4), (1, 0);" +EXCEPTION_TEXT="Some constraints are not satisfied" +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (3, 4), (1, 0);" 2>&1 \ + | grep -q "$EXCEPTION_TEXT" && echo "Exception ok" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" $CLICKHOUSE_CLIENT --query="DROP TABLE test_constraints;" @@ -36,11 +36,15 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE test_constraints ENGINE = MergeTree ORDER BY (a);" # This one must throw an exception -$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" +EXCEPTION_TEXT="Some constraints are not satisfied" +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" 2>&1 \ + | grep -q "$EXCEPTION_TEXT" && echo "Exception ok" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must throw an exception -$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (5, 16), (10, 11);" +EXCEPTION_TEXT="Some constraints are not satisfied" +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (5, 16), (10, 11);" 2>&1 \ + | grep -q "$EXCEPTION_TEXT" && echo "Exception ok" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must succeed From ff6cdaeb9846ae63ddba399e215809f0be641793 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Mon, 27 May 2019 09:30:18 +0300 Subject: [PATCH 042/509] Removed word "exception" from test reference --- .../queries/0_stateless/00952_basic_constraints.reference | 6 +++--- dbms/tests/queries/0_stateless/00952_basic_constraints.sh | 8 +++++--- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00952_basic_constraints.reference b/dbms/tests/queries/0_stateless/00952_basic_constraints.reference index 1bede18351d..4d98efd8939 100644 --- a/dbms/tests/queries/0_stateless/00952_basic_constraints.reference +++ b/dbms/tests/queries/0_stateless/00952_basic_constraints.reference @@ -1,7 +1,7 @@ 1 2 -Exception ok +ok 1 2 -Exception ok -Exception ok +ok +ok 0 11 7 18 diff --git a/dbms/tests/queries/0_stateless/00952_basic_constraints.sh b/dbms/tests/queries/0_stateless/00952_basic_constraints.sh index b214982da5e..93fa16ce4af 100755 --- a/dbms/tests/queries/0_stateless/00952_basic_constraints.sh +++ b/dbms/tests/queries/0_stateless/00952_basic_constraints.sh @@ -3,6 +3,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh +EXCEPTION_SUCCESS_TEXT=ok + $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_constraints;" $CLICKHOUSE_CLIENT --query="CREATE TABLE test_constraints @@ -20,7 +22,7 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must throw and exception EXCEPTION_TEXT="Some constraints are not satisfied" $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (3, 4), (1, 0);" 2>&1 \ - | grep -q "$EXCEPTION_TEXT" && echo "Exception ok" || echo "Did not thrown an exception" + | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" $CLICKHOUSE_CLIENT --query="DROP TABLE test_constraints;" @@ -38,13 +40,13 @@ ENGINE = MergeTree ORDER BY (a);" # This one must throw an exception EXCEPTION_TEXT="Some constraints are not satisfied" $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" 2>&1 \ - | grep -q "$EXCEPTION_TEXT" && echo "Exception ok" || echo "Did not thrown an exception" + | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must throw an exception EXCEPTION_TEXT="Some constraints are not satisfied" $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (5, 16), (10, 11);" 2>&1 \ - | grep -q "$EXCEPTION_TEXT" && echo "Exception ok" || echo "Did not thrown an exception" + | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must succeed From d112695c42a36b905a74eacd9b057718c0867a2c Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Tue, 28 May 2019 17:17:48 -0400 Subject: [PATCH 043/509] * Adding other changes to support LIVE VIEWs --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Common/UInt128.h | 10 +- dbms/src/Core/Block.h | 3 +- dbms/src/Core/Defines.h | 4 + dbms/src/Core/Settings.h | 6 +- .../DataStreams/LiveViewBlockInputStream.h | 8 +- .../LiveViewEventsBlockInputStream.h | 227 +++++++++ .../PushingToViewsBlockOutputStream.cpp | 12 +- .../PushingToViewsBlockOutputStream.h | 2 +- dbms/src/Databases/DatabasesCommon.cpp | 2 +- .../Interpreters/InterpreterAlterQuery.cpp | 20 + dbms/src/Interpreters/InterpreterFactory.cpp | 6 + dbms/src/Parsers/ASTAlterQuery.cpp | 52 +- dbms/src/Parsers/ASTAlterQuery.h | 25 + dbms/src/Parsers/ASTCreateQuery.cpp | 13 + dbms/src/Parsers/ASTCreateQuery.h | 3 + dbms/src/Parsers/ParserAlterQuery.cpp | 475 +++++++++++------- dbms/src/Parsers/ParserAlterQuery.h | 21 + dbms/src/Parsers/ParserCreateQuery.cpp | 91 ++++ dbms/src/Parsers/ParserCreateQuery.h | 10 +- dbms/src/Parsers/ParserQueryWithOutput.cpp | 5 +- dbms/src/Storages/IStorage.h | 30 ++ dbms/src/Storages/StorageFactory.cpp | 28 ++ dbms/src/Storages/StorageLiveView.cpp | 64 ++- dbms/src/Storages/StorageLiveView.h | 47 +- dbms/src/Storages/registerStorages.cpp | 4 + 26 files changed, 940 insertions(+), 229 deletions(-) create mode 100644 dbms/src/DataStreams/LiveViewEventsBlockInputStream.h diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 7b1aa175c92..752146d9a5b 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -427,6 +427,7 @@ namespace ErrorCodes extern const int BAD_TTL_EXPRESSION = 450; extern const int BAD_TTL_FILE = 451; extern const int SETTING_CONSTRAINT_VIOLATION = 452; + extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW = 453; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/UInt128.h b/dbms/src/Common/UInt128.h index 72fb1a2503a..491d3a980cb 100644 --- a/dbms/src/Common/UInt128.h +++ b/dbms/src/Common/UInt128.h @@ -1,7 +1,8 @@ #pragma once #include - +#include +#include #include #include @@ -33,6 +34,13 @@ struct UInt128 auto tuple() const { return std::tie(high, low); } + String toHexString() const + { + std::ostringstream os; + os << std::setw(16) << std::setfill('0') << std::hex << high << low; + return String(os.str()); + } + bool inline operator== (const UInt128 rhs) const { return tuple() == rhs.tuple(); } bool inline operator!= (const UInt128 rhs) const { return tuple() != rhs.tuple(); } bool inline operator< (const UInt128 rhs) const { return tuple() < rhs.tuple(); } diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index d3ce0c67b79..4a93e5ed803 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -144,7 +144,8 @@ private: using Blocks = std::vector; using BlocksList = std::list; - +using BlocksPtr = std::shared_ptr; +using BlocksPtrs = std::shared_ptr>; /// Compare number of columns, data types, column types, column names, and values of constant columns. bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs); diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index b8db4d64843..82e882d29f2 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -32,7 +32,11 @@ */ #define DEFAULT_MERGE_BLOCK_SIZE 8192 +#define DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC 5 +#define DEFAULT_TEMPORARY_LIVE_CHANNEL_TIMEOUT_SEC 15 +#define DEFAULT_ALTER_LIVE_CHANNEL_WAIT_MS 10000 #define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160) +#define DEFAULT_HEARTBEAT_DELAY 15000000 #define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024 #define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES 3 /// each period reduces the error counter by 2 times diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 21eb792e20c..ae7f891a17d 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -325,7 +325,11 @@ struct Settings : public SettingsCollection M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \ M(SettingBool, allow_simdjson, 1, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \ \ - M(SettingUInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.") + M(SettingUInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.") \ + M(SettingUInt64, heartbeat_delay, DEFAULT_HEARTBEAT_DELAY, "The interval in microseconds to indicate live query is alive.") \ + M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.") \ + M(SettingSeconds, temporary_live_channel_timeout, DEFAULT_TEMPORARY_LIVE_CHANNEL_TIMEOUT_SEC, "Timeout after which temporary live channel is deleted.") \ + M(SettingMilliseconds, alter_channel_wait_ms, DEFAULT_ALTER_LIVE_CHANNEL_WAIT_MS, "The wait time for alter channel request.") DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) diff --git a/dbms/src/DataStreams/LiveViewBlockInputStream.h b/dbms/src/DataStreams/LiveViewBlockInputStream.h index 3bb0a52efae..62ab983aad1 100644 --- a/dbms/src/DataStreams/LiveViewBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewBlockInputStream.h @@ -41,9 +41,11 @@ public: } /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update /// and LIMIT 0 just returns data without waiting for any updates - LiveViewBlockInputStream(StorageLiveView & storage_, std::shared_ptr blocks_ptr_, std::shared_ptr active_ptr_, Poco::Condition & condition_, Poco::FastMutex & mutex_, + LiveViewBlockInputStream(StorageLiveView & storage_, std::shared_ptr blocks_ptr_, + std::shared_ptr blocks_metadata_ptr_, + std::shared_ptr active_ptr_, Poco::Condition & condition_, Poco::FastMutex & mutex_, int64_t length_, const UInt64 & heartbeat_delay_) - : storage(storage_), blocks_ptr(blocks_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_delay(heartbeat_delay_), blocks_hash("") + : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_delay(heartbeat_delay_), blocks_hash("") { /// grab active pointer active = active_ptr.lock(); @@ -190,9 +192,11 @@ protected: private: StorageLiveView & storage; std::shared_ptr blocks_ptr; + std::shared_ptr blocks_metadata_ptr; std::weak_ptr active_ptr; std::shared_ptr active; BlocksPtr blocks; + BlocksMetadataPtr blocks_metadata; Blocks::iterator it; Blocks::iterator end; Blocks::iterator begin; diff --git a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h new file mode 100644 index 00000000000..d9fdda97d21 --- /dev/null +++ b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h @@ -0,0 +1,227 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include + +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** + */ + +class LiveViewEventsBlockInputStream : public IBlockInputStream +{ + +using NonBlockingResult = std::pair; + +public: + ~LiveViewEventsBlockInputStream() override + { + /// Start storage no users thread + /// if we are the last active user + if (!storage.is_dropped && blocks_ptr.use_count() < 3) + storage.startNoUsersThread(); + } + /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update + /// and LIMIT 0 just returns data without waiting for any updates + LiveViewEventsBlockInputStream(StorageLiveView & storage_, std::shared_ptr blocks_ptr_, std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, Poco::Condition & condition_, Poco::FastMutex & mutex_, + int64_t length_, const UInt64 & heartbeat_delay_) + : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_delay(heartbeat_delay_) + { + /// grab active pointer + active = active_ptr.lock(); + } + + String getName() const override { return "LiveViewEventsBlockInputStream"; } + + void cancel(bool kill) override + { + if (isCancelled() || storage.is_dropped) + return; + IBlockInputStream::cancel(kill); + Poco::FastMutex::ScopedLock lock(mutex); + condition.broadcast(); + } + + Block getHeader() const override { return storage.getHeader(); } + + void refresh() + { + if (active && blocks && it == end) + it = blocks->begin(); + } + + void suspend() + { + active.reset(); + } + + void resume() + { + active = active_ptr.lock(); + { + if (!blocks || blocks.get() != (*blocks_ptr).get()) { + blocks = (*blocks_ptr); + blocks_metadata = (*blocks_metadata_ptr); + } + } + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + + NonBlockingResult tryRead() + { + return tryRead_(false); + } + +protected: + Block readImpl() override + { + /// try reading + return tryRead_(true).first; + } + + /** tryRead method attempts to read a block in either blocking + * or non-blocking mode. If blocking is set to false + * then method return empty block with flag set to false + * to indicate that method would block to get the next block. + */ + NonBlockingResult tryRead_(bool blocking) + { + if (length == 0) + { + return { Block(), true }; + } + /// If blocks were never assigned get blocks + if (!blocks) + { + Poco::FastMutex::ScopedLock lock(mutex); + if (!active) + return { Block(), false }; + blocks = (*blocks_ptr); + blocks_metadata = (*blocks_metadata_ptr); + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + + if (isCancelled() || storage.is_dropped) + { + return { Block(), true }; + } + + if (it == end) + { + { + Poco::FastMutex::ScopedLock lock(mutex); + if (!active) + return { Block(), false }; + /// If we are done iterating over our blocks + /// and there are new blocks availble then get them + if (blocks.get() != (*blocks_ptr).get()) + { + blocks = (*blocks_ptr); + blocks_metadata = (*blocks_metadata_ptr); + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + /// No new blocks available wait for new ones + else + { + if (!blocking) + { + return { Block(), false }; + } + while (true) + { + bool signaled = condition.tryWait(mutex, std::max((UInt64)0, heartbeat_delay - ((UInt64)timestamp.epochMicroseconds() - last_event_timestamp)) / 1000); + + if (isCancelled() || storage.is_dropped) + { + return { Block(), true }; + } + if (signaled) + { + break; + } + else + { + // return Block(version, hash) + //hashmap["blocks"] = blocks_hash; + last_event_timestamp = (UInt64)timestamp.epochMicroseconds(); + //heartbeat(Heartbeat(last_event_timestamp, std::move(hashmap))); + } + } + } + } + return tryRead_(blocking); + } + + // move right to the end + it = end; + + if (it == end) + { + if (length > 0) + --length; + } + + last_event_timestamp = (UInt64)timestamp.epochMicroseconds(); + + Block res{ + ColumnWithTypeAndName( + DataTypeUInt64().createColumnConst(1, blocks_metadata->version)->convertToFullColumnIfConst(), + std::make_shared(), + "version"), + ColumnWithTypeAndName( + DataTypeString().createColumnConst(1, blocks_metadata->hash)->convertToFullColumnIfConst(), + std::make_shared(), + "hash"), + + }; + + return { res, true }; + } + +private: + StorageLiveView & storage; + std::shared_ptr blocks_ptr; + std::shared_ptr blocks_metadata_ptr; + std::weak_ptr active_ptr; + std::shared_ptr active; + BlocksPtr blocks; + BlocksMetadataPtr blocks_metadata; + Blocks::iterator it; + Blocks::iterator end; + Blocks::iterator begin; + Poco::Condition & condition; + Poco::FastMutex & mutex; + /// Length specifies number of updates to send, default -1 (no limit) + int64_t length; + UInt64 heartbeat_delay; + UInt64 last_event_timestamp{0}; + Poco::Timestamp timestamp; +}; + +} diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 195c5edcb07..9ac8fd2ce50 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -72,8 +73,15 @@ void PushingToViewsBlockOutputStream::write(const Block & block) */ Nested::validateArraySizes(block); - if (output) - output->write(block); + if (auto * live_view = dynamic_cast(storage.get())) + { + StorageLiveView::writeIntoLiveView(*live_view, block, context, output); + } + else + { + if (output) + output->write(block); + } /// Don't process materialized views if this block is duplicate if (replicated_output && replicated_output->lastBlockIsDuplicate()) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h index 3381a828ff0..8aa927ca84f 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -5,7 +5,7 @@ #include #include #include - +#include namespace DB { diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index 2d0fc6f4b33..5d0aec84e8b 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -40,7 +40,7 @@ String getTableDefinitionFromCreateQuery(const ASTPtr & query) create.replace_view = false; /// For views it is necessary to save the SELECT query itself, for the rest - on the contrary - if (!create.is_view && !create.is_materialized_view) + if (!create.is_view && !create.is_materialized_view && !create.is_live_view) create.select = nullptr; create.format = nullptr; diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index 8751ff067b1..6d2fc61f080 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -8,7 +8,9 @@ #include #include #include +#include #include +#include #include @@ -48,6 +50,7 @@ BlockIO InterpreterAlterQuery::execute() AlterCommands alter_commands; PartitionCommands partition_commands; MutationCommands mutation_commands; + LiveViewCommands live_view_commands; for (ASTAlterCommand * command_ast : alter.command_list->commands) { if (auto alter_command = AlterCommand::parse(command_ast)) @@ -56,6 +59,8 @@ BlockIO InterpreterAlterQuery::execute() partition_commands.emplace_back(std::move(*partition_command)); else if (auto mut_command = MutationCommand::parse(command_ast)) mutation_commands.emplace_back(std::move(*mut_command)); + else if (auto live_view_command = LiveViewCommand::parse(command_ast)) + live_view_commands.emplace_back(std::move(*live_view_command)); else throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR); } @@ -72,6 +77,21 @@ BlockIO InterpreterAlterQuery::execute() table->alterPartition(query_ptr, partition_commands, context); } + if (!live_view_commands.empty()) + { + live_view_commands.validate(*table); + for (const LiveViewCommand & command : live_view_commands) + { + auto live_view = std::dynamic_pointer_cast(table); + switch (command.type) + { + case LiveViewCommand::REFRESH: + live_view->refresh(context); + break; + } + } + } + if (!alter_commands.empty()) { auto table_lock_holder = table->lockAlterIntention(context.getCurrentQueryId()); diff --git a/dbms/src/Interpreters/InterpreterFactory.cpp b/dbms/src/Interpreters/InterpreterFactory.cpp index 54611860227..eaba1d1d9dc 100644 --- a/dbms/src/Interpreters/InterpreterFactory.cpp +++ b/dbms/src/Interpreters/InterpreterFactory.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -35,6 +36,7 @@ #include #include #include +#include #include @@ -173,6 +175,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & throwIfNoAccess(context); return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY); } diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index e614f64d208..15d89f5aaa7 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -45,6 +45,11 @@ ASTPtr ASTAlterCommand::clone() const res->ttl = ttl->clone(); res->children.push_back(res->ttl); } + if (values) + { + res->values = values->clone(); + res->children.push_back(res->values); + } return res; } @@ -172,6 +177,46 @@ void ASTAlterCommand::formatImpl( settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : ""); predicate->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::LIVE_VIEW_REFRESH) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : ""); + } + else if (type == ASTAlterCommand::LIVE_CHANNEL_ADD) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD " << (settings.hilite ? hilite_none : ""); + + values->formatImpl(settings, state, frame); + } + else if (type == ASTAlterCommand::LIVE_CHANNEL_DROP) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DROP " << (settings.hilite ? hilite_none : ""); + + values->formatImpl(settings, state, frame); + } + else if (type == ASTAlterCommand::LIVE_CHANNEL_MODIFY) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY " << (settings.hilite ? hilite_none : ""); + + values->formatImpl(settings, state, frame); + } + else if (type == ASTAlterCommand::LIVE_CHANNEL_SUSPEND) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "SUSPEND " << (settings.hilite ? hilite_none : ""); + + values->formatImpl(settings, state, frame); + } + else if (type == ASTAlterCommand::LIVE_CHANNEL_RESUME) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "RESUME " << (settings.hilite ? hilite_none : ""); + + values->formatImpl(settings, state, frame); + } + else if (type == ASTAlterCommand::LIVE_CHANNEL_REFRESH) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : ""); + + values->formatImpl(settings, state, frame); + } else if (type == ASTAlterCommand::COMMENT_COLUMN) { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "COMMENT COLUMN " << (settings.hilite ? hilite_none : ""); @@ -236,7 +281,12 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (settings.hilite ? hilite_none : ""); + if (is_live_view) + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER LIVE VIEW " << (settings.hilite ? hilite_none : ""); + else if (is_live_channel) + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER LIVE CHANNEL " << (settings.hilite ? hilite_none : ""); + else + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (settings.hilite ? hilite_none : ""); if (!table.empty()) { diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index 2c4b3ddbaf1..3f81eab6ba6 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -15,6 +15,15 @@ namespace DB * MODIFY COLUMN col_name type, * DROP PARTITION partition, * COMMENT_COLUMN col_name 'comment', + * ALTER LIVE VIEW [db.]name_type + * REFRESH + * ALTER CHANNEL [db.]name_type + * ADD live_view,... + * DROP live_view,... + * SUSPEND live_view,... + * RESUME live_view,... + * REFRESH live_view,... + * MODIFY live_view,... */ class ASTAlterCommand : public IAST @@ -43,6 +52,15 @@ public: UPDATE, NO_TYPE, + + LIVE_VIEW_REFRESH, + + LIVE_CHANNEL_ADD, + LIVE_CHANNEL_DROP, + LIVE_CHANNEL_SUSPEND, + LIVE_CHANNEL_RESUME, + LIVE_CHANNEL_REFRESH, + LIVE_CHANNEL_MODIFY }; Type type = NO_TYPE; @@ -88,6 +106,10 @@ public: /// For MODIFY TTL query ASTPtr ttl; + /** In ALTER CHANNEL, ADD, DROP, SUSPEND, RESUME, REFRESH, MODIFY queries, the list of live views is stored here + */ + ASTPtr values; + bool detach = false; /// true for DETACH PARTITION bool part = false; /// true for ATTACH PART @@ -142,6 +164,9 @@ protected: class ASTAlterQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster { public: + bool is_live_view{false}; /// true for ALTER LIVE VIEW + bool is_live_channel{false}; /// true for ALTER LIVE CHANNEL + ASTAlterCommandList * command_list = nullptr; String getID(char) const override; diff --git a/dbms/src/Parsers/ASTCreateQuery.cpp b/dbms/src/Parsers/ASTCreateQuery.cpp index e99c543f5ec..1ed7e06ec45 100644 --- a/dbms/src/Parsers/ASTCreateQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuery.cpp @@ -173,6 +173,8 @@ ASTPtr ASTCreateQuery::clone() const res->set(res->storage, storage->clone()); if (select) res->set(res->select, select->clone()); + if (tables) + res->set(res->tables, tables->clone()); cloneOutputOptions(*res); @@ -204,6 +206,11 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat what = "VIEW"; if (is_materialized_view) what = "MATERIALIZED VIEW"; + if (is_live_view) + what = "LIVE VIEW"; + if (is_live_channel) + what = "LIVE CHANNEL"; + settings.ostr << (settings.hilite ? hilite_keyword : "") @@ -253,6 +260,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS" << settings.nl_or_ws << (settings.hilite ? hilite_none : ""); select->formatImpl(settings, state, frame); } + + if (tables) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH " << (settings.hilite ? hilite_none : ""); + tables->formatImpl(settings, state, frame); + } } } diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 2755e1a3d78..dedec37a4ad 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -55,9 +55,12 @@ public: bool if_not_exists{false}; bool is_view{false}; bool is_materialized_view{false}; + bool is_live_view{false}; + bool is_live_channel{false}; bool is_populate{false}; bool replace_view{false}; /// CREATE OR REPLACE VIEW ASTColumns * columns_list = nullptr; + ASTExpressionList *tables = nullptr; String to_database; /// For CREATE MATERIALIZED VIEW mv TO table. String to_table; ASTStorage * storage = nullptr; diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 98891bbdf5f..f8803cc3bc8 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -32,6 +32,13 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_add_index("ADD INDEX"); ParserKeyword s_drop_index("DROP INDEX"); + ParserKeyword s_add("ADD"); + ParserKeyword s_drop("DROP"); + ParserKeyword s_suspend("SUSPEND"); + ParserKeyword s_resume("RESUME"); + ParserKeyword s_refresh("REFRESH"); + ParserKeyword s_modify("MODIFY"); + ParserKeyword s_attach_partition("ATTACH PARTITION"); ParserKeyword s_detach_partition("DETACH PARTITION"); ParserKeyword s_drop_partition("DROP PARTITION"); @@ -63,234 +70,294 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserList parser_assignment_list( std::make_unique(), std::make_unique(TokenType::Comma), /* allow_empty = */ false); + ParserNameList values_p; - if (s_add_column.ignore(pos, expected)) + if (is_live_view) { - if (s_if_not_exists.ignore(pos, expected)) - command->if_not_exists = true; - - if (!parser_col_decl.parse(pos, command->col_decl, expected)) - return false; - - if (s_after.ignore(pos, expected)) + if (s_refresh.ignore(pos, expected)) { - if (!parser_name.parse(pos, command->column, expected)) - return false; + command->type = ASTAlterCommand::LIVE_VIEW_REFRESH; } - - command->type = ASTAlterCommand::ADD_COLUMN; - } - else if (s_drop_partition.ignore(pos, expected)) - { - if (!parser_partition.parse(pos, command->partition, expected)) + else return false; - - command->type = ASTAlterCommand::DROP_PARTITION; } - else if (s_drop_column.ignore(pos, expected)) + else if (is_live_channel) { - if (s_if_exists.ignore(pos, expected)) - command->if_exists = true; - - if (!parser_name.parse(pos, command->column, expected)) - return false; - - command->type = ASTAlterCommand::DROP_COLUMN; - command->detach = false; - } - else if (s_add_index.ignore(pos, expected)) - { - if (s_if_not_exists.ignore(pos, expected)) - command->if_not_exists = true; - - if (!parser_idx_decl.parse(pos, command->index_decl, expected)) - return false; - - if (s_after.ignore(pos, expected)) + if (s_add.ignore(pos, expected)) { - if (!parser_name.parse(pos, command->index, expected)) + if (!values_p.parse(pos, command->values, expected)) return false; + + command->type = ASTAlterCommand::LIVE_CHANNEL_ADD; } + else if (s_drop.ignore(pos, expected)) + { + if (!values_p.parse(pos, command->values, expected)) + return false; - command->type = ASTAlterCommand::ADD_INDEX; - } - else if (s_drop_index.ignore(pos, expected)) - { - if (s_if_exists.ignore(pos, expected)) - command->if_exists = true; + command->type = ASTAlterCommand::LIVE_CHANNEL_DROP; + } + else if (s_suspend.ignore(pos, expected)) + { + if (!values_p.parse(pos, command->values, expected)) + return false; - if (!parser_name.parse(pos, command->index, expected)) + command->type = ASTAlterCommand::LIVE_CHANNEL_SUSPEND; + } + else if (s_resume.ignore(pos, expected)) + { + if (!values_p.parse(pos, command->values, expected)) + return false; + + command->type = ASTAlterCommand::LIVE_CHANNEL_RESUME; + } + else if (s_refresh.ignore(pos, expected)) + { + if (!values_p.parse(pos, command->values, expected)) + return false; + + command->type = ASTAlterCommand::LIVE_CHANNEL_REFRESH; + } + else if (s_modify.ignore(pos, expected)) + { + if (!values_p.parse(pos, command->values, expected)) + return false; + + command->type = ASTAlterCommand::LIVE_CHANNEL_MODIFY; + } + else return false; - - command->type = ASTAlterCommand::DROP_INDEX; - command->detach = false; } - else if (s_clear_column.ignore(pos, expected)) + else { - if (s_if_exists.ignore(pos, expected)) - command->if_exists = true; + if (s_add_column.ignore(pos, expected)) + { + if (s_if_not_exists.ignore(pos, expected)) + command->if_not_exists = true; - if (!parser_name.parse(pos, command->column, expected)) - return false; + if (!parser_col_decl.parse(pos, command->col_decl, expected)) + return false; - command->type = ASTAlterCommand::DROP_COLUMN; - command->clear_column = true; - command->detach = false; + if (s_after.ignore(pos, expected)) + { + if (!parser_name.parse(pos, command->column, expected)) + return false; + } - if (s_in_partition.ignore(pos, expected)) + command->type = ASTAlterCommand::ADD_COLUMN; + } + else if (s_drop_partition.ignore(pos, expected)) { if (!parser_partition.parse(pos, command->partition, expected)) return false; + + command->type = ASTAlterCommand::DROP_PARTITION; } - } - else if (s_detach_partition.ignore(pos, expected)) - { - if (!parser_partition.parse(pos, command->partition, expected)) - return false; - - command->type = ASTAlterCommand::DROP_PARTITION; - command->detach = true; - } - else if (s_attach_partition.ignore(pos, expected)) - { - if (!parser_partition.parse(pos, command->partition, expected)) - return false; - - if (s_from.ignore(pos)) + else if (s_drop_column.ignore(pos, expected)) { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->column, expected)) + return false; + + command->type = ASTAlterCommand::DROP_COLUMN; + command->detach = false; + } + else if (s_add_index.ignore(pos, expected)) + { + if (s_if_not_exists.ignore(pos, expected)) + command->if_not_exists = true; + + if (!parser_idx_decl.parse(pos, command->index_decl, expected)) + return false; + + if (s_after.ignore(pos, expected)) + { + if (!parser_name.parse(pos, command->index, expected)) + return false; + } + + command->type = ASTAlterCommand::ADD_INDEX; + } + else if (s_drop_index.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->index, expected)) + return false; + + command->type = ASTAlterCommand::DROP_INDEX; + command->detach = false; + } + else if (s_clear_column.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->column, expected)) + return false; + + command->type = ASTAlterCommand::DROP_COLUMN; + command->clear_column = true; + command->detach = false; + + if (s_in_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + } + } + else if (s_detach_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + + command->type = ASTAlterCommand::DROP_PARTITION; + command->detach = true; + } + else if (s_attach_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + + if (s_from.ignore(pos)) + { + if (!parseDatabaseAndTableName(pos, expected, command->from_database, command->from_table)) + return false; + + command->replace = false; + command->type = ASTAlterCommand::REPLACE_PARTITION; + } + else + { + command->type = ASTAlterCommand::ATTACH_PARTITION; + } + } + else if (s_replace_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + + if (!s_from.ignore(pos, expected)) + return false; + if (!parseDatabaseAndTableName(pos, expected, command->from_database, command->from_table)) return false; - command->replace = false; + command->replace = true; command->type = ASTAlterCommand::REPLACE_PARTITION; } - else + else if (s_attach_part.ignore(pos, expected)) { + if (!parser_string_literal.parse(pos, command->partition, expected)) + return false; + + command->part = true; command->type = ASTAlterCommand::ATTACH_PARTITION; } - } - else if (s_replace_partition.ignore(pos, expected)) - { - if (!parser_partition.parse(pos, command->partition, expected)) - return false; - - if (!s_from.ignore(pos, expected)) - return false; - - if (!parseDatabaseAndTableName(pos, expected, command->from_database, command->from_table)) - return false; - - command->replace = true; - command->type = ASTAlterCommand::REPLACE_PARTITION; - } - else if (s_attach_part.ignore(pos, expected)) - { - if (!parser_string_literal.parse(pos, command->partition, expected)) - return false; - - command->part = true; - command->type = ASTAlterCommand::ATTACH_PARTITION; - } - else if (s_fetch_partition.ignore(pos, expected)) - { - if (!parser_partition.parse(pos, command->partition, expected)) - return false; - - if (!s_from.ignore(pos, expected)) - return false; - - ASTPtr ast_from; - if (!parser_string_literal.parse(pos, ast_from, expected)) - return false; - - command->from = ast_from->as().value.get(); - command->type = ASTAlterCommand::FETCH_PARTITION; - } - else if (s_freeze.ignore(pos, expected)) - { - if (s_partition.ignore(pos, expected)) + else if (s_fetch_partition.ignore(pos, expected)) { if (!parser_partition.parse(pos, command->partition, expected)) return false; - command->type = ASTAlterCommand::FREEZE_PARTITION; + if (!s_from.ignore(pos, expected)) + return false; + + ASTPtr ast_from; + if (!parser_string_literal.parse(pos, ast_from, expected)) + return false; + + command->from = ast_from->as().value.get(); + command->type = ASTAlterCommand::FETCH_PARTITION; + } + else if (s_freeze.ignore(pos, expected)) + { + if (s_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + + command->type = ASTAlterCommand::FREEZE_PARTITION; + } + else + { + command->type = ASTAlterCommand::FREEZE_ALL; + } + + /// WITH NAME 'name' - place local backup to directory with specified name + if (s_with.ignore(pos, expected)) + { + if (!s_name.ignore(pos, expected)) + return false; + + ASTPtr ast_with_name; + if (!parser_string_literal.parse(pos, ast_with_name, expected)) + return false; + + command->with_name = ast_with_name->as().value.get(); + } + } + else if (s_modify_column.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_modify_col_decl.parse(pos, command->col_decl, expected)) + return false; + + command->type = ASTAlterCommand::MODIFY_COLUMN; + } + else if (s_modify_order_by.ignore(pos, expected)) + { + if (!parser_exp_elem.parse(pos, command->order_by, expected)) + return false; + + command->type = ASTAlterCommand::MODIFY_ORDER_BY; + } + else if (s_delete_where.ignore(pos, expected)) + { + if (!parser_exp_elem.parse(pos, command->predicate, expected)) + return false; + + command->type = ASTAlterCommand::DELETE; + } + else if (s_update.ignore(pos, expected)) + { + if (!parser_assignment_list.parse(pos, command->update_assignments, expected)) + return false; + + if (!s_where.ignore(pos, expected)) + return false; + + if (!parser_exp_elem.parse(pos, command->predicate, expected)) + return false; + + command->type = ASTAlterCommand::UPDATE; + } + else if (s_comment_column.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->column, expected)) + return false; + + if (!parser_string_literal.parse(pos, command->comment, expected)) + return false; + + command->type = ASTAlterCommand::COMMENT_COLUMN; + } + else if (s_modify_ttl.ignore(pos, expected)) + { + if (!parser_exp_elem.parse(pos, command->ttl, expected)) + return false; + command->type = ASTAlterCommand::MODIFY_TTL; } else - { - command->type = ASTAlterCommand::FREEZE_ALL; - } - - /// WITH NAME 'name' - place local backup to directory with specified name - if (s_with.ignore(pos, expected)) - { - if (!s_name.ignore(pos, expected)) - return false; - - ASTPtr ast_with_name; - if (!parser_string_literal.parse(pos, ast_with_name, expected)) - return false; - - command->with_name = ast_with_name->as().value.get(); - } + return false; } - else if (s_modify_column.ignore(pos, expected)) - { - if (s_if_exists.ignore(pos, expected)) - command->if_exists = true; - - if (!parser_modify_col_decl.parse(pos, command->col_decl, expected)) - return false; - - command->type = ASTAlterCommand::MODIFY_COLUMN; - } - else if (s_modify_order_by.ignore(pos, expected)) - { - if (!parser_exp_elem.parse(pos, command->order_by, expected)) - return false; - - command->type = ASTAlterCommand::MODIFY_ORDER_BY; - } - else if (s_delete_where.ignore(pos, expected)) - { - if (!parser_exp_elem.parse(pos, command->predicate, expected)) - return false; - - command->type = ASTAlterCommand::DELETE; - } - else if (s_update.ignore(pos, expected)) - { - if (!parser_assignment_list.parse(pos, command->update_assignments, expected)) - return false; - - if (!s_where.ignore(pos, expected)) - return false; - - if (!parser_exp_elem.parse(pos, command->predicate, expected)) - return false; - - command->type = ASTAlterCommand::UPDATE; - } - else if (s_comment_column.ignore(pos, expected)) - { - if (s_if_exists.ignore(pos, expected)) - command->if_exists = true; - - if (!parser_name.parse(pos, command->column, expected)) - return false; - - if (!parser_string_literal.parse(pos, command->comment, expected)) - return false; - - command->type = ASTAlterCommand::COMMENT_COLUMN; - } - else if (s_modify_ttl.ignore(pos, expected)) - { - if (!parser_exp_elem.parse(pos, command->ttl, expected)) - return false; - command->type = ASTAlterCommand::MODIFY_TTL; - } - else - return false; if (command->col_decl) command->children.push_back(command->col_decl); @@ -304,6 +371,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->children.push_back(command->predicate); if (command->update_assignments) command->children.push_back(command->update_assignments); + if (command->values) + command->children.push_back(command->values); if (command->comment) command->children.push_back(command->comment); if (command->ttl) @@ -319,7 +388,7 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe node = command_list; ParserToken s_comma(TokenType::Comma); - ParserAlterCommand p_command; + ParserAlterCommand p_command(is_live_view, is_live_channel); do { @@ -368,8 +437,28 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) node = query; ParserKeyword s_alter_table("ALTER TABLE"); + ParserKeyword s_alter_live_view("ALTER LIVE VIEW"); + ParserKeyword s_alter_live_channel("ALTER LIVE CHANNEL"); + + bool is_live_view = false; + bool is_live_channel = false; + if (!s_alter_table.ignore(pos, expected)) - return false; + { + if (!s_alter_live_view.ignore(pos, expected)) + if (!s_alter_live_channel.ignore(pos, expected)) + return false; + else + is_live_channel = true; + else + is_live_view = true; + } + + if (is_live_view) + query->is_live_view = true; + + if (is_live_channel) + query->is_live_channel = true; if (!parseDatabaseAndTableName(pos, expected, query->database, query->table)) return false; @@ -382,7 +471,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } query->cluster = cluster_str; - ParserAlterCommandList p_command_list; + ParserAlterCommandList p_command_list(is_live_view, is_live_channel); ASTPtr command_list; if (!p_command_list.parse(pos, command_list, expected)) return false; diff --git a/dbms/src/Parsers/ParserAlterQuery.h b/dbms/src/Parsers/ParserAlterQuery.h index 282a4277e17..3dd7f89062b 100644 --- a/dbms/src/Parsers/ParserAlterQuery.h +++ b/dbms/src/Parsers/ParserAlterQuery.h @@ -19,6 +19,15 @@ namespace DB * [FREEZE [PARTITION] [WITH NAME name]] * [DELETE WHERE ...] * [UPDATE col_name = expr, ... WHERE ...] + * ALTER LIVE VIEW [db.name] + * [REFRESH] + * ALTER LIVE CHANNEL [db.name] [ON CLUSTER cluster] + * [ADD live_view, ...] + * [DROP live_view, ...] + * [SUSPEND live_view, ...] + * [RESUME live_view, ...] + * [REFRESH live_view, ...] + * [MODIFY live_view, ...] */ class ParserAlterQuery : public IParserBase @@ -34,6 +43,12 @@ class ParserAlterCommandList : public IParserBase protected: const char * getName() const { return "a list of ALTER commands"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); + +public: + bool is_live_view; + bool is_live_channel; + + ParserAlterCommandList(bool is_live_view = false, bool is_live_channel = false) : is_live_view(is_live_view), is_live_channel(is_live_channel) {} }; @@ -42,6 +57,12 @@ class ParserAlterCommand : public IParserBase protected: const char * getName() const { return "ALTER command"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); + +public: + bool is_live_view; + bool is_live_channel; + + ParserAlterCommand(bool is_live_view = false, bool is_live_channel = false) : is_live_view(is_live_view), is_live_channel(is_live_channel) {} }; diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index fd6665a5a2c..4064dde5213 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -94,6 +94,12 @@ bool ParserColumnDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & .parse(pos, node, expected); } +bool ParserNameList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) + .parse(pos, node, expected); +} + bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_type("TYPE"); @@ -309,7 +315,10 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_if_not_exists("IF NOT EXISTS"); ParserKeyword s_as("AS"); ParserKeyword s_view("VIEW"); + ParserKeyword s_with("WITH"); ParserKeyword s_materialized("MATERIALIZED"); + ParserKeyword s_live("LIVE"); + ParserKeyword s_channel("CHANNEL"); ParserKeyword s_populate("POPULATE"); ParserKeyword s_or_replace("OR REPLACE"); ParserToken s_dot(TokenType::Dot); @@ -319,6 +328,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserIdentifier name_p; ParserColumnsOrIndicesDeclarationList columns_or_indices_p; ParserSelectWithUnionQuery select_p; + ParserNameList names_p; ASTPtr database; ASTPtr table; @@ -329,11 +339,15 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr as_database; ASTPtr as_table; ASTPtr select; + ASTPtr tables; + String cluster_str; bool attach = false; bool if_not_exists = false; bool is_view = false; bool is_materialized_view = false; + bool is_live_view = false; + bool is_live_channel = false; bool is_populate = false; bool is_temporary = false; bool replace_view = false; @@ -426,6 +440,80 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } } } + else if (s_live.ignore(pos, expected)) + { + if (s_channel.ignore(pos, expected)) + is_live_channel = true; + else if (s_view.ignore(pos, expected)) + is_live_view = true; + else + return false; + + if (s_if_not_exists.ignore(pos, expected)) + if_not_exists = true; + + if (!name_p.parse(pos, table, expected)) + return false; + + if (s_dot.ignore(pos, expected)) + { + database = table; + if (!name_p.parse(pos, table, expected)) + return false; + } + + if (ParserKeyword{"ON"}.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + if (!is_live_channel) + { + // TO [db.]table + if (ParserKeyword{"TO"}.ignore(pos, expected)) + { + if (!name_p.parse(pos, to_table, expected)) + return false; + + if (s_dot.ignore(pos, expected)) + { + to_database = to_table; + if (!name_p.parse(pos, to_table, expected)) + return false; + } + } + } + + /// Optional - a list of columns can be specified. It must fully comply with SELECT. + if (s_lparen.ignore(pos, expected)) + { + if (!columns_or_indices_p.parse(pos, columns_list, expected)) + return false; + + if (!s_rparen.ignore(pos, expected)) + return false; + } + + if (is_live_channel) + { + if (s_with.ignore(pos, expected)) + { + if (!names_p.parse(pos, tables, expected)) + return false; + } + } + else + { + /// AS SELECT ... + if (!s_as.ignore(pos, expected)) + return false; + + ParserSelectWithUnionQuery select_p; + if (!select_p.parse(pos, select, expected)) + return false; + } + } else if (is_temporary) return false; else if (s_database.ignore(pos, expected)) @@ -530,6 +618,8 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->if_not_exists = if_not_exists; query->is_view = is_view; query->is_materialized_view = is_materialized_view; + query->is_live_view = is_live_view; + query->is_live_channel = is_live_channel; query->is_populate = is_populate; query->temporary = is_temporary; query->replace_view = replace_view; @@ -543,6 +633,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->set(query->columns_list, columns_list); query->set(query->storage, storage); + query->set(query->tables, tables); getIdentifierName(as_database, query->as_database); getIdentifierName(as_table, query->as_table); diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index bd3c8f671f0..2a6dc44e2db 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -91,6 +91,14 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); }; +/** List of table names. */ +class ParserNameList : public IParserBase +{ +protected: + const char * getName() const { return "name list"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); +}; + template class IParserColumnDeclaration : public IParserBase @@ -300,7 +308,7 @@ protected: * CREATE|ATTACH DATABASE db [ENGINE = engine] * * Or: - * CREATE [OR REPLACE]|ATTACH [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... + * CREATE[OR REPLACE]|ATTACH [[MATERIALIZED] VIEW] | [[TEMPORARY] LIVE [CHANNEL] | [VIEW]] [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... */ class ParserCreateQuery : public IParserBase { diff --git a/dbms/src/Parsers/ParserQueryWithOutput.cpp b/dbms/src/Parsers/ParserQueryWithOutput.cpp index c41e0946a96..1c44c639848 100644 --- a/dbms/src/Parsers/ParserQueryWithOutput.cpp +++ b/dbms/src/Parsers/ParserQueryWithOutput.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -32,6 +33,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserCheckQuery check_p; ParserOptimizeQuery optimize_p; ParserKillQueryQuery kill_query_p; + ParserWatchQuery watch_p; ASTPtr query; @@ -57,7 +59,8 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec || drop_p.parse(pos, query, expected) || check_p.parse(pos, query, expected) || kill_query_p.parse(pos, query, expected) - || optimize_p.parse(pos, query, expected); + || optimize_p.parse(pos, query, expected) + || watch_p.parse(pos, query, expected); if (!parsed) return false; diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index f18592ebce5..96b0dbd63ab 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -144,6 +144,36 @@ public: */ virtual QueryProcessingStage::Enum getQueryProcessingStage(const Context &) const { return QueryProcessingStage::FetchColumns; } + /** Watch live changes to the table. + * Accepts a list of columns to read, as well as a description of the query, + * from which information can be extracted about how to retrieve data + * (indexes, locks, etc.) + * Returns a stream with which you can read data sequentially + * or multiple streams for parallel data reading. + * The `processed_stage` info is also written to what stage the request was processed. + * (Normally, the function only reads the columns from the list, but in other cases, + * for example, the request can be partially processed on a remote server.) + * + * context contains settings for one query. + * Usually Storage does not care about these settings, since they are used in the interpreter. + * But, for example, for distributed query processing, the settings are passed to the remote server. + * + * num_streams - a recommendation, how many streams to return, + * if the storage can return a different number of streams. + * + * It is guaranteed that the structure of the table will not change over the lifetime of the returned streams (that is, there will not be ALTER, RENAME and DROP). + */ + virtual BlockInputStreams watch( + const Names & /*column_names*/, + const SelectQueryInfo & /*query_info*/, + const Context & /*context*/, + QueryProcessingStage::Enum & /*processed_stage*/, + size_t /*max_block_size*/, + unsigned /*num_streams*/) + { + throw Exception("Method watch is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + /** Read a set of columns from the table. * Accepts a list of columns to read, as well as a description of the query, * from which information can be extracted about how to retrieve data diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index 1dd49e5c99b..7d92ce0ea2c 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -60,6 +60,22 @@ StoragePtr StorageFactory::get( name = "View"; } + else if (query.is_live_view) + { + + if (query.storage) + throw Exception("Specifying ENGINE is not allowed for a LiveView", ErrorCodes::INCORRECT_QUERY); + + name = "LiveView"; + } + else if (query.is_live_channel) + { + + if (query.storage) + throw Exception("Specifying ENGINE is not allowed for a LiveChannel", ErrorCodes::INCORRECT_QUERY); + + name = "LiveChannel"; + } else { /// Check for some special types, that are not allowed to be stored in tables. Example: NULL data type. @@ -115,6 +131,18 @@ StoragePtr StorageFactory::get( "Direct creation of tables with ENGINE MaterializedView is not supported, use CREATE MATERIALIZED VIEW statement", ErrorCodes::INCORRECT_QUERY); } + else if (name == "LiveView") + { + throw Exception( + "Direct creation of tables with ENGINE LiveView is not supported, use CREATE LIVE VIEW statement", + ErrorCodes::INCORRECT_QUERY); + } + else if (name == "LiveChannel") + { + throw Exception( + "Direct creation of tables with ENGINE LiveChannel is not supported, use CREATE LIVE CHANNEL statement", + ErrorCodes::INCORRECT_QUERY); + } } } diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index 5a4e225e52c..f7012c1d007 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -19,6 +19,7 @@ limitations under the License. */ #include #include #include +#include #include #include @@ -133,6 +134,7 @@ StorageLiveView::StorageLiveView( is_temporary = query.temporary; blocks_ptr = std::make_shared(); + blocks_metadata_ptr = std::make_shared(); active_ptr = std::make_shared(true); } @@ -153,6 +155,7 @@ bool StorageLiveView::getNewBlocks() SipHash hash; UInt128 key; BlocksPtr new_blocks = std::make_shared(); + BlocksMetadataPtr new_blocks_metadata = std::make_shared(); BlocksPtr new_mergeable_blocks = std::make_shared(); InterpreterSelectQuery interpreter(inner_query->clone(), global_context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); @@ -180,14 +183,16 @@ bool StorageLiveView::getNewBlocks() /// if blocks are not in the same order bool updated = false; { - if (hash_key != key.toHexString()) + if (getBlocksHashKey() != key.toHexString()) { if (new_blocks->empty()) { new_blocks->push_back(getHeader()); } + new_blocks_metadata->hash = key.toHexString(); + new_blocks_metadata->version = getBlocksVersion() + 1; (*blocks_ptr) = new_blocks; - hash_key = key.toHexString(); + (*blocks_metadata_ptr) = new_blocks_metadata; updated = true; } } @@ -366,27 +371,54 @@ BlockInputStreams StorageLiveView::watch( if (query.limit_length) length = (int64_t)safeGet(typeid_cast(*query.limit_length).value); - auto reader = std::make_shared(*this, blocks_ptr, active_ptr, condition, mutex, length, context.getSettingsRef().heartbeat_delay); - - if (no_users_thread.joinable()) + if (query.is_watch_events) { - Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); - noUsersThreadWakeUp = true; - noUsersThreadCondition.signal(); - } + auto reader = std::make_shared(*this, blocks_ptr, blocks_metadata_ptr, active_ptr, condition, mutex, length, context.getSettingsRef().heartbeat_delay); - { - Poco::FastMutex::ScopedLock lock(mutex); - if (!(*blocks_ptr)) + if (no_users_thread.joinable()) { - if (getNewBlocks()) - condition.broadcast(); + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); + noUsersThreadWakeUp = true; + noUsersThreadCondition.signal(); } + + { + Poco::FastMutex::ScopedLock lock(mutex); + if (!(*blocks_ptr)) + { + if (getNewBlocks()) + condition.broadcast(); + } + } + + processed_stage = QueryProcessingStage::Complete; + + return { reader }; } + else + { + auto reader = std::make_shared(*this, blocks_ptr, blocks_metadata_ptr, active_ptr, condition, mutex, length, context.getSettingsRef().heartbeat_delay); - processed_stage = QueryProcessingStage::Complete; + if (no_users_thread.joinable()) + { + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); + noUsersThreadWakeUp = true; + noUsersThreadCondition.signal(); + } - return { reader }; + { + Poco::FastMutex::ScopedLock lock(mutex); + if (!(*blocks_ptr)) + { + if (getNewBlocks()) + condition.broadcast(); + } + } + + processed_stage = QueryProcessingStage::Complete; + + return { reader }; + } } BlockOutputStreamPtr StorageLiveView::write(const ASTPtr & /*query*/, const Context & /*context*/) diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index cb90e2b3e71..9f54f19cffb 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -29,6 +29,14 @@ namespace DB class IAST; using ASTPtr = std::shared_ptr; +struct BlocksMetadata { + String hash; + UInt64 version; +}; + +using BlocksMetadataPtr = std::shared_ptr; +using SipHashPtr = std::shared_ptr; + class StorageLiveView : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; @@ -79,7 +87,16 @@ public: String getBlocksHashKey() { - return hash_key; + if (*blocks_metadata_ptr) + return (*blocks_metadata_ptr)->hash; + return ""; + } + + UInt64 getBlocksVersion() + { + if (*blocks_metadata_ptr) + return (*blocks_metadata_ptr)->version; + return -1; } /// Reset blocks @@ -87,8 +104,8 @@ public: void reset() { (*blocks_ptr).reset(); + (*blocks_metadata_ptr).reset(); mergeable_blocks.reset(); - hash_key = ""; } void checkTableCanBeDropped() const override; @@ -246,12 +263,11 @@ private: /// Current data blocks that store query result std::shared_ptr blocks_ptr; BlocksPtr new_blocks; + /// Current data blocks metadata + std::shared_ptr blocks_metadata_ptr; + BlocksMetadataPtr new_blocks_metadata; BlocksPtrs mergeable_blocks; - /// Current blocks hash key - String hash_key; - String new_hash_key; - void noUsersThread(); std::thread no_users_thread; std::atomic shutdown_called{false}; @@ -273,13 +289,27 @@ public: void write(const Block & block) override { + UInt128 key; + if (!new_blocks) + { new_blocks = std::make_shared(); + new_blocks_metadata = std::make_shared(); + new_hash = std::make_shared(); + } new_blocks->push_back(block); - // FIXME: do I need to calculate block hash? + block.updateHash(*new_hash); + new_hash->get128(key.low, key.high); + new_blocks_metadata->hash = key.toHexString(); + new_blocks_metadata->version = storage.getBlocksVersion() + 1; + (*storage.blocks_ptr) = new_blocks; + (*storage.blocks_metadata_ptr) = new_blocks_metadata; + new_blocks.reset(); + new_blocks_metadata.reset(); + new_hash.reset(); storage.condition.broadcast(); } @@ -287,7 +317,8 @@ public: private: BlocksPtr new_blocks; - String new_hash_key; + BlocksMetadataPtr new_blocks_metadata; + SipHashPtr new_hash; StorageLiveView & storage; }; diff --git a/dbms/src/Storages/registerStorages.cpp b/dbms/src/Storages/registerStorages.cpp index 4bd2d995104..fa90bb44817 100644 --- a/dbms/src/Storages/registerStorages.cpp +++ b/dbms/src/Storages/registerStorages.cpp @@ -23,6 +23,8 @@ void registerStorageSet(StorageFactory & factory); void registerStorageJoin(StorageFactory & factory); void registerStorageView(StorageFactory & factory); void registerStorageMaterializedView(StorageFactory & factory); +void registerStorageLiveView(StorageFactory & factory); +//void registerStorageLiveChannel(StorageFactory & factory); #if USE_HDFS void registerStorageHDFS(StorageFactory & factory); @@ -63,6 +65,8 @@ void registerStorages() registerStorageJoin(factory); registerStorageView(factory); registerStorageMaterializedView(factory); + registerStorageLiveView(factory); + //registerStorageLiveChannel(factory); #if USE_HDFS registerStorageHDFS(factory); From 673d1a46a05cf36a80051ad60c090410ffb51c9f Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Wed, 29 May 2019 17:52:13 -0400 Subject: [PATCH 044/509] * Adding support for _version virtual column to LIVE VIEWs when using SELECT * Adding support for _version column to LIVE VIEWS when using WATCH query * Adding initial support for WATCH query on LIVE VIEWs --- .../PushingToViewsBlockOutputStream.cpp | 35 ++++++++++++---- .../Interpreters/InterpreterCreateQuery.cpp | 9 +++- dbms/src/Storages/StorageLiveView.cpp | 25 +++++++++++ dbms/src/Storages/StorageLiveView.h | 41 ++++++++++++------- 4 files changed, 85 insertions(+), 25 deletions(-) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 9ac8fd2ce50..c826835b007 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -43,14 +43,25 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( for (const auto & database_table : dependencies) { auto dependent_table = context.getTable(database_table.first, database_table.second); - auto & materialized_view = dynamic_cast(*dependent_table); - if (StoragePtr inner_table = materialized_view.tryGetTargetTable()) - addTableLock(inner_table->lockStructureForShare(true, context.getCurrentQueryId())); + ASTPtr query; + BlockOutputStreamPtr out; + + if (auto * materialized_view = dynamic_cast(dependent_table.get())) + { + if (StoragePtr inner_table = materialized_view->tryGetTargetTable()) + addTableLock(inner_table->lockStructureForShare(true, context.getCurrentQueryId())); + + query = materialized_view->getInnerQuery(); + } + + if (dynamic_cast(dependent_table.get())) + out = std::make_shared( + database_table.first, database_table.second, dependent_table, *views_context, ASTPtr(), true); + else + out = std::make_shared( + database_table.first, database_table.second, dependent_table, *views_context, ASTPtr()); - auto query = materialized_view.getInnerQuery(); - BlockOutputStreamPtr out = std::make_shared( - database_table.first, database_table.second, dependent_table, *views_context, ASTPtr()); views.emplace_back(ViewInfo{std::move(query), database_table.first, database_table.second, std::move(out)}); } } @@ -75,7 +86,8 @@ void PushingToViewsBlockOutputStream::write(const Block & block) if (auto * live_view = dynamic_cast(storage.get())) { - StorageLiveView::writeIntoLiveView(*live_view, block, context, output); + BlockOutputStreamPtr output = storage->write(query_ptr, context); + StorageLiveView::writeIntoLiveView(*live_view, block, context, output); } else { @@ -169,8 +181,13 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n try { BlockInputStreamPtr from = std::make_shared(block); - InterpreterSelectQuery select(view.query, *views_context, from); - BlockInputStreamPtr in = std::make_shared(select.execute().in); + BlockInputStreamPtr in = from; + + if (view.query) + { + InterpreterSelectQuery select(view.query, *views_context, from); + in = std::make_shared(select.execute().in); + } /// Squashing is needed here because the materialized view query can generate a lot of blocks /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY /// and two-level aggregation is triggered). diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 035b299ac5e..92b1eccb952 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -459,6 +459,11 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const "Cannot CREATE a table AS " + as_database_name + "." + as_table_name + ", it is a View", ErrorCodes::INCORRECT_QUERY); + if (as_create.is_live_view) + throw Exception( + "Cannot CREATE a table AS " + as_database_name + "." + as_table_name + ", it is a Live View", + ErrorCodes::INCORRECT_QUERY); + create.set(create.storage, as_create.storage->ptr()); } } @@ -499,7 +504,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (create.to_database.empty()) create.to_database = current_database; - if (create.select && (create.is_view || create.is_materialized_view)) + if (create.select && (create.is_view || create.is_materialized_view || create.is_live_view)) { AddDefaultDatabaseVisitor visitor(current_database); visitor.visit(*create.select); @@ -596,7 +601,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) /// If the query is a CREATE SELECT, insert the data into the table. if (create.select && !create.attach - && !create.is_view && (!create.is_materialized_view || create.is_populate)) + && !create.is_view && !create.is_live_view && (!create.is_materialized_view || create.is_populate)) { auto insert = std::make_shared(); diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index f7012c1d007..7f010c2b3de 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -138,12 +138,32 @@ StorageLiveView::StorageLiveView( active_ptr = std::make_shared(true); } +NameAndTypePair StorageLiveView::getColumn(const String & column_name) const +{ + if ( column_name == "_version" ) + return NameAndTypePair("_version", std::make_shared()); + + return IStorage::getColumn(column_name); +} + +bool StorageLiveView::hasColumn(const String & column_name) const +{ + if ( column_name == "_version" ) + return true; + + return IStorage::hasColumn(column_name); +} + Block StorageLiveView::getHeader() const { if (!sample_block) { auto storage = global_context.getTable(select_database_name, select_table_name); sample_block = InterpreterSelectQuery(inner_query, global_context, storage).getSampleBlock(); + sample_block.insert({DataTypeUInt64().createColumnConst( + sample_block.rows(), 0)->convertToFullColumnIfConst(), + std::make_shared(), + "_version"}); } return sample_block; @@ -172,6 +192,11 @@ bool StorageLiveView::getNewBlocks() BlockInputStreamPtr data = std::make_shared(select.execute().in); while (Block block = data->read()) { + /// add result version meta column + block.insert({DataTypeUInt64().createColumnConst( + block.rows(), getBlocksVersion() + 1)->convertToFullColumnIfConst(), + std::make_shared(), + "_version"}); block.updateHash(hash); new_blocks->push_back(block); } diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index 9f54f19cffb..0c6801f9f61 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -18,6 +18,7 @@ limitations under the License. */ #include #include #include +#include #include #include #include @@ -50,6 +51,9 @@ public: String getSelectDatabaseName() const { return select_database_name; } String getSelectTableName() const { return select_table_name; } + NameAndTypePair getColumn(const String & column_name) const override; + bool hasColumn(const String & column_name) const override; + // const NamesAndTypesList & getColumnsListImpl() const override { return *columns; } ASTPtr getInnerQuery() const { return inner_query->clone(); }; @@ -96,7 +100,7 @@ public: { if (*blocks_metadata_ptr) return (*blocks_metadata_ptr)->version; - return -1; + return 0; } /// Reset blocks @@ -167,7 +171,6 @@ public: BlocksPtr blocks = std::make_shared(); BlocksPtrs mergeable_blocks; BlocksPtr new_mergeable_blocks = std::make_shared(); - { auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); BlockInputStreams streams = {std::make_shared(block)}; @@ -227,6 +230,7 @@ public: auto proxy_storage = std::make_shared(parent_storage, std::move(from), QueryProcessingStage::WithMergeableState); InterpreterSelectQuery select(live_view.getInnerQuery(), context, proxy_storage, QueryProcessingStage::Complete); BlockInputStreamPtr data = std::make_shared(select.execute().in); + while (Block this_block = data->read()) { this_block.updateHash(hash); @@ -242,7 +246,6 @@ public: auto sample_block = blocks->front().cloneEmpty(); BlockInputStreamPtr new_data = std::make_shared(std::make_shared(blocks), sample_block); { - Poco::FastMutex::ScopedLock lock(live_view.mutex); copyData(*new_data, *output); } } @@ -287,22 +290,21 @@ class LiveViewBlockOutputStream : public IBlockOutputStream public: explicit LiveViewBlockOutputStream(StorageLiveView & storage_) : storage(storage_) {} - void write(const Block & block) override + void writePrefix() override { + new_blocks = std::make_shared(); + new_blocks_metadata = std::make_shared(); + new_hash = std::make_shared(); + new_blocks_metadata->version = storage.getBlocksVersion() + 1; + } + + void writeSuffix() override + { + Poco::FastMutex::ScopedLock lock(storage.mutex); UInt128 key; - if (!new_blocks) - { - new_blocks = std::make_shared(); - new_blocks_metadata = std::make_shared(); - new_hash = std::make_shared(); - } - - new_blocks->push_back(block); - block.updateHash(*new_hash); new_hash->get128(key.low, key.high); new_blocks_metadata->hash = key.toHexString(); - new_blocks_metadata->version = storage.getBlocksVersion() + 1; (*storage.blocks_ptr) = new_blocks; (*storage.blocks_metadata_ptr) = new_blocks_metadata; @@ -310,9 +312,20 @@ public: new_blocks.reset(); new_blocks_metadata.reset(); new_hash.reset(); + storage.condition.broadcast(); } + void write(const Block & block) override + { + new_blocks->push_back(block); + new_blocks->back().insert({DataTypeUInt64().createColumnConst( + block.rows(), new_blocks_metadata->version)->convertToFullColumnIfConst(), + std::make_shared(), + "_version"}); + block.updateHash(*new_hash); + } + Block getHeader() const override { return storage.getHeader(); } private: From f06f0e3947a94782d8dedb0f7305941149731fb4 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 30 May 2019 17:29:30 -0400 Subject: [PATCH 045/509] * Adding support for CREATE TEMPORARY LIVE VIEW * Fixing issue with setting _version virtual column --- .../Interpreters/InterpreterCreateQuery.cpp | 2 +- .../Interpreters/InterpreterWatchQuery.cpp | 6 ++++++ dbms/src/Storages/StorageLiveView.cpp | 11 +++++----- dbms/src/Storages/StorageLiveView.h | 20 ++++++++++++------- 4 files changed, 26 insertions(+), 13 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 92b1eccb952..aed7a60b805 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -436,7 +436,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const return; } - if (create.temporary) + if (create.temporary && !create.is_live_view) { auto engine_ast = std::make_shared(); engine_ast->name = "Memory"; diff --git a/dbms/src/Interpreters/InterpreterWatchQuery.cpp b/dbms/src/Interpreters/InterpreterWatchQuery.cpp index 4672a42a304..3ba8e2eadaa 100644 --- a/dbms/src/Interpreters/InterpreterWatchQuery.cpp +++ b/dbms/src/Interpreters/InterpreterWatchQuery.cpp @@ -23,6 +23,7 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_STORAGE; + extern const int UNKNOWN_TABLE; extern const int TOO_MANY_COLUMNS; } @@ -49,6 +50,11 @@ BlockIO InterpreterWatchQuery::execute() /// Get storage storage = context.tryGetTable(database, table); + if (!storage) + throw Exception("Table " + backQuoteIfNeed(database) + "." + + backQuoteIfNeed(table) + " doesn't exist.", + ErrorCodes::UNKNOWN_TABLE); + /// List of columns to read to execute the query. Names required_columns = storage->getColumns().getNamesOfPhysical(); diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index 7f010c2b3de..799a2193932 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -192,12 +192,13 @@ bool StorageLiveView::getNewBlocks() BlockInputStreamPtr data = std::make_shared(select.execute().in); while (Block block = data->read()) { + /// calculate hash before virtual column is added + block.updateHash(hash); /// add result version meta column block.insert({DataTypeUInt64().createColumnConst( block.rows(), getBlocksVersion() + 1)->convertToFullColumnIfConst(), std::make_shared(), "_version"}); - block.updateHash(hash); new_blocks->push_back(block); } @@ -411,8 +412,8 @@ BlockInputStreams StorageLiveView::watch( Poco::FastMutex::ScopedLock lock(mutex); if (!(*blocks_ptr)) { - if (getNewBlocks()) - condition.broadcast(); + if (getNewBlocks()) + condition.broadcast(); } } @@ -435,8 +436,8 @@ BlockInputStreams StorageLiveView::watch( Poco::FastMutex::ScopedLock lock(mutex); if (!(*blocks_ptr)) { - if (getNewBlocks()) - condition.broadcast(); + if (getNewBlocks()) + condition.broadcast(); } } diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index 0c6801f9f61..d16a80ee888 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -88,14 +88,16 @@ public: Poco::FastMutex noUsersThreadMutex; bool noUsersThreadWakeUp{false}; Poco::Condition noUsersThreadCondition; - + /// Get blocks hash + /// must be called with mutex locked String getBlocksHashKey() { if (*blocks_metadata_ptr) return (*blocks_metadata_ptr)->hash; return ""; } - + /// Get blocks version + /// must be called with mutex locked UInt64 getBlocksVersion() { if (*blocks_metadata_ptr) @@ -295,7 +297,6 @@ public: new_blocks = std::make_shared(); new_blocks_metadata = std::make_shared(); new_hash = std::make_shared(); - new_blocks_metadata->version = storage.getBlocksVersion() + 1; } void writeSuffix() override @@ -305,6 +306,15 @@ public: new_hash->get128(key.low, key.high); new_blocks_metadata->hash = key.toHexString(); + new_blocks_metadata->version = storage.getBlocksVersion() + 1; + + for (auto & block : *new_blocks) + { + block.insert({DataTypeUInt64().createColumnConst( + block.rows(), new_blocks_metadata->version)->convertToFullColumnIfConst(), + std::make_shared(), + "_version"}); + } (*storage.blocks_ptr) = new_blocks; (*storage.blocks_metadata_ptr) = new_blocks_metadata; @@ -319,10 +329,6 @@ public: void write(const Block & block) override { new_blocks->push_back(block); - new_blocks->back().insert({DataTypeUInt64().createColumnConst( - block.rows(), new_blocks_metadata->version)->convertToFullColumnIfConst(), - std::make_shared(), - "_version"}); block.updateHash(*new_hash); } From e7293486bd39103a25d4d51ca4e8e8597c1c11ce Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 2 Jun 2019 17:41:12 +0300 Subject: [PATCH 046/509] Added ALTER TABLE support to constraints (ADD CONSTRAINT, DROP CONSTRAINT) --- dbms/src/Databases/DatabaseDictionary.cpp | 1 + dbms/src/Databases/DatabaseDictionary.h | 1 + dbms/src/Databases/DatabaseMemory.cpp | 1 + dbms/src/Databases/DatabaseMemory.h | 1 + dbms/src/Databases/DatabaseOrdinary.cpp | 7 ++ dbms/src/Databases/DatabaseOrdinary.h | 1 + dbms/src/Databases/IDatabase.h | 2 + dbms/src/Parsers/ASTAlterQuery.cpp | 11 +++ dbms/src/Parsers/ASTAlterQuery.h | 13 ++- dbms/src/Parsers/ParserAlterQuery.cpp | 25 ++++++ dbms/src/Storages/AlterCommands.cpp | 80 ++++++++++++++++++- dbms/src/Storages/AlterCommands.h | 17 +++- dbms/src/Storages/IStorage.cpp | 3 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 13 +-- dbms/src/Storages/MergeTree/MergeTreeData.h | 3 +- .../ReplicatedMergeTreeTableMetadata.cpp | 16 ++++ .../ReplicatedMergeTreeTableMetadata.h | 6 +- dbms/src/Storages/StorageBuffer.cpp | 3 +- dbms/src/Storages/StorageDistributed.cpp | 3 +- dbms/src/Storages/StorageMerge.cpp | 3 +- dbms/src/Storages/StorageMergeTree.cpp | 16 ++-- dbms/src/Storages/StorageNull.cpp | 3 +- .../Storages/StorageReplicatedMergeTree.cpp | 14 +++- .../00953_constraints_operations.reference | 4 + .../00953_constraints_operations.sh | 40 ++++++++++ 25 files changed, 255 insertions(+), 32 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00953_constraints_operations.reference create mode 100755 dbms/tests/queries/0_stateless/00953_constraints_operations.sh diff --git a/dbms/src/Databases/DatabaseDictionary.cpp b/dbms/src/Databases/DatabaseDictionary.cpp index 195dcea5287..8add0fa8911 100644 --- a/dbms/src/Databases/DatabaseDictionary.cpp +++ b/dbms/src/Databases/DatabaseDictionary.cpp @@ -131,6 +131,7 @@ void DatabaseDictionary::alterTable( const String &, const ColumnsDescription &, const IndicesDescription &, + const ConstraintsDescription &, const ASTModifier &) { throw Exception("DatabaseDictionary: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); diff --git a/dbms/src/Databases/DatabaseDictionary.h b/dbms/src/Databases/DatabaseDictionary.h index 3bff84c36b8..d0ea33c6e4e 100644 --- a/dbms/src/Databases/DatabaseDictionary.h +++ b/dbms/src/Databases/DatabaseDictionary.h @@ -72,6 +72,7 @@ public: const String & name, const ColumnsDescription & columns, const IndicesDescription & indices, + const ConstraintsDescription & constraints, const ASTModifier & engine_modifier) override; time_t getTableMetadataModificationTime( diff --git a/dbms/src/Databases/DatabaseMemory.cpp b/dbms/src/Databases/DatabaseMemory.cpp index 3eea0bc666a..c53309ca6c1 100644 --- a/dbms/src/Databases/DatabaseMemory.cpp +++ b/dbms/src/Databases/DatabaseMemory.cpp @@ -54,6 +54,7 @@ void DatabaseMemory::alterTable( const String &, const ColumnsDescription &, const IndicesDescription &, + const ConstraintsDescription &, const ASTModifier &) { throw Exception("DatabaseMemory: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index fe7cc783ba3..dc770373360 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -49,6 +49,7 @@ public: const String & name, const ColumnsDescription & columns, const IndicesDescription & indices, + const ConstraintsDescription & constraints, const ASTModifier & engine_modifier) override; time_t getTableMetadataModificationTime( diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 9fa7d1b1196..1ef67dfd9f4 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -516,6 +516,7 @@ void DatabaseOrdinary::alterTable( const String & table_name, const ColumnsDescription & columns, const IndicesDescription & indices, + const ConstraintsDescription & constraints, const ASTModifier & storage_modifier) { /// Read the definition of the table and replace the necessary parts with new ones. @@ -538,6 +539,7 @@ void DatabaseOrdinary::alterTable( ASTPtr new_columns = InterpreterCreateQuery::formatColumns(columns); ASTPtr new_indices = InterpreterCreateQuery::formatIndices(indices); + ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(constraints); ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns); @@ -546,6 +548,11 @@ void DatabaseOrdinary::alterTable( else ast_create_query.columns_list->set(ast_create_query.columns_list->indices, new_indices); + if (ast_create_query.columns_list->constraints) + ast_create_query.columns_list->replace(ast_create_query.columns_list->constraints, new_constraints); + else + ast_create_query.columns_list->set(ast_create_query.columns_list->constraints, new_constraints); + if (storage_modifier) storage_modifier(*ast_create_query.storage); diff --git a/dbms/src/Databases/DatabaseOrdinary.h b/dbms/src/Databases/DatabaseOrdinary.h index 887bf101d62..2ed1a426d64 100644 --- a/dbms/src/Databases/DatabaseOrdinary.h +++ b/dbms/src/Databases/DatabaseOrdinary.h @@ -43,6 +43,7 @@ public: const String & name, const ColumnsDescription & columns, const IndicesDescription & indices, + const ConstraintsDescription & constraints, const ASTModifier & engine_modifier) override; time_t getTableMetadataModificationTime( diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index d53de1dfcb5..37ee1e676e9 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -114,6 +115,7 @@ public: const String & name, const ColumnsDescription & columns, const IndicesDescription & indices, + const ConstraintsDescription & constraints, const ASTModifier & engine_modifier) = 0; /// Returns time of table's metadata change, 0 if there is no corresponding metadata file. diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index e614f64d208..a85890d3cd0 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -105,6 +105,17 @@ void ASTAlterCommand::formatImpl( << "DROP INDEX " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); index->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::ADD_CONSTRAINT) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD CONSTRAINT" << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : ""); + constraint_decl->formatImpl(settings, state, frame); + } + else if (type == ASTAlterCommand::DROP_CONSTRAINT) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str + << "DROP CONSTRAINT " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); + constraint->formatImpl(settings, state, frame); + } else if (type == ASTAlterCommand::DROP_PARTITION) { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << (detach ? "DETACH" : "DROP") << " PARTITION " diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index 2c4b3ddbaf1..2ca88f3145d 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -32,6 +32,9 @@ public: ADD_INDEX, DROP_INDEX, + ADD_CONSTRAINT, + DROP_CONSTRAINT, + DROP_PARTITION, ATTACH_PARTITION, REPLACE_PARTITION, @@ -69,7 +72,15 @@ public: /** The ADD INDEX query stores the name of the index following AFTER. * The DROP INDEX query stores the name for deletion. */ - ASTPtr index; + ASTPtr index; + + /** The ADD CONSTRAINT query stores the ConstraintDeclaration there. + */ + ASTPtr constraint_decl; + + /** The DROP CONSTRAINT query stores the name for deletion. + */ + ASTPtr constraint; /** Used in DROP PARTITION and ATTACH PARTITION FROM queries. * The value or ID of the partition is stored here. diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 98891bbdf5f..e4220fda868 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -32,6 +32,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_add_index("ADD INDEX"); ParserKeyword s_drop_index("DROP INDEX"); + ParserKeyword s_add_constraint("ADD CONSTRAINT"); + ParserKeyword s_drop_constraint("DROP CONSTRAINT"); + ParserKeyword s_attach_partition("ATTACH PARTITION"); ParserKeyword s_detach_partition("DETACH PARTITION"); ParserKeyword s_drop_partition("DROP PARTITION"); @@ -57,6 +60,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserStringLiteral parser_string_literal; ParserCompoundColumnDeclaration parser_col_decl; ParserIndexDeclaration parser_idx_decl; + ParserConstraintDeclaration parser_constraint_decl; ParserCompoundColumnDeclaration parser_modify_col_decl(false); ParserPartition parser_partition; ParserExpression parser_exp_elem; @@ -125,6 +129,27 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::DROP_INDEX; command->detach = false; } + else if (s_add_constraint.ignore(pos, expected)) + { + if (s_if_not_exists.ignore(pos, expected)) + command->if_not_exists = true; + + if (!parser_constraint_decl.parse(pos, command->constraint_decl, expected)) + return false; + + command->type = ASTAlterCommand::ADD_CONSTRAINT; + } + else if (s_drop_constraint.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->constraint, expected)) + return false; + + command->type = ASTAlterCommand::DROP_CONSTRAINT; + command->detach = false; + } else if (s_clear_column.ignore(pos, expected)) { if (s_if_exists.ignore(pos, expected)) diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 88f3e909f49..22095ec4ae3 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -165,6 +166,32 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ return command; } + else if (command_ast->type == ASTAlterCommand::ADD_CONSTRAINT) + { + AlterCommand command; + command.constraint_decl = command_ast->constraint_decl; + command.type = AlterCommand::ADD_CONSTRAINT; + + const auto & ast_constraint_decl = command_ast->constraint_decl->as(); + + command.constraint_name = ast_constraint_decl.name; + + command.if_not_exists = command_ast->if_not_exists; + + return command; + } + else if (command_ast->type == ASTAlterCommand::DROP_CONSTRAINT) + { + if (command_ast->clear_column) + throw Exception("\"ALTER TABLE table CLEAR COLUMN column\" queries are not supported yet. Use \"CLEAR COLUMN column IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED); + + AlterCommand command; + command.type = AlterCommand::DROP_CONSTRAINT; + command.constraint_name = command_ast->constraint->as().name; + command.if_exists = command_ast->if_exists; + + return command; + } else if (command_ast->type == ASTAlterCommand::MODIFY_TTL) { AlterCommand command; @@ -178,7 +205,8 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, - ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const + ConstraintsDescription & constraints_description, + ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const { if (type == ADD_COLUMN) { @@ -298,6 +326,44 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri indices_description.indices.erase(erase_it); } + else if (type == ADD_CONSTRAINT) + { + if (std::any_of( + constraints_description.constraints.cbegin(), + constraints_description.constraints.cend(), + [this](const ASTPtr & constraint_ast) + { + return constraint_ast->as().name == constraint_name; + })) + { + if (if_not_exists) + return; + throw Exception("Cannot add constraint " + constraint_name + ": constraint with this name already exists", + ErrorCodes::ILLEGAL_COLUMN); + } + + auto insert_it = constraints_description.constraints.end(); + + constraints_description.constraints.emplace(insert_it, std::dynamic_pointer_cast(constraint_decl)); + } + else if (type == DROP_CONSTRAINT) + { + auto erase_it = std::find_if( + constraints_description.constraints.begin(), + constraints_description.constraints.end(), + [this](const ASTPtr & constraint_ast) + { + return constraint_ast->as().name == constraint_name; + }); + + if (erase_it == constraints_description.constraints.end()) { + if (if_exists) + return; + throw Exception("Wrong constraint name. Cannot find constraint `" + constraint_name + "` to drop.", + ErrorCodes::LOGICAL_ERROR); + } + constraints_description.constraints.erase(erase_it); + } else if (type == MODIFY_TTL) { ttl_table_ast = ttl; @@ -317,20 +383,23 @@ bool AlterCommand::isMutable() const } void AlterCommands::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, - ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const + ConstraintsDescription & constraints_description, + ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const { auto new_columns_description = columns_description; auto new_indices_description = indices_description; + auto new_constraints_description = constraints_description; auto new_order_by_ast = order_by_ast; auto new_primary_key_ast = primary_key_ast; auto new_ttl_table_ast = ttl_table_ast; for (const AlterCommand & command : *this) if (!command.ignore) - command.apply(new_columns_description, new_indices_description, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast); + command.apply(new_columns_description, new_indices_description, new_constraints_description, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast); columns_description = std::move(new_columns_description); indices_description = std::move(new_indices_description); + constraints_description = std::move(new_constraints_description); order_by_ast = std::move(new_order_by_ast); primary_key_ast = std::move(new_primary_key_ast); ttl_table_ast = std::move(new_ttl_table_ast); @@ -518,10 +587,11 @@ void AlterCommands::apply(ColumnsDescription & columns_description) const { auto out_columns_description = columns_description; IndicesDescription indices_description; + ConstraintsDescription constraints_description; ASTPtr out_order_by; ASTPtr out_primary_key; ASTPtr out_ttl_table; - apply(out_columns_description, indices_description, out_order_by, out_primary_key, out_ttl_table); + apply(out_columns_description, indices_description, constraints_description, out_order_by, out_primary_key, out_ttl_table); if (out_order_by) throw Exception("Storage doesn't support modifying ORDER BY expression", ErrorCodes::NOT_IMPLEMENTED); @@ -529,6 +599,8 @@ void AlterCommands::apply(ColumnsDescription & columns_description) const throw Exception("Storage doesn't support modifying PRIMARY KEY expression", ErrorCodes::NOT_IMPLEMENTED); if (!indices_description.indices.empty()) throw Exception("Storage doesn't support modifying indices", ErrorCodes::NOT_IMPLEMENTED); + if (!constraints_description.constraints.empty()) + throw Exception("Storage doesn't support modifying constraints", ErrorCodes::NOT_IMPLEMENTED); if (out_ttl_table) throw Exception("Storage doesn't support modifying TTL expression", ErrorCodes::NOT_IMPLEMENTED); diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index 4905b80f92f..1dfd46f9617 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -25,6 +26,8 @@ struct AlterCommand MODIFY_ORDER_BY, ADD_INDEX, DROP_INDEX, + ADD_CONSTRAINT, + DROP_CONSTRAINT, MODIFY_TTL, UKNOWN_TYPE, }; @@ -62,6 +65,12 @@ struct AlterCommand /// For ADD/DROP INDEX String index_name; + // For ADD CONSTRAINT + ASTPtr constraint_decl; + + // For ADD/DROP CONSTRAINT + String constraint_name; + /// For MODIFY TTL ASTPtr ttl; @@ -84,7 +93,8 @@ struct AlterCommand static std::optional parse(const ASTAlterCommand * command); void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, - ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const; + ConstraintsDescription & constraints_description, + ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const; /// Checks that not only metadata touched by that command bool isMutable() const; @@ -95,8 +105,9 @@ class Context; class AlterCommands : public std::vector { public: - void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, ASTPtr & order_by_ast, - ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const; + void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, + ConstraintsDescription & constraints_description, + ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const; /// For storages that don't support MODIFY_ORDER_BY. void apply(ColumnsDescription & columns_description) const; diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 7c19fd94aea..07f52749d2d 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -346,8 +346,9 @@ void IStorage::alter( lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); auto new_columns = getColumns(); auto new_indices = getIndices(); + auto new_constraints = getConstraints(); params.apply(new_columns); - context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, {}); + context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {}); setColumns(std::move(new_columns)); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 8427982efd7..144c6591ed9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -121,7 +121,7 @@ MergeTreeData::MergeTreeData( data_parts_by_info(data_parts_indexes.get()), data_parts_by_state_and_info(data_parts_indexes.get()) { - setPrimaryKeyIndicesAndColumns(order_by_ast_, primary_key_ast_, columns_, indices_); + setPrimaryKeyIndicesAndColumns(order_by_ast_, primary_key_ast_, columns_, indices_, constraints_); setConstraints(constraints_); /// NOTE: using the same columns list as is read when performing actual merges. @@ -255,7 +255,8 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam void MergeTreeData::setPrimaryKeyIndicesAndColumns( const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast, - const ColumnsDescription & new_columns, const IndicesDescription & indices_description, bool only_check) + const ColumnsDescription & new_columns, const IndicesDescription & indices_description, + const ConstraintsDescription & constraints_description, bool only_check) { if (!new_order_by_ast) throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS); @@ -425,6 +426,8 @@ void MergeTreeData::setPrimaryKeyIndicesAndColumns( setIndices(indices_description); skip_indices = std::move(new_indices); + setConstraints(constraints_description); + primary_key_and_skip_indices_expr = new_indices_with_primary_key_expr; sorting_key_and_skip_indices_expr = new_indices_with_sorting_key_expr; } @@ -1180,11 +1183,11 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c /// Check that needed transformations can be applied to the list of columns without considering type conversions. auto new_columns = getColumns(); auto new_indices = getIndices(); + auto new_constraints = getConstraints(); ASTPtr new_order_by_ast = order_by_ast; ASTPtr new_primary_key_ast = primary_key_ast; ASTPtr new_ttl_table_ast = ttl_table_ast; - commands.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast); - + commands.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast); if (getIndices().empty() && !new_indices.empty() && !context.getSettingsRef().allow_experimental_data_skipping_indices) throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \ @@ -1267,7 +1270,7 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c } setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, - new_columns, new_indices, /* only_check = */ true); + new_columns, new_indices, new_constraints, /* only_check = */ true); setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast, /* only_check = */ true); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index d589bb77013..bfcfbaba9a1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -778,7 +778,8 @@ protected: void setPrimaryKeyIndicesAndColumns(const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast, const ColumnsDescription & new_columns, - const IndicesDescription & indices_description, bool only_check = false); + const IndicesDescription & indices_description, + const ConstraintsDescription & constraints_description, bool only_check = false); void initPartitionKey(); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index b122785c5fd..8549264e2c2 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -46,6 +46,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr partition_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.partition_by_ast)); skip_indices = data.getIndices().toString(); + constraints = data.getConstraints().toString(); index_granularity_bytes = data.index_granularity_info.index_granularity_bytes; ttl_table = formattedAST(data.ttl_table_ast); } @@ -229,6 +230,21 @@ ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTabl ErrorCodes::METADATA_MISMATCH); } + if (constraints != from_zk.constraints) + { + if (allow_alter) + { + diff.constraints_changed = true; + diff.new_constraints = from_zk.constraints; + } + else + throw Exception( + "Existing table metadata in ZooKeeper differs in constraints." + " Stored in ZooKeeper: " + from_zk.constraints + + ", local: " + constraints, + ErrorCodes::METADATA_MISMATCH); + } + if (index_granularity_bytes != from_zk.index_granularity_bytes) throw Exception("Existing table metadata in ZooKeeper differs in index granularity bytes." " Stored in ZooKeeper: " + DB::toString(from_zk.index_granularity_bytes) + diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h index 55cfdb1494d..b28a7306e33 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h @@ -26,6 +26,7 @@ struct ReplicatedMergeTreeTableMetadata String partition_key; String sorting_key; String skip_indices; + String constraints; UInt64 index_granularity_bytes; String ttl_table; @@ -46,10 +47,13 @@ struct ReplicatedMergeTreeTableMetadata bool skip_indices_changed = false; String new_skip_indices; + bool constraints_changed = false; + String new_constraints; + bool ttl_table_changed = false; String new_ttl_table; - bool empty() const { return !sorting_key_changed && !skip_indices_changed; } + bool empty() const { return !sorting_key_changed && !skip_indices_changed && !constraints_changed; } }; Diff checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk, bool allow_alter) const; diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 24e1ecef2e3..5a4409600d8 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -701,8 +701,9 @@ void StorageBuffer::alter(const AlterCommands & params, const String & database_ auto new_columns = getColumns(); auto new_indices = getIndices(); + auto new_constraints = getConstraints(); params.apply(new_columns); - context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, {}); + context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {}); setColumns(std::move(new_columns)); } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 4440d2b96ee..42e769019d6 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -346,8 +346,9 @@ void StorageDistributed::alter( auto new_columns = getColumns(); auto new_indices = getIndices(); + auto new_constraints = getConstraints(); params.apply(new_columns); - context.getDatabase(database_name)->alterTable(context, current_table_name, new_columns, new_indices, {}); + context.getDatabase(database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {}); setColumns(std::move(new_columns)); } diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 1fbde64fd85..9f3331a033e 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -417,8 +417,9 @@ void StorageMerge::alter( auto new_columns = getColumns(); auto new_indices = getIndices(); + auto new_constraints = getConstraints(); params.apply(new_columns); - context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, {}); + context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {}); setColumns(new_columns); } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 01049bfbf76..10709cd8181 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -242,8 +242,9 @@ void StorageMergeTree::alter( lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); auto new_columns = getColumns(); auto new_indices = getIndices(); + auto new_constraints = getConstraints(); params.apply(new_columns); - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {}); + context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {}); setColumns(std::move(new_columns)); return; } @@ -252,15 +253,14 @@ void StorageMergeTree::alter( auto merge_blocker = merger_mutator.actions_blocker.cancel(); lockNewDataStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - checkAlter(params, context); - auto new_columns = getColumns(); auto new_indices = getIndices(); + auto new_constraints = getConstraints(); ASTPtr new_order_by_ast = order_by_ast; ASTPtr new_primary_key_ast = primary_key_ast; ASTPtr new_ttl_table_ast = ttl_table_ast; - params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast); + params.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast); auto transactions = prepareAlterTransactions(new_columns, new_indices, context); @@ -279,11 +279,10 @@ void StorageMergeTree::alter( if (new_ttl_table_ast.get() != ttl_table_ast.get()) storage_ast.set(storage_ast.ttl_table, new_ttl_table_ast); }; - - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, storage_modifier); + context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, storage_modifier); /// Reinitialize primary key because primary key column types might have changed. - setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices); + setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints); setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast); @@ -834,10 +833,11 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi auto new_columns = getColumns(); auto new_indices = getIndices(); + auto new_constraints = getConstraints(); ASTPtr ignored_order_by_ast; ASTPtr ignored_primary_key_ast; ASTPtr ignored_ttl_table_ast; - alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast); + alter_command.apply(new_columns, new_indices, new_constraints, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast); auto columns_for_parts = new_columns.getAllPhysical(); for (const auto & part : parts) diff --git a/dbms/src/Storages/StorageNull.cpp b/dbms/src/Storages/StorageNull.cpp index 1762c8372f5..73cb0243c11 100644 --- a/dbms/src/Storages/StorageNull.cpp +++ b/dbms/src/Storages/StorageNull.cpp @@ -38,8 +38,9 @@ void StorageNull::alter( ColumnsDescription new_columns = getColumns(); IndicesDescription new_indices = getIndices(); + ConstraintsDescription new_constraints = getConstraints(); params.apply(new_columns); - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {}); + context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {}); setColumns(std::move(new_columns)); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index aaf8f18b65b..250a9dc96e8 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -422,6 +422,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column ASTPtr new_primary_key_ast = primary_key_ast; ASTPtr new_order_by_ast = order_by_ast; auto new_indices = getIndices(); + auto new_constraints = getConstraints(); ASTPtr new_ttl_table_ast = ttl_table_ast; IDatabase::ASTModifier storage_modifier; if (!metadata_diff.empty()) @@ -451,6 +452,9 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column if (metadata_diff.skip_indices_changed) new_indices = IndicesDescription::parse(metadata_diff.new_skip_indices); + if (metadata_diff.constraints_changed) + new_constraints = ConstraintsDescription::parse(metadata_diff.new_constraints); + if (metadata_diff.ttl_table_changed) { ParserExpression parser; @@ -476,11 +480,11 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column }; } - global_context.getDatabase(database_name)->alterTable(global_context, table_name, new_columns, new_indices, storage_modifier); + global_context.getDatabase(database_name)->alterTable(global_context, table_name, new_columns, new_indices, new_constraints, storage_modifier); /// Even if the primary/sorting keys didn't change we must reinitialize it /// because primary key column types might have changed. - setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices); + setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints); setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast); } @@ -1507,10 +1511,11 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & auto new_columns = getColumns(); auto new_indices = getIndices(); + auto new_constraints = getConstraints(); ASTPtr ignored_order_by_ast; ASTPtr ignored_primary_key_ast; ASTPtr ignored_ttl_table_ast; - alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast); + alter_command.apply(new_columns, new_indices, new_constraints, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast); size_t modified_parts = 0; auto parts = getDataParts(); @@ -3114,10 +3119,11 @@ void StorageReplicatedMergeTree::alter( ColumnsDescription new_columns = getColumns(); IndicesDescription new_indices = getIndices(); + ConstraintsDescription new_constraints = getConstraints(); ASTPtr new_order_by_ast = order_by_ast; ASTPtr new_primary_key_ast = primary_key_ast; ASTPtr new_ttl_table_ast = ttl_table_ast; - params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast); + params.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast); String new_columns_str = new_columns.toString(); if (new_columns_str != getColumns().toString()) diff --git a/dbms/tests/queries/0_stateless/00953_constraints_operations.reference b/dbms/tests/queries/0_stateless/00953_constraints_operations.reference new file mode 100644 index 00000000000..5713da9fef5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00953_constraints_operations.reference @@ -0,0 +1,4 @@ +1 2 +ok +1 2 +ok diff --git a/dbms/tests/queries/0_stateless/00953_constraints_operations.sh b/dbms/tests/queries/0_stateless/00953_constraints_operations.sh new file mode 100755 index 00000000000..917719c3e46 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00953_constraints_operations.sh @@ -0,0 +1,40 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +EXCEPTION_SUCCESS_TEXT=ok + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_constraints;" + +$CLICKHOUSE_CLIENT --query="CREATE TABLE test_constraints +( + a UInt32, + b UInt32, + CONSTRAINT b_constraint CHECK b > 0 +) +ENGINE = MergeTree ORDER BY (a);" + +# This one must succeed +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" +$CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" + +# This one must throw and exception +EXCEPTION_TEXT="Some constraints are not satisfied" +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 0);" 2>&1 \ + | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" +$CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" + +$CLICKHOUSE_CLIENT --query="ALTER TABLE test_constraints DROP CONSTRAINT b_constraint;" + +# This one must suceed now +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 0);" + +$CLICKHOUSE_CLIENT --query="ALTER TABLE test_constraints ADD CONSTRAINT b_constraint CHECK b > 10;" + +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 10);" 2>&1 \ + | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" + +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 11);" + +$CLICKHOUSE_CLIENT --query="DROP TABLE test_constraints;" \ No newline at end of file From 33f0ebd8ab83ef61990a66f7fb04d424f25cff5f Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 2 Jun 2019 18:08:28 +0300 Subject: [PATCH 047/509] Brace style fix --- dbms/src/Storages/AlterCommands.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 22095ec4ae3..0d20847727a 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -356,7 +356,8 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri return constraint_ast->as().name == constraint_name; }); - if (erase_it == constraints_description.constraints.end()) { + if (erase_it == constraints_description.constraints.end()) + { if (if_exists) return; throw Exception("Wrong constraint name. Cannot find constraint `" + constraint_name + "` to drop.", From fc21200bddafd553d75ea64c2debb202254f4ab4 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Mon, 3 Jun 2019 15:53:11 -0400 Subject: [PATCH 048/509] * Started to add LIVE VIEW table tests * Added uexpect.py module * Fixed support for CREATE TEMPORARY LIVE VIEW --- .../Interpreters/InterpreterCreateQuery.cpp | 6 +- dbms/src/Storages/StorageLiveView.cpp | 3 +- dbms/tests/clickhouse-test | 2 +- .../00952_live_view_create.reference | 1 + .../0_stateless/00952_live_view_create.sql | 9 + .../00953_live_view_select.reference | 4 + .../0_stateless/00953_live_view_select.sql | 14 ++ .../00954_live_view_select_version.reference | 4 + .../00954_live_view_select_version.sql | 14 ++ ...ive_view_select_with_aggregation.reference | 3 + ...0955_live_view_select_with_aggregation.sql | 18 ++ .../00956_live_view_watch_events.reference | 4 + .../00956_live_view_watch_events.sql | 20 ++ .../00957_live_view_watch.reference | 4 + .../0_stateless/00957_live_view_watch.sql | 20 ++ .../0_stateless/00958_live_view_watch_live.py | 46 +++++ .../00958_live_view_watch_live.reference | 0 ...00959_create_temporary_live_view.reference | 4 + .../00959_create_temporary_live_view.sql | 14 ++ .../00960_temporary_live_view_watch.reference | 4 + .../00960_temporary_live_view_watch.sql | 20 ++ .../00961_temporary_live_view_watch_live.py | 46 +++++ ...1_temporary_live_view_watch_live.reference | 0 ..._temporary_live_view_watch_live_timeout.py | 57 ++++++ ...ary_live_view_watch_live_timeout.reference | 0 dbms/tests/queries/0_stateless/uexpect.py | 187 ++++++++++++++++++ 26 files changed, 499 insertions(+), 5 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00952_live_view_create.reference create mode 100644 dbms/tests/queries/0_stateless/00952_live_view_create.sql create mode 100644 dbms/tests/queries/0_stateless/00953_live_view_select.reference create mode 100644 dbms/tests/queries/0_stateless/00953_live_view_select.sql create mode 100644 dbms/tests/queries/0_stateless/00954_live_view_select_version.reference create mode 100644 dbms/tests/queries/0_stateless/00954_live_view_select_version.sql create mode 100644 dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.reference create mode 100644 dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.sql create mode 100644 dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference create mode 100644 dbms/tests/queries/0_stateless/00956_live_view_watch_events.sql create mode 100644 dbms/tests/queries/0_stateless/00957_live_view_watch.reference create mode 100644 dbms/tests/queries/0_stateless/00957_live_view_watch.sql create mode 100755 dbms/tests/queries/0_stateless/00958_live_view_watch_live.py create mode 100644 dbms/tests/queries/0_stateless/00958_live_view_watch_live.reference create mode 100644 dbms/tests/queries/0_stateless/00959_create_temporary_live_view.reference create mode 100644 dbms/tests/queries/0_stateless/00959_create_temporary_live_view.sql create mode 100644 dbms/tests/queries/0_stateless/00960_temporary_live_view_watch.reference create mode 100644 dbms/tests/queries/0_stateless/00960_temporary_live_view_watch.sql create mode 100755 dbms/tests/queries/0_stateless/00961_temporary_live_view_watch_live.py create mode 100644 dbms/tests/queries/0_stateless/00961_temporary_live_view_watch_live.reference create mode 100755 dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live_timeout.py create mode 100644 dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live_timeout.reference create mode 100644 dbms/tests/queries/0_stateless/uexpect.py diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index aed7a60b805..5ce9af7f936 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -481,7 +481,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) } /// Temporary tables are created out of databases. - if (create.temporary && !create.database.empty()) + if (create.temporary && !create.database.empty() && !create.is_live_view) throw Exception("Temporary tables cannot be inside a database. You should not specify a database for a temporary table.", ErrorCodes::BAD_DATABASE_FOR_TEMPORARY_TABLE); @@ -539,7 +539,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) String data_path; DatabasePtr database; - if (!create.temporary) + if (!create.temporary || create.is_live_view) { database = context.getDatabase(database_name); data_path = database->getDataPath(); @@ -582,7 +582,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) create.attach, false); - if (create.temporary) + if (create.temporary && !create.is_live_view) context.getSessionContext().addExternalTable(table_name, res, query_ptr); else database->createTable(context, table_name, res, query_ptr); diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index 799a2193932..57e4695706b 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -243,9 +243,9 @@ void StorageLiveView::noUsersThread() bool drop_table = false; { - Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); while (1) { + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); if (!noUsersThreadWakeUp && !noUsersThreadCondition.tryWait(noUsersThreadMutex, global_context.getSettingsRef().temporary_live_view_timeout.totalSeconds() * 1000)) { noUsersThreadWakeUp = false; @@ -271,6 +271,7 @@ void StorageLiveView::noUsersThread() auto drop_query = std::make_shared(); drop_query->database = database_name; drop_query->table = table_name; + drop_query->kind = ASTDropQuery::Kind::Drop; ASTPtr ast_drop_query = drop_query; InterpreterDropQuery drop_interpreter(ast_drop_query, global_context); drop_interpreter.execute(); diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index fe7d45415bd..ba6f738b81c 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -258,7 +258,7 @@ def main(args): # Keep same default values as in queries/shell_config.sh os.environ.setdefault("CLICKHOUSE_BINARY", args.binary) - #os.environ.setdefault("CLICKHOUSE_CLIENT", args.client) + os.environ.setdefault("CLICKHOUSE_CLIENT", args.client) os.environ.setdefault("CLICKHOUSE_CONFIG", args.configserver) if args.configclient: os.environ.setdefault("CLICKHOUSE_CONFIG_CLIENT", args.configclient) diff --git a/dbms/tests/queries/0_stateless/00952_live_view_create.reference b/dbms/tests/queries/0_stateless/00952_live_view_create.reference new file mode 100644 index 00000000000..c39f21eaf5e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00952_live_view_create.reference @@ -0,0 +1 @@ +lv diff --git a/dbms/tests/queries/0_stateless/00952_live_view_create.sql b/dbms/tests/queries/0_stateless/00952_live_view_create.sql new file mode 100644 index 00000000000..0b64ed3a876 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00952_live_view_create.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; + +SHOW TABLES LIKE 'lv'; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00953_live_view_select.reference b/dbms/tests/queries/0_stateless/00953_live_view_select.reference new file mode 100644 index 00000000000..9c44fe02ff1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00953_live_view_select.reference @@ -0,0 +1,4 @@ +lv +1 +2 +3 diff --git a/dbms/tests/queries/0_stateless/00953_live_view_select.sql b/dbms/tests/queries/0_stateless/00953_live_view_select.sql new file mode 100644 index 00000000000..a51dfb112c1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00953_live_view_select.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; + +SHOW TABLES LIKE 'lv'; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT * FROM test.lv; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00954_live_view_select_version.reference b/dbms/tests/queries/0_stateless/00954_live_view_select_version.reference new file mode 100644 index 00000000000..07b5ce41af1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00954_live_view_select_version.reference @@ -0,0 +1,4 @@ +lv +1 1 +2 1 +3 1 diff --git a/dbms/tests/queries/0_stateless/00954_live_view_select_version.sql b/dbms/tests/queries/0_stateless/00954_live_view_select_version.sql new file mode 100644 index 00000000000..569cf5b0ab2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00954_live_view_select_version.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; + +SHOW TABLES LIKE 'lv'; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT *,_version FROM test.lv; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.reference b/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.reference new file mode 100644 index 00000000000..cb865431ffd --- /dev/null +++ b/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.reference @@ -0,0 +1,3 @@ +lv +6 +21 diff --git a/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.sql b/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.sql new file mode 100644 index 00000000000..cdac382cae2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; + +SHOW TABLES LIKE 'lv'; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT sum(a) FROM test.lv; + +INSERT INTO test.mt VALUES (4),(5),(6); + +SELECT sum(a) FROM test.lv; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference b/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference new file mode 100644 index 00000000000..86a72083821 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference @@ -0,0 +1,4 @@ +lv +1 c9d39b11cce79112219a73aaa319b475 +1 4cd0592103888d4682de9a32a23602e3 +1 2186dbea325ee4c56b67e9b792e993a3 diff --git a/dbms/tests/queries/0_stateless/00956_live_view_watch_events.sql b/dbms/tests/queries/0_stateless/00956_live_view_watch_events.sql new file mode 100644 index 00000000000..1dbc201bdca --- /dev/null +++ b/dbms/tests/queries/0_stateless/00956_live_view_watch_events.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +SHOW TABLES LIKE 'lv'; + +WATCH test.lv EVENTS LIMIT 0; + +INSERT INTO test.mt VALUES (1),(2),(3); + +WATCH test.lv EVENTS LIMIT 0; + +INSERT INTO test.mt VALUES (4),(5),(6); + +WATCH test.lv EVENTS LIMIT 0; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00957_live_view_watch.reference b/dbms/tests/queries/0_stateless/00957_live_view_watch.reference new file mode 100644 index 00000000000..69a461b3fc4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00957_live_view_watch.reference @@ -0,0 +1,4 @@ +lv +0 1 +6 1 +21 1 diff --git a/dbms/tests/queries/0_stateless/00957_live_view_watch.sql b/dbms/tests/queries/0_stateless/00957_live_view_watch.sql new file mode 100644 index 00000000000..ac304fec0b9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00957_live_view_watch.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +SHOW TABLES LIKE 'lv'; + +WATCH test.lv LIMIT 0; + +INSERT INTO test.mt VALUES (1),(2),(3); + +WATCH test.lv LIMIT 0; + +INSERT INTO test.mt VALUES (4),(5),(6); + +WATCH test.lv LIMIT 0; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py new file mode 100755 index 00000000000..9fb58cb556c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py @@ -0,0 +1,46 @@ +#!/usr/bin/env python +import imp +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) + +def client(name=''): + client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) + client.eol('\r') + # Note: uncomment this line for debugging + #client.logger(sys.stdout, prefix=name) + client.timeout(2) + return client + +prompt = ':\) ' +end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +client1 = client('client1>') +client2 = client('client2>') + +client1.expect(prompt) +client2.expect(prompt) + +client1.send('DROP TABLE IF EXISTS test.lv') +client1.expect(prompt) +client1.send(' DROP TABLE IF EXISTS test.mt') +client1.expect(prompt) +client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') +client1.expect(prompt) +client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') +client1.expect(prompt) +client1.send('WATCH test.lv') +client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') +client1.expect(r'6.*2' + end_of_block) +client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') +client1.expect(r'21.*3' + end_of_block) +# send Ctrl-C +os.kill(client1.process.pid,signal.SIGINT) +client1.expect(prompt) +client1.send('DROP TABLE test.lv') +client1.expect(prompt) +client1.send('DROP TABLE test.mt') +client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.reference b/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.reference new file mode 100644 index 00000000000..4bd209d4424 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.reference @@ -0,0 +1,4 @@ +temporary_live_view_timeout 5 +lv +0 +0 diff --git a/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.sql b/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.sql new file mode 100644 index 00000000000..4a4a7ae5eed --- /dev/null +++ b/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +SELECT name, value from system.settings WHERE name = 'temporary_live_view_timeout'; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +SHOW TABLES LIKE 'lv'; +SELECT sleep(3); +SELECT sleep(2); +SHOW TABLES LIKE 'lv'; + +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00960_temporary_live_view_watch.reference b/dbms/tests/queries/0_stateless/00960_temporary_live_view_watch.reference new file mode 100644 index 00000000000..69a461b3fc4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00960_temporary_live_view_watch.reference @@ -0,0 +1,4 @@ +lv +0 1 +6 1 +21 1 diff --git a/dbms/tests/queries/0_stateless/00960_temporary_live_view_watch.sql b/dbms/tests/queries/0_stateless/00960_temporary_live_view_watch.sql new file mode 100644 index 00000000000..eb99e054514 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00960_temporary_live_view_watch.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +SHOW TABLES LIKE 'lv'; + +WATCH test.lv LIMIT 0; + +INSERT INTO test.mt VALUES (1),(2),(3); + +WATCH test.lv LIMIT 0; + +INSERT INTO test.mt VALUES (4),(5),(6); + +WATCH test.lv LIMIT 0; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch_live.py new file mode 100755 index 00000000000..0b0260adeed --- /dev/null +++ b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch_live.py @@ -0,0 +1,46 @@ +#!/usr/bin/env python +import imp +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) + +def client(name=''): + client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) + client.eol('\r') + # Note: uncomment this line for debugging + #client.logger(sys.stdout, prefix=name) + client.timeout(2) + return client + +prompt = ':\) ' +end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +client1 = client('client1>') +client2 = client('client2>') + +client1.expect(prompt) +client2.expect(prompt) + +client1.send('DROP TABLE IF EXISTS test.lv') +client1.expect(prompt) +client1.send(' DROP TABLE IF EXISTS test.mt') +client1.expect(prompt) +client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') +client1.expect(prompt) +client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') +client1.expect(prompt) +client1.send('WATCH test.lv') +client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') +client1.expect(r'6.*2' + end_of_block) +client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') +client1.expect(r'21.*3' + end_of_block) +# send Ctrl-C +os.kill(client1.process.pid,signal.SIGINT) +client1.expect(prompt) +client1.send('DROP TABLE test.lv') +client1.expect(prompt) +client1.send('DROP TABLE test.mt') +client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch_live.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live_timeout.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live_timeout.py new file mode 100755 index 00000000000..43bf748dc2b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live_timeout.py @@ -0,0 +1,57 @@ +#!/usr/bin/env python +import imp +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) + +def client(name=''): + client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) + client.eol('\r') + # Note: uncomment this line for debugging + #client.logger(sys.stdout, prefix=name) + client.timeout(2) + return client + +prompt = ':\) ' +end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +client1 = client('client1>') +client2 = client('client2>') + +client1.expect(prompt) +client2.expect(prompt) + +client1.send('DROP TABLE IF EXISTS test.lv') +client1.expect(prompt) +client1.send(' DROP TABLE IF EXISTS test.mt') +client1.expect(prompt) +client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') +client1.expect(prompt) +client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') +client1.expect(prompt) +client1.send('SELECT 4') +client1.expect(prompt, timeout=4) +client1.send('WATCH test.lv') +client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') +client2.expect(prompt) +client1.expect(r'6.*2' + end_of_block) +client2.send('SELECT sleep(3)') +client2.expect(prompt, timeout=4) +client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') +client2.expect(prompt) +client1.expect(r'21.*3' + end_of_block) +# send Ctrl-C +os.kill(client1.process.pid,signal.SIGINT) +client1.expect(prompt) +client1.send('SELECT sleep(3)') +client1.expect(prompt, timeout=4) +client1.send('SELECT sleep(3)') +client1.expect(prompt, timeout=4) +client1.send('DROP TABLE test.lv') +client1.expect('Table test.lv doesn\'t exist') +client1.expect(prompt) +client1.send('DROP TABLE test.mt') +client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live_timeout.reference b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live_timeout.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/uexpect.py b/dbms/tests/queries/0_stateless/uexpect.py new file mode 100644 index 00000000000..0b57890b593 --- /dev/null +++ b/dbms/tests/queries/0_stateless/uexpect.py @@ -0,0 +1,187 @@ +# Copyright (c) 2019 Vitaliy Zakaznikov +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +import os +import pty +import time +import sys +import re + +from threading import Thread +from subprocess import Popen +from Queue import Queue, Empty + +class TimeoutError(Exception): + def __init__(self, timeout): + self.timeout = timeout + + def __str__(self): + return 'Timeout %.3fs' % float(self.timeout) + +class ExpectTimeoutError(Exception): + def __init__(self, pattern, timeout, buffer): + self.pattern = pattern + self.timeout = timeout + self.buffer = buffer + + def __str__(self): + return ('Timeout %.3fs ' % float(self.timeout) + + 'for %s ' % repr(self.pattern.pattern) + + 'buffer ends with %s ' % repr(self.buffer[-80:]) + + 'or \'%s\'' % ','.join(['%x' % ord(c) for c in self.buffer[-80:]])) + +class IO(object): + class EOF(object): + pass + + class Timeout(object): + pass + + EOF = EOF + TIMEOUT = Timeout + + class Logger(object): + def __init__(self, logger, prefix=''): + self._logger = logger + self._prefix = prefix + + def write(self, data): + self._logger.write(('\n' + data).replace('\n','\n' + self._prefix)) + + def flush(self): + self._logger.flush() + + def __init__(self, process, master, queue): + self.process = process + self.master = master + self.queue = queue + self.buffer = None + self.before = None + self.after = None + self.match = None + self.pattern = None + self._timeout = None + self._logger = None + self._eol = '' + + def logger(self, logger=None, prefix=''): + if logger: + self._logger = self.Logger(logger, prefix=prefix) + return self._logger + + def timeout(self, timeout=None): + if timeout: + self._timeout = timeout + return self._timeout + + def eol(self, eol=None): + if eol: + self._eol = eol + return self._eol + + def close(self): + if self._logger: + self._logger.write('\n') + self._logger.flush() + + def send(self, data, eol=None): + if eol is None: + eol = self._eol + return self.write(data + eol) + + def write(self, data): + return os.write(self.master, data) + + def expect(self, pattern, timeout=None): + self.match = None + self.before = None + self.after = None + pattern = re.compile(pattern) + if timeout is None: + timeout = self._timeout + while timeout >= 0: + start_time = time.time() + try: + data = self.read(timeout=timeout, raise_exception=True) + except TimeoutError: + if self._logger: + self._logger.write(self.buffer + '\n') + self._logger.flush() + exception = ExpectTimeoutError(pattern, timeout, self.buffer) + self.buffer = None + raise exception + timeout -= (time.time() - start_time) + if data: + self.buffer = self.buffer + data if self.buffer else data + self.match = pattern.search(self.buffer, 0) + if self.match: + self.after = self.buffer[self.match.start():self.match.end()] + self.before = self.buffer[:self.match.start()] + self.buffer = self.buffer[self.match.end():] + break + if self._logger: + self._logger.write(self.before + self.after) + self._logger.flush() + return self.match + + def read(self, timeout=0, raise_exception=False): + data = '' + try: + while timeout >= 0 : + start_time = time.time() + data += self.queue.get(timeout=timeout) + if data: + break + timeout -= (time.time() - start_time) + except Empty: + if data: + return data + if raise_exception: + raise TimeoutError(timeout) + pass + return data + +def spawn(*command): + master, slave = pty.openpty() + process = Popen(command, preexec_fn=os.setsid, stdout=slave, stdin=slave, stderr=slave, bufsize=1) + os.close(slave) + + queue = Queue() + thread = Thread(target=reader, args=(master, queue)) + thread.daemon = True + thread.start() + + return IO(process, master, queue) + +def reader(out, queue): + while True: + data = os.read(out, 65536) + queue.put(data) + +if __name__ == '__main__': + io = spawn('/bin/bash') + prompt = '\$ ' + io.logger(sys.stdout) + io.timeout(2) + io.eol('\r') + + io.expect(prompt) + io.send('clickhouse-client') + prompt = ':\) ' + io.expect(prompt) + io.send('SELECT 1') + io.expect(prompt) + io.send('SHOW TABLES') + io.expect('test') + io.expect(prompt) + io.close() From b63623d0146965191cc5dcec728d25ec7d84ab46 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Wed, 5 Jun 2019 10:33:34 +0300 Subject: [PATCH 049/509] Extended constraint exception with constraint name and expression --- .../DataStreams/CheckConstraintsBlockOutputStream.cpp | 10 +++++++--- .../queries/0_stateless/00952_basic_constraints.sh | 8 ++++---- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 1cc271e5578..da77e4a1c2e 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -1,14 +1,18 @@ #include - +#include namespace DB { void CheckConstraintsBlockOutputStream::write(const Block & block) { - for (auto & constraint_expr: expressions) + for (size_t i = 0; i < expressions.size(); ++i) + { + auto constraint_expr = expressions[i]; if (!checkConstraintOnBlock(block, constraint_expr)) - throw Exception{"Some constraints are not satisfied", ErrorCodes::QUERY_WAS_CANCELLED}; + throw Exception{"Constraint " + constraints.constraints[i]->name + " is not satisfied at, constraint expression: " + + serializeAST(*(constraints.constraints[i]->expr), true), ErrorCodes::LOGICAL_ERROR}; + } output->write(block); } diff --git a/dbms/tests/queries/0_stateless/00952_basic_constraints.sh b/dbms/tests/queries/0_stateless/00952_basic_constraints.sh index 93fa16ce4af..1d2a46dae61 100755 --- a/dbms/tests/queries/0_stateless/00952_basic_constraints.sh +++ b/dbms/tests/queries/0_stateless/00952_basic_constraints.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh EXCEPTION_SUCCESS_TEXT=ok - +$CLICKHOUSE_CLIENT --query="CREATE DATABASE IF NOT EXISTS test;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_constraints;" $CLICKHOUSE_CLIENT --query="CREATE TABLE test_constraints @@ -20,7 +20,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must throw and exception -EXCEPTION_TEXT="Some constraints are not satisfied" +EXCEPTION_TEXT="Constraint b_constraint is not satisfied" $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (3, 4), (1, 0);" 2>&1 \ | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" @@ -38,13 +38,13 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE test_constraints ENGINE = MergeTree ORDER BY (a);" # This one must throw an exception -EXCEPTION_TEXT="Some constraints are not satisfied" +EXCEPTION_TEXT="Constraint b_constraint is not satisfied" $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" 2>&1 \ | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must throw an exception -EXCEPTION_TEXT="Some constraints are not satisfied" +EXCEPTION_TEXT="Constraint a_constraint is not satisfied" $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (5, 16), (10, 11);" 2>&1 \ | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" From f413b1e346270e732988826e84e548857dc8174e Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Wed, 5 Jun 2019 11:05:46 +0300 Subject: [PATCH 050/509] Implemented memoryIsByte, replaced memoryIsZero with it, implemented memory check in CheckConstraintsBlockOutputStream --- dbms/src/Columns/ColumnsCommon.cpp | 43 ++++--------------- dbms/src/Columns/ColumnsCommon.h | 2 +- .../CheckConstraintsBlockOutputStream.cpp | 21 ++++++++- .../CheckConstraintsBlockOutputStream.h | 2 + 4 files changed, 32 insertions(+), 36 deletions(-) diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index 6ad3d0907ab..0745a3d5b9f 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -61,43 +61,18 @@ std::vector countColumnsSizeInSelector(IColumn::ColumnIndex num_columns, return counts; } -/** clang 4 generates better code than gcc 6. - * And both gcc and clang could not vectorize trivial loop by bytes automatically. - */ -bool memoryIsZero(const void * data, size_t size) +bool memoryIsByte(const void * data, size_t size, uint8_t byte) { - const Int8 * pos = reinterpret_cast(data); - const Int8 * end = pos + size; - -#ifdef __SSE2__ - const __m128 zero16 = _mm_setzero_ps(); - const Int8 * end64 = pos + size / 64 * 64; - - for (; pos < end64; pos += 64) - if (_mm_movemask_ps(_mm_cmpneq_ps( - _mm_loadu_ps(reinterpret_cast(pos)), - zero16)) - | _mm_movemask_ps(_mm_cmpneq_ps( - _mm_loadu_ps(reinterpret_cast(pos + 16)), - zero16)) - | _mm_movemask_ps(_mm_cmpneq_ps( - _mm_loadu_ps(reinterpret_cast(pos + 32)), - zero16)) - | _mm_movemask_ps(_mm_cmpneq_ps( - _mm_loadu_ps(reinterpret_cast(pos + 48)), - zero16))) - return false; - - /// TODO Add duff device for tail? -#endif - - for (; pos < end; ++pos) - if (*pos) - return false; - - return true; + if (size == 0) + return true; + auto ptr = reinterpret_cast(data); + return *ptr == byte && memcmp(ptr, ptr + 1, size - 1) == 0; } +bool memoryIsZero(const void * data, size_t size) +{ + return memoryIsByte(data, size, 0x0); +} namespace ErrorCodes { diff --git a/dbms/src/Columns/ColumnsCommon.h b/dbms/src/Columns/ColumnsCommon.h index 0b14b76ad39..46c6c0e9df3 100644 --- a/dbms/src/Columns/ColumnsCommon.h +++ b/dbms/src/Columns/ColumnsCommon.h @@ -22,7 +22,7 @@ std::vector countColumnsSizeInSelector(IColumn::ColumnIndex num_columns, /// Returns true, if the memory contains only zeros. bool memoryIsZero(const void * data, size_t size); - +bool memoryIsByte(const void * data, size_t size, uint8_t byte); /// The general implementation of `filter` function for ColumnArray and ColumnString. template diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index da77e4a1c2e..ac432694d83 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -1,5 +1,8 @@ #include +#include +#include #include +#include namespace DB { @@ -31,17 +34,33 @@ void CheckConstraintsBlockOutputStream::writeSuffix() output->writeSuffix(); } -bool CheckConstraintsBlockOutputStream::checkConstraintOnBlock(const Block & block, const ExpressionActionsPtr & constraint) +bool CheckConstraintsBlockOutputStream::checkImplMemory(const Block & block, const ExpressionActionsPtr & constraint) +{ + Block res = block; + constraint->execute(res); + assert(block.columns() == res.columns() - 1); + ColumnWithTypeAndName res_column = res.safeGetByPosition(res.columns() - 1); + auto res_column_uint8 = checkAndGetColumn(res_column.column.get()); + return memoryIsByte(res_column_uint8->getRawDataBegin<1>(), res_column_uint8->byteSize(), 0x1); +} + +bool CheckConstraintsBlockOutputStream::checkImplBool(const Block & block, const ExpressionActionsPtr & constraint) { Block res = block; constraint->execute(res); assert(block.columns() == res.columns() - 1); ColumnWithTypeAndName res_column = res.safeGetByPosition(res.columns() - 1); size_t column_size = res_column.column->size(); + // std::cerr << "Sizes of constraints: " << res_column.column->size() << ' ' << res_column.column->get << '\n'; for (size_t i = 0; i < column_size; ++i) if (!res_column.column->getBool(i)) return false; return true; } +bool CheckConstraintsBlockOutputStream::checkConstraintOnBlock(const Block & block, const ExpressionActionsPtr & constraint) +{ + return checkImplMemory(block, constraint); +} + } diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h index 16b240eb758..6ea42cf44af 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h @@ -35,6 +35,8 @@ public: void writePrefix() override; void writeSuffix() override; + bool checkImplMemory(const Block & block, const ExpressionActionsPtr & constraint); + bool checkImplBool(const Block & block, const ExpressionActionsPtr & constraint); bool checkConstraintOnBlock(const Block & block, const ExpressionActionsPtr & constraint); private: From 5f3be8134dcaded078e266f6c5a2165be569f43e Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Wed, 5 Jun 2019 07:30:29 -0400 Subject: [PATCH 051/509] * Disallowing writes into LIVE VIEW tables * Adding support for watching live views via HTTPHandler * Small fix to WriteBufferValidUTF8.cpp to propagate next() call * Updated copyData.cpp to treat block with no rows as "flush block" * Updated PushingToViewsBlockOutputStream.cpp to directly use LIVE VIEW output stream instead of calling its write method --- .../DataStreams/LiveViewBlockInputStream.h | 11 +++- .../LiveViewEventsBlockInputStream.h | 56 +++++++++++++------ .../PushingToViewsBlockOutputStream.cpp | 2 +- dbms/src/DataStreams/copyData.cpp | 2 + dbms/src/IO/WriteBufferValidUTF8.cpp | 3 + dbms/src/Storages/StorageLiveView.cpp | 5 -- dbms/src/Storages/StorageLiveView.h | 15 +++-- 7 files changed, 61 insertions(+), 33 deletions(-) diff --git a/dbms/src/DataStreams/LiveViewBlockInputStream.h b/dbms/src/DataStreams/LiveViewBlockInputStream.h index 62ab983aad1..a6c9c3533ee 100644 --- a/dbms/src/DataStreams/LiveViewBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewBlockInputStream.h @@ -151,6 +151,11 @@ protected: { return { Block(), false }; } + if (!end_of_blocks) + { + end_of_blocks = true; + return { getHeader(), true }; + } while (true) { bool signaled = condition.tryWait(mutex, std::max((UInt64)0, heartbeat_delay - ((UInt64)timestamp.epochMicroseconds() - last_event_timestamp)) / 1000); @@ -165,9 +170,9 @@ protected: } else { - //hashmap["blocks"] = blocks_hash; + // heartbeat last_event_timestamp = (UInt64)timestamp.epochMicroseconds(); - //heartbeat(Heartbeat(last_event_timestamp, std::move(hashmap))); + return { getHeader(), true }; } } } @@ -181,6 +186,7 @@ protected: if (it == end) { + end_of_blocks = false; if (length > 0) --length; } @@ -204,6 +210,7 @@ private: Poco::FastMutex & mutex; /// Length specifies number of updates to send, default -1 (no limit) int64_t length; + bool end_of_blocks{0}; UInt64 heartbeat_delay; String blocks_hash; UInt64 last_event_timestamp{0}; diff --git a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h index d9fdda97d21..32290c3333b 100644 --- a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h @@ -17,6 +17,8 @@ limitations under the License. */ #include #include #include +#include +#include #include #include #include @@ -62,7 +64,18 @@ public: condition.broadcast(); } - Block getHeader() const override { return storage.getHeader(); } + Block getHeader() const override { + return { + ColumnWithTypeAndName( + ColumnUInt64::create(), + std::make_shared(), + "version"), + ColumnWithTypeAndName( + ColumnString::create(), + std::make_shared(), + "hash") + }; + } void refresh() { @@ -94,6 +107,21 @@ public: return tryRead_(false); } + Block getEventBlock() + { + Block res{ + ColumnWithTypeAndName( + DataTypeUInt64().createColumnConst(1, blocks_metadata->version)->convertToFullColumnIfConst(), + std::make_shared(), + "version"), + ColumnWithTypeAndName( + DataTypeString().createColumnConst(1, blocks_metadata->hash)->convertToFullColumnIfConst(), + std::make_shared(), + "hash"), + + }; + return res; + } protected: Block readImpl() override { @@ -153,6 +181,11 @@ protected: { return { Block(), false }; } + if (!end_of_blocks) + { + end_of_blocks = true; + return { getHeader(), true }; + } while (true) { bool signaled = condition.tryWait(mutex, std::max((UInt64)0, heartbeat_delay - ((UInt64)timestamp.epochMicroseconds() - last_event_timestamp)) / 1000); @@ -167,10 +200,9 @@ protected: } else { - // return Block(version, hash) - //hashmap["blocks"] = blocks_hash; + // repeat the event block as a heartbeat last_event_timestamp = (UInt64)timestamp.epochMicroseconds(); - //heartbeat(Heartbeat(last_event_timestamp, std::move(hashmap))); + return { getHeader(), true }; } } } @@ -183,25 +215,14 @@ protected: if (it == end) { + end_of_blocks = false; if (length > 0) --length; } last_event_timestamp = (UInt64)timestamp.epochMicroseconds(); - Block res{ - ColumnWithTypeAndName( - DataTypeUInt64().createColumnConst(1, blocks_metadata->version)->convertToFullColumnIfConst(), - std::make_shared(), - "version"), - ColumnWithTypeAndName( - DataTypeString().createColumnConst(1, blocks_metadata->hash)->convertToFullColumnIfConst(), - std::make_shared(), - "hash"), - - }; - - return { res, true }; + return { getEventBlock(), true }; } private: @@ -219,6 +240,7 @@ private: Poco::FastMutex & mutex; /// Length specifies number of updates to send, default -1 (no limit) int64_t length; + bool end_of_blocks{0}; UInt64 heartbeat_delay; UInt64 last_event_timestamp{0}; Poco::Timestamp timestamp; diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index c826835b007..06f9a1edacf 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -86,7 +86,7 @@ void PushingToViewsBlockOutputStream::write(const Block & block) if (auto * live_view = dynamic_cast(storage.get())) { - BlockOutputStreamPtr output = storage->write(query_ptr, context); + BlockOutputStreamPtr output = std::make_shared(*live_view); StorageLiveView::writeIntoLiveView(*live_view, block, context, output); } else diff --git a/dbms/src/DataStreams/copyData.cpp b/dbms/src/DataStreams/copyData.cpp index 9d17596fc8d..5000c87be7c 100644 --- a/dbms/src/DataStreams/copyData.cpp +++ b/dbms/src/DataStreams/copyData.cpp @@ -28,6 +28,8 @@ void copyDataImpl(IBlockInputStream & from, IBlockOutputStream & to, TCancelCall break; to.write(block); + if (!block.rows()) + to.flush(); progress(block); } diff --git a/dbms/src/IO/WriteBufferValidUTF8.cpp b/dbms/src/IO/WriteBufferValidUTF8.cpp index 01e011982cd..14d713de574 100644 --- a/dbms/src/IO/WriteBufferValidUTF8.cpp +++ b/dbms/src/IO/WriteBufferValidUTF8.cpp @@ -117,6 +117,9 @@ void WriteBufferValidUTF8::nextImpl() memory[i] = p[i]; working_buffer = Buffer(&memory[cnt], memory.data() + memory.size()); + + /// Propagate next() to the output buffer + output_buffer.next(); } diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index 57e4695706b..77578bc386d 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -448,11 +448,6 @@ BlockInputStreams StorageLiveView::watch( } } -BlockOutputStreamPtr StorageLiveView::write(const ASTPtr & /*query*/, const Context & /*context*/) -{ - return std::make_shared(*this); -} - void registerStorageLiveView(StorageFactory & factory) { factory.registerStorage("LiveView", [](const StorageFactory::Arguments & args) diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index d16a80ee888..bb4377fbed6 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -121,10 +121,6 @@ public: void refresh(const Context & context); - BlockOutputStreamPtr write( - const ASTPtr &, - const Context &) override; - BlockInputStreams read( const Names & column_names, const SelectQueryInfo & query_info, @@ -177,9 +173,11 @@ public: auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); BlockInputStreams streams = {std::make_shared(block)}; auto proxy_storage = std::make_shared(parent_storage, std::move(streams), QueryProcessingStage::FetchColumns); - InterpreterSelectQuery select_block(live_view.getInnerQuery(), context, proxy_storage, - QueryProcessingStage::WithMergeableState); - auto data_mergeable_stream = std::make_shared(select_block.execute().in); + InterpreterSelectQuery select_block(live_view.getInnerQuery(), + context, proxy_storage, + QueryProcessingStage::WithMergeableState); + auto data_mergeable_stream = std::make_shared( + select_block.execute().in); while (Block this_block = data_mergeable_stream->read()) new_mergeable_blocks->push_back(this_block); } @@ -196,7 +194,8 @@ public: mergeable_blocks = std::make_shared>(); BlocksPtr base_mergeable_blocks = std::make_shared(); InterpreterSelectQuery interpreter(live_view.getInnerQuery(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names{}); - auto view_mergeable_stream = std::make_shared(interpreter.execute().in); + auto view_mergeable_stream = std::make_shared( + interpreter.execute().in); while (Block this_block = view_mergeable_stream->read()) base_mergeable_blocks->push_back(this_block); mergeable_blocks->push_back(base_mergeable_blocks); From 7d69496fbbf056da254560717a5e28383a2b7052 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Wed, 5 Jun 2019 08:10:34 -0400 Subject: [PATCH 052/509] * Adding a NOTICE file to include the original header that was inside some of the files. --- NOTICE | 41 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 41 insertions(+) create mode 100644 NOTICE diff --git a/NOTICE b/NOTICE new file mode 100644 index 00000000000..59c6923e66a --- /dev/null +++ b/NOTICE @@ -0,0 +1,41 @@ +-- +The following notice shall be applied to the files listed below. + +Some modifications Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + +dbms/src/Common/ErrorCodes.cpp +dbms/src/Common/UInt128.h +dbms/src/Core/Block.h +dbms/src/Core/Defines.h +dbms/src/Core/Settings.h +dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +dbms/src/DataStreams/PushingToViewsBlockOutputStream.h +dbms/src/DataStreams/copyData.cpp +dbms/src/Databases/DatabasesCommon.cpp +dbms/src/IO/WriteBufferValidUTF8.cpp +dbms/src/Interpreters/InterpreterAlterQuery.cpp +dbms/src/Interpreters/InterpreterCreateQuery.cpp +dbms/src/Interpreters/InterpreterFactory.cpp +dbms/src/Parsers/ASTAlterQuery.cpp +dbms/src/Parsers/ASTAlterQuery.h +dbms/src/Parsers/ASTCreateQuery.cpp +dbms/src/Parsers/ASTCreateQuery.h +dbms/src/Parsers/ParserAlterQuery.cpp +dbms/src/Parsers/ParserAlterQuery.h +dbms/src/Parsers/ParserCreateQuery.cpp +dbms/src/Parsers/ParserCreateQuery.h +dbms/src/Parsers/ParserQueryWithOutput.cpp +dbms/src/Storages/IStorage.h +dbms/src/Storages/StorageFactory.cpp +dbms/src/Storages/registerStorages.cpp +-- From bba3b33bdca89da53a8ac1707234b02959e8486c Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Wed, 5 Jun 2019 15:17:53 +0300 Subject: [PATCH 053/509] Fixed exception text in 00953_constraints_operations --- dbms/tests/queries/0_stateless/00953_constraints_operations.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00953_constraints_operations.sh b/dbms/tests/queries/0_stateless/00953_constraints_operations.sh index 917719c3e46..f0fc5b71fbf 100755 --- a/dbms/tests/queries/0_stateless/00953_constraints_operations.sh +++ b/dbms/tests/queries/0_stateless/00953_constraints_operations.sh @@ -20,7 +20,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must throw and exception -EXCEPTION_TEXT="Some constraints are not satisfied" +EXCEPTION_TEXT="Constraint b_constraint is not satisfied" $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 0);" 2>&1 \ | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" From 375e4640867c0318b47fbab3149a35a53d0d46bd Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Thu, 6 Jun 2019 00:25:48 +0300 Subject: [PATCH 054/509] Fixed exception text --- dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index ac432694d83..75067b6afa7 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -13,7 +13,7 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) { auto constraint_expr = expressions[i]; if (!checkConstraintOnBlock(block, constraint_expr)) - throw Exception{"Constraint " + constraints.constraints[i]->name + " is not satisfied at, constraint expression: " + + throw Exception{"Constraint " + constraints.constraints[i]->name + " is not satisfied, constraint expression: " + serializeAST(*(constraints.constraints[i]->expr), true), ErrorCodes::LOGICAL_ERROR}; } output->write(block); From 76772d1de095ff3c41174a0a914d544162d60c02 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Thu, 6 Jun 2019 01:25:57 +0300 Subject: [PATCH 055/509] Documentation on constraints (RU, EN) --- docs/en/query_language/alter.md | 16 ++++++++++++++++ docs/en/query_language/create.md | 20 ++++++++++++++++++++ docs/en/query_language/insert_into.md | 3 +++ docs/ru/query_language/alter.md | 16 ++++++++++++++++ docs/ru/query_language/create.md | 20 ++++++++++++++++++++ docs/ru/query_language/insert_into.md | 4 ++++ 6 files changed, 79 insertions(+) diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index 85941987ce9..fc42fc636e7 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -166,6 +166,22 @@ are available: These commands are lightweight in a sense that they only change metadata or remove files. Also, they are replicated (syncing indices metadata through ZooKeeper). +### Manipulations with constraints + +See more on [constraints](create.md#constraints) + +Constraints could be added or deleted using following syntax: +``` +ALTER TABLE [db].name ADD CONSTRAINT constraint_name CHECK expression; +ALTER TABLE [db].name DROP CONSTRAINT constraint_name; +``` + +Queries will add or remove metadata about constraints from table so they are processed immediately. + +Constraint check *will not be executed* on existing table if it was added. For now, we recommend to create new table and use `INSERT SELECT` query to fill new table. + +All changes on distributed tables are broadcasting to ZooKeeper so will be applied on other replicas. + ### Manipulations With Partitions and Parts {#alter_manipulations-with-partitions} The following operations with [partitions](../operations/table_engines/custom_partitioning_key.md) are available: diff --git a/docs/en/query_language/create.md b/docs/en/query_language/create.md index 9ebd50839d3..573388195e3 100644 --- a/docs/en/query_language/create.md +++ b/docs/en/query_language/create.md @@ -80,6 +80,26 @@ If you add a new column to a table but later change its default expression, the It is not possible to set default values for elements in nested data structures. +### Constraints {#constraints} + +WARNING: This feature is experimental. Correct work is not guaranteed on non-MergeTree family engines. + +Along with columns descriptions constraints could be defined: + +``sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec] [TTL expr1], + ... + CONSTRAINT constraint_name_1 CHECK boolean_expr_1, + ... +) ENGINE = engine +``` + +`boolean_expr_1` could by any boolean expression. If constraints are defined for the table, each of them will be checked for every row in `INSERT` query. If any constraint is not satisfied — server will raise an exception with constraint name and checking expression. + +Adding large amount of constraints can negatively affect performance of big `INSERT` queries. + ### TTL expression Can be specified only for MergeTree-family tables. An expression for setting storage time for values. It must depends on `Date` or `DateTime` column and has one `Date` or `DateTime` column as a result. Example: diff --git a/docs/en/query_language/insert_into.md b/docs/en/query_language/insert_into.md index 914c3b2917f..c0cb9f8c3b1 100644 --- a/docs/en/query_language/insert_into.md +++ b/docs/en/query_language/insert_into.md @@ -40,6 +40,9 @@ INSERT INTO t FORMAT TabSeparated You can insert data separately from the query by using the command-line client or the HTTP interface. For more information, see the section "[Interfaces](../interfaces/index.md#interfaces)". +### Constraints + +If table has [constraints](create.md#constraints), their expressions will be checked for each row of inserted data. If any of those constraints is not satisfied — server will raise an exception containing constraint name and expression, the query will be stopped. ### Inserting The Results of `SELECT` {#insert_query_insert-select} diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index 2367386172a..5e847abce4d 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -165,6 +165,22 @@ ALTER TABLE [db].name DROP INDEX name Запрос на изменение индексов реплицируется, сохраняя новые метаданные в ZooKeeper и применяя изменения на всех репликах. +### Манипуляции с ограничениями (constraints) + +Про ограничения подробнее написано [тут](create.md#constraints). + +Добавить или удалить ограничение можно с помощью запросов +``` +ALTER TABLE [db].name ADD CONSTRAINT constraint_name CHECK expression; +ALTER TABLE [db].name DROP CONSTRAINT constraint_name; +``` + +Запросы выполняют добавление или удаление метаданных об ограничениях таблицы `[db].name`, поэтому выполняются мнгновенно. + +Если ограничение появилось для непустой таблицы, то *проверка ограничения вызвана не будет*. Если же важно добавить ограничение на существующую таблицу, то рекомендуется создать новую таблицу с нужным ограничением и выполнить `INSERT SELECT` запрос для перекачки данных из одной таблицы в другую. + +Запрос на изменение ограничений так же, как и с индексами, реплицируется через ZooKeeper. + ### Манипуляции с партициями и кусками {#alter_manipulations-with-partitions} Для работы с [партициями](../operations/table_engines/custom_partitioning_key.md) доступны следующие операции: diff --git a/docs/ru/query_language/create.md b/docs/ru/query_language/create.md index ee6dc3c7820..0b842e22e47 100644 --- a/docs/ru/query_language/create.md +++ b/docs/ru/query_language/create.md @@ -80,6 +80,26 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ... Отсутствует возможность задать значения по умолчанию для элементов вложенных структур данных. +### Ограничения (constraints) {#constraints} + +WARNING: Находится в экспериментальном режиме, поддержано в MergeTree (работоспособность на других типах движков таблиц не гарантируется). + +Наряду с объявлением столбцов можно объявить ограчения на значения в столбцах таблицы: + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec] [TTL expr1], + ... + CONSTRAINT constraint_name_1 CHECK boolean_expr_1, + ... +) ENGINE = engine +``` + +`boolean_expr_1` может быть любым булевым выражением, состоящим из операторов сравнения или функций. При наличии одного или нескольких ограничений в момент вставки данных выражения ограничений будут проверяться на истинность для каждой вставляемой строки данных. В случае, если в теле INSERT запроса придут некорректные данные — клиентов будет выкинуто исключение с нарушенным ограничением. + +Добавление большого числа ограничений может негативно повлиять на производительность объёмных `INSERT` запросов. + ### Выражение для TTL Может быть указано только для таблиц семейства MergeTree. Выражение для указания времени хранения значений. Оно должно зависеть от стобца типа `Date` или `DateTime` и в качестве результата вычислять столбец типа `Date` или `DateTime`. Пример: diff --git a/docs/ru/query_language/insert_into.md b/docs/ru/query_language/insert_into.md index 356b720e157..454339ebcdb 100644 --- a/docs/ru/query_language/insert_into.md +++ b/docs/ru/query_language/insert_into.md @@ -40,6 +40,10 @@ INSERT INTO t FORMAT TabSeparated С помощью консольного клиента или HTTP интерфейса можно вставлять данные отдельно от запроса. Как это сделать, читайте в разделе "[Интерфейсы](../interfaces/index.md#interfaces)". +### Ограничения (constraints) + +Если в таблице объявлены [ограничения](create.md#constraints), то их выполнимость будет проверена для каждой вставляемой строки. Если для хотя бы одной строки ограничения не будут выполнены, запрос будет остановлен. + ### Вставка результатов `SELECT` {#insert_query_insert-select} ``` sql From a946d09f1b73efa1d04d9287da9e340a94520ac3 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Wed, 5 Jun 2019 18:27:36 -0400 Subject: [PATCH 056/509] * Adding more LIVE VIEW tests --- .../00960_live_view_watch_events_live.py | 46 +++++++++++++++++ ...960_live_view_watch_events_live.reference} | 0 ...00961_temporary_live_view_watch.reference} | 0 ...ql => 00961_temporary_live_view_watch.sql} | 0 ...> 00962_temporary_live_view_watch_live.py} | 0 ..._temporary_live_view_watch_live.reference} | 0 ...temporary_live_view_watch_live_timeout.py} | 0 ...ary_live_view_watch_live_timeout.reference | 0 .../00964_live_view_watch_events_heartbeat.py | 47 +++++++++++++++++ ...live_view_watch_events_heartbeat.reference | 0 .../00965_live_view_watch_heartbeat.py | 47 +++++++++++++++++ .../00965_live_view_watch_heartbeat.reference | 0 .../00966_live_view_watch_events_http.py | 51 +++++++++++++++++++ ...0966_live_view_watch_events_http.reference | 0 .../0_stateless/00967_live_view_watch_http.py | 51 +++++++++++++++++++ .../00967_live_view_watch_http.reference | 0 dbms/tests/queries/0_stateless/uexpect.py | 10 ++-- 17 files changed, 248 insertions(+), 4 deletions(-) create mode 100755 dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py rename dbms/tests/queries/0_stateless/{00961_temporary_live_view_watch_live.reference => 00960_live_view_watch_events_live.reference} (100%) rename dbms/tests/queries/0_stateless/{00960_temporary_live_view_watch.reference => 00961_temporary_live_view_watch.reference} (100%) rename dbms/tests/queries/0_stateless/{00960_temporary_live_view_watch.sql => 00961_temporary_live_view_watch.sql} (100%) rename dbms/tests/queries/0_stateless/{00961_temporary_live_view_watch_live.py => 00962_temporary_live_view_watch_live.py} (100%) rename dbms/tests/queries/0_stateless/{00962_temporary_live_view_watch_live_timeout.reference => 00962_temporary_live_view_watch_live.reference} (100%) rename dbms/tests/queries/0_stateless/{00962_temporary_live_view_watch_live_timeout.py => 00963_temporary_live_view_watch_live_timeout.py} (100%) create mode 100644 dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.reference create mode 100755 dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py create mode 100644 dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.reference create mode 100755 dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py create mode 100644 dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.reference create mode 100755 dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py create mode 100644 dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.reference create mode 100755 dbms/tests/queries/0_stateless/00967_live_view_watch_http.py create mode 100644 dbms/tests/queries/0_stateless/00967_live_view_watch_http.reference diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py new file mode 100755 index 00000000000..fa96331e3fb --- /dev/null +++ b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py @@ -0,0 +1,46 @@ +#!/usr/bin/env python +import imp +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) + +def client(name=''): + client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) + client.eol('\r') + # Note: uncomment this line for debugging + #client.logger(sys.stdout, prefix=name) + client.timeout(2) + return client + +prompt = ':\) ' +end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +client1 = client('client1>') +client2 = client('client2>') + +client1.expect(prompt) +client2.expect(prompt) + +client1.send('DROP TABLE IF EXISTS test.lv') +client1.expect(prompt) +client1.send(' DROP TABLE IF EXISTS test.mt') +client1.expect(prompt) +client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') +client1.expect(prompt) +client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') +client1.expect(prompt) +client1.send('WATCH test.lv EVENTS') +client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') +client1.expect('2.*4cd0592103888d4682de9a32a23602e3' + end_of_block) +client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') +client1.expect('3.*2186dbea325ee4c56b67e9b792e993a3' + end_of_block) +# send Ctrl-C +os.kill(client1.process.pid,signal.SIGINT) +client1.expect(prompt) +client1.send('DROP TABLE test.lv') +client1.expect(prompt) +client1.send('DROP TABLE test.mt') +client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00961_temporary_live_view_watch_live.reference rename to dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.reference diff --git a/dbms/tests/queries/0_stateless/00960_temporary_live_view_watch.reference b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00960_temporary_live_view_watch.reference rename to dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference diff --git a/dbms/tests/queries/0_stateless/00960_temporary_live_view_watch.sql b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql similarity index 100% rename from dbms/tests/queries/0_stateless/00960_temporary_live_view_watch.sql rename to dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py similarity index 100% rename from dbms/tests/queries/0_stateless/00961_temporary_live_view_watch_live.py rename to dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live_timeout.reference b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live_timeout.reference rename to dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.reference diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live_timeout.py b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py similarity index 100% rename from dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live_timeout.py rename to dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.reference b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py new file mode 100755 index 00000000000..9d4f1d0716c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -0,0 +1,47 @@ +#!/usr/bin/env python +import imp +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) + +def client(name=''): + client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) + client.eol('\r') + # Note: uncomment this line for debugging + #client.logger(sys.stdout, prefix=name) + client.timeout(2) + return client + +prompt = ':\) ' +end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +client1 = client('client1>') +client2 = client('client2>') + +client1.expect(prompt) +client2.expect(prompt) + +client1.send('DROP TABLE IF EXISTS test.lv') +client1.expect(prompt) +client1.send(' DROP TABLE IF EXISTS test.mt') +client1.expect(prompt) +client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') +client1.expect(prompt) +client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') +client1.expect(prompt) +client1.send('WATCH test.lv EVENTS') +client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') +client1.expect('2.*4cd0592103888d4682de9a32a23602e3' + end_of_block) +client1.expect('Progress: 2.00 rows.*\)') +# wait for heartbeat +client1.expect('Progress: 2.00 rows.*\)', timeout=15) +# send Ctrl-C +os.kill(client1.process.pid,signal.SIGINT) +client1.expect(prompt) +client1.send('DROP TABLE test.lv') +client1.expect(prompt) +client1.send('DROP TABLE test.mt') +client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.reference b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py new file mode 100755 index 00000000000..cafbcd833b1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py @@ -0,0 +1,47 @@ +#!/usr/bin/env python +import imp +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) + +def client(name=''): + client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) + client.eol('\r') + # Note: uncomment this line for debugging + #client.logger(sys.stdout, prefix=name) + client.timeout(2) + return client + +prompt = ':\) ' +end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +client1 = client('client1>') +client2 = client('client2>') + +client1.expect(prompt) +client2.expect(prompt) + +client1.send('DROP TABLE IF EXISTS test.lv') +client1.expect(prompt) +client1.send(' DROP TABLE IF EXISTS test.mt') +client1.expect(prompt) +client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') +client1.expect(prompt) +client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') +client1.expect(prompt) +client1.send('WATCH test.lv') +client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') +client1.expect(r'6.*2' + end_of_block) +client1.expect('Progress: 2.00 rows.*\)') +# wait for heartbeat +client1.expect('Progress: 2.00 rows.*\)', timeout=15) +# send Ctrl-C +os.kill(client1.process.pid,signal.SIGINT) +client1.expect(prompt) +client1.send('DROP TABLE test.lv') +client1.expect(prompt) +client1.send('DROP TABLE test.mt') +client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.reference b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py new file mode 100755 index 00000000000..be1cd220679 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py @@ -0,0 +1,51 @@ +#!/usr/bin/env python +import imp +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) + +def client(name='', command=None): + if command is None: + client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) + else: + client = uexpect.spawn(command) + client.eol('\r') + # Note: uncomment this line for debugging + #client.logger(sys.stdout, prefix=name) + client.timeout(2) + return client + +prompt = ':\) ' +end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +client1 = client('client1>') +client1.expect(prompt) + +client1.send('DROP TABLE IF EXISTS test.lv') +client1.expect(prompt) +client1.send(' DROP TABLE IF EXISTS test.mt') +client1.expect(prompt) +client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') +client1.expect(prompt) +client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') +client1.expect(prompt) + +client2 = client('client2>', ['bash', '--noediting']) +client2.expect('\$ ') +client2.send('wget -O- -q "http://localhost:8123/?query=WATCH test.lv EVENTS"') +client2.expect('.*1\tc9d39b11cce79112219a73aaa319b475\r\n') + +client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') +client1.expect(prompt) + +client2.expect('.*2\t.*\r\n') +## send Ctrl-C +os.kill(client2.process.pid,signal.SIGINT) + +client1.send('DROP TABLE test.lv') +client1.expect(prompt) +client1.send('DROP TABLE test.mt') +client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.reference b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py new file mode 100755 index 00000000000..0a844579f5d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py @@ -0,0 +1,51 @@ +#!/usr/bin/env python +import imp +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) + +def client(name='', command=None): + if command is None: + client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) + else: + client = uexpect.spawn(command) + client.eol('\r') + # Note: uncomment this line for debugging + #client.logger(sys.stdout, prefix=name) + client.timeout(2) + return client + +prompt = ':\) ' +end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +client1 = client('client1>') +client1.expect(prompt) + +client1.send('DROP TABLE IF EXISTS test.lv') +client1.expect(prompt) +client1.send(' DROP TABLE IF EXISTS test.mt') +client1.expect(prompt) +client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') +client1.expect(prompt) +client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') +client1.expect(prompt) + +client2 = client('client2>', ['bash', '--noediting']) +client2.expect('\$ ') +client2.send('wget -O- -q "http://localhost:8123/?query=WATCH test.lv"') +client2.expect('.*0\t1\r\n') + +client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') +client1.expect(prompt) + +client2.expect('.*6\t2\r\n') +## send Ctrl-C +os.kill(client2.process.pid,signal.SIGINT) + +client1.send('DROP TABLE test.lv') +client1.expect(prompt) +client1.send('DROP TABLE test.mt') +client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.reference b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/uexpect.py b/dbms/tests/queries/0_stateless/uexpect.py index 0b57890b593..412facb8389 100644 --- a/dbms/tests/queries/0_stateless/uexpect.py +++ b/dbms/tests/queries/0_stateless/uexpect.py @@ -102,10 +102,12 @@ class IO(object): def write(self, data): return os.write(self.master, data) - def expect(self, pattern, timeout=None): + def expect(self, pattern, timeout=None, escape=False): self.match = None self.before = None self.after = None + if escape: + pattern = re.escape(pattern) pattern = re.compile(pattern) if timeout is None: timeout = self._timeout @@ -151,7 +153,7 @@ class IO(object): pass return data -def spawn(*command): +def spawn(command): master, slave = pty.openpty() process = Popen(command, preexec_fn=os.setsid, stdout=slave, stdin=slave, stderr=slave, bufsize=1) os.close(slave) @@ -169,7 +171,7 @@ def reader(out, queue): queue.put(data) if __name__ == '__main__': - io = spawn('/bin/bash') + io = spawn(['/bin/bash','--noediting']) prompt = '\$ ' io.logger(sys.stdout) io.timeout(2) @@ -182,6 +184,6 @@ if __name__ == '__main__': io.send('SELECT 1') io.expect(prompt) io.send('SHOW TABLES') - io.expect('test') + io.expect('.*\r\n.*') io.expect(prompt) io.close() From e51f2c1dd189159a48f733884e4c155b80c7f0e9 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Wed, 5 Jun 2019 18:52:31 -0400 Subject: [PATCH 057/509] Revert "Merge branch 'master' of https://github.com/vzakaznikov/ClickHouse into liveview" This reverts commit 160ff4d519cdebf3816694534e19a47485193f24, reversing changes made to a946d09f1b73efa1d04d9287da9e340a94520ac3. --- .gitmodules | 9 + contrib/googletest | 1 + .../boost/archive/archive_exception.hpp | 100 - .../boost/archive/basic_archive.hpp | 304 --- .../boost/archive/basic_binary_iarchive.hpp | 204 -- .../boost/archive/basic_binary_iprimitive.hpp | 198 -- .../boost/archive/basic_binary_oarchive.hpp | 185 -- .../boost/archive/basic_binary_oprimitive.hpp | 188 -- .../archive/basic_streambuf_locale_saver.hpp | 108 -- .../boost/archive/basic_text_iarchive.hpp | 96 - .../boost/archive/basic_text_iprimitive.hpp | 142 -- .../boost/archive/basic_text_oarchive.hpp | 119 -- .../boost/archive/basic_text_oprimitive.hpp | 209 -- .../boost/archive/basic_xml_archive.hpp | 67 - .../boost/archive/basic_xml_iarchive.hpp | 119 -- .../boost/archive/basic_xml_oarchive.hpp | 138 -- .../boost/archive/binary_iarchive.hpp | 64 - .../boost/archive/binary_iarchive_impl.hpp | 105 - .../boost/archive/binary_oarchive.hpp | 64 - .../boost/archive/binary_oarchive_impl.hpp | 106 - .../boost/archive/binary_wiarchive.hpp | 56 - .../boost/archive/binary_woarchive.hpp | 59 - .../boost/archive/codecvt_null.hpp | 109 -- .../boost/archive/detail/abi_prefix.hpp | 16 - .../boost/archive/detail/abi_suffix.hpp | 15 - .../archive/detail/archive_serializer_map.hpp | 54 - .../archive/detail/auto_link_archive.hpp | 48 - .../archive/detail/auto_link_warchive.hpp | 47 - .../boost/archive/detail/basic_iarchive.hpp | 105 - .../archive/detail/basic_iserializer.hpp | 91 - .../boost/archive/detail/basic_oarchive.hpp | 94 - .../archive/detail/basic_oserializer.hpp | 89 - .../detail/basic_pointer_iserializer.hpp | 70 - .../detail/basic_pointer_oserializer.hpp | 68 - .../boost/archive/detail/basic_serializer.hpp | 77 - .../archive/detail/basic_serializer_map.hpp | 69 - .../boost/archive/detail/check.hpp | 169 -- .../boost/archive/detail/common_iarchive.hpp | 88 - .../boost/archive/detail/common_oarchive.hpp | 88 - .../boost/archive/detail/decl.hpp | 57 - .../archive/detail/helper_collection.hpp | 99 - .../archive/detail/interface_iarchive.hpp | 85 - .../archive/detail/interface_oarchive.hpp | 87 - .../boost/archive/detail/iserializer.hpp | 631 ------ .../boost/archive/detail/oserializer.hpp | 540 ------ .../detail/polymorphic_iarchive_route.hpp | 218 --- .../detail/polymorphic_oarchive_route.hpp | 209 -- .../boost/archive/detail/register_archive.hpp | 91 - .../archive/detail/utf8_codecvt_facet.hpp | 39 - .../boost_1_65_0/boost/archive/dinkumware.hpp | 224 --- .../archive/impl/archive_serializer_map.ipp | 75 - .../archive/impl/basic_binary_iarchive.ipp | 134 -- .../archive/impl/basic_binary_iprimitive.ipp | 171 -- .../archive/impl/basic_binary_oarchive.ipp | 42 - .../archive/impl/basic_binary_oprimitive.ipp | 126 -- .../archive/impl/basic_text_iarchive.ipp | 76 - .../archive/impl/basic_text_iprimitive.ipp | 137 -- .../archive/impl/basic_text_oarchive.ipp | 62 - .../archive/impl/basic_text_oprimitive.ipp | 115 -- .../boost/archive/impl/basic_xml_grammar.hpp | 173 -- .../boost/archive/impl/basic_xml_iarchive.ipp | 115 -- .../boost/archive/impl/basic_xml_oarchive.ipp | 272 --- .../boost/archive/impl/text_iarchive_impl.ipp | 128 -- .../boost/archive/impl/text_oarchive_impl.ipp | 122 -- .../archive/impl/text_wiarchive_impl.ipp | 118 -- .../archive/impl/text_woarchive_impl.ipp | 85 - .../boost/archive/impl/xml_iarchive_impl.ipp | 199 -- .../boost/archive/impl/xml_oarchive_impl.ipp | 142 -- .../boost/archive/impl/xml_wiarchive_impl.ipp | 189 -- .../boost/archive/impl/xml_woarchive_impl.ipp | 171 -- .../archive/iterators/base64_exception.hpp | 68 - .../archive/iterators/base64_from_binary.hpp | 109 -- .../archive/iterators/binary_from_base64.hpp | 118 -- .../boost/archive/iterators/dataflow.hpp | 102 - .../archive/iterators/dataflow_exception.hpp | 80 - .../boost/archive/iterators/escape.hpp | 115 -- .../archive/iterators/insert_linebreaks.hpp | 99 - .../archive/iterators/istream_iterator.hpp | 92 - .../boost/archive/iterators/mb_from_wchar.hpp | 139 -- .../archive/iterators/ostream_iterator.hpp | 83 - .../archive/iterators/remove_whitespace.hpp | 167 -- .../archive/iterators/transform_width.hpp | 177 -- .../boost/archive/iterators/unescape.hpp | 89 - .../boost/archive/iterators/wchar_from_mb.hpp | 194 -- .../boost/archive/iterators/xml_escape.hpp | 121 -- .../boost/archive/iterators/xml_unescape.hpp | 125 -- .../iterators/xml_unescape_exception.hpp | 49 - .../archive/polymorphic_binary_iarchive.hpp | 54 - .../archive/polymorphic_binary_oarchive.hpp | 43 - .../boost/archive/polymorphic_iarchive.hpp | 168 -- .../boost/archive/polymorphic_oarchive.hpp | 154 -- .../archive/polymorphic_text_iarchive.hpp | 54 - .../archive/polymorphic_text_oarchive.hpp | 39 - .../archive/polymorphic_text_wiarchive.hpp | 59 - .../archive/polymorphic_text_woarchive.hpp | 44 - .../archive/polymorphic_xml_iarchive.hpp | 54 - .../archive/polymorphic_xml_oarchive.hpp | 39 - .../archive/polymorphic_xml_wiarchive.hpp | 50 - .../archive/polymorphic_xml_woarchive.hpp | 44 - .../boost/archive/text_iarchive.hpp | 132 -- .../boost/archive/text_oarchive.hpp | 121 -- .../boost/archive/text_wiarchive.hpp | 137 -- .../boost/archive/text_woarchive.hpp | 155 -- .../boost_1_65_0/boost/archive/tmpdir.hpp | 50 - .../boost_1_65_0/boost/archive/wcslen.hpp | 58 - .../boost/archive/xml_archive_exception.hpp | 57 - .../boost/archive/xml_iarchive.hpp | 142 -- .../boost/archive/xml_oarchive.hpp | 137 -- .../boost/archive/xml_wiarchive.hpp | 149 -- .../boost/archive/xml_woarchive.hpp | 134 -- .../boost_1_65_0/boost/foreach_fwd.hpp | 51 - .../boost/multi_index/composite_key.hpp | 1513 --------------- .../multi_index/detail/access_specifier.hpp | 54 - .../boost/multi_index/detail/adl_swap.hpp | 44 - .../detail/archive_constructed.hpp | 83 - .../boost/multi_index/detail/auto_space.hpp | 91 - .../boost/multi_index/detail/base_type.hpp | 74 - .../detail/bidir_node_iterator.hpp | 114 -- .../boost/multi_index/detail/bucket_array.hpp | 243 --- .../multi_index/detail/cons_stdtuple.hpp | 93 - .../boost/multi_index/detail/converter.hpp | 52 - .../boost/multi_index/detail/copy_map.hpp | 142 -- .../detail/do_not_copy_elements_tag.hpp | 34 - .../detail/duplicates_iterator.hpp | 120 -- .../boost/multi_index/detail/has_tag.hpp | 42 - .../multi_index/detail/hash_index_args.hpp | 105 - .../detail/hash_index_iterator.hpp | 166 -- .../multi_index/detail/hash_index_node.hpp | 778 -------- .../multi_index/detail/header_holder.hpp | 50 - .../detail/ignore_wstrict_aliasing.hpp | 18 - .../boost/multi_index/detail/index_base.hpp | 293 --- .../boost/multi_index/detail/index_loader.hpp | 139 -- .../multi_index/detail/index_matcher.hpp | 249 --- .../multi_index/detail/index_node_base.hpp | 135 -- .../boost/multi_index/detail/index_saver.hpp | 135 -- .../multi_index/detail/invariant_assert.hpp | 21 - .../multi_index/detail/is_index_list.hpp | 40 - .../multi_index/detail/is_transparent.hpp | 135 -- .../boost/multi_index/detail/iter_adaptor.hpp | 321 --- .../multi_index/detail/modify_key_adaptor.hpp | 49 - .../multi_index/detail/no_duplicate_tags.hpp | 97 - .../boost/multi_index/detail/node_type.hpp | 66 - .../multi_index/detail/ord_index_args.hpp | 83 - .../multi_index/detail/ord_index_impl.hpp | 1567 --------------- .../multi_index/detail/ord_index_impl_fwd.hpp | 128 -- .../multi_index/detail/ord_index_node.hpp | 658 ------- .../multi_index/detail/ord_index_ops.hpp | 266 --- .../boost/multi_index/detail/promotes_arg.hpp | 83 - .../boost/multi_index/detail/raw_ptr.hpp | 52 - .../detail/restore_wstrict_aliasing.hpp | 11 - .../multi_index/detail/rnd_index_loader.hpp | 173 -- .../multi_index/detail/rnd_index_node.hpp | 273 --- .../multi_index/detail/rnd_index_ops.hpp | 203 -- .../detail/rnd_index_ptr_array.hpp | 144 -- .../multi_index/detail/rnd_node_iterator.hpp | 140 -- .../multi_index/detail/rnk_index_ops.hpp | 300 --- .../boost/multi_index/detail/safe_mode.hpp | 588 ------ .../boost/multi_index/detail/scope_guard.hpp | 453 ----- .../multi_index/detail/seq_index_node.hpp | 217 --- .../multi_index/detail/seq_index_ops.hpp | 203 -- .../detail/serialization_version.hpp | 73 - .../boost/multi_index/detail/uintptr_type.hpp | 76 - .../boost/multi_index/detail/unbounded.hpp | 66 - .../multi_index/detail/value_compare.hpp | 56 - .../multi_index/detail/vartempl_support.hpp | 247 --- .../boost/multi_index/global_fun.hpp | 185 -- .../boost/multi_index/hashed_index.hpp | 1725 ----------------- .../boost/multi_index/hashed_index_fwd.hpp | 74 - .../boost/multi_index/identity.hpp | 145 -- .../boost/multi_index/identity_fwd.hpp | 26 - .../boost/multi_index/indexed_by.hpp | 68 - .../boost/multi_index/key_extractors.hpp | 22 - .../boost/multi_index/mem_fun.hpp | 205 -- .../boost_1_65_0/boost/multi_index/member.hpp | 262 --- .../boost/multi_index/ordered_index.hpp | 114 -- .../boost/multi_index/ordered_index_fwd.hpp | 35 - .../boost/multi_index/random_access_index.hpp | 1167 ----------- .../multi_index/random_access_index_fwd.hpp | 91 - .../boost/multi_index/ranked_index.hpp | 382 ---- .../boost/multi_index/ranked_index_fwd.hpp | 35 - .../boost/multi_index/safe_mode_errors.hpp | 48 - .../boost/multi_index/sequenced_index.hpp | 1062 ---------- .../boost/multi_index/sequenced_index_fwd.hpp | 91 - .../boost_1_65_0/boost/multi_index/tag.hpp | 88 - .../boost/multi_index_container.hpp | 1362 ------------- .../boost/multi_index_container_fwd.hpp | 121 -- .../boost/serialization/access.hpp | 145 -- .../archive_input_unordered_map.hpp | 85 - .../archive_input_unordered_set.hpp | 72 - .../boost/serialization/array.hpp | 48 - .../serialization/array_optimization.hpp | 37 - .../boost/serialization/array_wrapper.hpp | 121 -- .../boost/serialization/assume_abstract.hpp | 60 - .../boost/serialization/base_object.hpp | 100 - .../boost/serialization/binary_object.hpp | 79 - .../boost/serialization/bitset.hpp | 75 - .../boost/serialization/boost_array.hpp | 33 - .../serialization/boost_unordered_map.hpp | 154 -- .../serialization/boost_unordered_set.hpp | 150 -- .../serialization/collection_size_type.hpp | 62 - .../boost/serialization/collection_traits.hpp | 79 - .../serialization/collections_load_imp.hpp | 106 - .../serialization/collections_save_imp.hpp | 82 - .../boost/serialization/complex.hpp | 81 - .../boost/serialization/config.hpp | 74 - .../boost/serialization/deque.hpp | 80 - .../detail/is_default_constructible.hpp | 54 - .../serialization/detail/shared_count_132.hpp | 551 ------ .../serialization/detail/shared_ptr_132.hpp | 443 ----- .../detail/shared_ptr_nmt_132.hpp | 182 -- .../detail/stack_constructor.hpp | 66 - .../boost/serialization/ephemeral.hpp | 72 - .../boost/serialization/export.hpp | 225 --- .../serialization/extended_type_info.hpp | 116 -- .../extended_type_info_no_rtti.hpp | 182 -- .../extended_type_info_typeid.hpp | 167 -- .../boost/serialization/factory.hpp | 102 - .../boost/serialization/force_include.hpp | 55 - .../boost/serialization/forward_list.hpp | 124 -- .../hash_collections_load_imp.hpp | 77 - .../hash_collections_save_imp.hpp | 97 - .../boost/serialization/hash_map.hpp | 232 --- .../boost/serialization/hash_set.hpp | 222 --- .../serialization/is_bitwise_serializable.hpp | 46 - .../boost/serialization/item_version_type.hpp | 68 - .../boost/serialization/level.hpp | 116 -- .../boost/serialization/level_enum.hpp | 55 - .../boost_1_65_0/boost/serialization/list.hpp | 85 - .../boost_1_65_0/boost/serialization/map.hpp | 139 -- .../boost_1_65_0/boost/serialization/nvp.hpp | 123 -- .../boost/serialization/optional.hpp | 107 - .../boost/serialization/priority_queue.hpp | 76 - .../boost/serialization/queue.hpp | 76 - .../boost/serialization/scoped_ptr.hpp | 58 - .../boost/serialization/serialization.hpp | 154 -- .../boost_1_65_0/boost/serialization/set.hpp | 137 -- .../boost/serialization/shared_ptr.hpp | 281 --- .../boost/serialization/shared_ptr_132.hpp | 222 --- .../boost/serialization/shared_ptr_helper.hpp | 209 -- .../boost/serialization/singleton.hpp | 166 -- .../boost/serialization/slist.hpp | 145 -- .../boost/serialization/smart_cast.hpp | 275 --- .../boost/serialization/split_free.hpp | 93 - .../boost/serialization/split_member.hpp | 86 - .../boost/serialization/stack.hpp | 76 - .../boost/serialization/state_saver.hpp | 96 - .../boost/serialization/static_warning.hpp | 103 - .../boost/serialization/string.hpp | 30 - .../boost/serialization/strong_typedef.hpp | 50 - .../boost/serialization/throw_exception.hpp | 44 - .../boost/serialization/tracking.hpp | 118 -- .../boost/serialization/tracking_enum.hpp | 41 - .../boost/serialization/traits.hpp | 65 - .../type_info_implementation.hpp | 73 - .../boost/serialization/unique_ptr.hpp | 68 - .../unordered_collections_load_imp.hpp | 73 - .../unordered_collections_save_imp.hpp | 86 - .../boost/serialization/unordered_map.hpp | 160 -- .../boost/serialization/unordered_set.hpp | 162 -- .../boost/serialization/utility.hpp | 56 - .../boost/serialization/valarray.hpp | 86 - .../boost/serialization/variant.hpp | 158 -- .../boost/serialization/vector.hpp | 233 --- .../boost/serialization/vector_135.hpp | 26 - .../boost/serialization/version.hpp | 107 - .../boost/serialization/void_cast.hpp | 298 --- .../boost/serialization/void_cast_fwd.hpp | 37 - .../boost/serialization/weak_ptr.hpp | 99 - .../boost/serialization/wrapper.hpp | 60 - contrib/poco | 1 + contrib/zlib-ng | 1 + dbms/src/Storages/MergeTree/MergeTreeData.cpp | 3 - 272 files changed, 12 insertions(+), 41591 deletions(-) create mode 160000 contrib/googletest delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/archive_exception.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_archive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iprimitive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oprimitive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_streambuf_locale_saver.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_text_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_text_iprimitive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_text_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_text_oprimitive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_xml_archive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_xml_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_xml_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive_impl.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive_impl.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_wiarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_woarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/codecvt_null.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/abi_prefix.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/abi_suffix.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/archive_serializer_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_archive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_warchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iserializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oserializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_iserializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_oserializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/check.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/common_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/common_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/decl.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/helper_collection.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/interface_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/interface_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/iserializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/oserializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_iarchive_route.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_oarchive_route.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/register_archive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/utf8_codecvt_facet.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/dinkumware.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/archive_serializer_map.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iarchive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iprimitive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oarchive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oprimitive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iarchive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iprimitive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oarchive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oprimitive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_grammar.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_iarchive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_oarchive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/text_iarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/text_oarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/text_wiarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/text_woarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/xml_iarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/xml_oarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/xml_wiarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/xml_woarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_exception.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_from_binary.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/binary_from_base64.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow_exception.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/escape.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/insert_linebreaks.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/istream_iterator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/mb_from_wchar.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/ostream_iterator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/remove_whitespace.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/transform_width.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/unescape.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/wchar_from_mb.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_escape.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape_exception.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_wiarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_woarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_wiarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_woarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/text_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/text_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/text_wiarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/text_woarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/tmpdir.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/wcslen.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_archive_exception.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_wiarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_woarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/foreach_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/composite_key.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/access_specifier.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/adl_swap.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/archive_constructed.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/auto_space.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/base_type.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/bidir_node_iterator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/bucket_array.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/cons_stdtuple.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/converter.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/copy_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/do_not_copy_elements_tag.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/duplicates_iterator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/has_tag.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_args.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_iterator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_node.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/header_holder.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ignore_wstrict_aliasing.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_base.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_loader.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_matcher.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_node_base.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_saver.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/invariant_assert.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_index_list.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_transparent.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/iter_adaptor.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/modify_key_adaptor.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/no_duplicate_tags.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/node_type.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_args.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_node.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_ops.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/promotes_arg.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/raw_ptr.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/restore_wstrict_aliasing.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_loader.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_node.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ops.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ptr_array.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_node_iterator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnk_index_ops.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/safe_mode.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/scope_guard.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_node.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_ops.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/serialization_version.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/uintptr_type.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/unbounded.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/value_compare.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/vartempl_support.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/global_fun.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/identity.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/identity_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/indexed_by.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/key_extractors.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/mem_fun.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/member.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/safe_mode_errors.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/tag.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index_container.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index_container_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/access.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_set.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/array.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/array_optimization.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/array_wrapper.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/assume_abstract.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/base_object.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/binary_object.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/bitset.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/boost_array.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_set.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/collection_size_type.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/collection_traits.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/collections_load_imp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/collections_save_imp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/complex.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/config.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/deque.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/is_default_constructible.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_count_132.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_132.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_nmt_132.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/stack_constructor.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/ephemeral.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/export.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_no_rtti.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_typeid.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/factory.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/force_include.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/forward_list.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_load_imp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_save_imp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/hash_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/hash_set.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/is_bitwise_serializable.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/item_version_type.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/level.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/level_enum.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/list.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/nvp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/optional.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/priority_queue.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/queue.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/scoped_ptr.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/serialization.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/set.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_132.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_helper.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/singleton.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/slist.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/smart_cast.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/split_free.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/split_member.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/stack.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/state_saver.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/static_warning.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/string.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/strong_typedef.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/throw_exception.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/tracking.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/tracking_enum.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/traits.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/type_info_implementation.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unique_ptr.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_load_imp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_save_imp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unordered_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unordered_set.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/utility.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/valarray.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/variant.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/vector.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/vector_135.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/version.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/void_cast.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/void_cast_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/weak_ptr.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/wrapper.hpp create mode 160000 contrib/poco create mode 160000 contrib/zlib-ng diff --git a/.gitmodules b/.gitmodules index 336c2a892e2..0fda654f07c 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,3 +1,6 @@ +[submodule "contrib/poco"] + path = contrib/poco + url = https://github.com/ClickHouse-Extras/poco [submodule "contrib/zstd"] path = contrib/zstd url = https://github.com/facebook/zstd.git @@ -10,6 +13,12 @@ [submodule "contrib/cctz"] path = contrib/cctz url = https://github.com/google/cctz.git +[submodule "contrib/zlib-ng"] + path = contrib/zlib-ng + url = https://github.com/ClickHouse-Extras/zlib-ng.git +[submodule "contrib/googletest"] + path = contrib/googletest + url = https://github.com/google/googletest.git [submodule "contrib/capnproto"] path = contrib/capnproto url = https://github.com/capnproto/capnproto.git diff --git a/contrib/googletest b/contrib/googletest new file mode 160000 index 00000000000..d175c8bf823 --- /dev/null +++ b/contrib/googletest @@ -0,0 +1 @@ +Subproject commit d175c8bf823e709d570772b038757fadf63bc632 diff --git a/contrib/libboost/boost_1_65_0/boost/archive/archive_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/archive_exception.hpp deleted file mode 100644 index fabcdb5fa71..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/archive_exception.hpp +++ /dev/null @@ -1,100 +0,0 @@ -#ifndef BOOST_ARCHIVE_ARCHIVE_EXCEPTION_HPP -#define BOOST_ARCHIVE_ARCHIVE_EXCEPTION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// archive/archive_exception.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#include -#include - -// note: the only reason this is in here is that windows header -// includes #define exception_code _exception_code (arrrgghhhh!). -// the most expedient way to address this is be sure that this -// header is always included whenever this header file is included. -#if defined(BOOST_WINDOWS) -#include -#endif - -#include // must be the last header - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// exceptions thrown by archives -// -class BOOST_SYMBOL_VISIBLE archive_exception : - public virtual std::exception -{ -private: - char m_buffer[128]; -protected: - BOOST_ARCHIVE_DECL unsigned int - append(unsigned int l, const char * a); - BOOST_ARCHIVE_DECL - archive_exception() BOOST_NOEXCEPT; -public: - typedef enum { - no_exception, // initialized without code - other_exception, // any excepton not listed below - unregistered_class, // attempt to serialize a pointer of - // an unregistered class - invalid_signature, // first line of archive does not contain - // expected string - unsupported_version,// archive created with library version - // subsequent to this one - pointer_conflict, // an attempt has been made to directly - // serialize an object which has - // already been serialized through a pointer. - // Were this permitted, the archive load would result - // in the creation of an extra copy of the obect. - incompatible_native_format, // attempt to read native binary format - // on incompatible platform - array_size_too_short,// array being loaded doesn't fit in array allocated - input_stream_error, // error on input stream - invalid_class_name, // class name greater than the maximum permitted. - // most likely a corrupted archive or an attempt - // to insert virus via buffer overrun method. - unregistered_cast, // base - derived relationship not registered with - // void_cast_register - unsupported_class_version, // type saved with a version # greater than the - // one used by the program. This indicates that the program - // needs to be rebuilt. - multiple_code_instantiation, // code for implementing serialization for some - // type has been instantiated in more than one module. - output_stream_error // error on input stream - } exception_code; - exception_code code; - - BOOST_ARCHIVE_DECL archive_exception( - exception_code c, - const char * e1 = NULL, - const char * e2 = NULL - ) BOOST_NOEXCEPT; - BOOST_ARCHIVE_DECL archive_exception(archive_exception const &) BOOST_NOEXCEPT ; - virtual BOOST_ARCHIVE_DECL ~archive_exception() BOOST_NOEXCEPT_OR_NOTHROW ; - virtual BOOST_ARCHIVE_DECL const char * what() const BOOST_NOEXCEPT_OR_NOTHROW ; -}; - -}// namespace archive -}// namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif //BOOST_ARCHIVE_ARCHIVE_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_archive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_archive.hpp deleted file mode 100644 index ce7ac99a6dd..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_archive.hpp +++ /dev/null @@ -1,304 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_ARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_ARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_archive.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include // count -#include -#include -#include // size_t -#include -#include - -#include -#include // must be the last header - -namespace boost { -namespace archive { - -#if defined(_MSC_VER) -#pragma warning( push ) -#pragma warning( disable : 4244 4267 ) -#endif - -/* NOTE : Warning : Warning : Warning : Warning : Warning - * Don't ever changes this. If you do, they previously created - * binary archives won't be readable !!! - */ -class library_version_type { -private: - typedef uint_least16_t base_type; - base_type t; -public: - library_version_type(): t(0) {}; - explicit library_version_type(const unsigned int & t_) : t(t_){ - BOOST_ASSERT(t_ <= boost::integer_traits::const_max); - } - library_version_type(const library_version_type & t_) : - t(t_.t) - {} - library_version_type & operator=(const library_version_type & rhs){ - t = rhs.t; - return *this; - } - // used for text output - operator base_type () const { - return t; - } - // used for text input - operator base_type & (){ - return t; - } - bool operator==(const library_version_type & rhs) const { - return t == rhs.t; - } - bool operator<(const library_version_type & rhs) const { - return t < rhs.t; - } -}; - -BOOST_ARCHIVE_DECL library_version_type -BOOST_ARCHIVE_VERSION(); - -class version_type { -private: - typedef uint_least32_t base_type; - base_type t; -public: - // should be private - but MPI fails if it's not!!! - version_type(): t(0) {}; - explicit version_type(const unsigned int & t_) : t(t_){ - BOOST_ASSERT(t_ <= boost::integer_traits::const_max); - } - version_type(const version_type & t_) : - t(t_.t) - {} - version_type & operator=(const version_type & rhs){ - t = rhs.t; - return *this; - } - // used for text output - operator base_type () const { - return t; - } - // used for text intput - operator base_type & (){ - return t; - } - bool operator==(const version_type & rhs) const { - return t == rhs.t; - } - bool operator<(const version_type & rhs) const { - return t < rhs.t; - } -}; - -class class_id_type { -private: - typedef int_least16_t base_type; - base_type t; -public: - // should be private - but then can't use BOOST_STRONG_TYPE below - class_id_type() : t(0) {}; - explicit class_id_type(const int t_) : t(t_){ - BOOST_ASSERT(t_ <= boost::integer_traits::const_max); - } - explicit class_id_type(const std::size_t t_) : t(t_){ - // BOOST_ASSERT(t_ <= boost::integer_traits::const_max); - } - class_id_type(const class_id_type & t_) : - t(t_.t) - {} - class_id_type & operator=(const class_id_type & rhs){ - t = rhs.t; - return *this; - } - - // used for text output - operator int () const { - return t; - } - // used for text input - operator int_least16_t &() { - return t; - } - bool operator==(const class_id_type & rhs) const { - return t == rhs.t; - } - bool operator<(const class_id_type & rhs) const { - return t < rhs.t; - } -}; - -#define NULL_POINTER_TAG boost::archive::class_id_type(-1) - -class object_id_type { -private: - typedef uint_least32_t base_type; - base_type t; -public: - object_id_type(): t(0) {}; - // note: presumes that size_t >= unsigned int. - explicit object_id_type(const std::size_t & t_) : t(t_){ - BOOST_ASSERT(t_ <= boost::integer_traits::const_max); - } - object_id_type(const object_id_type & t_) : - t(t_.t) - {} - object_id_type & operator=(const object_id_type & rhs){ - t = rhs.t; - return *this; - } - // used for text output - operator uint_least32_t () const { - return t; - } - // used for text input - operator uint_least32_t & () { - return t; - } - bool operator==(const object_id_type & rhs) const { - return t == rhs.t; - } - bool operator<(const object_id_type & rhs) const { - return t < rhs.t; - } -}; - -#if defined(_MSC_VER) -#pragma warning( pop ) -#endif - -struct tracking_type { - bool t; - explicit tracking_type(const bool t_ = false) - : t(t_) - {}; - tracking_type(const tracking_type & t_) - : t(t_.t) - {} - operator bool () const { - return t; - }; - operator bool & () { - return t; - }; - tracking_type & operator=(const bool t_){ - t = t_; - return *this; - } - bool operator==(const tracking_type & rhs) const { - return t == rhs.t; - } - bool operator==(const bool & rhs) const { - return t == rhs; - } - tracking_type & operator=(const tracking_type & rhs){ - t = rhs.t; - return *this; - } -}; - -struct class_name_type : - private boost::noncopyable -{ - char *t; - operator const char * & () const { - return const_cast(t); - } - operator char * () { - return t; - } - std::size_t size() const { - return std::strlen(t); - } - explicit class_name_type(const char *key_) - : t(const_cast(key_)){} - explicit class_name_type(char *key_) - : t(key_){} - class_name_type & operator=(const class_name_type & rhs){ - t = rhs.t; - return *this; - } -}; - -enum archive_flags { - no_header = 1, // suppress archive header info - no_codecvt = 2, // suppress alteration of codecvt facet - no_xml_tag_checking = 4, // suppress checking of xml tags - no_tracking = 8, // suppress ALL tracking - flags_last = 8 -}; - -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_SIGNATURE(); - -/* NOTE : Warning : Warning : Warning : Warning : Warning - * If any of these are changed to different sized types, - * binary_iarchive won't be able to read older archives - * unless you rev the library version and include conditional - * code based on the library version. There is nothing - * inherently wrong in doing this - but you have to be super - * careful because it's easy to get wrong and start breaking - * old archives !!! - */ - -#define BOOST_ARCHIVE_STRONG_TYPEDEF(T, D) \ - class D : public T { \ - public: \ - explicit D(const T tt) : T(tt){} \ - }; \ -/**/ - -BOOST_ARCHIVE_STRONG_TYPEDEF(class_id_type, class_id_reference_type) -BOOST_ARCHIVE_STRONG_TYPEDEF(class_id_type, class_id_optional_type) -BOOST_ARCHIVE_STRONG_TYPEDEF(object_id_type, object_reference_type) - -}// namespace archive -}// namespace boost - -#include // pops abi_suffix.hpp pragmas - -#include - -// set implementation level to primitive for all types -// used internally by the serialization library - -BOOST_CLASS_IMPLEMENTATION(boost::archive::library_version_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::version_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::class_id_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::class_id_reference_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::class_id_optional_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::class_name_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::object_id_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::object_reference_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::tracking_type, primitive_type) - -#include - -// set types used internally by the serialization library -// to be bitwise serializable - -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::library_version_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::version_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::class_id_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::class_id_reference_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::class_id_optional_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::class_name_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::object_id_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::object_reference_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::tracking_type) - -#endif //BOOST_ARCHIVE_BASIC_ARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iarchive.hpp deleted file mode 100644 index c0cc655c997..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iarchive.hpp +++ /dev/null @@ -1,204 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_BINARY_IARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_BINARY_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_iarchive.hpp -// -// archives stored as native binary - this should be the fastest way -// to archive the state of a group of obects. It makes no attempt to -// convert to any canonical form. - -// IN GENERAL, ARCHIVES CREATED WITH THIS CLASS WILL NOT BE READABLE -// ON PLATFORM APART FROM THE ONE THEY ARE CREATED ON - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -#include // must be the last header - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -///////////////////////////////////////////////////////////////////////// -// class basic_binary_iarchive - read serialized objects from a input binary stream -template -class BOOST_SYMBOL_VISIBLE basic_binary_iarchive : - public detail::common_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_iarchive; - #else - friend class detail::interface_iarchive; - #endif -#endif - // intermediate level to support override of operators - // fot templates in the absence of partial function - // template ordering. If we get here pass to base class - // note extra nonsense to sneak it pass the borland compiers - typedef detail::common_iarchive detail_common_iarchive; - template - void load_override(T & t){ - this->detail_common_iarchive::load_override(t); - } - - // include these to trap a change in binary format which - // isn't specifically handled - // upto 32K classes - BOOST_STATIC_ASSERT(sizeof(class_id_type) == sizeof(int_least16_t)); - BOOST_STATIC_ASSERT(sizeof(class_id_reference_type) == sizeof(int_least16_t)); - // upto 2G objects - BOOST_STATIC_ASSERT(sizeof(object_id_type) == sizeof(uint_least32_t)); - BOOST_STATIC_ASSERT(sizeof(object_reference_type) == sizeof(uint_least32_t)); - - // binary files don't include the optional information - void load_override(class_id_optional_type & /* t */){} - - void load_override(tracking_type & t, int /*version*/){ - library_version_type lvt = this->get_library_version(); - if(boost::archive::library_version_type(6) < lvt){ - int_least8_t x=0; - * this->This() >> x; - t = boost::archive::tracking_type(x); - } - else{ - bool x=0; - * this->This() >> x; - t = boost::archive::tracking_type(x); - } - } - void load_override(class_id_type & t){ - library_version_type lvt = this->get_library_version(); - if(boost::archive::library_version_type(7) < lvt){ - this->detail_common_iarchive::load_override(t); - } - else - if(boost::archive::library_version_type(6) < lvt){ - int_least16_t x=0; - * this->This() >> x; - t = boost::archive::class_id_type(x); - } - else{ - int x=0; - * this->This() >> x; - t = boost::archive::class_id_type(x); - } - } - void load_override(class_id_reference_type & t){ - load_override(static_cast(t)); - } - - void load_override(version_type & t){ - library_version_type lvt = this->get_library_version(); - if(boost::archive::library_version_type(7) < lvt){ - this->detail_common_iarchive::load_override(t); - } - else - if(boost::archive::library_version_type(6) < lvt){ - uint_least8_t x=0; - * this->This() >> x; - t = boost::archive::version_type(x); - } - else - if(boost::archive::library_version_type(5) < lvt){ - uint_least16_t x=0; - * this->This() >> x; - t = boost::archive::version_type(x); - } - else - if(boost::archive::library_version_type(2) < lvt){ - // upto 255 versions - unsigned char x=0; - * this->This() >> x; - t = version_type(x); - } - else{ - unsigned int x=0; - * this->This() >> x; - t = boost::archive::version_type(x); - } - } - - void load_override(boost::serialization::item_version_type & t){ - library_version_type lvt = this->get_library_version(); -// if(boost::archive::library_version_type(7) < lvt){ - if(boost::archive::library_version_type(6) < lvt){ - this->detail_common_iarchive::load_override(t); - } - else - if(boost::archive::library_version_type(6) < lvt){ - uint_least16_t x=0; - * this->This() >> x; - t = boost::serialization::item_version_type(x); - } - else{ - unsigned int x=0; - * this->This() >> x; - t = boost::serialization::item_version_type(x); - } - } - - void load_override(serialization::collection_size_type & t){ - if(boost::archive::library_version_type(5) < this->get_library_version()){ - this->detail_common_iarchive::load_override(t); - } - else{ - unsigned int x=0; - * this->This() >> x; - t = serialization::collection_size_type(x); - } - } - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_override(class_name_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(); - - basic_binary_iarchive(unsigned int flags) : - detail::common_iarchive(flags) - {} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_BINARY_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iprimitive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iprimitive.hpp deleted file mode 100644 index 665d3e81e1f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iprimitive.hpp +++ /dev/null @@ -1,198 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_IPRIMITIVE_HPP -#define BOOST_ARCHIVE_BINARY_IPRIMITIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -#if defined(_MSC_VER) -#pragma warning( disable : 4800 ) -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_iprimitive.hpp -// -// archives stored as native binary - this should be the fastest way -// to archive the state of a group of obects. It makes no attempt to -// convert to any canonical form. - -// IN GENERAL, ARCHIVES CREATED WITH THIS CLASS WILL NOT BE READABLE -// ON PLATFORM APART FROM THE ONE THEY ARE CREATED ON - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include // std::memcpy -#include // std::size_t -#include // basic_streambuf -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include - -//#include -#include -#include - -#include -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace archive { - -///////////////////////////////////////////////////////////////////////////// -// class binary_iarchive - read serialized objects from a input binary stream -template -class BOOST_SYMBOL_VISIBLE basic_binary_iprimitive { -#ifndef BOOST_NO_MEMBER_TEMPLATE_FRIENDS - friend class load_access; -protected: -#else -public: -#endif - std::basic_streambuf & m_sb; - // return a pointer to the most derived class - Archive * This(){ - return static_cast(this); - } - - #ifndef BOOST_NO_STD_LOCALE - // note order! - if you change this, libstd++ will fail! - // a) create new locale with new codecvt facet - // b) save current locale - // c) change locale to new one - // d) use stream buffer - // e) change locale back to original - // f) destroy new codecvt facet - boost::archive::codecvt_null codecvt_null_facet; - basic_streambuf_locale_saver locale_saver; - std::locale archive_locale; - #endif - - // main template for serilization of primitive types - template - void load(T & t){ - load_binary(& t, sizeof(T)); - } - - ///////////////////////////////////////////////////////// - // fundamental types that need special treatment - - // trap usage of invalid uninitialized boolean - void load(bool & t){ - load_binary(& t, sizeof(t)); - int i = t; - BOOST_ASSERT(0 == i || 1 == i); - (void)i; // warning suppression for release builds. - } - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load(std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load(std::wstring &ws); - #endif - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load(char * t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load(wchar_t * t); - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - basic_binary_iprimitive( - std::basic_streambuf & sb, - bool no_codecvt - ); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - ~basic_binary_iprimitive(); -public: - // we provide an optimized load for all fundamental types - // typedef serialization::is_bitwise_serializable - // use_array_optimization; - struct use_array_optimization { - template - #if defined(BOOST_NO_DEPENDENT_NESTED_DERIVATIONS) - struct apply { - typedef typename boost::serialization::is_bitwise_serializable< T >::type type; - }; - #else - struct apply : public boost::serialization::is_bitwise_serializable< T > {}; - #endif - }; - - // the optimized load_array dispatches to load_binary - template - void load_array(serialization::array_wrapper& a, unsigned int) - { - load_binary(a.address(),a.count()*sizeof(ValueType)); - } - - void - load_binary(void *address, std::size_t count); -}; - -template -inline void -basic_binary_iprimitive::load_binary( - void *address, - std::size_t count -){ - // note: an optimizer should eliminate the following for char files - BOOST_ASSERT( - static_cast(count / sizeof(Elem)) - <= boost::integer_traits::const_max - ); - std::streamsize s = static_cast(count / sizeof(Elem)); - std::streamsize scount = m_sb.sgetn( - static_cast(address), - s - ); - if(scount != s) - boost::serialization::throw_exception( - archive_exception(archive_exception::input_stream_error) - ); - // note: an optimizer should eliminate the following for char files - BOOST_ASSERT(count % sizeof(Elem) <= boost::integer_traits::const_max); - s = static_cast(count % sizeof(Elem)); - if(0 < s){ -// if(is.fail()) -// boost::serialization::throw_exception( -// archive_exception(archive_exception::stream_error) -// ); - Elem t; - scount = m_sb.sgetn(& t, 1); - if(scount != 1) - boost::serialization::throw_exception( - archive_exception(archive_exception::input_stream_error) - ); - std::memcpy(static_cast(address) + (count - s), &t, static_cast(s)); - } -} - -} // namespace archive -} // namespace boost - -#include // pop pragmas - -#endif // BOOST_ARCHIVE_BINARY_IPRIMITIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oarchive.hpp deleted file mode 100644 index f05f2f86d55..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oarchive.hpp +++ /dev/null @@ -1,185 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_BINARY_OARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_BINARY_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// archives stored as native binary - this should be the fastest way -// to archive the state of a group of obects. It makes no attempt to -// convert to any canonical form. - -// IN GENERAL, ARCHIVES CREATED WITH THIS CLASS WILL NOT BE READABLE -// ON PLATFORM APART FROM THE ONE THEY ARE CREATE ON - -#include -#include -#include - -#include -#include - -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -////////////////////////////////////////////////////////////////////// -// class basic_binary_oarchive - write serialized objects to a binary output stream -// note: this archive has no pretensions to portability. Archive format -// may vary across machine architectures and compilers. About the only -// guarentee is that an archive created with this code will be readable -// by a program built with the same tools for the same machne. This class -// does have the virtue of buiding the smalles archive in the minimum amount -// of time. So under some circumstances it may be he right choice. -template -class BOOST_SYMBOL_VISIBLE basic_binary_oarchive : - public detail::common_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_oarchive; - #else - friend class detail::interface_oarchive; - #endif -#endif - // any datatype not specifed below will be handled by base class - typedef detail::common_oarchive detail_common_oarchive; - template - void save_override(const T & t){ - this->detail_common_oarchive::save_override(t); - } - - // include these to trap a change in binary format which - // isn't specifically handled - BOOST_STATIC_ASSERT(sizeof(tracking_type) == sizeof(bool)); - // upto 32K classes - BOOST_STATIC_ASSERT(sizeof(class_id_type) == sizeof(int_least16_t)); - BOOST_STATIC_ASSERT(sizeof(class_id_reference_type) == sizeof(int_least16_t)); - // upto 2G objects - BOOST_STATIC_ASSERT(sizeof(object_id_type) == sizeof(uint_least32_t)); - BOOST_STATIC_ASSERT(sizeof(object_reference_type) == sizeof(uint_least32_t)); - - // binary files don't include the optional information - void save_override(const class_id_optional_type & /* t */){} - - // enable this if we decide to support generation of previous versions - #if 0 - void save_override(const boost::archive::version_type & t){ - library_version_type lvt = this->get_library_version(); - if(boost::archive::library_version_type(7) < lvt){ - this->detail_common_oarchive::save_override(t); - } - else - if(boost::archive::library_version_type(6) < lvt){ - const boost::uint_least16_t x = t; - * this->This() << x; - } - else{ - const unsigned int x = t; - * this->This() << x; - } - } - void save_override(const boost::serialization::item_version_type & t){ - library_version_type lvt = this->get_library_version(); - if(boost::archive::library_version_type(7) < lvt){ - this->detail_common_oarchive::save_override(t); - } - else - if(boost::archive::library_version_type(6) < lvt){ - const boost::uint_least16_t x = t; - * this->This() << x; - } - else{ - const unsigned int x = t; - * this->This() << x; - } - } - - void save_override(class_id_type & t){ - library_version_type lvt = this->get_library_version(); - if(boost::archive::library_version_type(7) < lvt){ - this->detail_common_oarchive::save_override(t); - } - else - if(boost::archive::library_version_type(6) < lvt){ - const boost::int_least16_t x = t; - * this->This() << x; - } - else{ - const int x = t; - * this->This() << x; - } - } - void save_override(class_id_reference_type & t){ - save_override(static_cast(t)); - } - - #endif - - // explicitly convert to char * to avoid compile ambiguities - void save_override(const class_name_type & t){ - const std::string s(t); - * this->This() << s; - } - - #if 0 - void save_override(const serialization::collection_size_type & t){ - if (get_library_version() < boost::archive::library_version_type(6)){ - unsigned int x=0; - * this->This() >> x; - t = serialization::collection_size_type(x); - } - else{ - * this->This() >> t; - } - } - #endif - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(); - - basic_binary_oarchive(unsigned int flags) : - detail::common_oarchive(flags) - {} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_BINARY_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oprimitive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oprimitive.hpp deleted file mode 100644 index 6dc770c60e8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oprimitive.hpp +++ /dev/null @@ -1,188 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_BINARY_OPRIMITIVE_HPP -#define BOOST_ARCHIVE_BASIC_BINARY_OPRIMITIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_oprimitive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// archives stored as native binary - this should be the fastest way -// to archive the state of a group of obects. It makes no attempt to -// convert to any canonical form. - -// IN GENERAL, ARCHIVES CREATED WITH THIS CLASS WILL NOT BE READABLE -// ON PLATFORM APART FROM THE ONE THEY ARE CREATE ON - -#include -#include -#include -#include // basic_streambuf -#include -#include // size_t - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include - -//#include -#include -#include - -#include -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace archive { - -///////////////////////////////////////////////////////////////////////// -// class basic_binary_oprimitive - binary output of prmitives - -template -class BOOST_SYMBOL_VISIBLE basic_binary_oprimitive { -#ifndef BOOST_NO_MEMBER_TEMPLATE_FRIENDS - friend class save_access; -protected: -#else -public: -#endif - std::basic_streambuf & m_sb; - // return a pointer to the most derived class - Archive * This(){ - return static_cast(this); - } - #ifndef BOOST_NO_STD_LOCALE - // note order! - if you change this, libstd++ will fail! - // a) create new locale with new codecvt facet - // b) save current locale - // c) change locale to new one - // d) use stream buffer - // e) change locale back to original - // f) destroy new codecvt facet - boost::archive::codecvt_null codecvt_null_facet; - basic_streambuf_locale_saver locale_saver; - std::locale archive_locale; - #endif - // default saving of primitives. - template - void save(const T & t) - { - save_binary(& t, sizeof(T)); - } - - ///////////////////////////////////////////////////////// - // fundamental types that need special treatment - - // trap usage of invalid uninitialized boolean which would - // otherwise crash on load. - void save(const bool t){ - BOOST_ASSERT(0 == static_cast(t) || 1 == static_cast(t)); - save_binary(& t, sizeof(t)); - } - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save(const std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save(const std::wstring &ws); - #endif - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save(const char * t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save(const wchar_t * t); - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(); - - BOOST_ARCHIVE_OR_WARCHIVE_DECL - basic_binary_oprimitive( - std::basic_streambuf & sb, - bool no_codecvt - ); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - ~basic_binary_oprimitive(); -public: - - // we provide an optimized save for all fundamental types - // typedef serialization::is_bitwise_serializable - // use_array_optimization; - // workaround without using mpl lambdas - struct use_array_optimization { - template - #if defined(BOOST_NO_DEPENDENT_NESTED_DERIVATIONS) - struct apply { - typedef typename boost::serialization::is_bitwise_serializable< T >::type type; - }; - #else - struct apply : public boost::serialization::is_bitwise_serializable< T > {}; - #endif - }; - - // the optimized save_array dispatches to save_binary - template - void save_array(boost::serialization::array_wrapper const& a, unsigned int) - { - save_binary(a.address(),a.count()*sizeof(ValueType)); - } - - void save_binary(const void *address, std::size_t count); -}; - -template -inline void -basic_binary_oprimitive::save_binary( - const void *address, - std::size_t count -){ - // BOOST_ASSERT(count <= std::size_t(boost::integer_traits::const_max)); - // note: if the following assertions fail - // a likely cause is that the output stream is set to "text" - // mode where by cr characters recieve special treatment. - // be sure that the output stream is opened with ios::binary - //if(os.fail()) - // boost::serialization::throw_exception( - // archive_exception(archive_exception::output_stream_error) - // ); - // figure number of elements to output - round up - count = ( count + sizeof(Elem) - 1) / sizeof(Elem); - std::streamsize scount = m_sb.sputn( - static_cast(address), - static_cast(count) - ); - if(count != static_cast(scount)) - boost::serialization::throw_exception( - archive_exception(archive_exception::output_stream_error) - ); - //os.write( - // static_cast(address), - // count - //); - //BOOST_ASSERT(os.good()); -} - -} //namespace boost -} //namespace archive - -#include // pop pragmas - -#endif // BOOST_ARCHIVE_BASIC_BINARY_OPRIMITIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_streambuf_locale_saver.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_streambuf_locale_saver.hpp deleted file mode 100644 index 5cd4b36f081..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_streambuf_locale_saver.hpp +++ /dev/null @@ -1,108 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_STREAMBUF_LOCALE_SAVER_HPP -#define BOOST_ARCHIVE_BASIC_STREAMBUF_LOCALE_SAVER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_streambuf_locale_saver.hpp - -// (C) Copyright 2005 Robert Ramey - http://www.rrsd.com - -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// note derived from boost/io/ios_state.hpp -// Copyright 2002, 2005 Daryle Walker. Use, modification, and distribution -// are subject to the Boost Software License, Version 1.0. (See accompanying -// file LICENSE_1_0.txt or a copy at .) - -// See for the library's home page. - -#ifndef BOOST_NO_STD_LOCALE - -#include // for std::locale -#include -#include // for std::basic_streambuf - -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost{ -namespace archive{ - -template < typename Ch, class Tr > -class basic_streambuf_locale_saver : - private boost::noncopyable -{ -public: - explicit basic_streambuf_locale_saver(std::basic_streambuf &s) : - m_streambuf(s), - m_locale(s.getloc()) - {} - ~basic_streambuf_locale_saver(){ - m_streambuf.pubsync(); - m_streambuf.pubimbue(m_locale); - } -private: - std::basic_streambuf & m_streambuf; - std::locale const m_locale; -}; - -template < typename Ch, class Tr > -class basic_istream_locale_saver : - private boost::noncopyable -{ -public: - explicit basic_istream_locale_saver(std::basic_istream &s) : - m_istream(s), - m_locale(s.getloc()) - {} - ~basic_istream_locale_saver(){ - // libstdc++ crashes without this - m_istream.sync(); - m_istream.imbue(m_locale); - } -private: - std::basic_istream & m_istream; - std::locale const m_locale; -}; - -template < typename Ch, class Tr > -class basic_ostream_locale_saver : - private boost::noncopyable -{ -public: - explicit basic_ostream_locale_saver(std::basic_ostream &s) : - m_ostream(s), - m_locale(s.getloc()) - {} - ~basic_ostream_locale_saver(){ - m_ostream.flush(); - m_ostream.imbue(m_locale); - } -private: - std::basic_ostream & m_ostream; - std::locale const m_locale; -}; - - -} // archive -} // boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_NO_STD_LOCALE -#endif // BOOST_ARCHIVE_BASIC_STREAMBUF_LOCALE_SAVER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iarchive.hpp deleted file mode 100644 index 48a646cc1f7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iarchive.hpp +++ /dev/null @@ -1,96 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_TEXT_IARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_TEXT_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// archives stored as text - note these ar templated on the basic -// stream templates to accommodate wide (and other?) kind of characters -// -// note the fact that on libraries without wide characters, ostream is -// is not a specialization of basic_ostream which in fact is not defined -// in such cases. So we can't use basic_istream but rather -// use two template parameters - -#include -#include - -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -///////////////////////////////////////////////////////////////////////// -// class basic_text_iarchive - read serialized objects from a input text stream -template -class BOOST_SYMBOL_VISIBLE basic_text_iarchive : - public detail::common_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_iarchive; - #else - friend class detail::interface_iarchive; - #endif -#endif - // intermediate level to support override of operators - // fot templates in the absence of partial function - // template ordering - typedef detail::common_iarchive detail_common_iarchive; - template - void load_override(T & t){ - this->detail_common_iarchive::load_override(t); - } - // text file don't include the optional information - void load_override(class_id_optional_type & /*t*/){} - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_override(class_name_type & t); - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(void); - - basic_text_iarchive(unsigned int flags) : - detail::common_iarchive(flags) - {} - ~basic_text_iarchive(){} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_TEXT_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iprimitive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iprimitive.hpp deleted file mode 100644 index bf936b55546..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iprimitive.hpp +++ /dev/null @@ -1,142 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_TEXT_IPRIMITIVE_HPP -#define BOOST_ARCHIVE_BASIC_TEXT_IPRIMITIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_iprimitive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// archives stored as text - note these are templated on the basic -// stream templates to accommodate wide (and other?) kind of characters -// -// Note the fact that on libraries without wide characters, ostream is -// not a specialization of basic_ostream which in fact is not defined -// in such cases. So we can't use basic_ostream but rather -// use two template parameters - -#include -#include // size_t - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; - #if ! defined(BOOST_DINKUMWARE_STDLIB) && ! defined(__SGI_STL_PORT) - using ::locale; - #endif -} // namespace std -#endif - -#include -#include - -#include -#if BOOST_WORKAROUND(BOOST_DINKUMWARE_STDLIB, == 1) -#include -#endif -#include -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace archive { - -///////////////////////////////////////////////////////////////////////// -// class basic_text_iarchive - load serialized objects from a input text stream -#if defined(_MSC_VER) -#pragma warning( push ) -#pragma warning( disable : 4244 4267 ) -#endif - -template -class BOOST_SYMBOL_VISIBLE basic_text_iprimitive { -protected: - IStream &is; - io::ios_flags_saver flags_saver; - io::ios_precision_saver precision_saver; - - #ifndef BOOST_NO_STD_LOCALE - // note order! - if you change this, libstd++ will fail! - // a) create new locale with new codecvt facet - // b) save current locale - // c) change locale to new one - // d) use stream buffer - // e) change locale back to original - // f) destroy new codecvt facet - boost::archive::codecvt_null codecvt_null_facet; - std::locale archive_locale; - basic_istream_locale_saver< - typename IStream::char_type, - typename IStream::traits_type - > locale_saver; - #endif - - template - void load(T & t) - { - if(is >> t) - return; - boost::serialization::throw_exception( - archive_exception(archive_exception::input_stream_error) - ); - } - - void load(char & t) - { - short int i; - load(i); - t = i; - } - void load(signed char & t) - { - short int i; - load(i); - t = i; - } - void load(unsigned char & t) - { - unsigned short int i; - load(i); - t = i; - } - - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - void load(wchar_t & t) - { - BOOST_STATIC_ASSERT(sizeof(wchar_t) <= sizeof(int)); - int i; - load(i); - t = i; - } - #endif - BOOST_ARCHIVE_OR_WARCHIVE_DECL - basic_text_iprimitive(IStream &is, bool no_codecvt); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - ~basic_text_iprimitive(); -public: - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_binary(void *address, std::size_t count); -}; - -#if defined(_MSC_VER) -#pragma warning( pop ) -#endif - -} // namespace archive -} // namespace boost - -#include // pop pragmas - -#endif // BOOST_ARCHIVE_BASIC_TEXT_IPRIMITIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oarchive.hpp deleted file mode 100644 index 6f7f8fb167d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oarchive.hpp +++ /dev/null @@ -1,119 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_TEXT_OARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_TEXT_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// archives stored as text - note these ar templated on the basic -// stream templates to accommodate wide (and other?) kind of characters -// -// note the fact that on libraries without wide characters, ostream is -// is not a specialization of basic_ostream which in fact is not defined -// in such cases. So we can't use basic_ostream but rather -// use two template parameters - -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -///////////////////////////////////////////////////////////////////////// -// class basic_text_oarchive -template -class BOOST_SYMBOL_VISIBLE basic_text_oarchive : - public detail::common_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_oarchive; - #else - friend class detail::interface_oarchive; - #endif -#endif - - enum { - none, - eol, - space - } delimiter; - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - newtoken(); - - void newline(){ - delimiter = eol; - } - - // default processing - kick back to base class. Note the - // extra stuff to get it passed borland compilers - typedef detail::common_oarchive detail_common_oarchive; - template - void save_override(T & t){ - this->detail_common_oarchive::save_override(t); - } - - // start new objects on a new line - void save_override(const object_id_type & t){ - this->This()->newline(); - this->detail_common_oarchive::save_override(t); - } - - // text file don't include the optional information - void save_override(const class_id_optional_type & /* t */){} - - void save_override(const class_name_type & t){ - const std::string s(t); - * this->This() << s; - } - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(); - - basic_text_oarchive(unsigned int flags) : - detail::common_oarchive(flags), - delimiter(none) - {} - ~basic_text_oarchive(){} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_TEXT_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oprimitive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oprimitive.hpp deleted file mode 100644 index 45f09358ece..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oprimitive.hpp +++ /dev/null @@ -1,209 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_TEXT_OPRIMITIVE_HPP -#define BOOST_ARCHIVE_BASIC_TEXT_OPRIMITIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_oprimitive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// archives stored as text - note these ar templated on the basic -// stream templates to accommodate wide (and other?) kind of characters -// -// note the fact that on libraries without wide characters, ostream is -// is not a specialization of basic_ostream which in fact is not defined -// in such cases. So we can't use basic_ostream but rather -// use two template parameters - -#include -#include -#include // size_t - -#include -#include -#include - -#include -#if BOOST_WORKAROUND(BOOST_DINKUMWARE_STDLIB, == 1) -#include -#endif - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; - #if ! defined(BOOST_DINKUMWARE_STDLIB) && ! defined(__SGI_STL_PORT) - using ::locale; - #endif -} // namespace std -#endif - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace archive { - -///////////////////////////////////////////////////////////////////////// -// class basic_text_oprimitive - output of prmitives to stream -template -class BOOST_SYMBOL_VISIBLE basic_text_oprimitive -{ -protected: - OStream &os; - io::ios_flags_saver flags_saver; - io::ios_precision_saver precision_saver; - - #ifndef BOOST_NO_STD_LOCALE - // note order! - if you change this, libstd++ will fail! - // a) create new locale with new codecvt facet - // b) save current locale - // c) change locale to new one - // d) use stream buffer - // e) change locale back to original - // f) destroy new codecvt facet - boost::archive::codecvt_null codecvt_null_facet; - std::locale archive_locale; - basic_ostream_locale_saver< - typename OStream::char_type, - typename OStream::traits_type - > locale_saver; - #endif - - ///////////////////////////////////////////////////////// - // fundamental types that need special treatment - void save(const bool t){ - // trap usage of invalid uninitialized boolean which would - // otherwise crash on load. - BOOST_ASSERT(0 == static_cast(t) || 1 == static_cast(t)); - if(os.fail()) - boost::serialization::throw_exception( - archive_exception(archive_exception::output_stream_error) - ); - os << t; - } - void save(const signed char t) - { - save(static_cast(t)); - } - void save(const unsigned char t) - { - save(static_cast(t)); - } - void save(const char t) - { - save(static_cast(t)); - } - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - void save(const wchar_t t) - { - BOOST_STATIC_ASSERT(sizeof(wchar_t) <= sizeof(int)); - save(static_cast(t)); - } - #endif - - ///////////////////////////////////////////////////////// - // saving of any types not listed above - - template - void save_impl(const T &t, boost::mpl::bool_ &){ - if(os.fail()) - boost::serialization::throw_exception( - archive_exception(archive_exception::output_stream_error) - ); - os << t; - } - - ///////////////////////////////////////////////////////// - // floating point types need even more special treatment - // the following determines whether the type T is some sort - // of floating point type. Note that we then assume that - // the stream << operator is defined on that type - if not - // we'll get a compile time error. This is meant to automatically - // support synthesized types which support floating point - // operations. Also it should handle compiler dependent types - // such long double. Due to John Maddock. - - template - struct is_float { - typedef typename mpl::bool_< - boost::is_floating_point::value - || (std::numeric_limits::is_specialized - && !std::numeric_limits::is_integer - && !std::numeric_limits::is_exact - && std::numeric_limits::max_exponent) - >::type type; - }; - - template - void save_impl(const T &t, boost::mpl::bool_ &){ - // must be a user mistake - can't serialize un-initialized data - if(os.fail()) - boost::serialization::throw_exception( - archive_exception(archive_exception::output_stream_error) - ); - // The formulae for the number of decimla digits required is given in - // http://www2.open-std.org/JTC1/SC22/WG21/docs/papers/2005/n1822.pdf - // which is derived from Kahan's paper: - // www.eecs.berkeley.edu/~wkahan/ieee754status/ieee754.ps - // const unsigned int digits = (std::numeric_limits::digits * 3010) / 10000; - // note: I've commented out the above because I didn't get good results. e.g. - // in one case I got a difference of 19 units. - #ifndef BOOST_NO_CXX11_NUMERIC_LIMITS - const unsigned int digits = std::numeric_limits::max_digits10; - #else - const unsigned int digits = std::numeric_limits::digits10 + 2; - #endif - os << std::setprecision(digits) << std::scientific << t; - } - - template - void save(const T & t){ - typename is_float::type tf; - save_impl(t, tf); - } - - BOOST_ARCHIVE_OR_WARCHIVE_DECL - basic_text_oprimitive(OStream & os, bool no_codecvt); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - ~basic_text_oprimitive(); -public: - // unformatted append of one character - void put(typename OStream::char_type c){ - if(os.fail()) - boost::serialization::throw_exception( - archive_exception(archive_exception::output_stream_error) - ); - os.put(c); - } - // unformatted append of null terminated string - void put(const char * s){ - while('\0' != *s) - os.put(*s++); - } - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_binary(const void *address, std::size_t count); -}; - -} //namespace boost -} //namespace archive - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_TEXT_OPRIMITIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_archive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_archive.hpp deleted file mode 100644 index bef368b973b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_archive.hpp +++ /dev/null @@ -1,67 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_XML_TEXT_ARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_XML_TEXT_ARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_xml_archive.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include // must be the last header - -namespace boost { -namespace archive { - -// constant strings used in xml i/o - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_OBJECT_ID(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_OBJECT_REFERENCE(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_CLASS_ID(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_CLASS_ID_REFERENCE(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_CLASS_NAME(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_TRACKING(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_VERSION(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_SIGNATURE(); - -}// namespace archive -}// namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_XML_TEXT_ARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_iarchive.hpp deleted file mode 100644 index e9f7482f744..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_iarchive.hpp +++ /dev/null @@ -1,119 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_XML_IARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_XML_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_xml_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -///////////////////////////////////////////////////////////////////////// -// class basic_xml_iarchive - read serialized objects from a input text stream -template -class BOOST_SYMBOL_VISIBLE basic_xml_iarchive : - public detail::common_iarchive -{ - unsigned int depth; -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_iarchive; -#endif - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_start(const char *name); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_end(const char *name); - - // Anything not an attribute and not a name-value pair is an - // should be trapped here. - template - void load_override(T & t) - { - // If your program fails to compile here, its most likely due to - // not specifying an nvp wrapper around the variable to - // be serialized. - BOOST_MPL_ASSERT((serialization::is_wrapper< T >)); - this->detail_common_iarchive::load_override(t); - } - - // Anything not an attribute - see below - should be a name value - // pair and be processed here - typedef detail::common_iarchive detail_common_iarchive; - template - void load_override( - const boost::serialization::nvp< T > & t - ){ - this->This()->load_start(t.name()); - this->detail_common_iarchive::load_override(t.value()); - this->This()->load_end(t.name()); - } - - // specific overrides for attributes - handle as - // primitives. These are not name-value pairs - // so they have to be intercepted here and passed on to load. - // although the class_id is included in the xml text file in order - // to make the file self describing, it isn't used when loading - // an xml archive. So we can skip it here. Note: we MUST override - // it otherwise it will be loaded as a normal primitive w/o tag and - // leaving the archive in an undetermined state - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_override(class_id_type & t); - void load_override(class_id_optional_type & /* t */){} - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_override(object_id_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_override(version_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_override(tracking_type & t); - // class_name_type can't be handled here as it depends upon the - // char type used by the stream. So require the derived implementation - // handle this. - // void load_override(class_name_type & t); - - BOOST_ARCHIVE_OR_WARCHIVE_DECL - basic_xml_iarchive(unsigned int flags); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - ~basic_xml_iarchive(); -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_XML_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_oarchive.hpp deleted file mode 100644 index 107fca4ec65..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_oarchive.hpp +++ /dev/null @@ -1,138 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_XML_OARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_XML_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_xml_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -////////////////////////////////////////////////////////////////////// -// class basic_xml_oarchive - write serialized objects to a xml output stream -template -class BOOST_SYMBOL_VISIBLE basic_xml_oarchive : - public detail::common_oarchive -{ - // special stuff for xml output - unsigned int depth; - bool pending_preamble; -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_oarchive; -#endif - bool indent_next; - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - indent(); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - windup(); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - write_attribute( - const char *attribute_name, - int t, - const char *conjunction = "=\"" - ); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - write_attribute( - const char *attribute_name, - const char *key - ); - // helpers used below - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_start(const char *name); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_end(const char *name); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - end_preamble(); - - // Anything not an attribute and not a name-value pair is an - // error and should be trapped here. - template - void save_override(T & t) - { - // If your program fails to compile here, its most likely due to - // not specifying an nvp wrapper around the variable to - // be serialized. - BOOST_MPL_ASSERT((serialization::is_wrapper< T >)); - this->detail_common_oarchive::save_override(t); - } - - // special treatment for name-value pairs. - typedef detail::common_oarchive detail_common_oarchive; - template - void save_override( - const ::boost::serialization::nvp< T > & t - ){ - this->This()->save_start(t.name()); - this->detail_common_oarchive::save_override(t.const_value()); - this->This()->save_end(t.name()); - } - - // specific overrides for attributes - not name value pairs so we - // want to trap them before the above "fall through" - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const class_id_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const class_id_optional_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const class_id_reference_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const object_id_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const object_reference_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const version_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const class_name_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const tracking_type & t); - - BOOST_ARCHIVE_OR_WARCHIVE_DECL - basic_xml_oarchive(unsigned int flags); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - ~basic_xml_oarchive(); -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_XML_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive.hpp deleted file mode 100644 index 785ce7610b1..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive.hpp +++ /dev/null @@ -1,64 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_IARCHIVE_HPP -#define BOOST_ARCHIVE_BINARY_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from binary_iarchive_impl instead. This will -// preserve correct static polymorphism. -class BOOST_SYMBOL_VISIBLE binary_iarchive : - public binary_iarchive_impl< - boost::archive::binary_iarchive, - std::istream::char_type, - std::istream::traits_type - >{ -public: - binary_iarchive(std::istream & is, unsigned int flags = 0) : - binary_iarchive_impl< - binary_iarchive, std::istream::char_type, std::istream::traits_type - >(is, flags) - {} - binary_iarchive(std::streambuf & bsb, unsigned int flags = 0) : - binary_iarchive_impl< - binary_iarchive, std::istream::char_type, std::istream::traits_type - >(bsb, flags) - {} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::binary_iarchive) -BOOST_SERIALIZATION_USE_ARRAY_OPTIMIZATION(boost::archive::binary_iarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_BINARY_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive_impl.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive_impl.hpp deleted file mode 100644 index b4747c98ece..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive_impl.hpp +++ /dev/null @@ -1,105 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_IARCHIVE_IMPL_HPP -#define BOOST_ARCHIVE_BINARY_IARCHIVE_IMPL_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_iarchive_impl.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE binary_iarchive_impl : - public basic_binary_iprimitive, - public basic_binary_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_iarchive; - friend basic_binary_iarchive; - friend load_access; - #else - friend class detail::interface_iarchive; - friend class basic_binary_iarchive; - friend class load_access; - #endif -#endif - template - void load_override(T & t){ - this->basic_binary_iarchive::load_override(t); - } - void init(unsigned int flags){ - if(0 != (flags & no_header)){ - return; - } - #if ! defined(__MWERKS__) - this->basic_binary_iarchive::init(); - this->basic_binary_iprimitive::init(); - #else - basic_binary_iarchive::init(); - basic_binary_iprimitive::init(); - #endif - } - binary_iarchive_impl( - std::basic_streambuf & bsb, - unsigned int flags - ) : - basic_binary_iprimitive( - bsb, - 0 != (flags & no_codecvt) - ), - basic_binary_iarchive(flags) - { - init(flags); - } - binary_iarchive_impl( - std::basic_istream & is, - unsigned int flags - ) : - basic_binary_iprimitive( - * is.rdbuf(), - 0 != (flags & no_codecvt) - ), - basic_binary_iarchive(flags) - { - init(flags); - } -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_BINARY_IARCHIVE_IMPL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive.hpp deleted file mode 100644 index e8313fd7c95..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive.hpp +++ /dev/null @@ -1,64 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_OARCHIVE_HPP -#define BOOST_ARCHIVE_BINARY_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from binary_oarchive_impl instead. This will -// preserve correct static polymorphism. -class BOOST_SYMBOL_VISIBLE binary_oarchive : - public binary_oarchive_impl< - binary_oarchive, std::ostream::char_type, std::ostream::traits_type - > -{ -public: - binary_oarchive(std::ostream & os, unsigned int flags = 0) : - binary_oarchive_impl< - binary_oarchive, std::ostream::char_type, std::ostream::traits_type - >(os, flags) - {} - binary_oarchive(std::streambuf & bsb, unsigned int flags = 0) : - binary_oarchive_impl< - binary_oarchive, std::ostream::char_type, std::ostream::traits_type - >(bsb, flags) - {} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::binary_oarchive) -BOOST_SERIALIZATION_USE_ARRAY_OPTIMIZATION(boost::archive::binary_oarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_BINARY_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive_impl.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive_impl.hpp deleted file mode 100644 index 6b4d018a564..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive_impl.hpp +++ /dev/null @@ -1,106 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_OARCHIVE_IMPL_HPP -#define BOOST_ARCHIVE_BINARY_OARCHIVE_IMPL_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_oarchive_impl.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE binary_oarchive_impl : - public basic_binary_oprimitive, - public basic_binary_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_oarchive; - friend basic_binary_oarchive; - friend save_access; - #else - friend class detail::interface_oarchive; - friend class basic_binary_oarchive; - friend class save_access; - #endif -#endif - template - void save_override(T & t){ - this->basic_binary_oarchive::save_override(t); - } - void init(unsigned int flags) { - if(0 != (flags & no_header)){ - return; - } - #if ! defined(__MWERKS__) - this->basic_binary_oarchive::init(); - this->basic_binary_oprimitive::init(); - #else - basic_binary_oarchive::init(); - basic_binary_oprimitive::init(); - #endif - } - binary_oarchive_impl( - std::basic_streambuf & bsb, - unsigned int flags - ) : - basic_binary_oprimitive( - bsb, - 0 != (flags & no_codecvt) - ), - basic_binary_oarchive(flags) - { - init(flags); - } - binary_oarchive_impl( - std::basic_ostream & os, - unsigned int flags - ) : - basic_binary_oprimitive( - * os.rdbuf(), - 0 != (flags & no_codecvt) - ), - basic_binary_oarchive(flags) - { - init(flags); - } -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_BINARY_OARCHIVE_IMPL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_wiarchive.hpp deleted file mode 100644 index 775d8f82726..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/binary_wiarchive.hpp +++ /dev/null @@ -1,56 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_WIARCHIVE_HPP -#define BOOST_ARCHIVE_BINARY_WIARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_wiarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include // wistream -#include -#include - -namespace boost { -namespace archive { - -class binary_wiarchive : - public binary_iarchive_impl< - binary_wiarchive, std::wistream::char_type, std::wistream::traits_type - > -{ -public: - binary_wiarchive(std::wistream & is, unsigned int flags = 0) : - binary_iarchive_impl< - binary_wiarchive, std::wistream::char_type, std::wistream::traits_type - >(is, flags) - {} - binary_wiarchive(std::wstreambuf & bsb, unsigned int flags = 0) : - binary_iarchive_impl< - binary_wiarchive, std::wistream::char_type, std::wistream::traits_type - >(bsb, flags) - {} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::binary_wiarchive) - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_BINARY_WIARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_woarchive.hpp deleted file mode 100644 index a8817d6f8b4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/binary_woarchive.hpp +++ /dev/null @@ -1,59 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_WOARCHIVE_HPP -#define BOOST_ARCHIVE_BINARY_WOARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_woarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include -#include -#include - -namespace boost { -namespace archive { - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from binary_oarchive_impl instead. This will -// preserve correct static polymorphism. -class binary_woarchive : - public binary_oarchive_impl< - binary_woarchive, std::wostream::char_type, std::wostream::traits_type - > -{ -public: - binary_woarchive(std::wostream & os, unsigned int flags = 0) : - binary_oarchive_impl< - binary_woarchive, std::wostream::char_type, std::wostream::traits_type - >(os, flags) - {} - binary_woarchive(std::wstreambuf & bsb, unsigned int flags = 0) : - binary_oarchive_impl< - binary_woarchive, std::wostream::char_type, std::wostream::traits_type - >(bsb, flags) - {} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::binary_woarchive) - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_BINARY_WOARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/codecvt_null.hpp b/contrib/libboost/boost_1_65_0/boost/archive/codecvt_null.hpp deleted file mode 100644 index 7bce2b9b329..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/codecvt_null.hpp +++ /dev/null @@ -1,109 +0,0 @@ -#ifndef BOOST_ARCHIVE_CODECVT_NULL_HPP -#define BOOST_ARCHIVE_CODECVT_NULL_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// codecvt_null.hpp: - -// (C) Copyright 2004 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL, size_t -#ifndef BOOST_NO_CWCHAR -#include // for mbstate_t -#endif -#include -#include -#include -#include // must be the last header - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std { -// For STLport on WinCE, BOOST_NO_STDC_NAMESPACE can get defined if STLport is putting symbols in its own namespace. -// In the case of codecvt, however, this does not mean that codecvt is in the global namespace (it will be in STLport's namespace) -# if !defined(__SGI_STL_PORT) && !defined(_STLPORT_VERSION) - using ::codecvt; -# endif - using ::mbstate_t; - using ::size_t; -} // namespace -#endif - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -template -class codecvt_null; - -template<> -class codecvt_null : public std::codecvt -{ - virtual bool do_always_noconv() const throw() { - return true; - } -public: - explicit codecvt_null(std::size_t no_locale_manage = 0) : - std::codecvt(no_locale_manage) - {} - virtual ~codecvt_null(){}; -}; - -template<> -class BOOST_SYMBOL_VISIBLE codecvt_null : public std::codecvt -{ - virtual BOOST_WARCHIVE_DECL BOOST_DLLEXPORT std::codecvt_base::result - do_out( - std::mbstate_t & state, - const wchar_t * first1, - const wchar_t * last1, - const wchar_t * & next1, - char * first2, - char * last2, - char * & next2 - ) const BOOST_USED; - virtual BOOST_WARCHIVE_DECL BOOST_DLLEXPORT std::codecvt_base::result - do_in( - std::mbstate_t & state, - const char * first1, - const char * last1, - const char * & next1, - wchar_t * first2, - wchar_t * last2, - wchar_t * & next2 - ) const BOOST_USED; - virtual int do_encoding( ) const throw( ){ - return sizeof(wchar_t) / sizeof(char); - } - virtual int do_max_length( ) const throw( ){ - return do_encoding(); - } -public: - BOOST_DLLEXPORT explicit codecvt_null(std::size_t no_locale_manage = 0) : - std::codecvt(no_locale_manage) - {} - virtual ~codecvt_null(){}; -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif -#include // pop pragmas - -#endif //BOOST_ARCHIVE_CODECVT_NULL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_prefix.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_prefix.hpp deleted file mode 100644 index debf79e9f0b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_prefix.hpp +++ /dev/null @@ -1,16 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// abi_prefix.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // must be the last header -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4251 4231 4660 4275) -#endif - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_suffix.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_suffix.hpp deleted file mode 100644 index 4e054d66214..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_suffix.hpp +++ /dev/null @@ -1,15 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// abi_suffix.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif -#include // pops abi_suffix.hpp pragmas - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/archive_serializer_map.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/archive_serializer_map.hpp deleted file mode 100644 index 5432bfc73e7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/archive_serializer_map.hpp +++ /dev/null @@ -1,54 +0,0 @@ -#ifndef BOOST_ARCHIVE_SERIALIZER_MAP_HPP -#define BOOST_ARCHIVE_SERIALIZER_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// archive_serializer_map.hpp: extenstion of type_info required for -// serialization. - -// (C) Copyright 2009 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// note: this is nothing more than the thinest of wrappers around -// basic_serializer_map so we can have a one map / archive type. - -#include -#include -#include // must be the last header - -namespace boost { - -namespace serialization { - class extended_type_info; -} // namespace serialization - -namespace archive { -namespace detail { - -class basic_serializer; - -template -class BOOST_SYMBOL_VISIBLE archive_serializer_map { -public: - static BOOST_ARCHIVE_OR_WARCHIVE_DECL bool insert(const basic_serializer * bs); - static BOOST_ARCHIVE_OR_WARCHIVE_DECL void erase(const basic_serializer * bs); - static BOOST_ARCHIVE_OR_WARCHIVE_DECL const basic_serializer * find( - const boost::serialization::extended_type_info & type_ - ); -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#include // must be the last header - -#endif //BOOST_ARCHIVE_SERIALIZER_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_archive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_archive.hpp deleted file mode 100644 index 79b0e490d65..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_archive.hpp +++ /dev/null @@ -1,48 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_AUTO_LINK_ARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_AUTO_LINK_ARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// auto_link_archive.hpp -// -// (c) Copyright Robert Ramey 2004 -// Use, modification, and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See library home page at http://www.boost.org/libs/serialization - -//----------------------------------------------------------------------------// - -// This header implements separate compilation features as described in -// http://www.boost.org/more/separate_compilation.html - -// enable automatic library variant selection ------------------------------// - -#include - -#if !defined(BOOST_ALL_NO_LIB) && !defined(BOOST_SERIALIZATION_NO_LIB) \ -&& !defined(BOOST_ARCHIVE_SOURCE) && !defined(BOOST_WARCHIVE_SOURCE) \ -&& !defined(BOOST_SERIALIZATION_SOURCE) - - // Set the name of our library, this will get undef'ed by auto_link.hpp - // once it's done with it: - // - #define BOOST_LIB_NAME boost_serialization - // - // If we're importing code from a dll, then tell auto_link.hpp about it: - // - #if defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) - # define BOOST_DYN_LINK - #endif - // - // And include the header that does the work: - // - #include -#endif // auto-linking disabled - -#endif // BOOST_ARCHIVE_DETAIL_AUTO_LINK_ARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_warchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_warchive.hpp deleted file mode 100644 index 683d191c20d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_warchive.hpp +++ /dev/null @@ -1,47 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_AUTO_LINK_WARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_AUTO_LINK_WARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// auto_link_warchive.hpp -// -// (c) Copyright Robert Ramey 2004 -// Use, modification, and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See library home page at http://www.boost.org/libs/serialization - -//----------------------------------------------------------------------------// - -// This header implements separate compilation features as described in -// http://www.boost.org/more/separate_compilation.html - -// enable automatic library variant selection ------------------------------// - -#include - -#if !defined(BOOST_WARCHIVE_SOURCE) \ -&& !defined(BOOST_ALL_NO_LIB) && !defined(BOOST_SERIALIZATION_NO_LIB) - -// Set the name of our library, this will get undef'ed by auto_link.hpp -// once it's done with it: -// -#define BOOST_LIB_NAME boost_wserialization -// -// If we're importing code from a dll, then tell auto_link.hpp about it: -// -#if defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) -# define BOOST_DYN_LINK -#endif -// -// And include the header that does the work: -// -#include -#endif // auto-linking disabled - -#endif // ARCHIVE_DETAIL_AUTO_LINK_ARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iarchive.hpp deleted file mode 100644 index 1f5a8bf63bf..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iarchive.hpp +++ /dev/null @@ -1,105 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_BASIC_IARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_BASIC_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_iarchive.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// can't use this - much as I'd like to as borland doesn't support it - -#include -#include -#include - -#include -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization - -namespace archive { -namespace detail { - -class basic_iarchive_impl; -class basic_iserializer; -class basic_pointer_iserializer; - -////////////////////////////////////////////////////////////////////// -// class basic_iarchive - read serialized objects from a input stream -class BOOST_SYMBOL_VISIBLE basic_iarchive : - private boost::noncopyable, - public boost::archive::detail::helper_collection -{ - friend class basic_iarchive_impl; - // hide implementation of this class to minimize header conclusion - boost::scoped_ptr pimpl; - - virtual void vload(version_type &t) = 0; - virtual void vload(object_id_type &t) = 0; - virtual void vload(class_id_type &t) = 0; - virtual void vload(class_id_optional_type &t) = 0; - virtual void vload(class_name_type &t) = 0; - virtual void vload(tracking_type &t) = 0; -protected: - BOOST_ARCHIVE_DECL basic_iarchive(unsigned int flags); - boost::archive::detail::helper_collection & - get_helper_collection(){ - return *this; - } -public: - // some msvc versions require that the following function be public - // otherwise it should really protected. - virtual BOOST_ARCHIVE_DECL ~basic_iarchive(); - // note: NOT part of the public API. - BOOST_ARCHIVE_DECL void next_object_pointer(void *t); - BOOST_ARCHIVE_DECL void register_basic_serializer( - const basic_iserializer & bis - ); - BOOST_ARCHIVE_DECL void load_object( - void *t, - const basic_iserializer & bis - ); - BOOST_ARCHIVE_DECL const basic_pointer_iserializer * - load_pointer( - void * & t, - const basic_pointer_iserializer * bpis_ptr, - const basic_pointer_iserializer * (*finder)( - const boost::serialization::extended_type_info & eti - ) - ); - // real public API starts here - BOOST_ARCHIVE_DECL void - set_library_version(library_version_type archive_library_version); - BOOST_ARCHIVE_DECL library_version_type - get_library_version() const; - BOOST_ARCHIVE_DECL unsigned int - get_flags() const; - BOOST_ARCHIVE_DECL void - reset_object_address(const void * new_address, const void * old_address); - BOOST_ARCHIVE_DECL void - delete_created_pointers(); -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif //BOOST_ARCHIVE_DETAIL_BASIC_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iserializer.hpp deleted file mode 100644 index 0d66674c349..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iserializer.hpp +++ /dev/null @@ -1,91 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_BASIC_ISERIALIZER_HPP -#define BOOST_ARCHIVE_DETAIL_BASIC_ISERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_iserializer.hpp: extenstion of type_info required for serialization. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include - -#include -#include -#include -#include -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization - -// forward declarations -namespace archive { -namespace detail { - -class basic_iarchive; -class basic_pointer_iserializer; - -class BOOST_SYMBOL_VISIBLE basic_iserializer : - public basic_serializer -{ -private: - basic_pointer_iserializer *m_bpis; -protected: - explicit BOOST_ARCHIVE_DECL basic_iserializer( - const boost::serialization::extended_type_info & type - ); - virtual BOOST_ARCHIVE_DECL ~basic_iserializer(); -public: - bool serialized_as_pointer() const { - return m_bpis != NULL; - } - void set_bpis(basic_pointer_iserializer *bpis){ - m_bpis = bpis; - } - const basic_pointer_iserializer * get_bpis_ptr() const { - return m_bpis; - } - virtual void load_object_data( - basic_iarchive & ar, - void *x, - const unsigned int file_version - ) const = 0; - // returns true if class_info should be saved - virtual bool class_info() const = 0 ; - // returns true if objects should be tracked - virtual bool tracking(const unsigned int) const = 0 ; - // returns class version - virtual version_type version() const = 0 ; - // returns true if this class is polymorphic - virtual bool is_polymorphic() const = 0; - virtual void destroy(/*const*/ void *address) const = 0 ; -}; - -} // namespae detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_DETAIL_BASIC_ISERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oarchive.hpp deleted file mode 100644 index c379108d584..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oarchive.hpp +++ /dev/null @@ -1,94 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_OARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_oarchive.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include -#include -#include - -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization - -namespace archive { -namespace detail { - -class basic_oarchive_impl; -class basic_oserializer; -class basic_pointer_oserializer; - -////////////////////////////////////////////////////////////////////// -// class basic_oarchive - write serialized objects to an output stream -class BOOST_SYMBOL_VISIBLE basic_oarchive : - private boost::noncopyable, - public boost::archive::detail::helper_collection -{ - friend class basic_oarchive_impl; - // hide implementation of this class to minimize header conclusion - boost::scoped_ptr pimpl; - - // overload these to bracket object attributes. Used to implement - // xml archives - virtual void vsave(const version_type t) = 0; - virtual void vsave(const object_id_type t) = 0; - virtual void vsave(const object_reference_type t) = 0; - virtual void vsave(const class_id_type t) = 0; - virtual void vsave(const class_id_optional_type t) = 0; - virtual void vsave(const class_id_reference_type t) = 0; - virtual void vsave(const class_name_type & t) = 0; - virtual void vsave(const tracking_type t) = 0; -protected: - BOOST_ARCHIVE_DECL basic_oarchive(unsigned int flags = 0); - BOOST_ARCHIVE_DECL boost::archive::detail::helper_collection & - get_helper_collection(); - virtual BOOST_ARCHIVE_DECL ~basic_oarchive(); -public: - // note: NOT part of the public interface - BOOST_ARCHIVE_DECL void register_basic_serializer( - const basic_oserializer & bos - ); - BOOST_ARCHIVE_DECL void save_object( - const void *x, - const basic_oserializer & bos - ); - BOOST_ARCHIVE_DECL void save_pointer( - const void * t, - const basic_pointer_oserializer * bpos_ptr - ); - void save_null_pointer(){ - vsave(NULL_POINTER_TAG); - } - // real public interface starts here - BOOST_ARCHIVE_DECL void end_preamble(); // default implementation does nothing - BOOST_ARCHIVE_DECL library_version_type get_library_version() const; - BOOST_ARCHIVE_DECL unsigned int get_flags() const; -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif //BOOST_ARCHIVE_BASIC_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oserializer.hpp deleted file mode 100644 index 94247e90056..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oserializer.hpp +++ /dev/null @@ -1,89 +0,0 @@ -#ifndef BOOST_SERIALIZATION_BASIC_OSERIALIZER_HPP -#define BOOST_SERIALIZATION_BASIC_OSERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_oserializer.hpp: extenstion of type_info required for serialization. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include -#include - -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization - -// forward declarations -namespace archive { -namespace detail { - -class basic_oarchive; -class basic_pointer_oserializer; - -class BOOST_SYMBOL_VISIBLE basic_oserializer : - public basic_serializer -{ -private: - basic_pointer_oserializer *m_bpos; -protected: - explicit BOOST_ARCHIVE_DECL basic_oserializer( - const boost::serialization::extended_type_info & type_ - ); - virtual BOOST_ARCHIVE_DECL ~basic_oserializer(); -public: - bool serialized_as_pointer() const { - return m_bpos != NULL; - } - void set_bpos(basic_pointer_oserializer *bpos){ - m_bpos = bpos; - } - const basic_pointer_oserializer * get_bpos() const { - return m_bpos; - } - virtual void save_object_data( - basic_oarchive & ar, const void * x - ) const = 0; - // returns true if class_info should be saved - virtual bool class_info() const = 0; - // returns true if objects should be tracked - virtual bool tracking(const unsigned int flags) const = 0; - // returns class version - virtual version_type version() const = 0; - // returns true if this class is polymorphic - virtual bool is_polymorphic() const = 0; -}; - -} // namespace detail -} // namespace serialization -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_SERIALIZATION_BASIC_OSERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_iserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_iserializer.hpp deleted file mode 100644 index 1fc4b14d6e9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_iserializer.hpp +++ /dev/null @@ -1,70 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_POINTER_ISERIALIZER_HPP -#define BOOST_ARCHIVE_BASIC_POINTER_ISERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_pointer_oserializer.hpp: extenstion of type_info required for -// serialization. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization - -// forward declarations -namespace archive { -namespace detail { - -class basic_iarchive; -class basic_iserializer; - -class BOOST_SYMBOL_VISIBLE basic_pointer_iserializer - : public basic_serializer { -protected: - explicit BOOST_ARCHIVE_DECL basic_pointer_iserializer( - const boost::serialization::extended_type_info & type_ - ); - virtual BOOST_ARCHIVE_DECL ~basic_pointer_iserializer(); -public: - virtual void * heap_allocation() const = 0; - virtual const basic_iserializer & get_basic_serializer() const = 0; - virtual void load_object_ptr( - basic_iarchive & ar, - void * x, - const unsigned int file_version - ) const = 0; -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_POINTER_ISERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_oserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_oserializer.hpp deleted file mode 100644 index 1a5d9549eab..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_oserializer.hpp +++ /dev/null @@ -1,68 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_POINTER_OSERIALIZER_HPP -#define BOOST_ARCHIVE_BASIC_POINTER_OSERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_pointer_oserializer.hpp: extenstion of type_info required for -// serialization. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization - -namespace archive { -namespace detail { - -class basic_oarchive; -class basic_oserializer; - -class BOOST_SYMBOL_VISIBLE basic_pointer_oserializer : - public basic_serializer -{ -protected: - explicit BOOST_ARCHIVE_DECL basic_pointer_oserializer( - const boost::serialization::extended_type_info & type_ - ); -public: - virtual BOOST_ARCHIVE_DECL ~basic_pointer_oserializer(); - virtual const basic_oserializer & get_basic_serializer() const = 0; - virtual void save_object_ptr( - basic_oarchive & ar, - const void * x - ) const = 0; -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_POINTER_OSERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer.hpp deleted file mode 100644 index f9c4203f862..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer.hpp +++ /dev/null @@ -1,77 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_SERIALIZER_HPP -#define BOOST_ARCHIVE_BASIC_SERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_serializer.hpp: extenstion of type_info required for serialization. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { -namespace detail { - -class basic_serializer : - private boost::noncopyable -{ - const boost::serialization::extended_type_info * m_eti; -protected: - explicit basic_serializer( - const boost::serialization::extended_type_info & eti - ) : - m_eti(& eti) - {} -public: - inline bool - operator<(const basic_serializer & rhs) const { - // can't compare address since there can be multiple eti records - // for the same type in different execution modules (that is, DLLS) - // leave this here as a reminder not to do this! - // return & lhs.get_eti() < & rhs.get_eti(); - return get_eti() < rhs.get_eti(); - } - const char * get_debug_info() const { - return m_eti->get_debug_info(); - } - const boost::serialization::extended_type_info & get_eti() const { - return * m_eti; - } -}; - -class basic_serializer_arg : public basic_serializer { -public: - basic_serializer_arg(const serialization::extended_type_info & eti) : - basic_serializer(eti) - {} -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_BASIC_SERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer_map.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer_map.hpp deleted file mode 100644 index 79341803367..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer_map.hpp +++ /dev/null @@ -1,69 +0,0 @@ -#ifndef BOOST_SERIALIZER_MAP_HPP -#define BOOST_SERIALIZER_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_serializer_map.hpp: extenstion of type_info required for serialization. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include -#include - -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} - -namespace archive { -namespace detail { - -class basic_serializer; - -class BOOST_SYMBOL_VISIBLE -basic_serializer_map : public - boost::noncopyable -{ - struct type_info_pointer_compare - { - bool operator()( - const basic_serializer * lhs, const basic_serializer * rhs - ) const ; - }; - typedef std::set< - const basic_serializer *, - type_info_pointer_compare - > map_type; - map_type m_map; -public: - BOOST_ARCHIVE_DECL bool insert(const basic_serializer * bs); - BOOST_ARCHIVE_DECL void erase(const basic_serializer * bs); - BOOST_ARCHIVE_DECL const basic_serializer * find( - const boost::serialization::extended_type_info & type_ - ) const; -private: - // cw 8.3 requires this - basic_serializer_map& operator=(basic_serializer_map const&); -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#include // must be the last header - -#endif // BOOST_SERIALIZER_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/check.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/check.hpp deleted file mode 100644 index 10034e7d101..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/check.hpp +++ /dev/null @@ -1,169 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_CHECK_HPP -#define BOOST_ARCHIVE_DETAIL_CHECK_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#pragma inline_depth(511) -#pragma inline_recursion(on) -#endif - -#if defined(__MWERKS__) -#pragma inline_depth(511) -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// check.hpp: interface for serialization system. - -// (C) Copyright 2009 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -namespace boost { -namespace archive { -namespace detail { - -// checks for objects - -template -inline void check_object_level(){ - typedef - typename mpl::greater_equal< - serialization::implementation_level< T >, - mpl::int_ - >::type typex; - - // trap attempts to serialize objects marked - // not_serializable - BOOST_STATIC_ASSERT(typex::value); -} - -template -inline void check_object_versioning(){ - typedef - typename mpl::or_< - typename mpl::greater< - serialization::implementation_level< T >, - mpl::int_ - >, - typename mpl::equal_to< - serialization::version< T >, - mpl::int_<0> - > - > typex; - // trap attempts to serialize with objects that don't - // save class information in the archive with versioning. - BOOST_STATIC_ASSERT(typex::value); -} - -template -inline void check_object_tracking(){ - // presume it has already been determined that - // T is not a const - BOOST_STATIC_ASSERT(! boost::is_const< T >::value); - typedef typename mpl::equal_to< - serialization::tracking_level< T >, - mpl::int_ - >::type typex; - // saving an non-const object of a type not marked "track_never) - - // may be an indicator of an error usage of the - // serialization library and should be double checked. - // See documentation on object tracking. Also, see the - // "rationale" section of the documenation - // for motivation for this checking. - - BOOST_STATIC_WARNING(typex::value); -} - -// checks for pointers - -template -inline void check_pointer_level(){ - // we should only invoke this once we KNOW that T - // has been used as a pointer!! - typedef - typename mpl::or_< - typename mpl::greater< - serialization::implementation_level< T >, - mpl::int_ - >, - typename mpl::not_< - typename mpl::equal_to< - serialization::tracking_level< T >, - mpl::int_ - > - > - > typex; - // Address the following when serializing to a pointer: - - // a) This type doesn't save class information in the - // archive. That is, the serialization trait implementation - // level <= object_serializable. - // b) Tracking for this type is set to "track selectively" - - // in this case, indication that an object is tracked is - // not stored in the archive itself - see level == object_serializable - // but rather the existence of the operation ar >> T * is used to - // infer that an object of this type should be tracked. So, if - // you save via a pointer but don't load via a pointer the operation - // will fail on load without given any valid reason for the failure. - - // So if your program traps here, consider changing the - // tracking or implementation level traits - or not - // serializing via a pointer. - BOOST_STATIC_WARNING(typex::value); -} - -template -void inline check_pointer_tracking(){ - typedef typename mpl::greater< - serialization::tracking_level< T >, - mpl::int_ - >::type typex; - // serializing an object of a type marked "track_never" through a pointer - // could result in creating more objects than were saved! - BOOST_STATIC_WARNING(typex::value); -} - -template -inline void check_const_loading(){ - typedef - typename mpl::or_< - typename boost::serialization::is_wrapper< T >, - typename mpl::not_< - typename boost::is_const< T > - > - >::type typex; - // cannot load data into a "const" object unless it's a - // wrapper around some other non-const object. - BOOST_STATIC_ASSERT(typex::value); -} - -} // detail -} // archive -} // boost - -#endif // BOOST_ARCHIVE_DETAIL_CHECK_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/common_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/common_iarchive.hpp deleted file mode 100644 index 82304f1e5ac..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/common_iarchive.hpp +++ /dev/null @@ -1,88 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_COMMON_IARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_COMMON_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// common_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { -namespace detail { - -class extended_type_info; - -// note: referred to as Curiously Recurring Template Patter (CRTP) -template -class BOOST_SYMBOL_VISIBLE common_iarchive : - public basic_iarchive, - public interface_iarchive -{ - friend class interface_iarchive; -private: - virtual void vload(version_type & t){ - * this->This() >> t; - } - virtual void vload(object_id_type & t){ - * this->This() >> t; - } - virtual void vload(class_id_type & t){ - * this->This() >> t; - } - virtual void vload(class_id_optional_type & t){ - * this->This() >> t; - } - virtual void vload(tracking_type & t){ - * this->This() >> t; - } - virtual void vload(class_name_type &s){ - * this->This() >> s; - } -protected: - // default processing - invoke serialization library - template - void load_override(T & t){ - archive::load(* this->This(), t); - } - // default implementations of functions which emit start/end tags for - // archive types that require them. - void load_start(const char * /*name*/){} - void load_end(const char * /*name*/){} - // default archive initialization - common_iarchive(unsigned int flags = 0) : - basic_iarchive(flags), - interface_iarchive() - {} -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_DETAIL_COMMON_IARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/common_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/common_oarchive.hpp deleted file mode 100644 index ee42bbe5976..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/common_oarchive.hpp +++ /dev/null @@ -1,88 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_COMMON_OARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_COMMON_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// common_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { -namespace detail { - -// note: referred to as Curiously Recurring Template Patter (CRTP) -template - -class BOOST_SYMBOL_VISIBLE common_oarchive : - public basic_oarchive, - public interface_oarchive -{ - friend class interface_oarchive; -private: - virtual void vsave(const version_type t){ - * this->This() << t; - } - virtual void vsave(const object_id_type t){ - * this->This() << t; - } - virtual void vsave(const object_reference_type t){ - * this->This() << t; - } - virtual void vsave(const class_id_type t){ - * this->This() << t; - } - virtual void vsave(const class_id_reference_type t){ - * this->This() << t; - } - virtual void vsave(const class_id_optional_type t){ - * this->This() << t; - } - virtual void vsave(const class_name_type & t){ - * this->This() << t; - } - virtual void vsave(const tracking_type t){ - * this->This() << t; - } -protected: - // default processing - invoke serialization library - template - void save_override(T & t){ - archive::save(* this->This(), t); - } - void save_start(const char * /*name*/){} - void save_end(const char * /*name*/){} - common_oarchive(unsigned int flags = 0) : - basic_oarchive(flags), - interface_oarchive() - {} -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_DETAIL_COMMON_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/decl.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/decl.hpp deleted file mode 100644 index 4f731cded37..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/decl.hpp +++ /dev/null @@ -1,57 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_DECL_HPP -#define BOOST_ARCHIVE_DETAIL_DECL_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2///////// 3/////////4/////////5/////////6/////////7/////////8 -// decl.hpp -// -// (c) Copyright Robert Ramey 2004 -// Use, modification, and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See library home page at http://www.boost.org/libs/serialization - -//----------------------------------------------------------------------------// - -// This header implements separate compilation features as described in -// http://www.boost.org/more/separate_compilation.html - -#include - -#if (defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK)) - #if defined(BOOST_ARCHIVE_SOURCE) - #define BOOST_ARCHIVE_DECL BOOST_SYMBOL_EXPORT - #else - #define BOOST_ARCHIVE_DECL BOOST_SYMBOL_IMPORT - #endif - - #if defined(BOOST_WARCHIVE_SOURCE) - #define BOOST_WARCHIVE_DECL BOOST_SYMBOL_EXPORT - #else - #define BOOST_WARCHIVE_DECL BOOST_SYMBOL_IMPORT - #endif - - #if defined(BOOST_WARCHIVE_SOURCE) || defined(BOOST_ARCHIVE_SOURCE) - #define BOOST_ARCHIVE_OR_WARCHIVE_DECL BOOST_SYMBOL_EXPORT - #else - #define BOOST_ARCHIVE_OR_WARCHIVE_DECL BOOST_SYMBOL_IMPORT - #endif - -#endif - -#if ! defined(BOOST_ARCHIVE_DECL) - #define BOOST_ARCHIVE_DECL -#endif -#if ! defined(BOOST_WARCHIVE_DECL) - #define BOOST_WARCHIVE_DECL -#endif -#if ! defined(BOOST_ARCHIVE_OR_WARCHIVE_DECL) - #define BOOST_ARCHIVE_OR_WARCHIVE_DECL -#endif - -#endif // BOOST_ARCHIVE_DETAIL_DECL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/helper_collection.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/helper_collection.hpp deleted file mode 100644 index edb4125e308..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/helper_collection.hpp +++ /dev/null @@ -1,99 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_HELPER_COLLECTION_HPP -#define BOOST_ARCHIVE_DETAIL_HELPER_COLLECTION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// helper_collection.hpp: archive support for run-time helpers - -// (C) Copyright 2002-2008 Robert Ramey and Joaquin M Lopez Munoz -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include -#include -#include -#include - -#include - -#include -#include - -namespace boost { - -namespace archive { -namespace detail { - -class helper_collection -{ - helper_collection(const helper_collection&); // non-copyable - helper_collection& operator = (const helper_collection&); // non-copyable - - // note: we dont' actually "share" the function object pointer - // we only use shared_ptr to make sure that it get's deleted - - typedef std::pair< - const void *, - boost::shared_ptr - > helper_value_type; - template - boost::shared_ptr make_helper_ptr(){ - // use boost::shared_ptr rather than std::shared_ptr to maintain - // c++03 compatibility - return boost::make_shared(); - } - - typedef std::vector collection; - collection m_collection; - - struct predicate { - BOOST_DELETED_FUNCTION(predicate & operator=(const predicate & rhs)) - public: - const void * const m_ti; - bool operator()(helper_value_type const &rhs) const { - return m_ti == rhs.first; - } - predicate(const void * ti) : - m_ti(ti) - {} - }; -protected: - helper_collection(){} - ~helper_collection(){} -public: - template - Helper& find_helper(void * const id = 0) { - collection::const_iterator it = - std::find_if( - m_collection.begin(), - m_collection.end(), - predicate(id) - ); - - void * rval = 0; - if(it == m_collection.end()){ - m_collection.push_back( - std::make_pair(id, make_helper_ptr()) - ); - rval = m_collection.back().second.get(); - } - else{ - rval = it->second.get(); - } - return *static_cast(rval); - } -}; - -} // namespace detail -} // namespace serialization -} // namespace boost - -#endif // BOOST_ARCHIVE_DETAIL_HELPER_COLLECTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_iarchive.hpp deleted file mode 100644 index 4a99e28b59f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_iarchive.hpp +++ /dev/null @@ -1,85 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_INTERFACE_IARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_INTERFACE_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// interface_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include // NULL -#include -#include -#include -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace archive { -namespace detail { - -class basic_pointer_iserializer; - -template -class interface_iarchive -{ -protected: - interface_iarchive(){}; -public: - ///////////////////////////////////////////////////////// - // archive public interface - typedef mpl::bool_ is_loading; - typedef mpl::bool_ is_saving; - - // return a pointer to the most derived class - Archive * This(){ - return static_cast(this); - } - - template - const basic_pointer_iserializer * - register_type(T * = NULL){ - const basic_pointer_iserializer & bpis = - boost::serialization::singleton< - pointer_iserializer - >::get_const_instance(); - this->This()->register_basic_serializer(bpis.get_basic_serializer()); - return & bpis; - } - template - Helper & - get_helper(void * const id = 0){ - helper_collection & hc = this->This()->get_helper_collection(); - return hc.template find_helper(id); - } - - template - Archive & operator>>(T & t){ - this->This()->load_override(t); - return * this->This(); - } - - // the & operator - template - Archive & operator&(T & t){ - return *(this->This()) >> t; - } -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_DETAIL_INTERFACE_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_oarchive.hpp deleted file mode 100644 index 359463ed9d8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_oarchive.hpp +++ /dev/null @@ -1,87 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_INTERFACE_OARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_INTERFACE_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// interface_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include // NULL -#include -#include - -#include -#include -#include // must be the last header - -#include - -namespace boost { -namespace archive { -namespace detail { - -class basic_pointer_oserializer; - -template -class interface_oarchive -{ -protected: - interface_oarchive(){}; -public: - ///////////////////////////////////////////////////////// - // archive public interface - typedef mpl::bool_ is_loading; - typedef mpl::bool_ is_saving; - - // return a pointer to the most derived class - Archive * This(){ - return static_cast(this); - } - - template - const basic_pointer_oserializer * - register_type(const T * = NULL){ - const basic_pointer_oserializer & bpos = - boost::serialization::singleton< - pointer_oserializer - >::get_const_instance(); - this->This()->register_basic_serializer(bpos.get_basic_serializer()); - return & bpos; - } - - template - Helper & - get_helper(void * const id = 0){ - helper_collection & hc = this->This()->get_helper_collection(); - return hc.template find_helper(id); - } - - template - Archive & operator<<(const T & t){ - this->This()->save_override(t); - return * this->This(); - } - - // the & operator - template - Archive & operator&(const T & t){ - return * this ->This() << t; - } -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_DETAIL_INTERFACE_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/iserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/iserializer.hpp deleted file mode 100644 index 193e98a82e4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/iserializer.hpp +++ /dev/null @@ -1,631 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_ISERIALIZER_HPP -#define BOOST_ARCHIVE_DETAIL_ISERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#pragma inline_depth(511) -#pragma inline_recursion(on) -#endif - -#if defined(__MWERKS__) -#pragma inline_depth(511) -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// iserializer.hpp: interface for serialization system. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // for placement new -#include // size_t, NULL - -#include -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include - -#include -#include -#include -#include -#include - -#ifndef BOOST_SERIALIZATION_DEFAULT_TYPE_INFO - #include -#endif -#include -#include -#include - -#include -#include -#include -#include -#include -#include - -#include - -#ifndef BOOST_MSVC - #define DONT_USE_HAS_NEW_OPERATOR ( \ - BOOST_WORKAROUND(__IBMCPP__, < 1210) \ - || defined(__SUNPRO_CC) && (__SUNPRO_CC < 0x590) \ - ) -#else - #define DONT_USE_HAS_NEW_OPERATOR 0 -#endif - -#if ! DONT_USE_HAS_NEW_OPERATOR -#include -#endif - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -// the following is need only for dynamic cast of polymorphic pointers -#include -#include -#include -#include -#include -#include - -namespace boost { - -namespace serialization { - class extended_type_info; -} // namespace serialization - -namespace archive { - -// an accessor to permit friend access to archives. Needed because -// some compilers don't handle friend templates completely -class load_access { -public: - template - static void load_primitive(Archive &ar, T &t){ - ar.load(t); - } -}; - -namespace detail { - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -template -class iserializer : public basic_iserializer -{ -private: - virtual void destroy(/*const*/ void *address) const { - boost::serialization::access::destroy(static_cast(address)); - } -protected: - // protected constructor since it's always created by singleton - explicit iserializer() : - basic_iserializer( - boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< T >::type - >::get_const_instance() - ) - {} -public: - virtual BOOST_DLLEXPORT void load_object_data( - basic_iarchive & ar, - void *x, - const unsigned int file_version - ) const BOOST_USED; - virtual bool class_info() const { - return boost::serialization::implementation_level< T >::value - >= boost::serialization::object_class_info; - } - virtual bool tracking(const unsigned int /* flags */) const { - return boost::serialization::tracking_level< T >::value - == boost::serialization::track_always - || ( boost::serialization::tracking_level< T >::value - == boost::serialization::track_selectively - && serialized_as_pointer()); - } - virtual version_type version() const { - return version_type(::boost::serialization::version< T >::value); - } - virtual bool is_polymorphic() const { - return boost::is_polymorphic< T >::value; - } - virtual ~iserializer(){}; -}; - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -template -BOOST_DLLEXPORT void iserializer::load_object_data( - basic_iarchive & ar, - void *x, - const unsigned int file_version -) const { - // note: we now comment this out. Before we permited archive - // version # to be very large. Now we don't. To permit - // readers of these old archives, we have to suppress this - // code. Perhaps in the future we might re-enable it but - // permit its suppression with a runtime switch. - #if 0 - // trap case where the program cannot handle the current version - if(file_version > static_cast(version())) - boost::serialization::throw_exception( - archive::archive_exception( - boost::archive::archive_exception::unsupported_class_version, - get_debug_info() - ) - ); - #endif - // make sure call is routed through the higest interface that might - // be specialized by the user. - boost::serialization::serialize_adl( - boost::serialization::smart_cast_reference(ar), - * static_cast(x), - file_version - ); -} - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -// the purpose of this code is to allocate memory for an object -// without requiring the constructor to be called. Presumably -// the allocated object will be subsequently initialized with -// "placement new". -// note: we have the boost type trait has_new_operator but we -// have no corresponding has_delete_operator. So we presume -// that the former being true would imply that the a delete -// operator is also defined for the class T. - -template -struct heap_allocation { - // boost::has_new_operator< T > doesn't work on these compilers - #if DONT_USE_HAS_NEW_OPERATOR - // This doesn't handle operator new overload for class T - static T * invoke_new(){ - return static_cast(operator new(sizeof(T))); - } - static void invoke_delete(T *t){ - (operator delete(t)); - } - #else - // note: we presume that a true value for has_new_operator - // implies the existence of a class specific delete operator as well - // as a class specific new operator. - struct has_new_operator { - static T * invoke_new() { - return static_cast((T::operator new)(sizeof(T))); - } - static void invoke_delete(T * t) { - // if compilation fails here, the likely cause that the class - // T has a class specific new operator but no class specific - // delete operator which matches the following signature. - // note that this solution addresses the issue that two - // possible signatures. But it doesn't address the possibility - // that the class might have class specific new with NO - // class specific delete at all. Patches (compatible with - // C++03) welcome! - delete t; - } - }; - struct doesnt_have_new_operator { - static T* invoke_new() { - return static_cast(operator new(sizeof(T))); - } - static void invoke_delete(T * t) { - // Note: I'm reliance upon automatic conversion from T * to void * here - delete t; - } - }; - static T * invoke_new() { - typedef typename - mpl::eval_if< - boost::has_new_operator< T >, - mpl::identity, - mpl::identity - >::type typex; - return typex::invoke_new(); - } - static void invoke_delete(T *t) { - typedef typename - mpl::eval_if< - boost::has_new_operator< T >, - mpl::identity, - mpl::identity - >::type typex; - typex::invoke_delete(t); - } - #endif - explicit heap_allocation(){ - m_p = invoke_new(); - } - ~heap_allocation(){ - if (0 != m_p) - invoke_delete(m_p); - } - T* get() const { - return m_p; - } - - T* release() { - T* p = m_p; - m_p = 0; - return p; - } -private: - T* m_p; -}; - -template -class pointer_iserializer : - public basic_pointer_iserializer -{ -private: - virtual void * heap_allocation() const { - detail::heap_allocation h; - T * t = h.get(); - h.release(); - return t; - } - virtual const basic_iserializer & get_basic_serializer() const { - return boost::serialization::singleton< - iserializer - >::get_const_instance(); - } - BOOST_DLLEXPORT virtual void load_object_ptr( - basic_iarchive & ar, - void * x, - const unsigned int file_version - ) const BOOST_USED; -protected: - // this should alway be a singleton so make the constructor protected - pointer_iserializer(); - ~pointer_iserializer(); -}; - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -// note: BOOST_DLLEXPORT is so that code for polymorphic class -// serialized only through base class won't get optimized out -template -BOOST_DLLEXPORT void pointer_iserializer::load_object_ptr( - basic_iarchive & ar, - void * t, - const unsigned int file_version -) const -{ - Archive & ar_impl = - boost::serialization::smart_cast_reference(ar); - - // note that the above will throw std::bad_alloc if the allocation - // fails so we don't have to address this contingency here. - - // catch exception during load_construct_data so that we don't - // automatically delete the t which is most likely not fully - // constructed - BOOST_TRY { - // this addresses an obscure situation that occurs when - // load_constructor de-serializes something through a pointer. - ar.next_object_pointer(t); - boost::serialization::load_construct_data_adl( - ar_impl, - static_cast(t), - file_version - ); - } - BOOST_CATCH(...){ - // if we get here the load_construct failed. The heap_allocation - // will be automatically deleted so we don't have to do anything - // special here. - BOOST_RETHROW; - } - BOOST_CATCH_END - - ar_impl >> boost::serialization::make_nvp(NULL, * static_cast(t)); -} - -template -pointer_iserializer::pointer_iserializer() : - basic_pointer_iserializer( - boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< T >::type - >::get_const_instance() - ) -{ - boost::serialization::singleton< - iserializer - >::get_mutable_instance().set_bpis(this); - archive_serializer_map::insert(this); -} - -template -pointer_iserializer::~pointer_iserializer(){ - archive_serializer_map::erase(this); -} - -template -struct load_non_pointer_type { - // note this bounces the call right back to the archive - // with no runtime overhead - struct load_primitive { - template - static void invoke(Archive & ar, T & t){ - load_access::load_primitive(ar, t); - } - }; - // note this bounces the call right back to the archive - // with no runtime overhead - struct load_only { - template - static void invoke(Archive & ar, const T & t){ - // short cut to user's serializer - // make sure call is routed through the higest interface that might - // be specialized by the user. - boost::serialization::serialize_adl( - ar, - const_cast(t), - boost::serialization::version< T >::value - ); - } - }; - - // note this save class information including version - // and serialization level to the archive - struct load_standard { - template - static void invoke(Archive &ar, const T & t){ - void * x = & const_cast(t); - ar.load_object( - x, - boost::serialization::singleton< - iserializer - >::get_const_instance() - ); - } - }; - - struct load_conditional { - template - static void invoke(Archive &ar, T &t){ - //if(0 == (ar.get_flags() & no_tracking)) - load_standard::invoke(ar, t); - //else - // load_only::invoke(ar, t); - } - }; - - template - static void invoke(Archive & ar, T &t){ - typedef typename mpl::eval_if< - // if its primitive - mpl::equal_to< - boost::serialization::implementation_level< T >, - mpl::int_ - >, - mpl::identity, - // else - typename mpl::eval_if< - // class info / version - mpl::greater_equal< - boost::serialization::implementation_level< T >, - mpl::int_ - >, - // do standard load - mpl::identity, - // else - typename mpl::eval_if< - // no tracking - mpl::equal_to< - boost::serialization::tracking_level< T >, - mpl::int_ - >, - // do a fast load - mpl::identity, - // else - // do a fast load only tracking is turned off - mpl::identity - > > >::type typex; - check_object_versioning< T >(); - check_object_level< T >(); - typex::invoke(ar, t); - } -}; - -template -struct load_pointer_type { - struct abstract - { - template - static const basic_pointer_iserializer * register_type(Archive & /* ar */){ - // it has? to be polymorphic - BOOST_STATIC_ASSERT(boost::is_polymorphic< T >::value); - return static_cast(NULL); - } - }; - - struct non_abstract - { - template - static const basic_pointer_iserializer * register_type(Archive & ar){ - return ar.register_type(static_cast(NULL)); - } - }; - - template - static const basic_pointer_iserializer * register_type(Archive &ar, const T & /*t*/){ - // there should never be any need to load an abstract polymorphic - // class pointer. Inhibiting code generation for this - // permits abstract base classes to be used - note: exception - // virtual serialize functions used for plug-ins - typedef typename - mpl::eval_if< - boost::serialization::is_abstract, - boost::mpl::identity, - boost::mpl::identity - >::type typex; - return typex::template register_type< T >(ar); - } - - template - static T * pointer_tweak( - const boost::serialization::extended_type_info & eti, - void const * const t, - const T & - ) { - // tweak the pointer back to the base class - void * upcast = const_cast( - boost::serialization::void_upcast( - eti, - boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< T >::type - >::get_const_instance(), - t - ) - ); - if(NULL == upcast) - boost::serialization::throw_exception( - archive_exception(archive_exception::unregistered_class) - ); - return static_cast(upcast); - } - - template - static void check_load(T & /* t */){ - check_pointer_level< T >(); - check_pointer_tracking< T >(); - } - - static const basic_pointer_iserializer * - find(const boost::serialization::extended_type_info & type){ - return static_cast( - archive_serializer_map::find(type) - ); - } - - template - static void invoke(Archive & ar, Tptr & t){ - check_load(*t); - const basic_pointer_iserializer * bpis_ptr = register_type(ar, *t); - const basic_pointer_iserializer * newbpis_ptr = ar.load_pointer( - // note major hack here !!! - // I tried every way to convert Tptr &t (where Tptr might - // include const) to void * &. This is the only way - // I could make it work. RR - (void * & )t, - bpis_ptr, - find - ); - // if the pointer isn't that of the base class - if(newbpis_ptr != bpis_ptr){ - t = pointer_tweak(newbpis_ptr->get_eti(), t, *t); - } - } -}; - -template -struct load_enum_type { - template - static void invoke(Archive &ar, T &t){ - // convert integers to correct enum to load - int i; - ar >> boost::serialization::make_nvp(NULL, i); - t = static_cast< T >(i); - } -}; - -template -struct load_array_type { - template - static void invoke(Archive &ar, T &t){ - typedef typename remove_extent< T >::type value_type; - - // convert integers to correct enum to load - // determine number of elements in the array. Consider the - // fact that some machines will align elements on boundries - // other than characters. - std::size_t current_count = sizeof(t) / ( - static_cast(static_cast(&t[1])) - - static_cast(static_cast(&t[0])) - ); - boost::serialization::collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(static_cast(count) > current_count) - boost::serialization::throw_exception( - archive::archive_exception( - boost::archive::archive_exception::array_size_too_short - ) - ); - // explict template arguments to pass intel C++ compiler - ar >> serialization::make_array< - value_type, - boost::serialization::collection_size_type - >( - static_cast(&t[0]), - count - ); - } -}; - -} // detail - -template -inline void load(Archive & ar, T &t){ - // if this assertion trips. It means we're trying to load a - // const object with a compiler that doesn't have correct - // function template ordering. On other compilers, this is - // handled below. - detail::check_const_loading< T >(); - typedef - typename mpl::eval_if, - mpl::identity > - ,//else - typename mpl::eval_if, - mpl::identity > - ,//else - typename mpl::eval_if, - mpl::identity > - ,//else - mpl::identity > - > - > - >::type typex; - typex::invoke(ar, t); -} - -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_DETAIL_ISERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/oserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/oserializer.hpp deleted file mode 100644 index c120ec55073..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/oserializer.hpp +++ /dev/null @@ -1,540 +0,0 @@ -#ifndef BOOST_ARCHIVE_OSERIALIZER_HPP -#define BOOST_ARCHIVE_OSERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#pragma inline_depth(511) -#pragma inline_recursion(on) -#endif - -#if defined(__MWERKS__) -#pragma inline_depth(511) -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// oserializer.hpp: interface for serialization system. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL - -#include -#include -#include - -#include -#include -#include -#include -#include - -#ifndef BOOST_SERIALIZATION_DEFAULT_TYPE_INFO - #include -#endif -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -#include -#include -#include -#include -#include -#include - -namespace boost { - -namespace serialization { - class extended_type_info; -} // namespace serialization - -namespace archive { - -// an accessor to permit friend access to archives. Needed because -// some compilers don't handle friend templates completely -class save_access { -public: - template - static void end_preamble(Archive & ar){ - ar.end_preamble(); - } - template - static void save_primitive(Archive & ar, const T & t){ - ar.end_preamble(); - ar.save(t); - } -}; - -namespace detail { - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -template -class oserializer : public basic_oserializer -{ -private: - // private constructor to inhibit any existence other than the - // static one -public: - explicit BOOST_DLLEXPORT oserializer() : - basic_oserializer( - boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< T >::type - >::get_const_instance() - ) - {} - virtual BOOST_DLLEXPORT void save_object_data( - basic_oarchive & ar, - const void *x - ) const BOOST_USED; - virtual bool class_info() const { - return boost::serialization::implementation_level< T >::value - >= boost::serialization::object_class_info; - } - virtual bool tracking(const unsigned int /* flags */) const { - return boost::serialization::tracking_level< T >::value == boost::serialization::track_always - || (boost::serialization::tracking_level< T >::value == boost::serialization::track_selectively - && serialized_as_pointer()); - } - virtual version_type version() const { - return version_type(::boost::serialization::version< T >::value); - } - virtual bool is_polymorphic() const { - return boost::is_polymorphic< T >::value; - } - virtual ~oserializer(){} -}; - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -template -BOOST_DLLEXPORT void oserializer::save_object_data( - basic_oarchive & ar, - const void *x -) const { - // make sure call is routed through the highest interface that might - // be specialized by the user. - BOOST_STATIC_ASSERT(boost::is_const< T >::value == false); - boost::serialization::serialize_adl( - boost::serialization::smart_cast_reference(ar), - * static_cast(const_cast(x)), - version() - ); -} - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -template -class pointer_oserializer : - public basic_pointer_oserializer -{ -private: - const basic_oserializer & - get_basic_serializer() const { - return boost::serialization::singleton< - oserializer - >::get_const_instance(); - } - virtual BOOST_DLLEXPORT void save_object_ptr( - basic_oarchive & ar, - const void * x - ) const BOOST_USED; -public: - pointer_oserializer(); - ~pointer_oserializer(); -}; - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -template -BOOST_DLLEXPORT void pointer_oserializer::save_object_ptr( - basic_oarchive & ar, - const void * x -) const { - BOOST_ASSERT(NULL != x); - // make sure call is routed through the highest interface that might - // be specialized by the user. - T * t = static_cast(const_cast(x)); - const unsigned int file_version = boost::serialization::version< T >::value; - Archive & ar_impl - = boost::serialization::smart_cast_reference(ar); - boost::serialization::save_construct_data_adl( - ar_impl, - t, - file_version - ); - ar_impl << boost::serialization::make_nvp(NULL, * t); -} - -template -pointer_oserializer::pointer_oserializer() : - basic_pointer_oserializer( - boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< T >::type - >::get_const_instance() - ) -{ - // make sure appropriate member function is instantiated - boost::serialization::singleton< - oserializer - >::get_mutable_instance().set_bpos(this); - archive_serializer_map::insert(this); -} - -template -pointer_oserializer::~pointer_oserializer(){ - archive_serializer_map::erase(this); -} - -template -struct save_non_pointer_type { - // note this bounces the call right back to the archive - // with no runtime overhead - struct save_primitive { - template - static void invoke(Archive & ar, const T & t){ - save_access::save_primitive(ar, t); - } - }; - // same as above but passes through serialization - struct save_only { - template - static void invoke(Archive & ar, const T & t){ - // make sure call is routed through the highest interface that might - // be specialized by the user. - boost::serialization::serialize_adl( - ar, - const_cast(t), - ::boost::serialization::version< T >::value - ); - } - }; - // adds class information to the archive. This includes - // serialization level and class version - struct save_standard { - template - static void invoke(Archive &ar, const T & t){ - ar.save_object( - & t, - boost::serialization::singleton< - oserializer - >::get_const_instance() - ); - } - }; - - // adds class information to the archive. This includes - // serialization level and class version - struct save_conditional { - template - static void invoke(Archive &ar, const T &t){ - //if(0 == (ar.get_flags() & no_tracking)) - save_standard::invoke(ar, t); - //else - // save_only::invoke(ar, t); - } - }; - - - template - static void invoke(Archive & ar, const T & t){ - typedef - typename mpl::eval_if< - // if its primitive - mpl::equal_to< - boost::serialization::implementation_level< T >, - mpl::int_ - >, - mpl::identity, - // else - typename mpl::eval_if< - // class info / version - mpl::greater_equal< - boost::serialization::implementation_level< T >, - mpl::int_ - >, - // do standard save - mpl::identity, - // else - typename mpl::eval_if< - // no tracking - mpl::equal_to< - boost::serialization::tracking_level< T >, - mpl::int_ - >, - // do a fast save - mpl::identity, - // else - // do a fast save only tracking is turned off - mpl::identity - > > >::type typex; - check_object_versioning< T >(); - typex::invoke(ar, t); - } - template - static void invoke(Archive & ar, T & t){ - check_object_level< T >(); - check_object_tracking< T >(); - invoke(ar, const_cast(t)); - } -}; - -template -struct save_pointer_type { - struct abstract - { - template - static const basic_pointer_oserializer * register_type(Archive & /* ar */){ - // it has? to be polymorphic - BOOST_STATIC_ASSERT(boost::is_polymorphic< T >::value); - return NULL; - } - }; - - struct non_abstract - { - template - static const basic_pointer_oserializer * register_type(Archive & ar){ - return ar.register_type(static_cast(NULL)); - } - }; - - template - static const basic_pointer_oserializer * register_type(Archive &ar, T & /*t*/){ - // there should never be any need to save an abstract polymorphic - // class pointer. Inhibiting code generation for this - // permits abstract base classes to be used - note: exception - // virtual serialize functions used for plug-ins - typedef - typename mpl::eval_if< - boost::serialization::is_abstract< T >, - mpl::identity, - mpl::identity - >::type typex; - return typex::template register_type< T >(ar); - } - - struct non_polymorphic - { - template - static void save( - Archive &ar, - T & t - ){ - const basic_pointer_oserializer & bpos = - boost::serialization::singleton< - pointer_oserializer - >::get_const_instance(); - // save the requested pointer type - ar.save_pointer(& t, & bpos); - } - }; - - struct polymorphic - { - template - static void save( - Archive &ar, - T & t - ){ - typename - boost::serialization::type_info_implementation< T >::type const - & i = boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< T >::type - >::get_const_instance(); - - boost::serialization::extended_type_info const * const this_type = & i; - - // retrieve the true type of the object pointed to - // if this assertion fails its an error in this library - BOOST_ASSERT(NULL != this_type); - - const boost::serialization::extended_type_info * true_type = - i.get_derived_extended_type_info(t); - - // note:if this exception is thrown, be sure that derived pointer - // is either registered or exported. - if(NULL == true_type){ - boost::serialization::throw_exception( - archive_exception( - archive_exception::unregistered_class, - "derived class not registered or exported" - ) - ); - } - - // if its not a pointer to a more derived type - const void *vp = static_cast(&t); - if(*this_type == *true_type){ - const basic_pointer_oserializer * bpos = register_type(ar, t); - ar.save_pointer(vp, bpos); - return; - } - // convert pointer to more derived type. if this is thrown - // it means that the base/derived relationship hasn't be registered - vp = serialization::void_downcast( - *true_type, - *this_type, - static_cast(&t) - ); - if(NULL == vp){ - boost::serialization::throw_exception( - archive_exception( - archive_exception::unregistered_cast, - true_type->get_debug_info(), - this_type->get_debug_info() - ) - ); - } - - // since true_type is valid, and this only gets made if the - // pointer oserializer object has been created, this should never - // fail - const basic_pointer_oserializer * bpos - = static_cast( - boost::serialization::singleton< - archive_serializer_map - >::get_const_instance().find(*true_type) - ); - BOOST_ASSERT(NULL != bpos); - if(NULL == bpos) - boost::serialization::throw_exception( - archive_exception( - archive_exception::unregistered_class, - "derived class not registered or exported" - ) - ); - ar.save_pointer(vp, bpos); - } - }; - - template - static void save( - Archive & ar, - const T & t - ){ - check_pointer_level< T >(); - check_pointer_tracking< T >(); - typedef typename mpl::eval_if< - is_polymorphic< T >, - mpl::identity, - mpl::identity - >::type type; - type::save(ar, const_cast(t)); - } - - template - static void invoke(Archive &ar, const TPtr t){ - register_type(ar, * t); - if(NULL == t){ - basic_oarchive & boa - = boost::serialization::smart_cast_reference(ar); - boa.save_null_pointer(); - save_access::end_preamble(ar); - return; - } - save(ar, * t); - } -}; - -template -struct save_enum_type -{ - template - static void invoke(Archive &ar, const T &t){ - // convert enum to integers on save - const int i = static_cast(t); - ar << boost::serialization::make_nvp(NULL, i); - } -}; - -template -struct save_array_type -{ - template - static void invoke(Archive &ar, const T &t){ - typedef typename boost::remove_extent< T >::type value_type; - - save_access::end_preamble(ar); - // consider alignment - std::size_t c = sizeof(t) / ( - static_cast(static_cast(&t[1])) - - static_cast(static_cast(&t[0])) - ); - boost::serialization::collection_size_type count(c); - ar << BOOST_SERIALIZATION_NVP(count); - // explict template arguments to pass intel C++ compiler - ar << serialization::make_array< - const value_type, - boost::serialization::collection_size_type - >( - static_cast(&t[0]), - count - ); - } -}; - -} // detail - -template -inline void save(Archive & ar, /*const*/ T &t){ - typedef - typename mpl::eval_if, - mpl::identity >, - //else - typename mpl::eval_if, - mpl::identity >, - //else - typename mpl::eval_if, - mpl::identity >, - //else - mpl::identity > - > - > - >::type typex; - typex::invoke(ar, t); -} - -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_OSERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_iarchive_route.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_iarchive_route.hpp deleted file mode 100644 index 105685ebbd8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_iarchive_route.hpp +++ /dev/null @@ -1,218 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_POLYMORPHIC_IARCHIVE_ROUTE_HPP -#define BOOST_ARCHIVE_DETAIL_POLYMORPHIC_IARCHIVE_ROUTE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_iarchive_route.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization -namespace archive { -namespace detail{ - -class basic_iserializer; -class basic_pointer_iserializer; - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -template -class polymorphic_iarchive_route : - public polymorphic_iarchive, - // note: gcc dynamic cross cast fails if the the derivation below is - // not public. I think this is a mistake. - public /*protected*/ ArchiveImplementation -{ -private: - // these are used by the serialization library. - virtual void load_object( - void *t, - const basic_iserializer & bis - ){ - ArchiveImplementation::load_object(t, bis); - } - virtual const basic_pointer_iserializer * load_pointer( - void * & t, - const basic_pointer_iserializer * bpis_ptr, - const basic_pointer_iserializer * (*finder)( - const boost::serialization::extended_type_info & type - ) - ){ - return ArchiveImplementation::load_pointer(t, bpis_ptr, finder); - } - virtual void set_library_version(library_version_type archive_library_version){ - ArchiveImplementation::set_library_version(archive_library_version); - } - virtual library_version_type get_library_version() const{ - return ArchiveImplementation::get_library_version(); - } - virtual unsigned int get_flags() const { - return ArchiveImplementation::get_flags(); - } - virtual void delete_created_pointers(){ - ArchiveImplementation::delete_created_pointers(); - } - virtual void reset_object_address( - const void * new_address, - const void * old_address - ){ - ArchiveImplementation::reset_object_address(new_address, old_address); - } - virtual void load_binary(void * t, std::size_t size){ - ArchiveImplementation::load_binary(t, size); - } - // primitive types the only ones permitted by polymorphic archives - virtual void load(bool & t){ - ArchiveImplementation::load(t); - } - virtual void load(char & t){ - ArchiveImplementation::load(t); - } - virtual void load(signed char & t){ - ArchiveImplementation::load(t); - } - virtual void load(unsigned char & t){ - ArchiveImplementation::load(t); - } - #ifndef BOOST_NO_CWCHAR - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - virtual void load(wchar_t & t){ - ArchiveImplementation::load(t); - } - #endif - #endif - virtual void load(short & t){ - ArchiveImplementation::load(t); - } - virtual void load(unsigned short & t){ - ArchiveImplementation::load(t); - } - virtual void load(int & t){ - ArchiveImplementation::load(t); - } - virtual void load(unsigned int & t){ - ArchiveImplementation::load(t); - } - virtual void load(long & t){ - ArchiveImplementation::load(t); - } - virtual void load(unsigned long & t){ - ArchiveImplementation::load(t); - } - #if defined(BOOST_HAS_LONG_LONG) - virtual void load(boost::long_long_type & t){ - ArchiveImplementation::load(t); - } - virtual void load(boost::ulong_long_type & t){ - ArchiveImplementation::load(t); - } - #elif defined(BOOST_HAS_MS_INT64) - virtual void load(__int64 & t){ - ArchiveImplementation::load(t); - } - virtual void load(unsigned __int64 & t){ - ArchiveImplementation::load(t); - } - #endif - virtual void load(float & t){ - ArchiveImplementation::load(t); - } - virtual void load(double & t){ - ArchiveImplementation::load(t); - } - virtual void load(std::string & t){ - ArchiveImplementation::load(t); - } - #ifndef BOOST_NO_STD_WSTRING - virtual void load(std::wstring & t){ - ArchiveImplementation::load(t); - } - #endif - // used for xml and other tagged formats default does nothing - virtual void load_start(const char * name){ - ArchiveImplementation::load_start(name); - } - virtual void load_end(const char * name){ - ArchiveImplementation::load_end(name); - } - virtual void register_basic_serializer(const basic_iserializer & bis){ - ArchiveImplementation::register_basic_serializer(bis); - } - virtual helper_collection & - get_helper_collection(){ - return ArchiveImplementation::get_helper_collection(); - } -public: - // this can't be inheriteded because they appear in mulitple - // parents - typedef mpl::bool_ is_loading; - typedef mpl::bool_ is_saving; - // the >> operator - template - polymorphic_iarchive & operator>>(T & t){ - return polymorphic_iarchive::operator>>(t); - } - // the & operator - template - polymorphic_iarchive & operator&(T & t){ - return polymorphic_iarchive::operator&(t); - } - // register type function - template - const basic_pointer_iserializer * - register_type(T * t = NULL){ - return ArchiveImplementation::register_type(t); - } - // all current archives take a stream as constructor argument - template - polymorphic_iarchive_route( - std::basic_istream<_Elem, _Tr> & is, - unsigned int flags = 0 - ) : - ArchiveImplementation(is, flags) - {} - virtual ~polymorphic_iarchive_route(){}; -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_DETAIL_POLYMORPHIC_IARCHIVE_DISPATCH_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_oarchive_route.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_oarchive_route.hpp deleted file mode 100644 index b23fd6bf39d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_oarchive_route.hpp +++ /dev/null @@ -1,209 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_POLYMORPHIC_OARCHIVE_ROUTE_HPP -#define BOOST_ARCHIVE_DETAIL_POLYMORPHIC_OARCHIVE_ROUTE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_oarchive_route.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include // size_t - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization -namespace archive { -namespace detail{ - -class basic_oserializer; -class basic_pointer_oserializer; - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -template -class polymorphic_oarchive_route : - public polymorphic_oarchive, - // note: gcc dynamic cross cast fails if the the derivation below is - // not public. I think this is a mistake. - public /*protected*/ ArchiveImplementation -{ -private: - // these are used by the serialization library. - virtual void save_object( - const void *x, - const detail::basic_oserializer & bos - ){ - ArchiveImplementation::save_object(x, bos); - } - virtual void save_pointer( - const void * t, - const detail::basic_pointer_oserializer * bpos_ptr - ){ - ArchiveImplementation::save_pointer(t, bpos_ptr); - } - virtual void save_null_pointer(){ - ArchiveImplementation::save_null_pointer(); - } - // primitive types the only ones permitted by polymorphic archives - virtual void save(const bool t){ - ArchiveImplementation::save(t); - } - virtual void save(const char t){ - ArchiveImplementation::save(t); - } - virtual void save(const signed char t){ - ArchiveImplementation::save(t); - } - virtual void save(const unsigned char t){ - ArchiveImplementation::save(t); - } - #ifndef BOOST_NO_CWCHAR - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - virtual void save(const wchar_t t){ - ArchiveImplementation::save(t); - } - #endif - #endif - virtual void save(const short t){ - ArchiveImplementation::save(t); - } - virtual void save(const unsigned short t){ - ArchiveImplementation::save(t); - } - virtual void save(const int t){ - ArchiveImplementation::save(t); - } - virtual void save(const unsigned int t){ - ArchiveImplementation::save(t); - } - virtual void save(const long t){ - ArchiveImplementation::save(t); - } - virtual void save(const unsigned long t){ - ArchiveImplementation::save(t); - } - #if defined(BOOST_HAS_LONG_LONG) - virtual void save(const boost::long_long_type t){ - ArchiveImplementation::save(t); - } - virtual void save(const boost::ulong_long_type t){ - ArchiveImplementation::save(t); - } - #elif defined(BOOST_HAS_MS_INT64) - virtual void save(const boost::int64_t t){ - ArchiveImplementation::save(t); - } - virtual void save(const boost::uint64_t t){ - ArchiveImplementation::save(t); - } - #endif - virtual void save(const float t){ - ArchiveImplementation::save(t); - } - virtual void save(const double t){ - ArchiveImplementation::save(t); - } - virtual void save(const std::string & t){ - ArchiveImplementation::save(t); - } - #ifndef BOOST_NO_STD_WSTRING - virtual void save(const std::wstring & t){ - ArchiveImplementation::save(t); - } - #endif - virtual library_version_type get_library_version() const{ - return ArchiveImplementation::get_library_version(); - } - virtual unsigned int get_flags() const { - return ArchiveImplementation::get_flags(); - } - virtual void save_binary(const void * t, std::size_t size){ - ArchiveImplementation::save_binary(t, size); - } - // used for xml and other tagged formats default does nothing - virtual void save_start(const char * name){ - ArchiveImplementation::save_start(name); - } - virtual void save_end(const char * name){ - ArchiveImplementation::save_end(name); - } - virtual void end_preamble(){ - ArchiveImplementation::end_preamble(); - } - virtual void register_basic_serializer(const detail::basic_oserializer & bos){ - ArchiveImplementation::register_basic_serializer(bos); - } - virtual helper_collection & - get_helper_collection(){ - return ArchiveImplementation::get_helper_collection(); - } -public: - // this can't be inheriteded because they appear in mulitple - // parents - typedef mpl::bool_ is_loading; - typedef mpl::bool_ is_saving; - // the << operator - template - polymorphic_oarchive & operator<<(T & t){ - return polymorphic_oarchive::operator<<(t); - } - // the & operator - template - polymorphic_oarchive & operator&(T & t){ - return polymorphic_oarchive::operator&(t); - } - // register type function - template - const basic_pointer_oserializer * - register_type(T * t = NULL){ - return ArchiveImplementation::register_type(t); - } - // all current archives take a stream as constructor argument - template - polymorphic_oarchive_route( - std::basic_ostream<_Elem, _Tr> & os, - unsigned int flags = 0 - ) : - ArchiveImplementation(os, flags) - {} - virtual ~polymorphic_oarchive_route(){}; -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_DETAIL_POLYMORPHIC_OARCHIVE_DISPATCH_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/register_archive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/register_archive.hpp deleted file mode 100644 index 5ffecc702ce..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/register_archive.hpp +++ /dev/null @@ -1,91 +0,0 @@ -// Copyright David Abrahams 2006. Distributed under the Boost -// Software License, Version 1.0. (See accompanying -// file LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) -#ifndef BOOST_ARCHIVE_DETAIL_REGISTER_ARCHIVE_DWA2006521_HPP -# define BOOST_ARCHIVE_DETAIL_REGISTER_ARCHIVE_DWA2006521_HPP - -namespace boost { namespace archive { namespace detail { - -// No instantiate_ptr_serialization overloads generated by -// BOOST_SERIALIZATION_REGISTER_ARCHIVE that lexically follow the call -// will be seen *unless* they are in an associated namespace of one of -// the arguments, so we pass one of these along to make sure this -// namespace is considered. See temp.dep.candidate (14.6.4.2) in the -// standard. -struct adl_tag {}; - -template -struct ptr_serialization_support; - -// We could've just used ptr_serialization_support, above, but using -// it with only a forward declaration causes vc6/7 to complain about a -// missing instantiate member, even if it has one. This is just a -// friendly layer of indirection. -template -struct _ptr_serialization_support - : ptr_serialization_support -{ - typedef int type; -}; - -#if defined(__SUNPRO_CC) && (__SUNPRO_CC < 0x5130) - -template -struct counter : counter {}; -template<> -struct counter<0> {}; - -template -void instantiate_ptr_serialization(Serializable* s, int, adl_tag) { - instantiate_ptr_serialization(s, counter<20>()); -} - -template -struct get_counter { - static const int value = sizeof(adjust_counter(counter<20>())); - typedef counter type; - typedef counter prior; - typedef char (&next)[value+1]; -}; - -char adjust_counter(counter<0>); -template -void instantiate_ptr_serialization(Serializable*, counter<0>) {} - -#define BOOST_SERIALIZATION_REGISTER_ARCHIVE(Archive) \ -namespace boost { namespace archive { namespace detail { \ - get_counter::next adjust_counter(get_counter::type);\ - template \ - void instantiate_ptr_serialization(Serializable* s, \ - get_counter::type) { \ - ptr_serialization_support x; \ - instantiate_ptr_serialization(s, get_counter::prior()); \ - }\ -}}} - - -#else - -// This function gets called, but its only purpose is to participate -// in overload resolution with the functions declared by -// BOOST_SERIALIZATION_REGISTER_ARCHIVE, below. -template -void instantiate_ptr_serialization(Serializable*, int, adl_tag ) {} - -// The function declaration generated by this macro never actually -// gets called, but its return type gets instantiated, and that's -// enough to cause registration of serialization functions between -// Archive and any exported Serializable type. See also: -// boost/serialization/export.hpp -# define BOOST_SERIALIZATION_REGISTER_ARCHIVE(Archive) \ -namespace boost { namespace archive { namespace detail { \ - \ -template \ -typename _ptr_serialization_support::type \ -instantiate_ptr_serialization( Serializable*, Archive*, adl_tag ); \ - \ -}}} -#endif -}}} // namespace boost::archive::detail - -#endif // BOOST_ARCHIVE_DETAIL_INSTANTIATE_SERIALIZE_DWA2006521_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/utf8_codecvt_facet.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/utf8_codecvt_facet.hpp deleted file mode 100644 index a40104abea6..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/utf8_codecvt_facet.hpp +++ /dev/null @@ -1,39 +0,0 @@ -// Copyright (c) 2001 Ronald Garcia, Indiana University (garcia@osl.iu.edu) -// Andrew Lumsdaine, Indiana University (lums@osl.iu.edu). -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#ifndef BOOST_ARCHIVE_DETAIL_UTF8_CODECVT_FACET_HPP -#define BOOST_ARCHIVE_DETAIL_UTF8_CODECVT_FACET_HPP - -#include - -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#endif - -// std::codecvt_utf8 doesn't seem to work for any versions of msvc - -#if defined(_MSC_VER) || defined(BOOST_NO_CXX11_HDR_CODECVT) - // use boost's utf8 codecvt facet - #include - #define BOOST_UTF8_BEGIN_NAMESPACE \ - namespace boost { namespace archive { namespace detail { - #define BOOST_UTF8_DECL BOOST_ARCHIVE_DECL - #define BOOST_UTF8_END_NAMESPACE }}} - - #include - - #undef BOOST_UTF8_END_NAMESPACE - #undef BOOST_UTF8_DECL - #undef BOOST_UTF8_BEGIN_NAMESPACE -#else - // use the standard vendor supplied facet - #include - namespace boost { namespace archive { namespace detail { - typedef std::codecvt_utf8 utf8_codecvt_facet; - } } } -#endif - -#endif // BOOST_ARCHIVE_DETAIL_UTF8_CODECVT_FACET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/dinkumware.hpp b/contrib/libboost/boost_1_65_0/boost/archive/dinkumware.hpp deleted file mode 100644 index 90ba6271cdd..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/dinkumware.hpp +++ /dev/null @@ -1,224 +0,0 @@ -#ifndef BOOST_ARCHIVE_DINKUMWARE_HPP -#define BOOST_ARCHIVE_DINKUMWARE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// dinkumware.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// this file adds a couple of things that are missing from the dinkumware -// implementation of the standard library. - -#include -#include - -#include -#include - -namespace std { - -// define i/o operators for 64 bit integers -template -basic_ostream & -operator<<(basic_ostream & os, boost::uint64_t t){ - // octal rendering of 64 bit number would be 22 octets + eos - CharType d[23]; - unsigned int radix; - - if(os.flags() & (int)std::ios_base::hex) - radix = 16; - else - if(os.flags() & (int)std::ios_base::oct) - radix = 8; - else - //if(s.flags() & (int)std::ios_base::dec) - radix = 10; - unsigned int i = 0; - do{ - unsigned int j = t % radix; - d[i++] = j + ((j < 10) ? '0' : ('a' - 10)); - t /= radix; - } - while(t > 0); - d[i--] = '\0'; - - // reverse digits - unsigned int j = 0; - while(j < i){ - CharType k = d[i]; - d[i] = d[j]; - d[j] = k; - --i;++j; - } - os << d; - return os; - -} - -template -basic_ostream & -operator<<(basic_ostream &os, boost::int64_t t){ - if(0 <= t){ - os << static_cast(t); - } - else{ - os.put('-'); - os << -t; - } - return os; -} - -template -basic_istream & -operator>>(basic_istream &is, boost::int64_t & t){ - CharType d; - do{ - d = is.get(); - } - while(::isspace(d)); - bool negative = (d == '-'); - if(negative) - d = is.get(); - unsigned int radix; - if(is.flags() & (int)std::ios_base::hex) - radix = 16; - else - if(is.flags() & (int)std::ios_base::oct) - radix = 8; - else - //if(s.flags() & (int)std::ios_base::dec) - radix = 10; - t = 0; - do{ - if('0' <= d && d <= '9') - t = t * radix + (d - '0'); - else - if('a' <= d && d <= 'f') - t = t * radix + (d - 'a' + 10); - else - break; - d = is.get(); - } - while(!is.fail()); - // restore the delimiter - is.putback(d); - is.clear(); - if(negative) - t = -t; - return is; -} - -template -basic_istream & -operator>>(basic_istream &is, boost::uint64_t & t){ - boost::int64_t it; - is >> it; - t = it; - return is; -} - -//#endif - -template<> -class back_insert_iterator > : public - iterator -{ -public: - typedef basic_string container_type; - typedef container_type::reference reference; - - explicit back_insert_iterator(container_type & s) - : container(& s) - {} // construct with container - - back_insert_iterator & operator=( - container_type::const_reference Val_ - ){ // push value into container - //container->push_back(Val_); - *container += Val_; - return (*this); - } - - back_insert_iterator & operator*(){ - return (*this); - } - - back_insert_iterator & operator++(){ - // pretend to preincrement - return (*this); - } - - back_insert_iterator operator++(int){ - // pretend to postincrement - return (*this); - } - -protected: - container_type *container; // pointer to container -}; - -template -inline back_insert_iterator > back_inserter( - basic_string & s -){ - return (std::back_insert_iterator >(s)); -} - -template<> -class back_insert_iterator > : public - iterator -{ -public: - typedef basic_string container_type; - typedef container_type::reference reference; - - explicit back_insert_iterator(container_type & s) - : container(& s) - {} // construct with container - - back_insert_iterator & operator=( - container_type::const_reference Val_ - ){ // push value into container - //container->push_back(Val_); - *container += Val_; - return (*this); - } - - back_insert_iterator & operator*(){ - return (*this); - } - - back_insert_iterator & operator++(){ - // pretend to preincrement - return (*this); - } - - back_insert_iterator operator++(int){ - // pretend to postincrement - return (*this); - } - -protected: - container_type *container; // pointer to container -}; - -template -inline back_insert_iterator > back_inserter( - basic_string & s -){ - return (std::back_insert_iterator >(s)); -} - -} // namespace std - -#endif //BOOST_ARCHIVE_DINKUMWARE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/archive_serializer_map.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/archive_serializer_map.ipp deleted file mode 100644 index 7f163ec4076..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/archive_serializer_map.ipp +++ /dev/null @@ -1,75 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// archive_serializer_map.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -////////////////////////////////////////////////////////////////////// -// implementation of basic_text_iprimitive overrides for the combination -// of template parameters used to implement a text_iprimitive - -#include -#include -#include -#include - -namespace boost { -namespace archive { -namespace detail { - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace extra_detail { // anon - template - class map : public basic_serializer_map - {}; -} - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL bool -archive_serializer_map::insert(const basic_serializer * bs){ - return boost::serialization::singleton< - extra_detail::map - >::get_mutable_instance().insert(bs); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -archive_serializer_map::erase(const basic_serializer * bs){ - BOOST_ASSERT(! boost::serialization::singleton< - extra_detail::map - >::is_destroyed() - ); - if(boost::serialization::singleton< - extra_detail::map - >::is_destroyed()) - return; - boost::serialization::singleton< - extra_detail::map - >::get_mutable_instance().erase(bs); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL const basic_serializer * -archive_serializer_map::find( - const boost::serialization::extended_type_info & eti -) { - return boost::serialization::singleton< - extra_detail::map - >::get_const_instance().find(eti); -} - -} // namespace detail -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iarchive.ipp deleted file mode 100644 index d5619ab6cf3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iarchive.ipp +++ /dev/null @@ -1,134 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_iarchive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; - using ::strlen; - using ::size_t; -} -#endif - -#include -#include - -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implementation of binary_binary_archive -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iarchive::load_override(class_name_type & t){ - std::string cn; - cn.reserve(BOOST_SERIALIZATION_MAX_KEY_SIZE); - load_override(cn); - if(cn.size() > (BOOST_SERIALIZATION_MAX_KEY_SIZE - 1)) - boost::serialization::throw_exception( - archive_exception(archive_exception::invalid_class_name) - ); - std::memcpy(t, cn.data(), cn.size()); - // borland tweak - t.t[cn.size()] = '\0'; -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iarchive::init(void){ - // read signature in an archive version independent manner - std::string file_signature; - - #if 0 // commented out since it interfers with derivation - BOOST_TRY { - std::size_t l; - this->This()->load(l); - if(l == std::strlen(BOOST_ARCHIVE_SIGNATURE())) { - // borland de-allocator fixup - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != file_signature.data()) - #endif - file_signature.resize(l); - // note breaking a rule here - could be a problem on some platform - if(0 < l) - this->This()->load_binary(&(*file_signature.begin()), l); - } - } - BOOST_CATCH(archive_exception const &) { // catch stream_error archive exceptions - // will cause invalid_signature archive exception to be thrown below - file_signature = ""; - } - BOOST_CATCH_END - #else - // https://svn.boost.org/trac/boost/ticket/7301 - * this->This() >> file_signature; - #endif - - if(file_signature != BOOST_ARCHIVE_SIGNATURE()) - boost::serialization::throw_exception( - archive_exception(archive_exception::invalid_signature) - ); - - // make sure the version of the reading archive library can - // support the format of the archive being read - library_version_type input_library_version; - //* this->This() >> input_library_version; - { - int v = 0; - v = this->This()->m_sb.sbumpc(); - #if defined(BOOST_LITTLE_ENDIAN) - if(v < 6){ - ; - } - else - if(v < 7){ - // version 6 - next byte should be zero - this->This()->m_sb.sbumpc(); - } - else - if(v < 8){ - int x1; - // version 7 = might be followed by zero or some other byte - x1 = this->This()->m_sb.sgetc(); - // it's =a zero, push it back - if(0 == x1) - this->This()->m_sb.sbumpc(); - } - else{ - // version 8+ followed by a zero - this->This()->m_sb.sbumpc(); - } - #elif defined(BOOST_BIG_ENDIAN) - if(v == 0) - v = this->This()->m_sb.sbumpc(); - #endif - input_library_version = static_cast(v); - } - - #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) - this->set_library_version(input_library_version); - #else - detail::basic_iarchive::set_library_version(input_library_version); - #endif - - if(BOOST_ARCHIVE_VERSION() < input_library_version) - boost::serialization::throw_exception( - archive_exception(archive_exception::unsupported_version) - ); -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iprimitive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iprimitive.ipp deleted file mode 100644 index bbe933ccf63..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iprimitive.ipp +++ /dev/null @@ -1,171 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_iprimitive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // size_t, NULL -#include // memcpy - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; - using ::memcpy; -} // namespace std -#endif - -#include -#include -#include -#include - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// implementation of basic_binary_iprimitive - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iprimitive::init() -{ - // Detect attempts to pass native binary archives across - // incompatible platforms. This is not fool proof but its - // better than nothing. - unsigned char size; - this->This()->load(size); - if(sizeof(int) != size) - boost::serialization::throw_exception( - archive_exception( - archive_exception::incompatible_native_format, - "size of int" - ) - ); - this->This()->load(size); - if(sizeof(long) != size) - boost::serialization::throw_exception( - archive_exception( - archive_exception::incompatible_native_format, - "size of long" - ) - ); - this->This()->load(size); - if(sizeof(float) != size) - boost::serialization::throw_exception( - archive_exception( - archive_exception::incompatible_native_format, - "size of float" - ) - ); - this->This()->load(size); - if(sizeof(double) != size) - boost::serialization::throw_exception( - archive_exception( - archive_exception::incompatible_native_format, - "size of double" - ) - ); - - // for checking endian - int i; - this->This()->load(i); - if(1 != i) - boost::serialization::throw_exception( - archive_exception( - archive_exception::incompatible_native_format, - "endian setting" - ) - ); -} - -#ifndef BOOST_NO_CWCHAR -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iprimitive::load(wchar_t * ws) -{ - std::size_t l; // number of wchar_t !!! - this->This()->load(l); - load_binary(ws, l * sizeof(wchar_t) / sizeof(char)); - ws[l] = L'\0'; -} -#endif - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iprimitive::load(std::string & s) -{ - std::size_t l; - this->This()->load(l); - // borland de-allocator fixup - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != s.data()) - #endif - s.resize(l); - // note breaking a rule here - could be a problem on some platform - if(0 < l) - load_binary(&(*s.begin()), l); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iprimitive::load(char * s) -{ - std::size_t l; - this->This()->load(l); - load_binary(s, l); - s[l] = '\0'; -} - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iprimitive::load(std::wstring & ws) -{ - std::size_t l; - this->This()->load(l); - // borland de-allocator fixup - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != ws.data()) - #endif - ws.resize(l); - // note breaking a rule here - is could be a problem on some platform - load_binary(const_cast(ws.data()), l * sizeof(wchar_t) / sizeof(char)); -} -#endif - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_binary_iprimitive::basic_binary_iprimitive( - std::basic_streambuf & sb, - bool no_codecvt -) : -#ifndef BOOST_NO_STD_LOCALE - m_sb(sb), - codecvt_null_facet(1), - locale_saver(m_sb), - archive_locale(sb.getloc(), & codecvt_null_facet) -{ - if(! no_codecvt){ - m_sb.pubsync(); - m_sb.pubimbue(archive_locale); - } -} -#else - m_sb(sb) -{} -#endif - -// scoped_ptr requires that g be a complete type at time of -// destruction so define destructor here rather than in the header -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_binary_iprimitive::~basic_binary_iprimitive(){} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oarchive.ipp deleted file mode 100644 index d5a019d32bc..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oarchive.ipp +++ /dev/null @@ -1,42 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_oarchive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; -} -#endif - -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implementation of binary_binary_oarchive - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_oarchive::init(){ - // write signature in an archive version independent manner - const std::string file_signature(BOOST_ARCHIVE_SIGNATURE()); - * this->This() << file_signature; - // write library version - const library_version_type v(BOOST_ARCHIVE_VERSION()); - * this->This() << v; -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oprimitive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oprimitive.ipp deleted file mode 100644 index 7b042173a48..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oprimitive.ipp +++ /dev/null @@ -1,126 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_oprimitive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL -#include - -#include - -#if defined(BOOST_NO_STDC_NAMESPACE) && ! defined(__LIBCOMO__) -namespace std{ - using ::strlen; -} // namespace std -#endif - -#ifndef BOOST_NO_CWCHAR -#include -#ifdef BOOST_NO_STDC_NAMESPACE -namespace std{ using ::wcslen; } -#endif -#endif - -#include -#include - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// implementation of basic_binary_oprimitive - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_oprimitive::init() -{ - // record native sizes of fundamental types - // this is to permit detection of attempts to pass - // native binary archives accross incompatible machines. - // This is not foolproof but its better than nothing. - this->This()->save(static_cast(sizeof(int))); - this->This()->save(static_cast(sizeof(long))); - this->This()->save(static_cast(sizeof(float))); - this->This()->save(static_cast(sizeof(double))); - // for checking endianness - this->This()->save(int(1)); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_oprimitive::save(const char * s) -{ - std::size_t l = std::strlen(s); - this->This()->save(l); - save_binary(s, l); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_oprimitive::save(const std::string &s) -{ - std::size_t l = static_cast(s.size()); - this->This()->save(l); - save_binary(s.data(), l); -} - -#ifndef BOOST_NO_CWCHAR -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_oprimitive::save(const wchar_t * ws) -{ - std::size_t l = std::wcslen(ws); - this->This()->save(l); - save_binary(ws, l * sizeof(wchar_t) / sizeof(char)); -} -#endif - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_oprimitive::save(const std::wstring &ws) -{ - std::size_t l = ws.size(); - this->This()->save(l); - save_binary(ws.data(), l * sizeof(wchar_t) / sizeof(char)); -} -#endif -#endif // BOOST_NO_CWCHAR - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_binary_oprimitive::basic_binary_oprimitive( - std::basic_streambuf & sb, - bool no_codecvt -) : -#ifndef BOOST_NO_STD_LOCALE - m_sb(sb), - codecvt_null_facet(1), - locale_saver(m_sb), - archive_locale(sb.getloc(), & codecvt_null_facet) -{ - if(! no_codecvt){ - m_sb.pubsync(); - m_sb.pubimbue(archive_locale); - } -} -#else - m_sb(sb) -{} -#endif - -// scoped_ptr requires that g be a complete type at time of -// destruction so define destructor here rather than in the header -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_binary_oprimitive::~basic_binary_oprimitive(){} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iarchive.ipp deleted file mode 100644 index 9ec8c6588c8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iarchive.ipp +++ /dev/null @@ -1,76 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_iarchive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; -} -#endif - -#include -#include -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implementation of text_text_archive - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_text_iarchive::load_override(class_name_type & t){ - std::string cn; - cn.reserve(BOOST_SERIALIZATION_MAX_KEY_SIZE); - load_override(cn); - if(cn.size() > (BOOST_SERIALIZATION_MAX_KEY_SIZE - 1)) - boost::serialization::throw_exception( - archive_exception(archive_exception::invalid_class_name) - ); - std::memcpy(t, cn.data(), cn.size()); - // borland tweak - t.t[cn.size()] = '\0'; -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_text_iarchive::init(void){ - // read signature in an archive version independent manner - std::string file_signature; - * this->This() >> file_signature; - if(file_signature != BOOST_ARCHIVE_SIGNATURE()) - boost::serialization::throw_exception( - archive_exception(archive_exception::invalid_signature) - ); - - // make sure the version of the reading archive library can - // support the format of the archive being read - library_version_type input_library_version; - * this->This() >> input_library_version; - - #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) - this->set_library_version(input_library_version); - #else - detail::basic_iarchive::set_library_version(input_library_version); - #endif - - // extra little .t is to get around borland quirk - if(BOOST_ARCHIVE_VERSION() < input_library_version) - boost::serialization::throw_exception( - archive_exception(archive_exception::unsupported_version) - ); -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iprimitive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iprimitive.ipp deleted file mode 100644 index 4e44728068d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iprimitive.ipp +++ /dev/null @@ -1,137 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_iprimitive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // size_t, NULL -#include // NULL - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include - -#include - -#include -#include -#include -#include - -namespace boost { -namespace archive { - -namespace detail { - template - static inline bool is_whitespace(CharType c); - - template<> - inline bool is_whitespace(char t){ - return 0 != std::isspace(t); - } - - #ifndef BOOST_NO_CWCHAR - template<> - inline bool is_whitespace(wchar_t t){ - return 0 != std::iswspace(t); - } - #endif -} // detail - -// translate base64 text into binary and copy into buffer -// until buffer is full. -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_text_iprimitive::load_binary( - void *address, - std::size_t count -){ - typedef typename IStream::char_type CharType; - - if(0 == count) - return; - - BOOST_ASSERT( - static_cast((std::numeric_limits::max)()) - > (count + sizeof(CharType) - 1)/sizeof(CharType) - ); - - if(is.fail()) - boost::serialization::throw_exception( - archive_exception(archive_exception::input_stream_error) - ); - // convert from base64 to binary - typedef typename - iterators::transform_width< - iterators::binary_from_base64< - iterators::remove_whitespace< - iterators::istream_iterator - > - ,typename IStream::int_type - > - ,8 - ,6 - ,CharType - > - binary; - - binary i = binary(iterators::istream_iterator(is)); - - char * caddr = static_cast(address); - - // take care that we don't increment anymore than necessary - while(count-- > 0){ - *caddr++ = static_cast(*i++); - } - - // skip over any excess input - for(;;){ - typename IStream::int_type r; - r = is.get(); - if(is.eof()) - break; - if(detail::is_whitespace(static_cast(r))) - break; - } -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_text_iprimitive::basic_text_iprimitive( - IStream &is_, - bool no_codecvt -) : - is(is_), - flags_saver(is_), - precision_saver(is_), -#ifndef BOOST_NO_STD_LOCALE - codecvt_null_facet(1), - archive_locale(is.getloc(), & codecvt_null_facet), - locale_saver(is) -{ - if(! no_codecvt){ - is_.sync(); - is_.imbue(archive_locale); - } - is_ >> std::noboolalpha; -} -#else -{} -#endif - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_text_iprimitive::~basic_text_iprimitive(){ -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oarchive.ipp deleted file mode 100644 index 44bc1401fd6..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oarchive.ipp +++ /dev/null @@ -1,62 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_oarchive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; -} -#endif - -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implementation of basic_text_oarchive - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_text_oarchive::newtoken() -{ - switch(delimiter){ - default: - BOOST_ASSERT(false); - break; - case eol: - this->This()->put('\n'); - delimiter = space; - break; - case space: - this->This()->put(' '); - break; - case none: - delimiter = space; - break; - } -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_text_oarchive::init(){ - // write signature in an archive version independent manner - const std::string file_signature(BOOST_ARCHIVE_SIGNATURE()); - * this->This() << file_signature; - // write library version - const library_version_type v(BOOST_ARCHIVE_VERSION()); - * this->This() << v; -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oprimitive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oprimitive.ipp deleted file mode 100644 index 6030fd44c57..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oprimitive.ipp +++ /dev/null @@ -1,115 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_oprimitive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include // std::copy -#include // std::uncaught_exception -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include - -#include -#include -#include -#include - -namespace boost { -namespace archive { - -// translate to base64 and copy in to buffer. -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_text_oprimitive::save_binary( - const void *address, - std::size_t count -){ - typedef typename OStream::char_type CharType; - - if(0 == count) - return; - - if(os.fail()) - boost::serialization::throw_exception( - archive_exception(archive_exception::output_stream_error) - ); - - os.put('\n'); - - typedef - boost::archive::iterators::insert_linebreaks< - boost::archive::iterators::base64_from_binary< - boost::archive::iterators::transform_width< - const char *, - 6, - 8 - > - > - ,76 - ,const char // cwpro8 needs this - > - base64_text; - - boost::archive::iterators::ostream_iterator oi(os); - std::copy( - base64_text(static_cast(address)), - base64_text( - static_cast(address) + count - ), - oi - ); - - std::size_t tail = count % 3; - if(tail > 0){ - *oi++ = '='; - if(tail < 2) - *oi = '='; - } -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_text_oprimitive::basic_text_oprimitive( - OStream & os_, - bool no_codecvt -) : - os(os_), - flags_saver(os_), - precision_saver(os_), -#ifndef BOOST_NO_STD_LOCALE - codecvt_null_facet(1), - archive_locale(os.getloc(), & codecvt_null_facet), - locale_saver(os) -{ - if(! no_codecvt){ - os_.flush(); - os_.imbue(archive_locale); - } - os_ << std::noboolalpha; -} -#else -{} -#endif - - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_text_oprimitive::~basic_text_oprimitive(){ - if(std::uncaught_exception()) - return; - os << std::endl; -} - -} //namespace boost -} //namespace archive diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_grammar.hpp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_grammar.hpp deleted file mode 100644 index 6d4e4683f6a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_grammar.hpp +++ /dev/null @@ -1,173 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_XML_GRAMMAR_HPP -#define BOOST_ARCHIVE_BASIC_XML_GRAMMAR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_xml_grammar.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// this module is derived from simplexml.cpp - an example shipped as part of -// the spirit parser. This example contains the following notice: -/*============================================================================= - simplexml.cpp - - Spirit V1.3 - URL: http://spirit.sourceforge.net/ - - Copyright (c) 2001, Daniel C. Nuffer - - This software is provided 'as-is', without any express or implied - warranty. In no event will the copyright holder be held liable for - any damages arising from the use of this software. - - Permission is granted to anyone to use this software for any purpose, - including commercial applications, and to alter it and redistribute - it freely, subject to the following restrictions: - - 1. The origin of this software must not be misrepresented; you must - not claim that you wrote the original software. If you use this - software in a product, an acknowledgment in the product documentation - would be appreciated but is not required. - - 2. Altered source versions must be plainly marked as such, and must - not be misrepresented as being the original software. - - 3. This notice may not be removed or altered from any source - distribution. -=============================================================================*/ -#include - -#include -#include - -#include -#include - -#include -#include -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// XML grammar parsing - -template -class basic_xml_grammar { -public: - // The following is not necessary according to DR45, but at least - // one compiler (Compaq C++ 6.5 in strict_ansi mode) chokes otherwise. - struct return_values; - friend struct return_values; - -private: - typedef typename std::basic_istream IStream; - typedef typename std::basic_string StringType; - typedef typename boost::spirit::classic::chset chset_t; - typedef typename boost::spirit::classic::chlit chlit_t; - typedef typename boost::spirit::classic::scanner< - typename std::basic_string::iterator - > scanner_t; - typedef typename boost::spirit::classic::rule rule_t; - // Start grammar definition - rule_t - Reference, - Eq, - STag, - ETag, - LetterOrUnderscoreOrColon, - AttValue, - CharRef1, - CharRef2, - CharRef, - AmpRef, - LTRef, - GTRef, - AposRef, - QuoteRef, - CharData, - CharDataChars, - content, - AmpName, - LTName, - GTName, - ClassNameChar, - ClassName, - Name, - XMLDecl, - XMLDeclChars, - DocTypeDecl, - DocTypeDeclChars, - ClassIDAttribute, - ObjectIDAttribute, - ClassNameAttribute, - TrackingAttribute, - VersionAttribute, - UnusedAttribute, - Attribute, - SignatureAttribute, - SerializationWrapper, - NameHead, - NameTail, - AttributeList, - S; - - // XML Character classes - chset_t - BaseChar, - Ideographic, - Char, - Letter, - Digit, - CombiningChar, - Extender, - Sch, - NameChar; - - void init_chset(); - - bool my_parse( - IStream & is, - const rule_t &rule_, - const CharType delimiter = L'>' - ) const ; -public: - struct return_values { - StringType object_name; - StringType contents; - //class_id_type class_id; - int_least16_t class_id; - //object_id_type object_id; - uint_least32_t object_id; - //version_type version; - unsigned int version; - tracking_type tracking_level; - StringType class_name; - return_values() : - version(0), - tracking_level(false) - {} - } rv; - bool parse_start_tag(IStream & is) /*const*/; - bool parse_end_tag(IStream & is) const; - bool parse_string(IStream & is, StringType & s) /*const*/; - void init(IStream & is); - bool windup(IStream & is); - basic_xml_grammar(); -}; - -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_BASIC_XML_GRAMMAR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_iarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_iarchive.ipp deleted file mode 100644 index 625458b9eb5..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_iarchive.ipp +++ /dev/null @@ -1,115 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_xml_iarchive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL -#include - -#include -#include -#include -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implementation of xml_text_archive - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_iarchive::load_start(const char *name){ - // if there's no name - if(NULL == name) - return; - bool result = this->This()->gimpl->parse_start_tag(this->This()->get_is()); - if(true != result){ - boost::serialization::throw_exception( - archive_exception(archive_exception::input_stream_error) - ); - } - // don't check start tag at highest level - ++depth; - return; -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_iarchive::load_end(const char *name){ - // if there's no name - if(NULL == name) - return; - bool result = this->This()->gimpl->parse_end_tag(this->This()->get_is()); - if(true != result){ - boost::serialization::throw_exception( - archive_exception(archive_exception::input_stream_error) - ); - } - - // don't check start tag at highest level - if(0 == --depth) - return; - - if(0 == (this->get_flags() & no_xml_tag_checking)){ - // double check that the tag matches what is expected - useful for debug - if(0 != name[this->This()->gimpl->rv.object_name.size()] - || ! std::equal( - this->This()->gimpl->rv.object_name.begin(), - this->This()->gimpl->rv.object_name.end(), - name - ) - ){ - boost::serialization::throw_exception( - xml_archive_exception( - xml_archive_exception::xml_archive_tag_mismatch, - name - ) - ); - } - } -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_iarchive::load_override(object_id_type & t){ - t = object_id_type(this->This()->gimpl->rv.object_id); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_iarchive::load_override(version_type & t){ - t = version_type(this->This()->gimpl->rv.version); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_iarchive::load_override(class_id_type & t){ - t = class_id_type(this->This()->gimpl->rv.class_id); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_iarchive::load_override(tracking_type & t){ - t = this->This()->gimpl->rv.tracking_level; -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_xml_iarchive::basic_xml_iarchive(unsigned int flags) : - detail::common_iarchive(flags), - depth(0) -{} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_xml_iarchive::~basic_xml_iarchive(){ -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_oarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_oarchive.ipp deleted file mode 100644 index 3184413f382..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_oarchive.ipp +++ /dev/null @@ -1,272 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_xml_oarchive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL -#include -#if defined(BOOST_NO_STDC_NAMESPACE) && ! defined(__LIBCOMO__) -namespace std{ - using ::strlen; -} // namespace std -#endif - -#include -#include -#include -#include - -namespace boost { -namespace archive { - -namespace detail { -template -struct XML_name { - void operator()(CharType t) const{ - const unsigned char lookup_table[] = { - 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, - 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, - 0,0,0,0,0,0,0,0,0,0,0,0,0,1,1,0, // -. - 1,1,1,1,1,1,1,1,1,1,0,0,0,0,0,0, // 0-9 - 0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1, // A- - 1,1,1,1,1,1,1,1,1,1,1,0,0,0,0,1, // -Z _ - 0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1, // a- - 1,1,1,1,1,1,1,1,1,1,1,0,0,0,0,0, // -z - 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, - 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 - }; - if((unsigned)t > 127) - return; - if(0 == lookup_table[(unsigned)t]) - boost::serialization::throw_exception( - xml_archive_exception( - xml_archive_exception::xml_archive_tag_name_error - ) - ); - } -}; - -} // namespace detail - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implemenations of functions common to both types of xml output - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::write_attribute( - const char *attribute_name, - int t, - const char *conjunction -){ - this->This()->put(' '); - this->This()->put(attribute_name); - this->This()->put(conjunction); - this->This()->save(t); - this->This()->put('"'); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::write_attribute( - const char *attribute_name, - const char *key -){ - this->This()->put(' '); - this->This()->put(attribute_name); - this->This()->put("=\""); - this->This()->save(key); - this->This()->put('"'); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::indent(){ - int i; - for(i = depth; i-- > 0;) - this->This()->put('\t'); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_start(const char *name) -{ - if(NULL == name) - return; - - // be sure name has no invalid characters - std::for_each(name, name + std::strlen(name), detail::XML_name()); - - end_preamble(); - if(depth > 0){ - this->This()->put('\n'); - indent(); - } - ++depth; - this->This()->put('<'); - this->This()->save(name); - pending_preamble = true; - indent_next = false; -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_end(const char *name) -{ - if(NULL == name) - return; - - // be sure name has no invalid characters - std::for_each(name, name + std::strlen(name), detail::XML_name()); - - end_preamble(); - --depth; - if(indent_next){ - this->This()->put('\n'); - indent(); - } - indent_next = true; - this->This()->put("This()->save(name); - this->This()->put('>'); - if(0 == depth) - this->This()->put('\n'); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::end_preamble(){ - if(pending_preamble){ - this->This()->put('>'); - pending_preamble = false; - } -} -#if 0 -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const object_id_type & t) -{ - int i = t.t; // extra .t is for borland - write_attribute(BOOST_ARCHIVE_XML_OBJECT_ID(), i, "=\"_"); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override( - const object_reference_type & t, - int -){ - int i = t.t; // extra .t is for borland - write_attribute(BOOST_ARCHIVE_XML_OBJECT_REFERENCE(), i, "=\"_"); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const version_type & t) -{ - int i = t.t; // extra .t is for borland - write_attribute(BOOST_ARCHIVE_XML_VERSION(), i); -} -#endif - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const object_id_type & t) -{ - // borland doesn't do conversion of STRONG_TYPEDEFs very well - const unsigned int i = t; - write_attribute(BOOST_ARCHIVE_XML_OBJECT_ID(), i, "=\"_"); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override( - const object_reference_type & t -){ - const unsigned int i = t; - write_attribute(BOOST_ARCHIVE_XML_OBJECT_REFERENCE(), i, "=\"_"); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const version_type & t) -{ - const unsigned int i = t; - write_attribute(BOOST_ARCHIVE_XML_VERSION(), i); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const class_id_type & t) -{ - write_attribute(BOOST_ARCHIVE_XML_CLASS_ID(), t); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override( - const class_id_reference_type & t -){ - write_attribute(BOOST_ARCHIVE_XML_CLASS_ID_REFERENCE(), t); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override( - const class_id_optional_type & t -){ - write_attribute(BOOST_ARCHIVE_XML_CLASS_ID(), t); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const class_name_type & t) -{ - const char * key = t; - if(NULL == key) - return; - write_attribute(BOOST_ARCHIVE_XML_CLASS_NAME(), key); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const tracking_type & t) -{ - write_attribute(BOOST_ARCHIVE_XML_TRACKING(), t.t); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::init(){ - // xml header - this->This()->put("\n"); - this->This()->put("\n"); - // xml document wrapper - outer root - this->This()->put("This()->put(">\n"); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::windup(){ - // xml_trailer - this->This()->put("\n"); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_xml_oarchive::basic_xml_oarchive(unsigned int flags) : - detail::common_oarchive(flags), - depth(0), - pending_preamble(false), - indent_next(false) -{ -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_xml_oarchive::~basic_xml_oarchive(){ -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_iarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_iarchive_impl.ipp deleted file mode 100644 index ae4e2750ce8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_iarchive_impl.ipp +++ /dev/null @@ -1,128 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_iarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -////////////////////////////////////////////////////////////////////// -// implementation of basic_text_iprimitive overrides for the combination -// of template parameters used to implement a text_iprimitive - -#include // size_t, NULL -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include // RogueWave - -#include - -namespace boost { -namespace archive { - -template -BOOST_ARCHIVE_DECL void -text_iarchive_impl::load(char *s) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - // Works on all tested platforms - is.read(s, size); - s[size] = '\0'; -} - -template -BOOST_ARCHIVE_DECL void -text_iarchive_impl::load(std::string &s) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - // borland de-allocator fixup - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != s.data()) - #endif - s.resize(size); - if(0 < size) - is.read(&(*s.begin()), size); -} - -#ifndef BOOST_NO_CWCHAR -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_ARCHIVE_DECL void -text_iarchive_impl::load(wchar_t *ws) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - is.read((char *)ws, size * sizeof(wchar_t)/sizeof(char)); - ws[size] = L'\0'; -} -#endif // BOOST_NO_INTRINSIC_WCHAR_T - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_ARCHIVE_DECL void -text_iarchive_impl::load(std::wstring &ws) -{ - std::size_t size; - * this->This() >> size; - // borland de-allocator fixup - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != ws.data()) - #endif - ws.resize(size); - // skip separating space - is.get(); - is.read((char *)ws.data(), size * sizeof(wchar_t)/sizeof(char)); -} - -#endif // BOOST_NO_STD_WSTRING -#endif // BOOST_NO_CWCHAR - -template -BOOST_ARCHIVE_DECL void -text_iarchive_impl::load_override(class_name_type & t){ - basic_text_iarchive::load_override(t); -} - -template -BOOST_ARCHIVE_DECL void -text_iarchive_impl::init(){ - basic_text_iarchive::init(); -} - -template -BOOST_ARCHIVE_DECL -text_iarchive_impl::text_iarchive_impl( - std::istream & is, - unsigned int flags -) : - basic_text_iprimitive( - is, - 0 != (flags & no_codecvt) - ), - basic_text_iarchive(flags) -{ - if(0 == (flags & no_header)) - #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) - this->init(); - #else - this->basic_text_iarchive::init(); - #endif -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_oarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_oarchive_impl.ipp deleted file mode 100644 index 37d8664a98c..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_oarchive_impl.ipp +++ /dev/null @@ -1,122 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_oarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include // size_t - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#ifndef BOOST_NO_CWCHAR -#include -#ifdef BOOST_NO_STDC_NAMESPACE -namespace std{ using ::wcslen; } -#endif -#endif - -#include - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// implementation of basic_text_oprimitive overrides for the combination -// of template parameters used to create a text_oprimitive - -template -BOOST_ARCHIVE_DECL void -text_oarchive_impl::save(const char * s) -{ - const std::size_t len = std::ostream::traits_type::length(s); - *this->This() << len; - this->This()->newtoken(); - os << s; -} - -template -BOOST_ARCHIVE_DECL void -text_oarchive_impl::save(const std::string &s) -{ - const std::size_t size = s.size(); - *this->This() << size; - this->This()->newtoken(); - os << s; -} - -#ifndef BOOST_NO_CWCHAR -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_ARCHIVE_DECL void -text_oarchive_impl::save(const wchar_t * ws) -{ - const std::size_t l = std::wcslen(ws); - * this->This() << l; - this->This()->newtoken(); - os.write((const char *)ws, l * sizeof(wchar_t)/sizeof(char)); -} -#endif - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_ARCHIVE_DECL void -text_oarchive_impl::save(const std::wstring &ws) -{ - const std::size_t l = ws.size(); - * this->This() << l; - this->This()->newtoken(); - os.write((const char *)(ws.data()), l * sizeof(wchar_t)/sizeof(char)); -} -#endif -#endif // BOOST_NO_CWCHAR - -template -BOOST_ARCHIVE_DECL -text_oarchive_impl::text_oarchive_impl( - std::ostream & os, - unsigned int flags -) : - basic_text_oprimitive( - os, - 0 != (flags & no_codecvt) - ), - basic_text_oarchive(flags) -{ - if(0 == (flags & no_header)) - #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) - this->init(); - #else - this->basic_text_oarchive::init(); - #endif -} - -template -BOOST_ARCHIVE_DECL void -text_oarchive_impl::save_binary(const void *address, std::size_t count){ - put('\n'); - this->end_preamble(); - #if ! defined(__MWERKS__) - this->basic_text_oprimitive::save_binary( - #else - this->basic_text_oprimitive::save_binary( - #endif - address, - count - ); - this->delimiter = this->eol; -} - -} // namespace archive -} // namespace boost - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_wiarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_wiarchive_impl.ipp deleted file mode 100644 index e85625ac326..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_wiarchive_impl.ipp +++ /dev/null @@ -1,118 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_text_wiarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // size_t, NULL - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include // fixup for RogueWave - -#ifndef BOOST_NO_STD_WSTREAMBUF -#include - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// implementation of wiprimtives functions -// -template -BOOST_WARCHIVE_DECL void -text_wiarchive_impl::load(char *s) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - while(size-- > 0){ - *s++ = is.narrow(is.get(), '\0'); - } - *s = '\0'; -} - -template -BOOST_WARCHIVE_DECL void -text_wiarchive_impl::load(std::string &s) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != s.data()) - #endif - s.resize(0); - s.reserve(size); - while(size-- > 0){ - char x = is.narrow(is.get(), '\0'); - s += x; - } -} - -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_WARCHIVE_DECL void -text_wiarchive_impl::load(wchar_t *s) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - // Works on all tested platforms - is.read(s, size); - s[size] = L'\0'; -} -#endif - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_WARCHIVE_DECL void -text_wiarchive_impl::load(std::wstring &ws) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - // borland complains about resize - // borland de-allocator fixup - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != ws.data()) - #endif - ws.resize(size); - // note breaking a rule here - is this a problem on some platform - is.read(const_cast(ws.data()), size); -} -#endif - -template -BOOST_WARCHIVE_DECL -text_wiarchive_impl::text_wiarchive_impl( - std::wistream & is, - unsigned int flags -) : - basic_text_iprimitive( - is, - 0 != (flags & no_codecvt) - ), - basic_text_iarchive(flags) -{ - if(0 == (flags & no_header)) - basic_text_iarchive::init(); -} - -} // archive -} // boost - -#endif // BOOST_NO_STD_WSTREAMBUF diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_woarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_woarchive_impl.ipp deleted file mode 100644 index 2b6d427cd3a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_woarchive_impl.ipp +++ /dev/null @@ -1,85 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_woarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifndef BOOST_NO_STD_WSTREAMBUF - -#include -#include // size_t -#if defined(BOOST_NO_STDC_NAMESPACE) && ! defined(__LIBCOMO__) -namespace std{ - using ::strlen; - using ::size_t; -} // namespace std -#endif - -#include - -#include - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// implementation of woarchive functions -// -template -BOOST_WARCHIVE_DECL void -text_woarchive_impl::save(const char *s) -{ - // note: superfluous local variable fixes borland warning - const std::size_t size = std::strlen(s); - * this->This() << size; - this->This()->newtoken(); - while(*s != '\0') - os.put(os.widen(*s++)); -} - -template -BOOST_WARCHIVE_DECL void -text_woarchive_impl::save(const std::string &s) -{ - const std::size_t size = s.size(); - * this->This() << size; - this->This()->newtoken(); - const char * cptr = s.data(); - for(std::size_t i = size; i-- > 0;) - os.put(os.widen(*cptr++)); -} - -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_WARCHIVE_DECL void -text_woarchive_impl::save(const wchar_t *ws) -{ - const std::size_t size = std::wostream::traits_type::length(ws); - * this->This() << size; - this->This()->newtoken(); - os.write(ws, size); -} -#endif - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_WARCHIVE_DECL void -text_woarchive_impl::save(const std::wstring &ws) -{ - const std::size_t size = ws.length(); - * this->This() << size; - this->This()->newtoken(); - os.write(ws.data(), size); -} -#endif - -} // namespace archive -} // namespace boost - -#endif - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_iarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_iarchive_impl.ipp deleted file mode 100644 index efc32e01632..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_iarchive_impl.ipp +++ /dev/null @@ -1,199 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_iarchive_impl.cpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // memcpy -#include // NULL -#include - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; -} // namespace std -#endif - -#ifndef BOOST_NO_CWCHAR -#include // mbstate_t and mbrtowc -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::mbstate_t; - using ::mbrtowc; - } // namespace std -#endif -#endif // BOOST_NO_CWCHAR - -#include // RogueWave and Dinkumware -#if BOOST_WORKAROUND(BOOST_DINKUMWARE_STDLIB, == 1) -#include -#endif - -#include - -#include -#include -#include -#include - -#include "basic_xml_grammar.hpp" - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implemenations of functions specific to char archives - -// wide char stuff used by char archives - -#ifndef BOOST_NO_CWCHAR -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_ARCHIVE_DECL void -xml_iarchive_impl::load(std::wstring &ws){ - std::string s; - bool result = gimpl->parse_string(is, s); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); - - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != ws.data()) - #endif - ws.resize(0); - std::mbstate_t mbs = std::mbstate_t(); - const char * start = s.data(); - const char * end = start + s.size(); - while(start < end){ - wchar_t wc; - std::size_t count = std::mbrtowc(&wc, start, end - start, &mbs); - if(count == static_cast(-1)) - boost::serialization::throw_exception( - iterators::dataflow_exception( - iterators::dataflow_exception::invalid_conversion - ) - ); - if(count == static_cast(-2)) - continue; - start += count; - ws += wc; - } -} -#endif // BOOST_NO_STD_WSTRING - -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_ARCHIVE_DECL void -xml_iarchive_impl::load(wchar_t * ws){ - std::string s; - bool result = gimpl->parse_string(is, s); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception( - xml_archive_exception::xml_archive_parsing_error - ) - ); - - std::mbstate_t mbs = std::mbstate_t(); - const char * start = s.data(); - const char * end = start + s.size(); - while(start < end){ - wchar_t wc; - std::size_t length = std::mbrtowc(&wc, start, end - start, &mbs); - if(static_cast(-1) == length) - boost::serialization::throw_exception( - iterators::dataflow_exception( - iterators::dataflow_exception::invalid_conversion - ) - ); - if(static_cast(-2) == length) - continue; - - start += length; - *ws++ = wc; - } - *ws = L'\0'; -} -#endif // BOOST_NO_INTRINSIC_WCHAR_T - -#endif // BOOST_NO_CWCHAR - -template -BOOST_ARCHIVE_DECL void -xml_iarchive_impl::load(std::string &s){ - bool result = gimpl->parse_string(is, s); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); -} - -template -BOOST_ARCHIVE_DECL void -xml_iarchive_impl::load(char * s){ - std::string tstring; - bool result = gimpl->parse_string(is, tstring); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); - std::memcpy(s, tstring.data(), tstring.size()); - s[tstring.size()] = 0; -} - -template -BOOST_ARCHIVE_DECL void -xml_iarchive_impl::load_override(class_name_type & t){ - const std::string & s = gimpl->rv.class_name; - if(s.size() > BOOST_SERIALIZATION_MAX_KEY_SIZE - 1) - boost::serialization::throw_exception( - archive_exception(archive_exception::invalid_class_name) - ); - char * tptr = t; - std::memcpy(tptr, s.data(), s.size()); - tptr[s.size()] = '\0'; -} - -template -BOOST_ARCHIVE_DECL void -xml_iarchive_impl::init(){ - gimpl->init(is); - this->set_library_version( - library_version_type(gimpl->rv.version) - ); -} - -template -BOOST_ARCHIVE_DECL -xml_iarchive_impl::xml_iarchive_impl( - std::istream &is_, - unsigned int flags -) : - basic_text_iprimitive( - is_, - 0 != (flags & no_codecvt) - ), - basic_xml_iarchive(flags), - gimpl(new xml_grammar()) -{ - if(0 == (flags & no_header)) - init(); -} - -template -BOOST_ARCHIVE_DECL -xml_iarchive_impl::~xml_iarchive_impl(){ - if(std::uncaught_exception()) - return; - if(0 == (this->get_flags() & no_header)){ - gimpl->windup(is); - } -} -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_oarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_oarchive_impl.ipp deleted file mode 100644 index 5ebd454e722..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_oarchive_impl.ipp +++ /dev/null @@ -1,142 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_oarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include -#include -#include // std::copy -#include -#include - -#include // strlen -#include // msvc 6.0 needs this to suppress warnings -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::strlen; -} // namespace std -#endif - -#include -#include - -#ifndef BOOST_NO_CWCHAR -#include -#include -#endif - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implemenations of functions specific to char archives - -// wide char stuff used by char archives -#ifndef BOOST_NO_CWCHAR -// copy chars to output escaping to xml and translating wide chars to mb chars -template -void save_iterator(std::ostream &os, InputIterator begin, InputIterator end){ - typedef boost::archive::iterators::mb_from_wchar< - boost::archive::iterators::xml_escape - > translator; - std::copy( - translator(begin), - translator(end), - boost::archive::iterators::ostream_iterator(os) - ); -} - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_ARCHIVE_DECL void -xml_oarchive_impl::save(const std::wstring & ws){ -// at least one library doesn't typedef value_type for strings -// so rather than using string directly make a pointer iterator out of it -// save_iterator(os, ws.data(), ws.data() + std::wcslen(ws.data())); - save_iterator(os, ws.data(), ws.data() + ws.size()); -} -#endif - -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_ARCHIVE_DECL void -xml_oarchive_impl::save(const wchar_t * ws){ - save_iterator(os, ws, ws + std::wcslen(ws)); -} -#endif - -#endif // BOOST_NO_CWCHAR - -template -BOOST_ARCHIVE_DECL void -xml_oarchive_impl::save(const std::string & s){ -// at least one library doesn't typedef value_type for strings -// so rather than using string directly make a pointer iterator out of it - typedef boost::archive::iterators::xml_escape< - const char * - > xml_escape_translator; - std::copy( - xml_escape_translator(s.data()), - xml_escape_translator(s.data()+ s.size()), - boost::archive::iterators::ostream_iterator(os) - ); -} - -template -BOOST_ARCHIVE_DECL void -xml_oarchive_impl::save(const char * s){ - typedef boost::archive::iterators::xml_escape< - const char * - > xml_escape_translator; - std::copy( - xml_escape_translator(s), - xml_escape_translator(s + std::strlen(s)), - boost::archive::iterators::ostream_iterator(os) - ); -} - -template -BOOST_ARCHIVE_DECL -xml_oarchive_impl::xml_oarchive_impl( - std::ostream & os_, - unsigned int flags -) : - basic_text_oprimitive( - os_, - 0 != (flags & no_codecvt) - ), - basic_xml_oarchive(flags) -{ - if(0 == (flags & no_header)) - this->init(); -} - -template -BOOST_ARCHIVE_DECL void -xml_oarchive_impl::save_binary(const void *address, std::size_t count){ - this->end_preamble(); - #if ! defined(__MWERKS__) - this->basic_text_oprimitive::save_binary( - #else - this->basic_text_oprimitive::save_binary( - #endif - address, - count - ); - this->indent_next = true; -} - -template -BOOST_ARCHIVE_DECL -xml_oarchive_impl::~xml_oarchive_impl(){ - if(std::uncaught_exception()) - return; - if(0 == (this->get_flags() & no_header)) - this->windup(); -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_wiarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_wiarchive_impl.ipp deleted file mode 100644 index ee66c1263e6..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_wiarchive_impl.ipp +++ /dev/null @@ -1,189 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_wiarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; -} //std -#endif - -#include // msvc 6.0 needs this to suppress warnings -#ifndef BOOST_NO_STD_WSTREAMBUF - -#include -#include // std::copy -#include // uncaught exception -#include // Dinkumware and RogueWave -#if BOOST_WORKAROUND(BOOST_DINKUMWARE_STDLIB, == 1) -#include -#endif - -#include -#include -#include - -#include -#include - -#include -#include - -#include - -#include "basic_xml_grammar.hpp" - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implemenations of functions specific to wide char archives - -namespace { // anonymous - -void copy_to_ptr(char * s, const std::wstring & ws){ - std::copy( - iterators::mb_from_wchar( - ws.begin() - ), - iterators::mb_from_wchar( - ws.end() - ), - s - ); - s[ws.size()] = 0; -} - -} // anonymous - -template -BOOST_WARCHIVE_DECL void -xml_wiarchive_impl::load(std::string & s){ - std::wstring ws; - bool result = gimpl->parse_string(is, ws); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != s.data()) - #endif - s.resize(0); - s.reserve(ws.size()); - std::copy( - iterators::mb_from_wchar( - ws.begin() - ), - iterators::mb_from_wchar( - ws.end() - ), - std::back_inserter(s) - ); -} - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_WARCHIVE_DECL void -xml_wiarchive_impl::load(std::wstring & ws){ - bool result = gimpl->parse_string(is, ws); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); -} -#endif - -template -BOOST_WARCHIVE_DECL void -xml_wiarchive_impl::load(char * s){ - std::wstring ws; - bool result = gimpl->parse_string(is, ws); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); - copy_to_ptr(s, ws); -} - -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_WARCHIVE_DECL void -xml_wiarchive_impl::load(wchar_t * ws){ - std::wstring twstring; - bool result = gimpl->parse_string(is, twstring); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); - std::memcpy(ws, twstring.c_str(), twstring.size()); - ws[twstring.size()] = L'\0'; -} -#endif - -template -BOOST_WARCHIVE_DECL void -xml_wiarchive_impl::load_override(class_name_type & t){ - const std::wstring & ws = gimpl->rv.class_name; - if(ws.size() > BOOST_SERIALIZATION_MAX_KEY_SIZE - 1) - boost::serialization::throw_exception( - archive_exception(archive_exception::invalid_class_name) - ); - copy_to_ptr(t, ws); -} - -template -BOOST_WARCHIVE_DECL void -xml_wiarchive_impl::init(){ - gimpl->init(is); - this->set_library_version( - library_version_type(gimpl->rv.version) - ); -} - -template -BOOST_WARCHIVE_DECL -xml_wiarchive_impl::xml_wiarchive_impl( - std::wistream &is_, - unsigned int flags -) : - basic_text_iprimitive( - is_, - true // don't change the codecvt - use the one below - ), - basic_xml_iarchive(flags), - gimpl(new xml_wgrammar()) -{ - if(0 == (flags & no_codecvt)){ - std::locale l = std::locale( - is_.getloc(), - new boost::archive::detail::utf8_codecvt_facet - ); - // libstdc++ crashes without this - is_.sync(); - is_.imbue(l); - } - if(0 == (flags & no_header)) - init(); -} - -template -BOOST_WARCHIVE_DECL -xml_wiarchive_impl::~xml_wiarchive_impl(){ - if(std::uncaught_exception()) - return; - if(0 == (this->get_flags() & no_header)){ - gimpl->windup(is); - } -} - -} // namespace archive -} // namespace boost - -#endif // BOOST_NO_STD_WSTREAMBUF diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_woarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_woarchive_impl.ipp deleted file mode 100644 index 01b1a052d51..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_woarchive_impl.ipp +++ /dev/null @@ -1,171 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_woarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include -#ifndef BOOST_NO_STD_WSTREAMBUF - -#include -#include -#include // std::copy -#include -#include - -#include // strlen -#include // mbtowc -#ifndef BOOST_NO_CWCHAR -#include // wcslen -#endif - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::strlen; - #if ! defined(BOOST_NO_INTRINSIC_WCHAR_T) - using ::mbtowc; - using ::wcslen; - #endif -} // namespace std -#endif - -#include -#include - -#include - -#include -#include -#include -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implemenations of functions specific to wide char archives - -// copy chars to output escaping to xml and widening characters as we go -template -void save_iterator(std::wostream &os, InputIterator begin, InputIterator end){ - typedef iterators::wchar_from_mb< - iterators::xml_escape - > xmbtows; - std::copy( - xmbtows(begin), - xmbtows(end), - boost::archive::iterators::ostream_iterator(os) - ); -} - -template -BOOST_WARCHIVE_DECL void -xml_woarchive_impl::save(const std::string & s){ - // note: we don't use s.begin() and s.end() because dinkumware - // doesn't have string::value_type defined. So use a wrapper - // around these values to implement the definitions. - const char * begin = s.data(); - const char * end = begin + s.size(); - save_iterator(os, begin, end); -} - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_WARCHIVE_DECL void -xml_woarchive_impl::save(const std::wstring & ws){ -#if 0 - typedef iterators::xml_escape xmbtows; - std::copy( - xmbtows(ws.begin()), - xmbtows(ws.end()), - boost::archive::iterators::ostream_iterator(os) - ); -#endif - typedef iterators::xml_escape xmbtows; - std::copy( - xmbtows(ws.data()), - xmbtows(ws.data() + ws.size()), - boost::archive::iterators::ostream_iterator(os) - ); -} -#endif //BOOST_NO_STD_WSTRING - -template -BOOST_WARCHIVE_DECL void -xml_woarchive_impl::save(const char * s){ - save_iterator(os, s, s + std::strlen(s)); -} - -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_WARCHIVE_DECL void -xml_woarchive_impl::save(const wchar_t * ws){ - os << ws; - typedef iterators::xml_escape xmbtows; - std::copy( - xmbtows(ws), - xmbtows(ws + std::wcslen(ws)), - boost::archive::iterators::ostream_iterator(os) - ); -} -#endif - -template -BOOST_WARCHIVE_DECL -xml_woarchive_impl::xml_woarchive_impl( - std::wostream & os_, - unsigned int flags -) : - basic_text_oprimitive( - os_, - true // don't change the codecvt - use the one below - ), - basic_xml_oarchive(flags) -{ - if(0 == (flags & no_codecvt)){ - std::locale l = std::locale( - os_.getloc(), - new boost::archive::detail::utf8_codecvt_facet - ); - os_.flush(); - os_.imbue(l); - } - if(0 == (flags & no_header)) - this->init(); -} - -template -BOOST_WARCHIVE_DECL -xml_woarchive_impl::~xml_woarchive_impl(){ - if(std::uncaught_exception()) - return; - if(0 == (this->get_flags() & no_header)){ - save(L"\n"); - } -} - -template -BOOST_WARCHIVE_DECL void -xml_woarchive_impl::save_binary( - const void *address, - std::size_t count -){ - this->end_preamble(); - #if ! defined(__MWERKS__) - this->basic_text_oprimitive::save_binary( - #else - this->basic_text_oprimitive::save_binary( - #endif - address, - count - ); - this->indent_next = true; -} - -} // namespace archive -} // namespace boost - -#endif //BOOST_NO_STD_WSTREAMBUF diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_exception.hpp deleted file mode 100644 index 8f9208b60ea..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_exception.hpp +++ /dev/null @@ -1,68 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_BASE64_EXCEPTION_HPP -#define BOOST_ARCHIVE_ITERATORS_BASE64_EXCEPTION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// base64_exception.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifndef BOOST_NO_EXCEPTIONS -#include - -#include - -namespace boost { -namespace archive { -namespace iterators { - -////////////////////////////////////////////////////////////////////// -// exceptions thrown by base64s -// -class base64_exception : public std::exception -{ -public: - typedef enum { - invalid_code, // attempt to encode a value > 6 bits - invalid_character, // decode a value not in base64 char set - other_exception - } exception_code; - exception_code code; - - base64_exception(exception_code c = other_exception) : code(c) - {} - - virtual const char *what( ) const throw( ) - { - const char *msg = "unknown exception code"; - switch(code){ - case invalid_code: - msg = "attempt to encode a value > 6 bits"; - break; - case invalid_character: - msg = "attempt to decode a value not in base64 char set"; - break; - default: - BOOST_ASSERT(false); - break; - } - return msg; - } -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif //BOOST_NO_EXCEPTIONS -#endif //BOOST_ARCHIVE_ITERATORS_ARCHIVE_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_from_binary.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_from_binary.hpp deleted file mode 100644 index ee849944397..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_from_binary.hpp +++ /dev/null @@ -1,109 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_BASE64_FROM_BINARY_HPP -#define BOOST_ARCHIVE_ITERATORS_BASE64_FROM_BINARY_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// base64_from_binary.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include // size_t -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// convert binary integers to base64 characters - -namespace detail { - -template -struct from_6_bit { - typedef CharType result_type; - CharType operator()(CharType t) const{ - static const char * lookup_table = - "ABCDEFGHIJKLMNOPQRSTUVWXYZ" - "abcdefghijklmnopqrstuvwxyz" - "0123456789" - "+/"; - BOOST_ASSERT(t < 64); - return lookup_table[static_cast(t)]; - } -}; - -} // namespace detail - -// note: what we would like to do is -// template -// typedef transform_iterator< -// from_6_bit, -// transform_width -// > base64_from_binary; -// but C++ won't accept this. Rather than using a "type generator" and -// using a different syntax, make a derivation which should be equivalent. -// -// Another issue addressed here is that the transform_iterator doesn't have -// a templated constructor. This makes it incompatible with the dataflow -// ideal. This is also addressed here. - -//template -template< - class Base, - class CharType = typename boost::iterator_value::type -> -class base64_from_binary : - public transform_iterator< - detail::from_6_bit, - Base - > -{ - friend class boost::iterator_core_access; - typedef transform_iterator< - typename detail::from_6_bit, - Base - > super_t; - -public: - // make composible buy using templated constructor - template - base64_from_binary(T start) : - super_t( - Base(static_cast< T >(start)), - detail::from_6_bit() - ) - {} - // intel 7.1 doesn't like default copy constructor - base64_from_binary(const base64_from_binary & rhs) : - super_t( - Base(rhs.base_reference()), - detail::from_6_bit() - ) - {} -// base64_from_binary(){}; -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_BASE64_FROM_BINARY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/binary_from_base64.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/binary_from_base64.hpp deleted file mode 100644 index 89b8f889da3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/binary_from_base64.hpp +++ /dev/null @@ -1,118 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_BINARY_FROM_BASE64_HPP -#define BOOST_ARCHIVE_ITERATORS_BINARY_FROM_BASE64_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_from_base64.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// convert base64 characters to binary data - -namespace detail { - -template -struct to_6_bit { - typedef CharType result_type; - CharType operator()(CharType t) const{ - static const signed char lookup_table[] = { - -1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1, - -1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1, - -1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,62,-1,-1,-1,63, - 52,53,54,55,56,57,58,59,60,61,-1,-1,-1, 0,-1,-1, // render '=' as 0 - -1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9,10,11,12,13,14, - 15,16,17,18,19,20,21,22,23,24,25,-1,-1,-1,-1,-1, - -1,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,-1,-1,-1,-1,-1 - }; - // metrowerks trips this assertion - how come? - #if ! defined(__MWERKS__) - BOOST_STATIC_ASSERT(128 == sizeof(lookup_table)); - #endif - signed char value = -1; - if((unsigned)t <= 127) - value = lookup_table[(unsigned)t]; - if(-1 == value) - boost::serialization::throw_exception( - dataflow_exception(dataflow_exception::invalid_base64_character) - ); - return value; - } -}; - -} // namespace detail - -// note: what we would like to do is -// template -// typedef transform_iterator< -// from_6_bit, -// transform_width -// > base64_from_binary; -// but C++ won't accept this. Rather than using a "type generator" and -// using a different syntax, make a derivation which should be equivalent. -// -// Another issue addressed here is that the transform_iterator doesn't have -// a templated constructor. This makes it incompatible with the dataflow -// ideal. This is also addressed here. - -template< - class Base, - class CharType = typename boost::iterator_value::type -> -class binary_from_base64 : public - transform_iterator< - detail::to_6_bit, - Base - > -{ - friend class boost::iterator_core_access; - typedef transform_iterator< - detail::to_6_bit, - Base - > super_t; -public: - // make composible buy using templated constructor - template - binary_from_base64(T start) : - super_t( - Base(static_cast< T >(start)), - detail::to_6_bit() - ) - {} - // intel 7.1 doesn't like default copy constructor - binary_from_base64(const binary_from_base64 & rhs) : - super_t( - Base(rhs.base_reference()), - detail::to_6_bit() - ) - {} -// binary_from_base64(){}; -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_BINARY_FROM_BASE64_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow.hpp deleted file mode 100644 index 07733d5fd62..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow.hpp +++ /dev/null @@ -1,102 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_DATAFLOW_HPP -#define BOOST_ARCHIVE_ITERATORS_DATAFLOW_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// dataflow.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -// poor man's tri-state -struct tri_state { - enum state_enum { - is_false = false, - is_true = true, - is_indeterminant - } m_state; - // convert to bool - operator bool (){ - BOOST_ASSERT(is_indeterminant != m_state); - return is_true == m_state ? true : false; - } - // assign from bool - tri_state & operator=(bool rhs) { - m_state = rhs ? is_true : is_false; - return *this; - } - tri_state(bool rhs) : - m_state(rhs ? is_true : is_false) - {} - tri_state(state_enum state) : - m_state(state) - {} - bool operator==(const tri_state & rhs) const { - return m_state == rhs.m_state; - } - bool operator!=(const tri_state & rhs) const { - return m_state != rhs.m_state; - } -}; - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implement functions common to dataflow iterators -template -class dataflow { - bool m_eoi; -protected: - // test for iterator equality - tri_state equal(const Derived & rhs) const { - if(m_eoi && rhs.m_eoi) - return true; - if(m_eoi || rhs.m_eoi) - return false; - return tri_state(tri_state::is_indeterminant); - } - void eoi(bool tf){ - m_eoi = tf; - } - bool eoi() const { - return m_eoi; - } -public: - dataflow(bool tf) : - m_eoi(tf) - {} - dataflow() : // used for iterator end - m_eoi(true) - {} -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_DATAFLOW_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow_exception.hpp deleted file mode 100644 index e3e18605b38..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow_exception.hpp +++ /dev/null @@ -1,80 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_DATAFLOW_EXCEPTION_HPP -#define BOOST_ARCHIVE_ITERATORS_DATAFLOW_EXCEPTION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// dataflow_exception.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifndef BOOST_NO_EXCEPTIONS -#include -#endif //BOOST_NO_EXCEPTIONS - -#include - -namespace boost { -namespace archive { -namespace iterators { - -////////////////////////////////////////////////////////////////////// -// exceptions thrown by dataflows -// -class dataflow_exception : public std::exception -{ -public: - typedef enum { - invalid_6_bitcode, - invalid_base64_character, - invalid_xml_escape_sequence, - comparison_not_permitted, - invalid_conversion, - other_exception - } exception_code; - exception_code code; - - dataflow_exception(exception_code c = other_exception) : code(c) - {} - - virtual const char *what( ) const throw( ) - { - const char *msg = "unknown exception code"; - switch(code){ - case invalid_6_bitcode: - msg = "attempt to encode a value > 6 bits"; - break; - case invalid_base64_character: - msg = "attempt to decode a value not in base64 char set"; - break; - case invalid_xml_escape_sequence: - msg = "invalid xml escape_sequence"; - break; - case comparison_not_permitted: - msg = "cannot invoke iterator comparison now"; - break; - case invalid_conversion: - msg = "invalid multbyte/wide char conversion"; - break; - default: - BOOST_ASSERT(false); - break; - } - return msg; - } -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif //BOOST_ARCHIVE_ITERATORS_DATAFLOW_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/escape.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/escape.hpp deleted file mode 100644 index 103b31e0fef..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/escape.hpp +++ /dev/null @@ -1,115 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_ESCAPE_HPP -#define BOOST_ARCHIVE_ITERATORS_ESCAPE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// escape.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// insert escapes into text - -template -class escape : - public boost::iterator_adaptor< - Derived, - Base, - typename boost::iterator_value::type, - single_pass_traversal_tag, - typename boost::iterator_value::type - > -{ - typedef typename boost::iterator_value::type base_value_type; - typedef typename boost::iterator_reference::type reference_type; - friend class boost::iterator_core_access; - - typedef typename boost::iterator_adaptor< - Derived, - Base, - base_value_type, - single_pass_traversal_tag, - base_value_type - > super_t; - - typedef escape this_t; - - void dereference_impl() { - m_current_value = static_cast(this)->fill(m_bnext, m_bend); - m_full = true; - } - - //Access the value referred to - reference_type dereference() const { - if(!m_full) - const_cast(this)->dereference_impl(); - return m_current_value; - } - - bool equal(const this_t & rhs) const { - if(m_full){ - if(! rhs.m_full) - const_cast(& rhs)->dereference_impl(); - } - else{ - if(rhs.m_full) - const_cast(this)->dereference_impl(); - } - if(m_bnext != rhs.m_bnext) - return false; - if(this->base_reference() != rhs.base_reference()) - return false; - return true; - } - - void increment(){ - if(++m_bnext < m_bend){ - m_current_value = *m_bnext; - return; - } - ++(this->base_reference()); - m_bnext = NULL; - m_bend = NULL; - m_full = false; - } - - // buffer to handle pending characters - const base_value_type *m_bnext; - const base_value_type *m_bend; - bool m_full; - base_value_type m_current_value; -public: - escape(Base base) : - super_t(base), - m_bnext(NULL), - m_bend(NULL), - m_full(false), - m_current_value(0) - { - } -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_ESCAPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/insert_linebreaks.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/insert_linebreaks.hpp deleted file mode 100644 index 2504b030db1..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/insert_linebreaks.hpp +++ /dev/null @@ -1,99 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_INSERT_LINEBREAKS_HPP -#define BOOST_ARCHIVE_ITERATORS_INSERT_LINEBREAKS_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// insert_linebreaks.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ using ::memcpy; } -#endif - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// insert line break every N characters -template< - class Base, - int N, - class CharType = typename boost::iterator_value::type -> -class insert_linebreaks : - public iterator_adaptor< - insert_linebreaks, - Base, - CharType, - single_pass_traversal_tag, - CharType - > -{ -private: - friend class boost::iterator_core_access; - typedef iterator_adaptor< - insert_linebreaks, - Base, - CharType, - single_pass_traversal_tag, - CharType - > super_t; - - bool equal(const insert_linebreaks & rhs) const { - return -// m_count == rhs.m_count -// && base_reference() == rhs.base_reference() - this->base_reference() == rhs.base_reference() - ; - } - - void increment() { - if(m_count == N){ - m_count = 0; - return; - } - ++m_count; - ++(this->base_reference()); - } - CharType dereference() const { - if(m_count == N) - return '\n'; - return * (this->base_reference()); - } - unsigned int m_count; -public: - // make composible buy using templated constructor - template - insert_linebreaks(T start) : - super_t(Base(static_cast< T >(start))), - m_count(0) - {} - // intel 7.1 doesn't like default copy constructor - insert_linebreaks(const insert_linebreaks & rhs) : - super_t(rhs.base_reference()), - m_count(rhs.m_count) - {} -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_INSERT_LINEBREAKS_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/istream_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/istream_iterator.hpp deleted file mode 100644 index a187f605e69..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/istream_iterator.hpp +++ /dev/null @@ -1,92 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_ISTREAM_ITERATOR_HPP -#define BOOST_ARCHIVE_ITERATORS_ISTREAM_ITERATOR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// istream_iterator.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// note: this is a custom version of the standard istream_iterator. -// This is necessary as the standard version doesn't work as expected -// for wchar_t based streams on systems for which wchar_t not a true -// type but rather a synonym for some integer type. - -#include // NULL -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -// given a type, make an input iterator based on a pointer to that type -template -class istream_iterator : - public boost::iterator_facade< - istream_iterator, - Elem, - std::input_iterator_tag, - Elem - > -{ - friend class boost::iterator_core_access; - typedef istream_iterator this_t ; - typedef typename boost::iterator_facade< - istream_iterator, - Elem, - std::input_iterator_tag, - Elem - > super_t; - typedef typename std::basic_istream istream_type; - - bool equal(const this_t & rhs) const { - // note: only works for comparison against end of stream - return m_istream == rhs.m_istream; - } - - //Access the value referred to - Elem dereference() const { - return static_cast(m_istream->peek()); - } - - void increment(){ - if(NULL != m_istream){ - m_istream->ignore(1); - } - } - - istream_type *m_istream; - Elem m_current_value; -public: - istream_iterator(istream_type & is) : - m_istream(& is) - { - //increment(); - } - - istream_iterator() : - m_istream(NULL), - m_current_value(NULL) - {} - - istream_iterator(const istream_iterator & rhs) : - m_istream(rhs.m_istream), - m_current_value(rhs.m_current_value) - {} -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_ISTREAM_ITERATOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/mb_from_wchar.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/mb_from_wchar.hpp deleted file mode 100644 index 05df71c258e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/mb_from_wchar.hpp +++ /dev/null @@ -1,139 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_MB_FROM_WCHAR_HPP -#define BOOST_ARCHIVE_ITERATORS_MB_FROM_WCHAR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// mb_from_wchar.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // size_t -#ifndef BOOST_NO_CWCHAR -#include // mbstate_t -#endif -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::mbstate_t; -} // namespace std -#endif - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// class used by text archives to translate wide strings and to char -// strings of the currently selected locale -template // the input iterator -class mb_from_wchar - : public boost::iterator_adaptor< - mb_from_wchar, - Base, - wchar_t, - single_pass_traversal_tag, - char - > -{ - friend class boost::iterator_core_access; - - typedef typename boost::iterator_adaptor< - mb_from_wchar, - Base, - wchar_t, - single_pass_traversal_tag, - char - > super_t; - - typedef mb_from_wchar this_t; - - char dereference_impl() { - if(! m_full){ - fill(); - m_full = true; - } - return m_buffer[m_bnext]; - } - - char dereference() const { - return (const_cast(this))->dereference_impl(); - } - // test for iterator equality - bool equal(const mb_from_wchar & rhs) const { - // once the value is filled, the base_reference has been incremented - // so don't permit comparison anymore. - return - 0 == m_bend - && 0 == m_bnext - && this->base_reference() == rhs.base_reference() - ; - } - - void fill(){ - wchar_t value = * this->base_reference(); - const wchar_t *wend; - char *bend; - std::codecvt_base::result r = m_codecvt_facet.out( - m_mbs, - & value, & value + 1, wend, - m_buffer, m_buffer + sizeof(m_buffer), bend - ); - BOOST_ASSERT(std::codecvt_base::ok == r); - m_bnext = 0; - m_bend = bend - m_buffer; - } - - void increment(){ - if(++m_bnext < m_bend) - return; - m_bend = - m_bnext = 0; - ++(this->base_reference()); - m_full = false; - } - - boost::archive::detail::utf8_codecvt_facet m_codecvt_facet; - std::mbstate_t m_mbs; - // buffer to handle pending characters - char m_buffer[9 /* MB_CUR_MAX */]; - std::size_t m_bend; - std::size_t m_bnext; - bool m_full; - -public: - // make composible buy using templated constructor - template - mb_from_wchar(T start) : - super_t(Base(static_cast< T >(start))), - m_mbs(std::mbstate_t()), - m_bend(0), - m_bnext(0), - m_full(false) - {} - // intel 7.1 doesn't like default copy constructor - mb_from_wchar(const mb_from_wchar & rhs) : - super_t(rhs.base_reference()), - m_bend(rhs.m_bend), - m_bnext(rhs.m_bnext), - m_full(rhs.m_full) - {} -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_MB_FROM_WCHAR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/ostream_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/ostream_iterator.hpp deleted file mode 100644 index 49a9b99034b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/ostream_iterator.hpp +++ /dev/null @@ -1,83 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_OSTREAM_ITERATOR_HPP -#define BOOST_ARCHIVE_ITERATORS_OSTREAM_ITERATOR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// ostream_iterator.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// note: this is a custom version of the standard ostream_iterator. -// This is necessary as the standard version doesn't work as expected -// for wchar_t based streams on systems for which wchar_t not a true -// type but rather a synonym for some integer type. - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -// given a type, make an input iterator based on a pointer to that type -template -class ostream_iterator : - public boost::iterator_facade< - ostream_iterator, - Elem, - std::output_iterator_tag, - ostream_iterator & - > -{ - friend class boost::iterator_core_access; - typedef ostream_iterator this_t ; - typedef Elem char_type; - typedef std::basic_ostream ostream_type; - - //emulate the behavior of std::ostream - ostream_iterator & dereference() const { - return const_cast(*this); - } - bool equal(const this_t & rhs) const { - return m_ostream == rhs.m_ostream; - } - void increment(){} -protected: - ostream_type *m_ostream; - void put_val(char_type e){ - if(NULL != m_ostream){ - m_ostream->put(e); - if(! m_ostream->good()) - m_ostream = NULL; - } - } -public: - this_t & operator=(char_type c){ - put_val(c); - return *this; - } - ostream_iterator(ostream_type & os) : - m_ostream (& os) - {} - ostream_iterator() : - m_ostream (NULL) - {} - ostream_iterator(const ostream_iterator & rhs) : - m_ostream (rhs.m_ostream) - {} -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_OSTREAM_ITERATOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/remove_whitespace.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/remove_whitespace.hpp deleted file mode 100644 index c3580ab258a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/remove_whitespace.hpp +++ /dev/null @@ -1,167 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_REMOVE_WHITESPACE_HPP -#define BOOST_ARCHIVE_ITERATORS_REMOVE_WHITESPACE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// remove_whitespace.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include -#include - -// here is the default standard implementation of the functor used -// by the filter iterator to remove spaces. Unfortunately usage -// of this implementation in combination with spirit trips a bug -// VC 6.5. The only way I can find to work around it is to -// implement a special non-standard version for this platform - -#ifndef BOOST_NO_CWCTYPE -#include // iswspace -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ using ::iswspace; } -#endif -#endif - -#include // isspace -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ using ::isspace; } -#endif - -#if defined(__STD_RWCOMPILER_H__) || defined(_RWSTD_VER) -// this is required for the RW STL on Linux and Tru64. -#undef isspace -#undef iswspace -#endif - -namespace { // anonymous - -template -struct remove_whitespace_predicate; - -template<> -struct remove_whitespace_predicate -{ - bool operator()(unsigned char t){ - return ! std::isspace(t); - } -}; - -#ifndef BOOST_NO_CWCHAR -template<> -struct remove_whitespace_predicate -{ - bool operator()(wchar_t t){ - return ! std::iswspace(t); - } -}; -#endif - -} // namespace anonymous - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// convert base64 file data (including whitespace and padding) to binary - -namespace boost { -namespace archive { -namespace iterators { - -// custom version of filter iterator which doesn't look ahead further than -// necessary - -template -class filter_iterator - : public boost::iterator_adaptor< - filter_iterator, - Base, - use_default, - single_pass_traversal_tag - > -{ - friend class boost::iterator_core_access; - typedef typename boost::iterator_adaptor< - filter_iterator, - Base, - use_default, - single_pass_traversal_tag - > super_t; - typedef filter_iterator this_t; - typedef typename super_t::reference reference_type; - - reference_type dereference_impl(){ - if(! m_full){ - while(! m_predicate(* this->base_reference())) - ++(this->base_reference()); - m_full = true; - } - return * this->base_reference(); - } - - reference_type dereference() const { - return const_cast(this)->dereference_impl(); - } - - Predicate m_predicate; - bool m_full; -public: - // note: this function is public only because comeau compiler complained - // I don't know if this is because the compiler is wrong or what - void increment(){ - m_full = false; - ++(this->base_reference()); - } - filter_iterator(Base start) : - super_t(start), - m_full(false) - {} - filter_iterator(){} -}; - -template -class remove_whitespace : - public filter_iterator< - remove_whitespace_predicate< - typename boost::iterator_value::type - //typename Base::value_type - >, - Base - > -{ - friend class boost::iterator_core_access; - typedef filter_iterator< - remove_whitespace_predicate< - typename boost::iterator_value::type - //typename Base::value_type - >, - Base - > super_t; -public: -// remove_whitespace(){} // why is this needed? - // make composible buy using templated constructor - template - remove_whitespace(T start) : - super_t(Base(static_cast< T >(start))) - {} - // intel 7.1 doesn't like default copy constructor - remove_whitespace(const remove_whitespace & rhs) : - super_t(rhs.base_reference()) - {} -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_REMOVE_WHITESPACE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/transform_width.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/transform_width.hpp deleted file mode 100644 index 09c050a9274..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/transform_width.hpp +++ /dev/null @@ -1,177 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_TRANSFORM_WIDTH_HPP -#define BOOST_ARCHIVE_ITERATORS_TRANSFORM_WIDTH_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// transform_width.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// iterator which takes elements of x bits and returns elements of y bits. -// used to change streams of 8 bit characters into streams of 6 bit characters. -// and vice-versa for implementing base64 encodeing/decoding. Be very careful -// when using and end iterator. end is only reliable detected when the input -// stream length is some common multiple of x and y. E.G. Base64 6 bit -// character and 8 bit bytes. Lowest common multiple is 24 => 4 6 bit characters -// or 3 8 bit characters - -#include -#include - -#include // std::min - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// class used by text archives to translate char strings to wchar_t -// strings of the currently selected locale -template< - class Base, - int BitsOut, - int BitsIn, - class CharType = typename boost::iterator_value::type // output character -> -class transform_width : - public boost::iterator_adaptor< - transform_width, - Base, - CharType, - single_pass_traversal_tag, - CharType - > -{ - friend class boost::iterator_core_access; - typedef typename boost::iterator_adaptor< - transform_width, - Base, - CharType, - single_pass_traversal_tag, - CharType - > super_t; - - typedef transform_width this_t; - typedef typename iterator_value::type base_value_type; - - void fill(); - - CharType dereference() const { - if(!m_buffer_out_full) - const_cast(this)->fill(); - return m_buffer_out; - } - - bool equal_impl(const this_t & rhs){ - if(BitsIn < BitsOut) // discard any left over bits - return this->base_reference() == rhs.base_reference(); - else{ - // BitsIn > BitsOut // zero fill - if(this->base_reference() == rhs.base_reference()){ - m_end_of_sequence = true; - return 0 == m_remaining_bits; - } - return false; - } - } - - // standard iterator interface - bool equal(const this_t & rhs) const { - return const_cast(this)->equal_impl(rhs); - } - - void increment(){ - m_buffer_out_full = false; - } - - bool m_buffer_out_full; - CharType m_buffer_out; - - // last read element from input - base_value_type m_buffer_in; - - // number of bits to left in the input buffer. - unsigned int m_remaining_bits; - - // flag to indicate we've reached end of data. - bool m_end_of_sequence; - -public: - // make composible buy using templated constructor - template - transform_width(T start) : - super_t(Base(static_cast< T >(start))), - m_buffer_out_full(false), - m_buffer_out(0), - // To disable GCC warning, but not truly necessary - //(m_buffer_in will be initialized later before being - //used because m_remaining_bits == 0) - m_buffer_in(0), - m_remaining_bits(0), - m_end_of_sequence(false) - {} - // intel 7.1 doesn't like default copy constructor - transform_width(const transform_width & rhs) : - super_t(rhs.base_reference()), - m_buffer_out_full(rhs.m_buffer_out_full), - m_buffer_out(rhs.m_buffer_out), - m_buffer_in(rhs.m_buffer_in), - m_remaining_bits(rhs.m_remaining_bits), - m_end_of_sequence(false) - {} -}; - -template< - class Base, - int BitsOut, - int BitsIn, - class CharType -> -void transform_width::fill() { - unsigned int missing_bits = BitsOut; - m_buffer_out = 0; - do{ - if(0 == m_remaining_bits){ - if(m_end_of_sequence){ - m_buffer_in = 0; - m_remaining_bits = missing_bits; - } - else{ - m_buffer_in = * this->base_reference()++; - m_remaining_bits = BitsIn; - } - } - - // append these bits to the next output - // up to the size of the output - unsigned int i = (std::min)(missing_bits, m_remaining_bits); - // shift interesting bits to least significant position - base_value_type j = m_buffer_in >> (m_remaining_bits - i); - // and mask off the un interesting higher bits - // note presumption of twos complement notation - j &= (1 << i) - 1; - // append then interesting bits to the output value - m_buffer_out <<= i; - m_buffer_out |= j; - - // and update counters - missing_bits -= i; - m_remaining_bits -= i; - }while(0 < missing_bits); - m_buffer_out_full = true; -} - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_TRANSFORM_WIDTH_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/unescape.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/unescape.hpp deleted file mode 100644 index abf62406088..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/unescape.hpp +++ /dev/null @@ -1,89 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_UNESCAPE_HPP -#define BOOST_ARCHIVE_ITERATORS_UNESCAPE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// unescape.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// class used by text archives to translate char strings to wchar_t -// strings of the currently selected locale -template -class unescape - : public boost::iterator_adaptor< - unescape, - Base, - typename pointee::type, - single_pass_traversal_tag, - typename pointee::type - > -{ - friend class boost::iterator_core_access; - typedef typename boost::iterator_adaptor< - unescape, - Base, - typename pointee::type, - single_pass_traversal_tag, - typename pointee::type - > super_t; - - typedef unescape this_t; -public: - typedef typename this_t::value_type value_type; - typedef typename this_t::reference reference; -private: - value_type dereference_impl() { - if(! m_full){ - m_current_value = static_cast(this)->drain(); - m_full = true; - } - return m_current_value; - } - - reference dereference() const { - return const_cast(this)->dereference_impl(); - } - - value_type m_current_value; - bool m_full; - - void increment(){ - ++(this->base_reference()); - dereference_impl(); - m_full = false; - }; - -public: - - unescape(Base base) : - super_t(base), - m_full(false) - {} - -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_UNESCAPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/wchar_from_mb.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/wchar_from_mb.hpp deleted file mode 100644 index 2af8f6401f2..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/wchar_from_mb.hpp +++ /dev/null @@ -1,194 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_WCHAR_FROM_MB_HPP -#define BOOST_ARCHIVE_ITERATORS_WCHAR_FROM_MB_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// wchar_from_mb.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include // size_t -#ifndef BOOST_NO_CWCHAR -#include // mbstate_t -#endif -#include // copy - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::mbstate_t; -} // namespace std -#endif -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// class used by text archives to translate char strings to wchar_t -// strings of the currently selected locale -template -class wchar_from_mb - : public boost::iterator_adaptor< - wchar_from_mb, - Base, - wchar_t, - single_pass_traversal_tag, - wchar_t - > -{ - friend class boost::iterator_core_access; - typedef typename boost::iterator_adaptor< - wchar_from_mb, - Base, - wchar_t, - single_pass_traversal_tag, - wchar_t - > super_t; - - typedef wchar_from_mb this_t; - - void drain(); - - wchar_t dereference() const { - if(m_output.m_next == m_output.m_next_available) - return static_cast(0); - return * m_output.m_next; - } - - void increment(){ - if(m_output.m_next == m_output.m_next_available) - return; - if(++m_output.m_next == m_output.m_next_available){ - if(m_input.m_done) - return; - drain(); - } - } - - bool equal(this_t const & rhs) const { - return dereference() == rhs.dereference(); - } - - boost::archive::detail::utf8_codecvt_facet m_codecvt_facet; - std::mbstate_t m_mbs; - - template - struct sliding_buffer { - boost::array m_buffer; - typename boost::array::const_iterator m_next_available; - typename boost::array::iterator m_next; - bool m_done; - // default ctor - sliding_buffer() : - m_next_available(m_buffer.begin()), - m_next(m_buffer.begin()), - m_done(false) - {} - // copy ctor - sliding_buffer(const sliding_buffer & rhs) : - m_next_available( - std::copy( - rhs.m_buffer.begin(), - rhs.m_next_available, - m_buffer.begin() - ) - ), - m_next( - m_buffer.begin() + (rhs.m_next - rhs.m_buffer.begin()) - ), - m_done(rhs.m_done) - {} - }; - - sliding_buffer::type> m_input; - sliding_buffer::type> m_output; - -public: - // make composible buy using templated constructor - template - wchar_from_mb(T start) : - super_t(Base(static_cast< T >(start))), - m_mbs(std::mbstate_t()) - { - BOOST_ASSERT(std::mbsinit(&m_mbs)); - drain(); - } - // default constructor used as an end iterator - wchar_from_mb(){} - - // copy ctor - wchar_from_mb(const wchar_from_mb & rhs) : - super_t(rhs.base_reference()), - m_mbs(rhs.m_mbs), - m_input(rhs.m_input), - m_output(rhs.m_output) - {} -}; - -template -void wchar_from_mb::drain(){ - BOOST_ASSERT(! m_input.m_done); - for(;;){ - typename boost::iterators::iterator_reference::type c = *(this->base_reference()); - // a null character in a multibyte stream is takes as end of string - if(0 == c){ - m_input.m_done = true; - break; - } - ++(this->base_reference()); - * const_cast::type *>( - (m_input.m_next_available++) - ) = c; - // if input buffer is full - we're done for now - if(m_input.m_buffer.end() == m_input.m_next_available) - break; - } - const typename boost::iterators::iterator_value::type * input_new_start; - typename iterator_value::type * next_available; - - std::codecvt_base::result r = m_codecvt_facet.in( - m_mbs, - m_input.m_buffer.begin(), - m_input.m_next_available, - input_new_start, - m_output.m_buffer.begin(), - m_output.m_buffer.end(), - next_available - ); - BOOST_ASSERT(std::codecvt_base::ok == r); - m_output.m_next_available = next_available; - m_output.m_next = m_output.m_buffer.begin(); - - // we're done with some of the input so shift left. - m_input.m_next_available = std::copy( - input_new_start, - m_input.m_next_available, - m_input.m_buffer.begin() - ); - m_input.m_next = m_input.m_buffer.begin(); -} - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_WCHAR_FROM_MB_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_escape.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_escape.hpp deleted file mode 100644 index c838a73b864..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_escape.hpp +++ /dev/null @@ -1,121 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_XML_ESCAPE_HPP -#define BOOST_ARCHIVE_ITERATORS_XML_ESCAPE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_escape.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// insert escapes into xml text - -template -class xml_escape - : public escape, Base> -{ - friend class boost::iterator_core_access; - - typedef escape, Base> super_t; - -public: - char fill(const char * & bstart, const char * & bend); - wchar_t fill(const wchar_t * & bstart, const wchar_t * & bend); - - template - xml_escape(T start) : - super_t(Base(static_cast< T >(start))) - {} - // intel 7.1 doesn't like default copy constructor - xml_escape(const xml_escape & rhs) : - super_t(rhs.base_reference()) - {} -}; - -template -char xml_escape::fill( - const char * & bstart, - const char * & bend -){ - char current_value = * this->base_reference(); - switch(current_value){ - case '<': - bstart = "<"; - bend = bstart + 4; - break; - case '>': - bstart = ">"; - bend = bstart + 4; - break; - case '&': - bstart = "&"; - bend = bstart + 5; - break; - case '"': - bstart = """; - bend = bstart + 6; - break; - case '\'': - bstart = "'"; - bend = bstart + 6; - break; - default: - return current_value; - } - return *bstart; -} - -template -wchar_t xml_escape::fill( - const wchar_t * & bstart, - const wchar_t * & bend -){ - wchar_t current_value = * this->base_reference(); - switch(current_value){ - case '<': - bstart = L"<"; - bend = bstart + 4; - break; - case '>': - bstart = L">"; - bend = bstart + 4; - break; - case '&': - bstart = L"&"; - bend = bstart + 5; - break; - case '"': - bstart = L"""; - bend = bstart + 6; - break; - case '\'': - bstart = L"'"; - bend = bstart + 6; - break; - default: - return current_value; - } - return *bstart; -} - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_XML_ESCAPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape.hpp deleted file mode 100644 index 69977404567..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape.hpp +++ /dev/null @@ -1,125 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_HPP -#define BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_unescape.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// replace &??? xml escape sequences with the corresponding characters -template -class xml_unescape - : public unescape, Base> -{ - friend class boost::iterator_core_access; - typedef xml_unescape this_t; - typedef unescape super_t; - typedef typename boost::iterator_reference reference_type; - - reference_type dereference() const { - return unescape, Base>::dereference(); - } -public: - // workaround msvc 7.1 ICU crash - #if defined(BOOST_MSVC) - typedef int value_type; - #else - typedef typename this_t::value_type value_type; - #endif - - void drain_residue(const char *literal); - value_type drain(); - - template - xml_unescape(T start) : - super_t(Base(static_cast< T >(start))) - {} - // intel 7.1 doesn't like default copy constructor - xml_unescape(const xml_unescape & rhs) : - super_t(rhs.base_reference()) - {} -}; - -template -void xml_unescape::drain_residue(const char * literal){ - do{ - if(* literal != * ++(this->base_reference())) - boost::serialization::throw_exception( - dataflow_exception( - dataflow_exception::invalid_xml_escape_sequence - ) - ); - } - while('\0' != * ++literal); -} - -// note key constraint on this function is that can't "look ahead" any -// more than necessary into base iterator. Doing so would alter the base -// iterator refenence which would make subsequent iterator comparisons -// incorrect and thereby break the composiblity of iterators. -template -typename xml_unescape::value_type -//int -xml_unescape::drain(){ - value_type retval = * this->base_reference(); - if('&' != retval){ - return retval; - } - retval = * ++(this->base_reference()); - switch(retval){ - case 'l': // < - drain_residue("t;"); - retval = '<'; - break; - case 'g': // > - drain_residue("t;"); - retval = '>'; - break; - case 'a': - retval = * ++(this->base_reference()); - switch(retval){ - case 'p': // ' - drain_residue("os;"); - retval = '\''; - break; - case 'm': // & - drain_residue("p;"); - retval = '&'; - break; - } - break; - case 'q': - drain_residue("uot;"); - retval = '"'; - break; - } - return retval; -} - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape_exception.hpp deleted file mode 100644 index 71a64378c20..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape_exception.hpp +++ /dev/null @@ -1,49 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_EXCEPTION_HPP -#define BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_EXCEPTION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_unescape_exception.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifndef BOOST_NO_EXCEPTIONS -#include - -#include - -namespace boost { -namespace archive { -namespace iterators { - -////////////////////////////////////////////////////////////////////// -// exceptions thrown by xml_unescapes -// -class xml_unescape_exception : public std::exception -{ -public: - xml_unescape_exception() - {} - - virtual const char *what( ) const throw( ) - { - return "xml contained un-recognized escape code"; - } -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif //BOOST_NO_EXCEPTIONS -#endif //BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_iarchive.hpp deleted file mode 100644 index 4a898a8ad16..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_iarchive.hpp +++ /dev/null @@ -1,54 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_BINARY_IARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_BINARY_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_binary_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class polymorphic_binary_iarchive : - public detail::polymorphic_iarchive_route -{ -public: - polymorphic_binary_iarchive(std::istream & is, unsigned int flags = 0) : - detail::polymorphic_iarchive_route(is, flags) - {} - ~polymorphic_binary_iarchive(){} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_binary_iarchive -) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_BINARY_IARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_oarchive.hpp deleted file mode 100644 index 931b243feb8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_oarchive.hpp +++ /dev/null @@ -1,43 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_BINARY_OARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_BINARY_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_binary_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -namespace boost { -namespace archive { - -typedef detail::polymorphic_oarchive_route< - binary_oarchive_impl< - binary_oarchive, - std::ostream::char_type, - std::ostream::traits_type - > - > polymorphic_binary_oarchive; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_binary_oarchive -) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_BINARY_OARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_iarchive.hpp deleted file mode 100644 index d3c59a9f0f4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_iarchive.hpp +++ /dev/null @@ -1,168 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_IARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // std::size_t -#include // ULONG_MAX -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include - -#include -#include -#include -#include - -#include -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization -namespace archive { -namespace detail { - class basic_iarchive; - class basic_iserializer; -} - -class polymorphic_iarchive; - -class BOOST_SYMBOL_VISIBLE polymorphic_iarchive_impl : - public detail::interface_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else - friend class detail::interface_iarchive; - friend class load_access; -#endif - // primitive types the only ones permitted by polymorphic archives - virtual void load(bool & t) = 0; - - virtual void load(char & t) = 0; - virtual void load(signed char & t) = 0; - virtual void load(unsigned char & t) = 0; - #ifndef BOOST_NO_CWCHAR - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - virtual void load(wchar_t & t) = 0; - #endif - #endif - virtual void load(short & t) = 0; - virtual void load(unsigned short & t) = 0; - virtual void load(int & t) = 0; - virtual void load(unsigned int & t) = 0; - virtual void load(long & t) = 0; - virtual void load(unsigned long & t) = 0; - - #if defined(BOOST_HAS_LONG_LONG) - virtual void load(boost::long_long_type & t) = 0; - virtual void load(boost::ulong_long_type & t) = 0; - #elif defined(BOOST_HAS_MS_INT64) - virtual void load(__int64 & t) = 0; - virtual void load(unsigned __int64 & t) = 0; - #endif - - virtual void load(float & t) = 0; - virtual void load(double & t) = 0; - - // string types are treated as primitives - virtual void load(std::string & t) = 0; - #ifndef BOOST_NO_STD_WSTRING - virtual void load(std::wstring & t) = 0; - #endif - - // used for xml and other tagged formats - virtual void load_start(const char * name) = 0; - virtual void load_end(const char * name) = 0; - virtual void register_basic_serializer(const detail::basic_iserializer & bis) = 0; - virtual detail::helper_collection & get_helper_collection() = 0; - - // msvc and borland won't automatically pass these to the base class so - // make it explicit here - template - void load_override(T & t) - { - archive::load(* this->This(), t); - } - // special treatment for name-value pairs. - template - void load_override( - const boost::serialization::nvp< T > & t - ){ - load_start(t.name()); - archive::load(* this->This(), t.value()); - load_end(t.name()); - } -protected: - virtual ~polymorphic_iarchive_impl(){}; -public: - // utility function implemented by all legal archives - virtual void set_library_version(library_version_type archive_library_version) = 0; - virtual library_version_type get_library_version() const = 0; - virtual unsigned int get_flags() const = 0; - virtual void delete_created_pointers() = 0; - virtual void reset_object_address( - const void * new_address, - const void * old_address - ) = 0; - - virtual void load_binary(void * t, std::size_t size) = 0; - - // these are used by the serialization library implementation. - virtual void load_object( - void *t, - const detail::basic_iserializer & bis - ) = 0; - virtual const detail::basic_pointer_iserializer * load_pointer( - void * & t, - const detail::basic_pointer_iserializer * bpis_ptr, - const detail::basic_pointer_iserializer * (*finder)( - const boost::serialization::extended_type_info & type - ) - ) = 0; -}; - -} // namespace archive -} // namespace boost - -#include // pops abi_suffix.hpp pragmas - -namespace boost { -namespace archive { - -class BOOST_SYMBOL_VISIBLE polymorphic_iarchive : - public polymorphic_iarchive_impl -{ -public: - virtual ~polymorphic_iarchive(){}; -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::polymorphic_iarchive) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_oarchive.hpp deleted file mode 100644 index edac4edb1e8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_oarchive.hpp +++ /dev/null @@ -1,154 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_OARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // size_t -#include // ULONG_MAX -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include - -#include -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization -namespace archive { -namespace detail { - class basic_oarchive; - class basic_oserializer; -} - -class polymorphic_oarchive; - -class BOOST_SYMBOL_VISIBLE polymorphic_oarchive_impl : - public detail::interface_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else - friend class detail::interface_oarchive; - friend class save_access; -#endif - // primitive types the only ones permitted by polymorphic archives - virtual void save(const bool t) = 0; - - virtual void save(const char t) = 0; - virtual void save(const signed char t) = 0; - virtual void save(const unsigned char t) = 0; - #ifndef BOOST_NO_CWCHAR - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - virtual void save(const wchar_t t) = 0; - #endif - #endif - virtual void save(const short t) = 0; - virtual void save(const unsigned short t) = 0; - virtual void save(const int t) = 0; - virtual void save(const unsigned int t) = 0; - virtual void save(const long t) = 0; - virtual void save(const unsigned long t) = 0; - - #if defined(BOOST_HAS_LONG_LONG) - virtual void save(const boost::long_long_type t) = 0; - virtual void save(const boost::ulong_long_type t) = 0; - #elif defined(BOOST_HAS_MS_INT64) - virtual void save(const __int64 t) = 0; - virtual void save(const unsigned __int64 t) = 0; - #endif - - virtual void save(const float t) = 0; - virtual void save(const double t) = 0; - - // string types are treated as primitives - virtual void save(const std::string & t) = 0; - #ifndef BOOST_NO_STD_WSTRING - virtual void save(const std::wstring & t) = 0; - #endif - - virtual void save_null_pointer() = 0; - // used for xml and other tagged formats - virtual void save_start(const char * name) = 0; - virtual void save_end(const char * name) = 0; - virtual void register_basic_serializer(const detail::basic_oserializer & bos) = 0; - virtual detail::helper_collection & get_helper_collection() = 0; - - virtual void end_preamble() = 0; - - // msvc and borland won't automatically pass these to the base class so - // make it explicit here - template - void save_override(T & t) - { - archive::save(* this->This(), t); - } - // special treatment for name-value pairs. - template - void save_override( - const ::boost::serialization::nvp< T > & t - ){ - save_start(t.name()); - archive::save(* this->This(), t.const_value()); - save_end(t.name()); - } -protected: - virtual ~polymorphic_oarchive_impl(){}; -public: - // utility functions implemented by all legal archives - virtual unsigned int get_flags() const = 0; - virtual library_version_type get_library_version() const = 0; - virtual void save_binary(const void * t, std::size_t size) = 0; - - virtual void save_object( - const void *x, - const detail::basic_oserializer & bos - ) = 0; - virtual void save_pointer( - const void * t, - const detail::basic_pointer_oserializer * bpos_ptr - ) = 0; -}; - -// note: preserve naming symmetry -class BOOST_SYMBOL_VISIBLE polymorphic_oarchive : - public polymorphic_oarchive_impl -{ -public: - virtual ~polymorphic_oarchive(){}; -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::polymorphic_oarchive) - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_POLYMORPHIC_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_iarchive.hpp deleted file mode 100644 index 7bef2927865..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_iarchive.hpp +++ /dev/null @@ -1,54 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_TEXT_IARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_TEXT_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_text_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class polymorphic_text_iarchive : - public detail::polymorphic_iarchive_route -{ -public: - polymorphic_text_iarchive(std::istream & is, unsigned int flags = 0) : - detail::polymorphic_iarchive_route(is, flags) - {} - ~polymorphic_text_iarchive(){} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_text_iarchive -) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_TEXT_IARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_oarchive.hpp deleted file mode 100644 index 457aad9fd75..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_oarchive.hpp +++ /dev/null @@ -1,39 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_TEXT_OARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_TEXT_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_text_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -namespace boost { -namespace archive { - -typedef detail::polymorphic_oarchive_route< - text_oarchive_impl -> polymorphic_text_oarchive; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_text_oarchive -) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_TEXT_OARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_wiarchive.hpp deleted file mode 100644 index 8466f05d6a6..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_wiarchive.hpp +++ /dev/null @@ -1,59 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_TEXT_WIARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_TEXT_WIARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_text_wiarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class polymorphic_text_wiarchive : - public detail::polymorphic_iarchive_route -{ -public: - polymorphic_text_wiarchive(std::wistream & is, unsigned int flags = 0) : - detail::polymorphic_iarchive_route(is, flags) - {} - ~polymorphic_text_wiarchive(){} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_text_wiarchive -) - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_POLYMORPHIC_TEXT_WIARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_woarchive.hpp deleted file mode 100644 index 295625d1bcf..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_woarchive.hpp +++ /dev/null @@ -1,44 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_TEXT_WOARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_TEXT_WOARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_text_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include -#include - -namespace boost { -namespace archive { - -typedef detail::polymorphic_oarchive_route< - text_woarchive_impl -> polymorphic_text_woarchive; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_text_woarchive -) - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_POLYMORPHIC_TEXT_WOARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_iarchive.hpp deleted file mode 100644 index 4dc3f894b38..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_iarchive.hpp +++ /dev/null @@ -1,54 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_XML_IARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_XML_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_xml_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class polymorphic_xml_iarchive : - public detail::polymorphic_iarchive_route -{ -public: - polymorphic_xml_iarchive(std::istream & is, unsigned int flags = 0) : - detail::polymorphic_iarchive_route(is, flags) - {} - ~polymorphic_xml_iarchive(){} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_xml_iarchive -) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_XML_IARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_oarchive.hpp deleted file mode 100644 index 514f9e530a8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_oarchive.hpp +++ /dev/null @@ -1,39 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_XML_OARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_XML_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_xml_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -namespace boost { -namespace archive { - -typedef detail::polymorphic_oarchive_route< - xml_oarchive_impl -> polymorphic_xml_oarchive; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_xml_oarchive -) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_XML_OARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_wiarchive.hpp deleted file mode 100644 index d4ab731267f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_wiarchive.hpp +++ /dev/null @@ -1,50 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_XML_WIARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_XML_WIARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_xml_wiarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include -#include - -namespace boost { -namespace archive { - -class polymorphic_xml_wiarchive : - public detail::polymorphic_iarchive_route -{ -public: - polymorphic_xml_wiarchive(std::wistream & is, unsigned int flags = 0) : - detail::polymorphic_iarchive_route(is, flags) - {} - ~polymorphic_xml_wiarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_xml_wiarchive -) - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_POLYMORPHIC_XML_WIARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_woarchive.hpp deleted file mode 100644 index dd8963fbb14..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_woarchive.hpp +++ /dev/null @@ -1,44 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_XML_WOARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_XML_WOARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_xml_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include -#include - -namespace boost { -namespace archive { - -typedef detail::polymorphic_oarchive_route< - xml_woarchive_impl -> polymorphic_xml_woarchive; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_xml_woarchive -) - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_POLYMORPHIC_XML_WOARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/text_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/text_iarchive.hpp deleted file mode 100644 index d9d60adf0b8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/text_iarchive.hpp +++ /dev/null @@ -1,132 +0,0 @@ -#ifndef BOOST_ARCHIVE_TEXT_IARCHIVE_HPP -#define BOOST_ARCHIVE_TEXT_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE text_iarchive_impl : - public basic_text_iprimitive, - public basic_text_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_iarchive; - friend class load_access; -#endif - template - void load(T & t){ - basic_text_iprimitive::load(t); - } - void load(version_type & t){ - unsigned int v; - load(v); - t = version_type(v); - } - void load(boost::serialization::item_version_type & t){ - unsigned int v; - load(v); - t = boost::serialization::item_version_type(v); - } - BOOST_ARCHIVE_DECL void - load(char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_ARCHIVE_DECL void - load(wchar_t * t); - #endif - BOOST_ARCHIVE_DECL void - load(std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_ARCHIVE_DECL void - load(std::wstring &ws); - #endif - template - void load_override(T & t){ - basic_text_iarchive::load_override(t); - } - BOOST_ARCHIVE_DECL void - load_override(class_name_type & t); - BOOST_ARCHIVE_DECL void - init(); - BOOST_ARCHIVE_DECL - text_iarchive_impl(std::istream & is, unsigned int flags); - // don't import inline definitions! leave this as a reminder. - //BOOST_ARCHIVE_DECL - ~text_iarchive_impl(){}; -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class BOOST_SYMBOL_VISIBLE text_iarchive : - public text_iarchive_impl{ -public: - text_iarchive(std::istream & is_, unsigned int flags = 0) : - // note: added _ to suppress useless gcc warning - text_iarchive_impl(is_, flags) - {} - ~text_iarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::text_iarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_TEXT_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/text_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/text_oarchive.hpp deleted file mode 100644 index 9ba0dafffb4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/text_oarchive.hpp +++ /dev/null @@ -1,121 +0,0 @@ -#ifndef BOOST_ARCHIVE_TEXT_OARCHIVE_HPP -#define BOOST_ARCHIVE_TEXT_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // std::size_t - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE text_oarchive_impl : - /* protected ? */ public basic_text_oprimitive, - public basic_text_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_oarchive; - friend class basic_text_oarchive; - friend class save_access; -#endif - template - void save(const T & t){ - this->newtoken(); - basic_text_oprimitive::save(t); - } - void save(const version_type & t){ - save(static_cast(t)); - } - void save(const boost::serialization::item_version_type & t){ - save(static_cast(t)); - } - BOOST_ARCHIVE_DECL void - save(const char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_ARCHIVE_DECL void - save(const wchar_t * t); - #endif - BOOST_ARCHIVE_DECL void - save(const std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_ARCHIVE_DECL void - save(const std::wstring &ws); - #endif - BOOST_ARCHIVE_DECL - text_oarchive_impl(std::ostream & os, unsigned int flags); - // don't import inline definitions! leave this as a reminder. - //BOOST_ARCHIVE_DECL - ~text_oarchive_impl(){}; -public: - BOOST_ARCHIVE_DECL void - save_binary(const void *address, std::size_t count); -}; - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from text_oarchive_impl instead. This will -// preserve correct static polymorphism. -class BOOST_SYMBOL_VISIBLE text_oarchive : - public text_oarchive_impl -{ -public: - text_oarchive(std::ostream & os_, unsigned int flags = 0) : - // note: added _ to suppress useless gcc warning - text_oarchive_impl(os_, flags) - {} - ~text_oarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::text_oarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_TEXT_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/text_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/text_wiarchive.hpp deleted file mode 100644 index 3adf068a51a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/text_wiarchive.hpp +++ /dev/null @@ -1,137 +0,0 @@ -#ifndef BOOST_ARCHIVE_TEXT_WIARCHIVE_HPP -#define BOOST_ARCHIVE_TEXT_WIARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_wiarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include - -#include -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE text_wiarchive_impl : - public basic_text_iprimitive, - public basic_text_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_iarchive; - friend load_access; - #else - friend class detail::interface_iarchive; - friend class load_access; - #endif -#endif - template - void load(T & t){ - basic_text_iprimitive::load(t); - } - void load(version_type & t){ - unsigned int v; - load(v); - t = version_type(v); - } - void load(boost::serialization::item_version_type & t){ - unsigned int v; - load(v); - t = boost::serialization::item_version_type(v); - } - BOOST_WARCHIVE_DECL void - load(char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_WARCHIVE_DECL void - load(wchar_t * t); - #endif - BOOST_WARCHIVE_DECL void - load(std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_WARCHIVE_DECL void - load(std::wstring &ws); - #endif - template - void load_override(T & t){ - basic_text_iarchive::load_override(t); - } - BOOST_WARCHIVE_DECL - text_wiarchive_impl(std::wistream & is, unsigned int flags); - ~text_wiarchive_impl(){}; -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class BOOST_SYMBOL_VISIBLE text_wiarchive : - public text_wiarchive_impl{ -public: - text_wiarchive(std::wistream & is, unsigned int flags = 0) : - text_wiarchive_impl(is, flags) - {} - ~text_wiarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::text_wiarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_TEXT_WIARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/text_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/text_woarchive.hpp deleted file mode 100644 index b6b4f8ed59a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/text_woarchive.hpp +++ /dev/null @@ -1,155 +0,0 @@ -#ifndef BOOST_ARCHIVE_TEXT_WOARCHIVE_HPP -#define BOOST_ARCHIVE_TEXT_WOARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_woarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include -#include // size_t - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE text_woarchive_impl : - public basic_text_oprimitive, - public basic_text_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_oarchive; - friend basic_text_oarchive; - friend save_access; - #else - friend class detail::interface_oarchive; - friend class basic_text_oarchive; - friend class save_access; - #endif -#endif - template - void save(const T & t){ - this->newtoken(); - basic_text_oprimitive::save(t); - } - void save(const version_type & t){ - save(static_cast(t)); - } - void save(const boost::serialization::item_version_type & t){ - save(static_cast(t)); - } - BOOST_WARCHIVE_DECL void - save(const char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_WARCHIVE_DECL void - save(const wchar_t * t); - #endif - BOOST_WARCHIVE_DECL void - save(const std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_WARCHIVE_DECL void - save(const std::wstring &ws); - #endif - text_woarchive_impl(std::wostream & os, unsigned int flags) : - basic_text_oprimitive( - os, - 0 != (flags & no_codecvt) - ), - basic_text_oarchive(flags) - { - if(0 == (flags & no_header)) - basic_text_oarchive::init(); - } -public: - void save_binary(const void *address, std::size_t count){ - put(static_cast('\n')); - this->end_preamble(); - #if ! defined(__MWERKS__) - this->basic_text_oprimitive::save_binary( - #else - this->basic_text_oprimitive::save_binary( - #endif - address, - count - ); - put(static_cast('\n')); - this->delimiter = this->none; - } - -}; - -// we use the following because we can't use -// typedef text_oarchive_impl > text_oarchive; - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from text_oarchive_impl instead. This will -// preserve correct static polymorphism. -class BOOST_SYMBOL_VISIBLE text_woarchive : - public text_woarchive_impl -{ -public: - text_woarchive(std::wostream & os, unsigned int flags = 0) : - text_woarchive_impl(os, flags) - {} - ~text_woarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::text_woarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_TEXT_WOARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/tmpdir.hpp b/contrib/libboost/boost_1_65_0/boost/archive/tmpdir.hpp deleted file mode 100644 index 400d23b9f68..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/tmpdir.hpp +++ /dev/null @@ -1,50 +0,0 @@ -#ifndef BOOST_ARCHIVE_TMPDIR_HPP -#define BOOST_ARCHIVE_TMPDIR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// tmpdir.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // getenv -#include // NULL -//#include - -#include -#ifdef BOOST_NO_STDC_NAMESPACE -namespace std { - using ::getenv; -} -#endif - -namespace boost { -namespace archive { - -inline const char * tmpdir(){ - const char *dirname; - dirname = std::getenv("TMP"); - if(NULL == dirname) - dirname = std::getenv("TMPDIR"); - if(NULL == dirname) - dirname = std::getenv("TEMP"); - if(NULL == dirname){ - //BOOST_ASSERT(false); // no temp directory found - dirname = "."; - } - return dirname; -} - -} // archive -} // boost - -#endif // BOOST_ARCHIVE_TMPDIR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/wcslen.hpp b/contrib/libboost/boost_1_65_0/boost/archive/wcslen.hpp deleted file mode 100644 index 0b60004f095..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/wcslen.hpp +++ /dev/null @@ -1,58 +0,0 @@ -#ifndef BOOST_ARCHIVE_WCSLEN_HPP -#define BOOST_ARCHIVE_WCSLEN_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// wcslen.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // size_t -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#ifndef BOOST_NO_CWCHAR - -// a couple of libraries which include wchar_t don't include -// wcslen - -#if defined(BOOST_DINKUMWARE_STDLIB) && BOOST_DINKUMWARE_STDLIB < 306 \ -|| defined(__LIBCOMO__) - -namespace std { -inline std::size_t wcslen(const wchar_t * ws) -{ - const wchar_t * eows = ws; - while(* eows != 0) - ++eows; - return eows - ws; -} -} // namespace std - -#else - -#ifndef BOOST_NO_CWCHAR -#include -#endif -#ifdef BOOST_NO_STDC_NAMESPACE -namespace std{ using ::wcslen; } -#endif - -#endif // wcslen - -#endif //BOOST_NO_CWCHAR - -#endif //BOOST_ARCHIVE_WCSLEN_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_archive_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_archive_exception.hpp deleted file mode 100644 index 82c53ef5d3e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/xml_archive_exception.hpp +++ /dev/null @@ -1,57 +0,0 @@ -#ifndef BOOST_ARCHIVE_XML_ARCHIVE_EXCEPTION_HPP -#define BOOST_ARCHIVE_XML_ARCHIVE_EXCEPTION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_archive_exception.hpp: - -// (C) Copyright 2007 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include - -#include // must be the last header - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// exceptions thrown by xml archives -// -class BOOST_SYMBOL_VISIBLE xml_archive_exception : - public virtual boost::archive::archive_exception -{ -public: - typedef enum { - xml_archive_parsing_error, // see save_register - xml_archive_tag_mismatch, - xml_archive_tag_name_error - } exception_code; - BOOST_ARCHIVE_DECL xml_archive_exception( - exception_code c, - const char * e1 = NULL, - const char * e2 = NULL - ); - BOOST_ARCHIVE_DECL xml_archive_exception(xml_archive_exception const &) ; - virtual BOOST_ARCHIVE_DECL ~xml_archive_exception() BOOST_NOEXCEPT_OR_NOTHROW ; -}; - -}// namespace archive -}// namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif //BOOST_XML_ARCHIVE_ARCHIVE_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_iarchive.hpp deleted file mode 100644 index abd2f9fc4e3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/xml_iarchive.hpp +++ /dev/null @@ -1,142 +0,0 @@ -#ifndef BOOST_ARCHIVE_XML_IARCHIVE_HPP -#define BOOST_ARCHIVE_XML_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -template -class basic_xml_grammar; -typedef basic_xml_grammar xml_grammar; - -template -class BOOST_SYMBOL_VISIBLE xml_iarchive_impl : - public basic_text_iprimitive, - public basic_xml_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_iarchive; - friend class basic_xml_iarchive; - friend class load_access; -#endif - // use boost:scoped_ptr to implement automatic deletion; - boost::scoped_ptr gimpl; - - std::istream & get_is(){ - return is; - } - template - void load(T & t){ - basic_text_iprimitive::load(t); - } - void - load(version_type & t){ - unsigned int v; - load(v); - t = version_type(v); - } - void - load(boost::serialization::item_version_type & t){ - unsigned int v; - load(v); - t = boost::serialization::item_version_type(v); - } - BOOST_ARCHIVE_DECL void - load(char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_ARCHIVE_DECL void - load(wchar_t * t); - #endif - BOOST_ARCHIVE_DECL void - load(std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_ARCHIVE_DECL void - load(std::wstring &ws); - #endif - template - void load_override(T & t){ - basic_xml_iarchive::load_override(t); - } - BOOST_ARCHIVE_DECL void - load_override(class_name_type & t); - BOOST_ARCHIVE_DECL void - init(); - BOOST_ARCHIVE_DECL - xml_iarchive_impl(std::istream & is, unsigned int flags); - BOOST_ARCHIVE_DECL - ~xml_iarchive_impl(); -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class BOOST_SYMBOL_VISIBLE xml_iarchive : - public xml_iarchive_impl{ -public: - xml_iarchive(std::istream & is, unsigned int flags = 0) : - xml_iarchive_impl(is, flags) - {} - ~xml_iarchive(){}; -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::xml_iarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_XML_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_oarchive.hpp deleted file mode 100644 index eea12680372..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/xml_oarchive.hpp +++ /dev/null @@ -1,137 +0,0 @@ -#ifndef BOOST_ARCHIVE_XML_OARCHIVE_HPP -#define BOOST_ARCHIVE_XML_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include // size_t -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE xml_oarchive_impl : - public basic_text_oprimitive, - public basic_xml_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_oarchive; - friend class basic_xml_oarchive; - friend class save_access; -#endif - template - void save(const T & t){ - basic_text_oprimitive::save(t); - } - void - save(const version_type & t){ - save(static_cast(t)); - } - void - save(const boost::serialization::item_version_type & t){ - save(static_cast(t)); - } - BOOST_ARCHIVE_DECL void - save(const char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_ARCHIVE_DECL void - save(const wchar_t * t); - #endif - BOOST_ARCHIVE_DECL void - save(const std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_ARCHIVE_DECL void - save(const std::wstring &ws); - #endif - BOOST_ARCHIVE_DECL - xml_oarchive_impl(std::ostream & os, unsigned int flags); - BOOST_ARCHIVE_DECL - ~xml_oarchive_impl(); -public: - BOOST_ARCHIVE_DECL - void save_binary(const void *address, std::size_t count); -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -// we use the following because we can't use -// typedef xml_oarchive_impl > xml_oarchive; - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from xml_oarchive_impl instead. This will -// preserve correct static polymorphism. -class BOOST_SYMBOL_VISIBLE xml_oarchive : - public xml_oarchive_impl -{ -public: - xml_oarchive(std::ostream & os, unsigned int flags = 0) : - xml_oarchive_impl(os, flags) - {} - ~xml_oarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::xml_oarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_XML_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_wiarchive.hpp deleted file mode 100644 index ac24289ac11..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/xml_wiarchive.hpp +++ /dev/null @@ -1,149 +0,0 @@ -#ifndef BOOST_ARCHIVE_XML_WIARCHIVE_HPP -#define BOOST_ARCHIVE_XML_WIARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_wiarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include - -#include -#include -#include -#include -#include -#include -// #include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -template -class basic_xml_grammar; -typedef basic_xml_grammar xml_wgrammar; - -template -class BOOST_SYMBOL_VISIBLE xml_wiarchive_impl : - public basic_text_iprimitive, - public basic_xml_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_iarchive; - friend class basic_xml_iarchive; - friend class load_access; -#endif - boost::scoped_ptr gimpl; - std::wistream & get_is(){ - return is; - } - template - void - load(T & t){ - basic_text_iprimitive::load(t); - } - void - load(version_type & t){ - unsigned int v; - load(v); - t = version_type(v); - } - void - load(boost::serialization::item_version_type & t){ - unsigned int v; - load(v); - t = boost::serialization::item_version_type(v); - } - BOOST_WARCHIVE_DECL void - load(char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_WARCHIVE_DECL void - load(wchar_t * t); - #endif - BOOST_WARCHIVE_DECL void - load(std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_WARCHIVE_DECL void - load(std::wstring &ws); - #endif - template - void load_override(T & t){ - basic_xml_iarchive::load_override(t); - } - BOOST_WARCHIVE_DECL void - load_override(class_name_type & t); - BOOST_WARCHIVE_DECL void - init(); - BOOST_WARCHIVE_DECL - xml_wiarchive_impl(std::wistream & is, unsigned int flags) ; - BOOST_WARCHIVE_DECL - ~xml_wiarchive_impl(); -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class BOOST_SYMBOL_VISIBLE xml_wiarchive : - public xml_wiarchive_impl{ -public: - xml_wiarchive(std::wistream & is, unsigned int flags = 0) : - xml_wiarchive_impl(is, flags) - {} - ~xml_wiarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::xml_wiarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_XML_WIARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_woarchive.hpp deleted file mode 100644 index cb7ce68cb6f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/xml_woarchive.hpp +++ /dev/null @@ -1,134 +0,0 @@ -#ifndef BOOST_ARCHIVE_XML_WOARCHIVE_HPP -#define BOOST_ARCHIVE_XML_WOARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_woarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else -#include // size_t -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include - -//#include -#include -#include -#include -#include -#include -//#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE xml_woarchive_impl : - public basic_text_oprimitive, - public basic_xml_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_oarchive; - friend class basic_xml_oarchive; - friend class save_access; -#endif - //void end_preamble(){ - // basic_xml_oarchive::end_preamble(); - //} - template - void - save(const T & t){ - basic_text_oprimitive::save(t); - } - void - save(const version_type & t){ - save(static_cast(t)); - } - void - save(const boost::serialization::item_version_type & t){ - save(static_cast(t)); - } - BOOST_WARCHIVE_DECL void - save(const char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_WARCHIVE_DECL void - save(const wchar_t * t); - #endif - BOOST_WARCHIVE_DECL void - save(const std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_WARCHIVE_DECL void - save(const std::wstring &ws); - #endif - BOOST_WARCHIVE_DECL - xml_woarchive_impl(std::wostream & os, unsigned int flags); - BOOST_WARCHIVE_DECL - ~xml_woarchive_impl(); -public: - BOOST_WARCHIVE_DECL void - save_binary(const void *address, std::size_t count); - -}; - -// we use the following because we can't use -// typedef xml_woarchive_impl > xml_woarchive; - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from xml_woarchive_impl instead. This will -// preserve correct static polymorphism. -class BOOST_SYMBOL_VISIBLE xml_woarchive : - public xml_woarchive_impl -{ -public: - xml_woarchive(std::wostream & os, unsigned int flags = 0) : - xml_woarchive_impl(os, flags) - {} - ~xml_woarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::xml_woarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_XML_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/foreach_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/foreach_fwd.hpp deleted file mode 100644 index 4e0bb370c2f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/foreach_fwd.hpp +++ /dev/null @@ -1,51 +0,0 @@ -/////////////////////////////////////////////////////////////////////////////// -// foreach.hpp header file -// -// Copyright 2010 Eric Niebler. -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// See http://www.boost.org/libs/foreach for documentation -// -// Credits: -// Kazutoshi Satoda: for suggesting the need for a _fwd header for foreach's -// customization points. - -#ifndef BOOST_FOREACH_FWD_HPP -#define BOOST_FOREACH_FWD_HPP - -// This must be at global scope, hence the uglified name -enum boost_foreach_argument_dependent_lookup_hack -{ - boost_foreach_argument_dependent_lookup_hack_value -}; - -namespace boost -{ - -namespace foreach -{ - /////////////////////////////////////////////////////////////////////////////// - // boost::foreach::tag - // - typedef boost_foreach_argument_dependent_lookup_hack tag; - - /////////////////////////////////////////////////////////////////////////////// - // boost::foreach::is_lightweight_proxy - // Specialize this for user-defined collection types if they are inexpensive to copy. - // This tells BOOST_FOREACH it can avoid the rvalue/lvalue detection stuff. - template - struct is_lightweight_proxy; - - /////////////////////////////////////////////////////////////////////////////// - // boost::foreach::is_noncopyable - // Specialize this for user-defined collection types if they cannot be copied. - // This also tells BOOST_FOREACH to avoid the rvalue/lvalue detection stuff. - template - struct is_noncopyable; - -} // namespace foreach - -} // namespace boost - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/composite_key.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/composite_key.hpp deleted file mode 100644 index 787cdf83195..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/composite_key.hpp +++ /dev/null @@ -1,1513 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_COMPOSITE_KEY_HPP -#define BOOST_MULTI_INDEX_COMPOSITE_KEY_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_FUNCTION_TEMPLATE_ORDERING) -#include -#endif - -#if !defined(BOOST_NO_SFINAE) -#include -#endif - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) -#include -#endif - -/* A composite key stores n key extractors and "computes" the - * result on a given value as a packed reference to the value and - * the composite key itself. Actual invocations to the component - * key extractors are lazily performed when executing an operation - * on composite_key results (equality, comparison, hashing.) - * As the other key extractors in Boost.MultiIndex, composite_key - * is overloaded to work on chained pointers to T and reference_wrappers - * of T. - */ - -/* This user_definable macro limits the number of elements of a composite - * key; useful for shortening resulting symbol names (MSVC++ 6.0, for - * instance has problems coping with very long symbol names.) - * NB: This cannot exceed the maximum number of arguments of - * boost::tuple. In Boost 1.32, the limit is 10. - */ - -#if !defined(BOOST_MULTI_INDEX_LIMIT_COMPOSITE_KEY_SIZE) -#define BOOST_MULTI_INDEX_LIMIT_COMPOSITE_KEY_SIZE 10 -#endif - -/* maximum number of key extractors in a composite key */ - -#if BOOST_MULTI_INDEX_LIMIT_COMPOSITE_KEY_SIZE<10 /* max length of a tuple */ -#define BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE \ - BOOST_MULTI_INDEX_LIMIT_COMPOSITE_KEY_SIZE -#else -#define BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE 10 -#endif - -/* BOOST_PP_ENUM of BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE elements */ - -#define BOOST_MULTI_INDEX_CK_ENUM(macro,data) \ - BOOST_PP_ENUM(BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE,macro,data) - -/* BOOST_PP_ENUM_PARAMS of BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE elements */ - -#define BOOST_MULTI_INDEX_CK_ENUM_PARAMS(param) \ - BOOST_PP_ENUM_PARAMS(BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE,param) - -/* if n==0 -> text0 - * otherwise -> textn=tuples::null_type - */ - -#define BOOST_MULTI_INDEX_CK_TEMPLATE_PARM(z,n,text) \ - typename BOOST_PP_CAT(text,n) BOOST_PP_EXPR_IF(n,=tuples::null_type) - -/* const textn& kn=textn() */ - -#define BOOST_MULTI_INDEX_CK_CTOR_ARG(z,n,text) \ - const BOOST_PP_CAT(text,n)& BOOST_PP_CAT(k,n) = BOOST_PP_CAT(text,n)() - -/* typename list(0)::type */ - -#define BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N(z,n,list) \ - BOOST_DEDUCED_TYPENAME BOOST_PP_LIST_AT(list,0)< \ - BOOST_PP_LIST_AT(list,1),n \ - >::type - -namespace boost{ - -template class reference_wrapper; /* fwd decl. */ - -namespace multi_index{ - -namespace detail{ - -/* n-th key extractor of a composite key */ - -template -struct nth_key_from_value -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename mpl::eval_if_c< - N::value, - tuples::element, - mpl::identity - >::type type; -}; - -/* nth_composite_key_##name::type yields - * functor >, or tuples::null_type - * if N exceeds the length of the composite key. - */ - -#define BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(name,functor) \ -template \ -struct BOOST_PP_CAT(key_,name) \ -{ \ - typedef functor type; \ -}; \ - \ -template<> \ -struct BOOST_PP_CAT(key_,name) \ -{ \ - typedef tuples::null_type type; \ -}; \ - \ -template \ -struct BOOST_PP_CAT(nth_composite_key_,name) \ -{ \ - typedef typename nth_key_from_value::type key_from_value; \ - typedef typename BOOST_PP_CAT(key_,name)::type type; \ -}; - -/* nth_composite_key_equal_to - * nth_composite_key_less - * nth_composite_key_greater - * nth_composite_key_hash - */ - -BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(equal_to,std::equal_to) -BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(less,std::less) -BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(greater,std::greater) -BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(hash,boost::hash) - -/* used for defining equality and comparison ops of composite_key_result */ - -#define BOOST_MULTI_INDEX_CK_IDENTITY_ENUM_MACRO(z,n,text) text - -struct generic_operator_equal -{ - template - bool operator()(const T& x,const Q& y)const{return x==y;} -}; - -typedef tuple< - BOOST_MULTI_INDEX_CK_ENUM( - BOOST_MULTI_INDEX_CK_IDENTITY_ENUM_MACRO, - detail::generic_operator_equal)> generic_operator_equal_tuple; - -struct generic_operator_less -{ - template - bool operator()(const T& x,const Q& y)const{return x generic_operator_less_tuple; - -/* Metaprogramming machinery for implementing equality, comparison and - * hashing operations of composite_key_result. - * - * equal_* checks for equality between composite_key_results and - * between those and tuples, accepting a tuple of basic equality functors. - * compare_* does lexicographical comparison. - * hash_* computes a combination of elementwise hash values. - */ - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename EqualCons -> -struct equal_ckey_ckey; /* fwd decl. */ - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename EqualCons -> -struct equal_ckey_ckey_terminal -{ - static bool compare( - const KeyCons1&,const Value1&, - const KeyCons2&,const Value2&, - const EqualCons&) - { - return true; - } -}; - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename EqualCons -> -struct equal_ckey_ckey_normal -{ - static bool compare( - const KeyCons1& c0,const Value1& v0, - const KeyCons2& c1,const Value2& v1, - const EqualCons& eq) - { - if(!eq.get_head()(c0.get_head()(v0),c1.get_head()(v1)))return false; - return equal_ckey_ckey< - BOOST_DEDUCED_TYPENAME KeyCons1::tail_type,Value1, - BOOST_DEDUCED_TYPENAME KeyCons2::tail_type,Value2, - BOOST_DEDUCED_TYPENAME EqualCons::tail_type - >::compare(c0.get_tail(),v0,c1.get_tail(),v1,eq.get_tail()); - } -}; - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename EqualCons -> -struct equal_ckey_ckey: - mpl::if_< - mpl::or_< - is_same, - is_same - >, - equal_ckey_ckey_terminal, - equal_ckey_ckey_normal - >::type -{ -}; - -template -< - typename KeyCons,typename Value, - typename ValCons,typename EqualCons -> -struct equal_ckey_cval; /* fwd decl. */ - -template -< - typename KeyCons,typename Value, - typename ValCons,typename EqualCons -> -struct equal_ckey_cval_terminal -{ - static bool compare( - const KeyCons&,const Value&,const ValCons&,const EqualCons&) - { - return true; - } - - static bool compare( - const ValCons&,const KeyCons&,const Value&,const EqualCons&) - { - return true; - } -}; - -template -< - typename KeyCons,typename Value, - typename ValCons,typename EqualCons -> -struct equal_ckey_cval_normal -{ - static bool compare( - const KeyCons& c,const Value& v,const ValCons& vc, - const EqualCons& eq) - { - if(!eq.get_head()(c.get_head()(v),vc.get_head()))return false; - return equal_ckey_cval< - BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, - BOOST_DEDUCED_TYPENAME ValCons::tail_type, - BOOST_DEDUCED_TYPENAME EqualCons::tail_type - >::compare(c.get_tail(),v,vc.get_tail(),eq.get_tail()); - } - - static bool compare( - const ValCons& vc,const KeyCons& c,const Value& v, - const EqualCons& eq) - { - if(!eq.get_head()(vc.get_head(),c.get_head()(v)))return false; - return equal_ckey_cval< - BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, - BOOST_DEDUCED_TYPENAME ValCons::tail_type, - BOOST_DEDUCED_TYPENAME EqualCons::tail_type - >::compare(vc.get_tail(),c.get_tail(),v,eq.get_tail()); - } -}; - -template -< - typename KeyCons,typename Value, - typename ValCons,typename EqualCons -> -struct equal_ckey_cval: - mpl::if_< - mpl::or_< - is_same, - is_same - >, - equal_ckey_cval_terminal, - equal_ckey_cval_normal - >::type -{ -}; - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename CompareCons -> -struct compare_ckey_ckey; /* fwd decl. */ - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename CompareCons -> -struct compare_ckey_ckey_terminal -{ - static bool compare( - const KeyCons1&,const Value1&, - const KeyCons2&,const Value2&, - const CompareCons&) - { - return false; - } -}; - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename CompareCons -> -struct compare_ckey_ckey_normal -{ - static bool compare( - const KeyCons1& c0,const Value1& v0, - const KeyCons2& c1,const Value2& v1, - const CompareCons& comp) - { - if(comp.get_head()(c0.get_head()(v0),c1.get_head()(v1)))return true; - if(comp.get_head()(c1.get_head()(v1),c0.get_head()(v0)))return false; - return compare_ckey_ckey< - BOOST_DEDUCED_TYPENAME KeyCons1::tail_type,Value1, - BOOST_DEDUCED_TYPENAME KeyCons2::tail_type,Value2, - BOOST_DEDUCED_TYPENAME CompareCons::tail_type - >::compare(c0.get_tail(),v0,c1.get_tail(),v1,comp.get_tail()); - } -}; - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename CompareCons -> -struct compare_ckey_ckey: - mpl::if_< - mpl::or_< - is_same, - is_same - >, - compare_ckey_ckey_terminal, - compare_ckey_ckey_normal - >::type -{ -}; - -template -< - typename KeyCons,typename Value, - typename ValCons,typename CompareCons -> -struct compare_ckey_cval; /* fwd decl. */ - -template -< - typename KeyCons,typename Value, - typename ValCons,typename CompareCons -> -struct compare_ckey_cval_terminal -{ - static bool compare( - const KeyCons&,const Value&,const ValCons&,const CompareCons&) - { - return false; - } - - static bool compare( - const ValCons&,const KeyCons&,const Value&,const CompareCons&) - { - return false; - } -}; - -template -< - typename KeyCons,typename Value, - typename ValCons,typename CompareCons -> -struct compare_ckey_cval_normal -{ - static bool compare( - const KeyCons& c,const Value& v,const ValCons& vc, - const CompareCons& comp) - { - if(comp.get_head()(c.get_head()(v),vc.get_head()))return true; - if(comp.get_head()(vc.get_head(),c.get_head()(v)))return false; - return compare_ckey_cval< - BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, - BOOST_DEDUCED_TYPENAME ValCons::tail_type, - BOOST_DEDUCED_TYPENAME CompareCons::tail_type - >::compare(c.get_tail(),v,vc.get_tail(),comp.get_tail()); - } - - static bool compare( - const ValCons& vc,const KeyCons& c,const Value& v, - const CompareCons& comp) - { - if(comp.get_head()(vc.get_head(),c.get_head()(v)))return true; - if(comp.get_head()(c.get_head()(v),vc.get_head()))return false; - return compare_ckey_cval< - BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, - BOOST_DEDUCED_TYPENAME ValCons::tail_type, - BOOST_DEDUCED_TYPENAME CompareCons::tail_type - >::compare(vc.get_tail(),c.get_tail(),v,comp.get_tail()); - } -}; - -template -< - typename KeyCons,typename Value, - typename ValCons,typename CompareCons -> -struct compare_ckey_cval: - mpl::if_< - mpl::or_< - is_same, - is_same - >, - compare_ckey_cval_terminal, - compare_ckey_cval_normal - >::type -{ -}; - -template -struct hash_ckey; /* fwd decl. */ - -template -struct hash_ckey_terminal -{ - static std::size_t hash( - const KeyCons&,const Value&,const HashCons&,std::size_t carry) - { - return carry; - } -}; - -template -struct hash_ckey_normal -{ - static std::size_t hash( - const KeyCons& c,const Value& v,const HashCons& h,std::size_t carry=0) - { - /* same hashing formula as boost::hash_combine */ - - carry^=h.get_head()(c.get_head()(v))+0x9e3779b9+(carry<<6)+(carry>>2); - return hash_ckey< - BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, - BOOST_DEDUCED_TYPENAME HashCons::tail_type - >::hash(c.get_tail(),v,h.get_tail(),carry); - } -}; - -template -struct hash_ckey: - mpl::if_< - is_same, - hash_ckey_terminal, - hash_ckey_normal - >::type -{ -}; - -template -struct hash_cval; /* fwd decl. */ - -template -struct hash_cval_terminal -{ - static std::size_t hash(const ValCons&,const HashCons&,std::size_t carry) - { - return carry; - } -}; - -template -struct hash_cval_normal -{ - static std::size_t hash( - const ValCons& vc,const HashCons& h,std::size_t carry=0) - { - carry^=h.get_head()(vc.get_head())+0x9e3779b9+(carry<<6)+(carry>>2); - return hash_cval< - BOOST_DEDUCED_TYPENAME ValCons::tail_type, - BOOST_DEDUCED_TYPENAME HashCons::tail_type - >::hash(vc.get_tail(),h.get_tail(),carry); - } -}; - -template -struct hash_cval: - mpl::if_< - is_same, - hash_cval_terminal, - hash_cval_normal - >::type -{ -}; - -} /* namespace multi_index::detail */ - -/* composite_key_result */ - -#if defined(BOOST_MSVC) -#pragma warning(push) -#pragma warning(disable:4512) -#endif - -template -struct composite_key_result -{ - typedef CompositeKey composite_key_type; - typedef typename composite_key_type::value_type value_type; - - composite_key_result( - const composite_key_type& composite_key_,const value_type& value_): - composite_key(composite_key_),value(value_) - {} - - const composite_key_type& composite_key; - const value_type& value; -}; - -#if defined(BOOST_MSVC) -#pragma warning(pop) -#endif - -/* composite_key */ - -template< - typename Value, - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_TEMPLATE_PARM,KeyFromValue) -> -struct composite_key: - private tuple -{ -private: - typedef tuple super; - -public: - typedef super key_extractor_tuple; - typedef Value value_type; - typedef composite_key_result result_type; - - composite_key( - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_CTOR_ARG,KeyFromValue)): - super(BOOST_MULTI_INDEX_CK_ENUM_PARAMS(k)) - {} - - composite_key(const key_extractor_tuple& x):super(x){} - - const key_extractor_tuple& key_extractors()const{return *this;} - key_extractor_tuple& key_extractors(){return *this;} - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,result_type>::type -#else - result_type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - result_type operator()(const value_type& x)const - { - return result_type(*this,x); - } - - result_type operator()(const reference_wrapper& x)const - { - return result_type(*this,x.get()); - } - - result_type operator()(const reference_wrapper& x)const - { - return result_type(*this,x.get()); - } -}; - -/* comparison operators */ - -/* == */ - -template -inline bool operator==( - const composite_key_result& x, - const composite_key_result& y) -{ - typedef typename CompositeKey1::key_extractor_tuple key_extractor_tuple1; - typedef typename CompositeKey1::value_type value_type1; - typedef typename CompositeKey2::key_extractor_tuple key_extractor_tuple2; - typedef typename CompositeKey2::value_type value_type2; - - BOOST_STATIC_ASSERT( - tuples::length::value== - tuples::length::value); - - return detail::equal_ckey_ckey< - key_extractor_tuple1,value_type1, - key_extractor_tuple2,value_type2, - detail::generic_operator_equal_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - y.composite_key.key_extractors(),y.value, - detail::generic_operator_equal_tuple()); -} - -template< - typename CompositeKey, - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value) -> -inline bool operator==( - const composite_key_result& x, - const tuple& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value== - tuples::length::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,detail::generic_operator_equal_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - y,detail::generic_operator_equal_tuple()); -} - -template -< - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), - typename CompositeKey -> -inline bool operator==( - const tuple& x, - const composite_key_result& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value== - tuples::length::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,detail::generic_operator_equal_tuple - >::compare( - x,y.composite_key.key_extractors(), - y.value,detail::generic_operator_equal_tuple()); -} - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) -template -inline bool operator==( - const composite_key_result& x, - const std::tuple& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - static_cast(tuples::length::value)== - std::tuple_size::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,detail::generic_operator_equal_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - detail::make_cons_stdtuple(y),detail::generic_operator_equal_tuple()); -} - -template -inline bool operator==( - const std::tuple& x, - const composite_key_result& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - static_cast(tuples::length::value)== - std::tuple_size::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,detail::generic_operator_equal_tuple - >::compare( - detail::make_cons_stdtuple(x),y.composite_key.key_extractors(), - y.value,detail::generic_operator_equal_tuple()); -} -#endif - -/* < */ - -template -inline bool operator<( - const composite_key_result& x, - const composite_key_result& y) -{ - typedef typename CompositeKey1::key_extractor_tuple key_extractor_tuple1; - typedef typename CompositeKey1::value_type value_type1; - typedef typename CompositeKey2::key_extractor_tuple key_extractor_tuple2; - typedef typename CompositeKey2::value_type value_type2; - - return detail::compare_ckey_ckey< - key_extractor_tuple1,value_type1, - key_extractor_tuple2,value_type2, - detail::generic_operator_less_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - y.composite_key.key_extractors(),y.value, - detail::generic_operator_less_tuple()); -} - -template -< - typename CompositeKey, - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value) -> -inline bool operator<( - const composite_key_result& x, - const tuple& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,detail::generic_operator_less_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - y,detail::generic_operator_less_tuple()); -} - -template -< - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), - typename CompositeKey -> -inline bool operator<( - const tuple& x, - const composite_key_result& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,detail::generic_operator_less_tuple - >::compare( - x,y.composite_key.key_extractors(), - y.value,detail::generic_operator_less_tuple()); -} - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) -template -inline bool operator<( - const composite_key_result& x, - const std::tuple& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,detail::generic_operator_less_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - detail::make_cons_stdtuple(y),detail::generic_operator_less_tuple()); -} - -template -inline bool operator<( - const std::tuple& x, - const composite_key_result& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,detail::generic_operator_less_tuple - >::compare( - detail::make_cons_stdtuple(x),y.composite_key.key_extractors(), - y.value,detail::generic_operator_less_tuple()); -} -#endif - -/* rest of comparison operators */ - -#define BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS(t1,t2,a1,a2) \ -template inline bool operator!=(const a1& x,const a2& y) \ -{ \ - return !(x==y); \ -} \ - \ -template inline bool operator>(const a1& x,const a2& y) \ -{ \ - return y inline bool operator>=(const a1& x,const a2& y) \ -{ \ - return !(x inline bool operator<=(const a1& x,const a2& y) \ -{ \ - return !(y, - composite_key_result -) - -BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS( - typename CompositeKey, - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), - composite_key_result, - tuple -) - -BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS( - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), - typename CompositeKey, - tuple, - composite_key_result -) - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) -BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS( - typename CompositeKey, - typename... Values, - composite_key_result, - std::tuple -) - -BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS( - typename CompositeKey, - typename... Values, - std::tuple, - composite_key_result -) -#endif - -/* composite_key_equal_to */ - -template -< - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_TEMPLATE_PARM,Pred) -> -struct composite_key_equal_to: - private tuple -{ -private: - typedef tuple super; - -public: - typedef super key_eq_tuple; - - composite_key_equal_to( - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_CTOR_ARG,Pred)): - super(BOOST_MULTI_INDEX_CK_ENUM_PARAMS(k)) - {} - - composite_key_equal_to(const key_eq_tuple& x):super(x){} - - const key_eq_tuple& key_eqs()const{return *this;} - key_eq_tuple& key_eqs(){return *this;} - - template - bool operator()( - const composite_key_result & x, - const composite_key_result & y)const - { - typedef typename CompositeKey1::key_extractor_tuple key_extractor_tuple1; - typedef typename CompositeKey1::value_type value_type1; - typedef typename CompositeKey2::key_extractor_tuple key_extractor_tuple2; - typedef typename CompositeKey2::value_type value_type2; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value&& - tuples::length::value== - tuples::length::value); - - return detail::equal_ckey_ckey< - key_extractor_tuple1,value_type1, - key_extractor_tuple2,value_type2, - key_eq_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - y.composite_key.key_extractors(),y.value, - key_eqs()); - } - - template - < - typename CompositeKey, - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value) - > - bool operator()( - const composite_key_result& x, - const tuple& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value&& - tuples::length::value== - tuples::length::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,key_eq_tuple - >::compare(x.composite_key.key_extractors(),x.value,y,key_eqs()); - } - - template - < - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), - typename CompositeKey - > - bool operator()( - const tuple& x, - const composite_key_result& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value&& - tuples::length::value== - tuples::length::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,key_eq_tuple - >::compare(x,y.composite_key.key_extractors(),y.value,key_eqs()); - } - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) - template - bool operator()( - const composite_key_result& x, - const std::tuple& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value&& - static_cast(tuples::length::value)== - std::tuple_size::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,key_eq_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - detail::make_cons_stdtuple(y),key_eqs()); - } - - template - bool operator()( - const std::tuple& x, - const composite_key_result& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - std::tuple_size::value<= - static_cast(tuples::length::value)&& - std::tuple_size::value== - static_cast(tuples::length::value)); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,key_eq_tuple - >::compare( - detail::make_cons_stdtuple(x),y.composite_key.key_extractors(), - y.value,key_eqs()); - } -#endif -}; - -/* composite_key_compare */ - -template -< - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_TEMPLATE_PARM,Compare) -> -struct composite_key_compare: - private tuple -{ -private: - typedef tuple super; - -public: - typedef super key_comp_tuple; - - composite_key_compare( - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_CTOR_ARG,Compare)): - super(BOOST_MULTI_INDEX_CK_ENUM_PARAMS(k)) - {} - - composite_key_compare(const key_comp_tuple& x):super(x){} - - const key_comp_tuple& key_comps()const{return *this;} - key_comp_tuple& key_comps(){return *this;} - - template - bool operator()( - const composite_key_result & x, - const composite_key_result & y)const - { - typedef typename CompositeKey1::key_extractor_tuple key_extractor_tuple1; - typedef typename CompositeKey1::value_type value_type1; - typedef typename CompositeKey2::key_extractor_tuple key_extractor_tuple2; - typedef typename CompositeKey2::value_type value_type2; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value|| - tuples::length::value<= - tuples::length::value); - - return detail::compare_ckey_ckey< - key_extractor_tuple1,value_type1, - key_extractor_tuple2,value_type2, - key_comp_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - y.composite_key.key_extractors(),y.value, - key_comps()); - } - -#if !defined(BOOST_NO_FUNCTION_TEMPLATE_ORDERING) - template - bool operator()( - const composite_key_result& x, - const Value& y)const - { - return operator()(x,boost::make_tuple(boost::cref(y))); - } -#endif - - template - < - typename CompositeKey, - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value) - > - bool operator()( - const composite_key_result& x, - const tuple& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value|| - tuples::length::value<= - tuples::length::value); - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,key_comp_tuple - >::compare(x.composite_key.key_extractors(),x.value,y,key_comps()); - } - -#if !defined(BOOST_NO_FUNCTION_TEMPLATE_ORDERING) - template - bool operator()( - const Value& x, - const composite_key_result& y)const - { - return operator()(boost::make_tuple(boost::cref(x)),y); - } -#endif - - template - < - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), - typename CompositeKey - > - bool operator()( - const tuple& x, - const composite_key_result& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value|| - tuples::length::value<= - tuples::length::value); - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,key_comp_tuple - >::compare(x,y.composite_key.key_extractors(),y.value,key_comps()); - } - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) - template - bool operator()( - const composite_key_result& x, - const std::tuple& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value|| - std::tuple_size::value<= - static_cast(tuples::length::value)); - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,key_comp_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - detail::make_cons_stdtuple(y),key_comps()); - } - - template - bool operator()( - const std::tuple& x, - const composite_key_result& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - std::tuple_size::value<= - static_cast(tuples::length::value)|| - tuples::length::value<= - tuples::length::value); - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,key_comp_tuple - >::compare( - detail::make_cons_stdtuple(x),y.composite_key.key_extractors(), - y.value,key_comps()); - } -#endif -}; - -/* composite_key_hash */ - -template -< - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_TEMPLATE_PARM,Hash) -> -struct composite_key_hash: - private tuple -{ -private: - typedef tuple super; - -public: - typedef super key_hasher_tuple; - - composite_key_hash( - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_CTOR_ARG,Hash)): - super(BOOST_MULTI_INDEX_CK_ENUM_PARAMS(k)) - {} - - composite_key_hash(const key_hasher_tuple& x):super(x){} - - const key_hasher_tuple& key_hash_functions()const{return *this;} - key_hasher_tuple& key_hash_functions(){return *this;} - - template - std::size_t operator()(const composite_key_result & x)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - - BOOST_STATIC_ASSERT( - tuples::length::value== - tuples::length::value); - - return detail::hash_ckey< - key_extractor_tuple,value_type, - key_hasher_tuple - >::hash(x.composite_key.key_extractors(),x.value,key_hash_functions()); - } - - template - std::size_t operator()( - const tuple& x)const - { - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value== - tuples::length::value); - - return detail::hash_cval< - key_tuple,key_hasher_tuple - >::hash(x,key_hash_functions()); - } - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) - template - std::size_t operator()(const std::tuple& x)const - { - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - std::tuple_size::value== - static_cast(tuples::length::value)); - - return detail::hash_cval< - cons_key_tuple,key_hasher_tuple - >::hash(detail::make_cons_stdtuple(x),key_hash_functions()); - } -#endif -}; - -/* Instantiations of the former functors with "natural" basic components: - * composite_key_result_equal_to uses std::equal_to of the values. - * composite_key_result_less uses std::less. - * composite_key_result_greater uses std::greater. - * composite_key_result_hash uses boost::hash. - */ - -#define BOOST_MULTI_INDEX_CK_RESULT_EQUAL_TO_SUPER \ -composite_key_equal_to< \ - BOOST_MULTI_INDEX_CK_ENUM( \ - BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N, \ - /* the argument is a PP list */ \ - (detail::nth_composite_key_equal_to, \ - (BOOST_DEDUCED_TYPENAME CompositeKeyResult::composite_key_type, \ - BOOST_PP_NIL))) \ - > - -template -struct composite_key_result_equal_to: -BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS -BOOST_MULTI_INDEX_CK_RESULT_EQUAL_TO_SUPER -{ -private: - typedef BOOST_MULTI_INDEX_CK_RESULT_EQUAL_TO_SUPER super; - -public: - typedef CompositeKeyResult first_argument_type; - typedef first_argument_type second_argument_type; - typedef bool result_type; - - using super::operator(); -}; - -#define BOOST_MULTI_INDEX_CK_RESULT_LESS_SUPER \ -composite_key_compare< \ - BOOST_MULTI_INDEX_CK_ENUM( \ - BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N, \ - /* the argument is a PP list */ \ - (detail::nth_composite_key_less, \ - (BOOST_DEDUCED_TYPENAME CompositeKeyResult::composite_key_type, \ - BOOST_PP_NIL))) \ - > - -template -struct composite_key_result_less: -BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS -BOOST_MULTI_INDEX_CK_RESULT_LESS_SUPER -{ -private: - typedef BOOST_MULTI_INDEX_CK_RESULT_LESS_SUPER super; - -public: - typedef CompositeKeyResult first_argument_type; - typedef first_argument_type second_argument_type; - typedef bool result_type; - - using super::operator(); -}; - -#define BOOST_MULTI_INDEX_CK_RESULT_GREATER_SUPER \ -composite_key_compare< \ - BOOST_MULTI_INDEX_CK_ENUM( \ - BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N, \ - /* the argument is a PP list */ \ - (detail::nth_composite_key_greater, \ - (BOOST_DEDUCED_TYPENAME CompositeKeyResult::composite_key_type, \ - BOOST_PP_NIL))) \ - > - -template -struct composite_key_result_greater: -BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS -BOOST_MULTI_INDEX_CK_RESULT_GREATER_SUPER -{ -private: - typedef BOOST_MULTI_INDEX_CK_RESULT_GREATER_SUPER super; - -public: - typedef CompositeKeyResult first_argument_type; - typedef first_argument_type second_argument_type; - typedef bool result_type; - - using super::operator(); -}; - -#define BOOST_MULTI_INDEX_CK_RESULT_HASH_SUPER \ -composite_key_hash< \ - BOOST_MULTI_INDEX_CK_ENUM( \ - BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N, \ - /* the argument is a PP list */ \ - (detail::nth_composite_key_hash, \ - (BOOST_DEDUCED_TYPENAME CompositeKeyResult::composite_key_type, \ - BOOST_PP_NIL))) \ - > - -template -struct composite_key_result_hash: -BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS -BOOST_MULTI_INDEX_CK_RESULT_HASH_SUPER -{ -private: - typedef BOOST_MULTI_INDEX_CK_RESULT_HASH_SUPER super; - -public: - typedef CompositeKeyResult argument_type; - typedef std::size_t result_type; - - using super::operator(); -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -/* Specializations of std::equal_to, std::less, std::greater and boost::hash - * for composite_key_results enabling interoperation with tuples of values. - */ - -namespace std{ - -template -struct equal_to >: - boost::multi_index::composite_key_result_equal_to< - boost::multi_index::composite_key_result - > -{ -}; - -template -struct less >: - boost::multi_index::composite_key_result_less< - boost::multi_index::composite_key_result - > -{ -}; - -template -struct greater >: - boost::multi_index::composite_key_result_greater< - boost::multi_index::composite_key_result - > -{ -}; - -} /* namespace std */ - -namespace boost{ - -template -struct hash >: - boost::multi_index::composite_key_result_hash< - boost::multi_index::composite_key_result - > -{ -}; - -} /* namespace boost */ - -#undef BOOST_MULTI_INDEX_CK_RESULT_HASH_SUPER -#undef BOOST_MULTI_INDEX_CK_RESULT_GREATER_SUPER -#undef BOOST_MULTI_INDEX_CK_RESULT_LESS_SUPER -#undef BOOST_MULTI_INDEX_CK_RESULT_EQUAL_TO_SUPER -#undef BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS -#undef BOOST_MULTI_INDEX_CK_IDENTITY_ENUM_MACRO -#undef BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR -#undef BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N -#undef BOOST_MULTI_INDEX_CK_CTOR_ARG -#undef BOOST_MULTI_INDEX_CK_TEMPLATE_PARM -#undef BOOST_MULTI_INDEX_CK_ENUM_PARAMS -#undef BOOST_MULTI_INDEX_CK_ENUM -#undef BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/access_specifier.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/access_specifier.hpp deleted file mode 100644 index f3346e836d4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/access_specifier.hpp +++ /dev/null @@ -1,54 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ACCESS_SPECIFIER_HPP -#define BOOST_MULTI_INDEX_DETAIL_ACCESS_SPECIFIER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include -#include - -/* In those compilers that do not accept the member template friend syntax, - * some protected and private sections might need to be specified as - * public. - */ - -#if defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) -#define BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS public -#define BOOST_MULTI_INDEX_PRIVATE_IF_MEMBER_TEMPLATE_FRIENDS public -#else -#define BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS protected -#define BOOST_MULTI_INDEX_PRIVATE_IF_MEMBER_TEMPLATE_FRIENDS private -#endif - -/* GCC does not correctly support in-class using declarations for template - * functions. See http://gcc.gnu.org/bugzilla/show_bug.cgi?id=9810 - * MSVC 7.1/8.0 seem to have a similar problem, though the conditions in - * which the error happens are not that simple. I have yet to isolate this - * into a snippet suitable for bug reporting. - * Sun Studio also has this problem, which might be related, from the - * information gathered at Sun forums, with a known issue notified at the - * internal bug report 6421933. The bug is present up to Studio Express 2, - * the latest preview version of the future Sun Studio 12. As of this writing - * (October 2006) it is not known whether a fix will finally make it into the - * official Sun Studio 12. - */ - -#if BOOST_WORKAROUND(__GNUC__,==3)&&(__GNUC_MINOR__<4)||\ - BOOST_WORKAROUND(BOOST_MSVC,==1310)||\ - BOOST_WORKAROUND(BOOST_MSVC,==1400)||\ - BOOST_WORKAROUND(__SUNPRO_CC,BOOST_TESTED_AT(0x590)) -#define BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS public -#else -#define BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS private -#endif - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/adl_swap.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/adl_swap.hpp deleted file mode 100644 index 02b06442290..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/adl_swap.hpp +++ /dev/null @@ -1,44 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ADL_SWAP_HPP -#define BOOST_MULTI_INDEX_DETAIL_ADL_SWAP_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -void adl_swap(T& x,T& y) -{ - -#if !defined(BOOST_FUNCTION_SCOPE_USING_DECLARATION_BREAKS_ADL) - using std::swap; - swap(x,y); -#else - std::swap(x,y); -#endif - -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/archive_constructed.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/archive_constructed.hpp deleted file mode 100644 index 0a7a26e0d4e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/archive_constructed.hpp +++ /dev/null @@ -1,83 +0,0 @@ -/* Copyright 2003-2016 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ARCHIVE_CONSTRUCTED_HPP -#define BOOST_MULTI_INDEX_DETAIL_ARCHIVE_CONSTRUCTED_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* constructs a stack-based object from a serialization archive */ - -template -struct archive_constructed:private noncopyable -{ - template - archive_constructed(Archive& ar,const unsigned int version) - { - serialization::load_construct_data_adl(ar,&get(),version); - BOOST_TRY{ - ar>>get(); - } - BOOST_CATCH(...){ - (&get())->~T(); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - template - archive_constructed(const char* name,Archive& ar,const unsigned int version) - { - serialization::load_construct_data_adl(ar,&get(),version); - BOOST_TRY{ - ar>>serialization::make_nvp(name,get()); - } - BOOST_CATCH(...){ - (&get())->~T(); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - ~archive_constructed() - { - (&get())->~T(); - } - -#include - - T& get(){return *reinterpret_cast(&space);} - -#include - -private: - typename aligned_storage::value>::type space; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/auto_space.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/auto_space.hpp deleted file mode 100644 index 9d78c3a363f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/auto_space.hpp +++ /dev/null @@ -1,91 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_AUTO_SPACE_HPP -#define BOOST_MULTI_INDEX_DETAIL_AUTO_SPACE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* auto_space provides uninitialized space suitably to store - * a given number of elements of a given type. - */ - -/* NB: it is not clear whether using an allocator to handle - * zero-sized arrays of elements is conformant or not. GCC 3.3.1 - * and prior fail here, other stdlibs handle the issue gracefully. - * To be on the safe side, the case n==0 is given special treatment. - * References: - * GCC Bugzilla, "standard allocator crashes when deallocating segment - * "of zero length", http://gcc.gnu.org/bugzilla/show_bug.cgi?id=14176 - * C++ Standard Library Defect Report List (Revision 28), issue 199 - * "What does allocate(0) return?", - * http://www.open-std.org/jtc1/sc22/wg21/docs/lwg-defects.html#199 - */ - -template > -struct auto_space:private noncopyable -{ - typedef typename boost::detail::allocator::rebind_to< - Allocator,T - >::type::pointer pointer; - - explicit auto_space(const Allocator& al=Allocator(),std::size_t n=1): - al_(al),n_(n),data_(n_?al_.allocate(n_):pointer(0)) - {} - - ~auto_space() - { - if(n_)al_.deallocate(data_,n_); - } - - Allocator get_allocator()const{return al_;} - - pointer data()const{return data_;} - - void swap(auto_space& x) - { - if(al_!=x.al_)adl_swap(al_,x.al_); - std::swap(n_,x.n_); - std::swap(data_,x.data_); - } - -private: - typename boost::detail::allocator::rebind_to< - Allocator,T>::type al_; - std::size_t n_; - pointer data_; -}; - -template -void swap(auto_space& x,auto_space& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/base_type.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/base_type.hpp deleted file mode 100644 index 8c9b62b716a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/base_type.hpp +++ /dev/null @@ -1,74 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_BASE_TYPE_HPP -#define BOOST_MULTI_INDEX_DETAIL_BASE_TYPE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* MPL machinery to construct a linear hierarchy of indices out of - * a index list. - */ - -struct index_applier -{ - template - struct apply - { - typedef typename IndexSpecifierMeta::type index_specifier; - typedef typename index_specifier:: - BOOST_NESTED_TEMPLATE index_class::type type; - }; -}; - -template -struct nth_layer -{ - BOOST_STATIC_CONSTANT(int,length=mpl::size::value); - - typedef typename mpl::eval_if_c< - N==length, - mpl::identity >, - mpl::apply2< - index_applier, - mpl::at_c, - nth_layer - > - >::type type; -}; - -template -struct multi_index_base_type:nth_layer<0,Value,IndexSpecifierList,Allocator> -{ - BOOST_STATIC_ASSERT(detail::is_index_list::value); -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bidir_node_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bidir_node_iterator.hpp deleted file mode 100644 index 9be5ec84b43..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bidir_node_iterator.hpp +++ /dev/null @@ -1,114 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_BIDIR_NODE_ITERATOR_HPP -#define BOOST_MULTI_INDEX_DETAIL_BIDIR_NODE_ITERATOR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Iterator class for node-based indices with bidirectional - * iterators (ordered and sequenced indices.) - */ - -template -class bidir_node_iterator: - public bidirectional_iterator_helper< - bidir_node_iterator, - typename Node::value_type, - std::ptrdiff_t, - const typename Node::value_type*, - const typename Node::value_type&> -{ -public: - /* coverity[uninit_ctor]: suppress warning */ - bidir_node_iterator(){} - explicit bidir_node_iterator(Node* node_):node(node_){} - - const typename Node::value_type& operator*()const - { - return node->value(); - } - - bidir_node_iterator& operator++() - { - Node::increment(node); - return *this; - } - - bidir_node_iterator& operator--() - { - Node::decrement(node); - return *this; - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* Serialization. As for why the following is public, - * see explanation in safe_mode_iterator notes in safe_mode.hpp. - */ - - BOOST_SERIALIZATION_SPLIT_MEMBER() - - typedef typename Node::base_type node_base_type; - - template - void save(Archive& ar,const unsigned int)const - { - node_base_type* bnode=node; - ar< - void load(Archive& ar,const unsigned int) - { - node_base_type* bnode; - ar>>serialization::make_nvp("pointer",bnode); - node=static_cast(bnode); - } -#endif - - /* get_node is not to be used by the user */ - - typedef Node node_type; - - Node* get_node()const{return node;} - -private: - Node* node; -}; - -template -bool operator==( - const bidir_node_iterator& x, - const bidir_node_iterator& y) -{ - return x.get_node()==y.get_node(); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bucket_array.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bucket_array.hpp deleted file mode 100644 index d9fa434d9a9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bucket_array.hpp +++ /dev/null @@ -1,243 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_BUCKET_ARRAY_HPP -#define BOOST_MULTI_INDEX_DETAIL_BUCKET_ARRAY_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* bucket structure for use by hashed indices */ - -#define BOOST_MULTI_INDEX_BA_SIZES_32BIT \ -(53ul)(97ul)(193ul)(389ul)(769ul) \ -(1543ul)(3079ul)(6151ul)(12289ul)(24593ul) \ -(49157ul)(98317ul)(196613ul)(393241ul)(786433ul) \ -(1572869ul)(3145739ul)(6291469ul)(12582917ul)(25165843ul) \ -(50331653ul)(100663319ul)(201326611ul)(402653189ul)(805306457ul) \ -(1610612741ul)(3221225473ul) - -#if ((((ULONG_MAX>>16)>>16)>>16)>>15)==0 /* unsigned long less than 64 bits */ -#define BOOST_MULTI_INDEX_BA_SIZES \ -BOOST_MULTI_INDEX_BA_SIZES_32BIT \ -(4294967291ul) -#else - /* obtained with aid from - * http://javaboutique.internet.com/prime_numb/ - * http://www.rsok.com/~jrm/next_ten_primes.html - * and verified with - * http://www.alpertron.com.ar/ECM.HTM - */ - -#define BOOST_MULTI_INDEX_BA_SIZES \ -BOOST_MULTI_INDEX_BA_SIZES_32BIT \ -(6442450939ul)(12884901893ul)(25769803751ul)(51539607551ul) \ -(103079215111ul)(206158430209ul)(412316860441ul)(824633720831ul) \ -(1649267441651ul)(3298534883309ul)(6597069766657ul)(13194139533299ul) \ -(26388279066623ul)(52776558133303ul)(105553116266489ul)(211106232532969ul) \ -(422212465066001ul)(844424930131963ul)(1688849860263953ul) \ -(3377699720527861ul)(6755399441055731ul)(13510798882111483ul) \ -(27021597764222939ul)(54043195528445957ul)(108086391056891903ul) \ -(216172782113783843ul)(432345564227567621ul)(864691128455135207ul) \ -(1729382256910270481ul)(3458764513820540933ul)(6917529027641081903ul) \ -(13835058055282163729ul)(18446744073709551557ul) -#endif - -template /* templatized to have in-header static var defs */ -class bucket_array_base:private noncopyable -{ -protected: - static const std::size_t sizes[ - BOOST_PP_SEQ_SIZE(BOOST_MULTI_INDEX_BA_SIZES)]; - - static std::size_t size_index(std::size_t n) - { - const std::size_t *bound=std::lower_bound(sizes,sizes+sizes_length,n); - if(bound==sizes+sizes_length)--bound; - return bound-sizes; - } - -#define BOOST_MULTI_INDEX_BA_POSITION_CASE(z,n,_) \ - case n:return hash%BOOST_PP_SEQ_ELEM(n,BOOST_MULTI_INDEX_BA_SIZES); - - static std::size_t position(std::size_t hash,std::size_t size_index_) - { - /* Accelerate hash%sizes[size_index_] by replacing with a switch on - * hash%Ci expressions, each Ci a compile-time constant, which the - * compiler can implement without using integer division. - */ - - switch(size_index_){ - default: /* never used */ - BOOST_PP_REPEAT( - BOOST_PP_SEQ_SIZE(BOOST_MULTI_INDEX_BA_SIZES), - BOOST_MULTI_INDEX_BA_POSITION_CASE,~) - } - } - -private: - static const std::size_t sizes_length; -}; - -template -const std::size_t bucket_array_base<_>::sizes[]={ - BOOST_PP_SEQ_ENUM(BOOST_MULTI_INDEX_BA_SIZES) -}; - -template -const std::size_t bucket_array_base<_>::sizes_length= - sizeof(bucket_array_base<_>::sizes)/ - sizeof(bucket_array_base<_>::sizes[0]); - -#undef BOOST_MULTI_INDEX_BA_POSITION_CASE -#undef BOOST_MULTI_INDEX_BA_SIZES -#undef BOOST_MULTI_INDEX_BA_SIZES_32BIT - -template -class bucket_array:bucket_array_base<> -{ - typedef bucket_array_base<> super; - typedef hashed_index_base_node_impl< - typename boost::detail::allocator::rebind_to< - Allocator, - char - >::type - > base_node_impl_type; - -public: - typedef typename base_node_impl_type::base_pointer base_pointer; - typedef typename base_node_impl_type::pointer pointer; - - bucket_array(const Allocator& al,pointer end_,std::size_t size_): - size_index_(super::size_index(size_)), - spc(al,super::sizes[size_index_]+1) - { - clear(end_); - } - - std::size_t size()const - { - return super::sizes[size_index_]; - } - - std::size_t position(std::size_t hash)const - { - return super::position(hash,size_index_); - } - - base_pointer begin()const{return buckets();} - base_pointer end()const{return buckets()+size();} - base_pointer at(std::size_t n)const{return buckets()+n;} - - void clear(pointer end_) - { - for(base_pointer x=begin(),y=end();x!=y;++x)x->prior()=pointer(0); - end()->prior()=end_->prior()=end_; - end_->next()=end(); - } - - void swap(bucket_array& x) - { - std::swap(size_index_,x.size_index_); - spc.swap(x.spc); - } - -private: - std::size_t size_index_; - auto_space spc; - - base_pointer buckets()const - { - return spc.data(); - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - friend class boost::serialization::access; - - /* bucket_arrays do not emit any kind of serialization info. They are - * fed to Boost.Serialization as hashed index iterators need to track - * them during serialization. - */ - - template - void serialize(Archive&,const unsigned int) - { - } -#endif -}; - -template -void swap(bucket_array& x,bucket_array& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -/* bucket_arrays never get constructed directly by Boost.Serialization, - * as archives are always fed pointers to previously existent - * arrays. So, if this is called it means we are dealing with a - * somehow invalid archive. - */ - -#if defined(BOOST_NO_ARGUMENT_DEPENDENT_LOOKUP) -namespace serialization{ -#else -namespace multi_index{ -namespace detail{ -#endif - -template -inline void load_construct_data( - Archive&,boost::multi_index::detail::bucket_array*, - const unsigned int) -{ - throw_exception( - archive::archive_exception(archive::archive_exception::other_exception)); -} - -#if defined(BOOST_NO_ARGUMENT_DEPENDENT_LOOKUP) -} /* namespace serialization */ -#else -} /* namespace multi_index::detail */ -} /* namespace multi_index */ -#endif - -#endif - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/cons_stdtuple.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/cons_stdtuple.hpp deleted file mode 100644 index 855c5e06aa9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/cons_stdtuple.hpp +++ /dev/null @@ -1,93 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_CONS_STDTUPLE_HPP -#define BOOST_MULTI_INDEX_DETAIL_CONS_STDTUPLE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* std::tuple wrapper providing the cons-based interface of boost::tuple for - * composite_key interoperability. - */ - -template -struct cons_stdtuple; - -struct cons_stdtuple_ctor_terminal -{ - typedef boost::tuples::null_type result_type; - - template - static result_type create(const StdTuple&) - { - return boost::tuples::null_type(); - } -}; - -template -struct cons_stdtuple_ctor_normal -{ - typedef cons_stdtuple result_type; - - static result_type create(const StdTuple& t) - { - return result_type(t); - } -}; - -template -struct cons_stdtuple_ctor: - boost::mpl::if_c< - N::value, - cons_stdtuple_ctor_normal, - cons_stdtuple_ctor_terminal - >::type -{}; - -template -struct cons_stdtuple -{ - typedef typename std::tuple_element::type head_type; - typedef cons_stdtuple_ctor tail_ctor; - typedef typename tail_ctor::result_type tail_type; - - cons_stdtuple(const StdTuple& t_):t(t_){} - - const head_type& get_head()const{return std::get(t);} - tail_type get_tail()const{return tail_ctor::create(t);} - - const StdTuple& t; -}; - -template -typename cons_stdtuple_ctor::result_type -make_cons_stdtuple(const StdTuple& t) -{ - return cons_stdtuple_ctor::create(t); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/converter.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/converter.hpp deleted file mode 100644 index 3e04a3e8295..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/converter.hpp +++ /dev/null @@ -1,52 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_CONVERTER_HPP -#define BOOST_MULTI_INDEX_DETAIL_CONVERTER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* converter offers means to access indices of a given multi_index_container - * and for convertibilty between index iterators, so providing a - * localized access point for get() and project() functions. - */ - -template -struct converter -{ - static const Index& index(const MultiIndexContainer& x){return x;} - static Index& index(MultiIndexContainer& x){return x;} - - static typename Index::const_iterator const_iterator( - const MultiIndexContainer& x,typename MultiIndexContainer::node_type* node) - { - return x.Index::make_iterator(node); - } - - static typename Index::iterator iterator( - MultiIndexContainer& x,typename MultiIndexContainer::node_type* node) - { - return x.Index::make_iterator(node); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/copy_map.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/copy_map.hpp deleted file mode 100644 index 9a34b259cf3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/copy_map.hpp +++ /dev/null @@ -1,142 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_COPY_MAP_HPP -#define BOOST_MULTI_INDEX_DETAIL_COPY_MAP_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* copy_map is used as an auxiliary structure during copy_() operations. - * When a container with n nodes is replicated, node_map holds the pairings - * between original and copied nodes, and provides a fast way to find a - * copied node from an original one. - * The semantics of the class are not simple, and no attempt has been made - * to enforce it: multi_index_container handles it right. On the other hand, - * the const interface, which is the one provided to index implementations, - * only allows for: - * - Enumeration of pairs of (original,copied) nodes (excluding the headers), - * - fast retrieval of copied nodes (including the headers.) - */ - -template -struct copy_map_entry -{ - copy_map_entry(Node* f,Node* s):first(f),second(s){} - - Node* first; - Node* second; - - bool operator<(const copy_map_entry& x)const - { - return std::less()(first,x.first); - } -}; - -template -class copy_map:private noncopyable -{ -public: - typedef const copy_map_entry* const_iterator; - - copy_map( - const Allocator& al,std::size_t size,Node* header_org,Node* header_cpy): - al_(al),size_(size),spc(al_,size_),n(0), - header_org_(header_org),header_cpy_(header_cpy),released(false) - {} - - ~copy_map() - { - if(!released){ - for(std::size_t i=0;isecond->value()); - deallocate((spc.data()+i)->second); - } - } - } - - const_iterator begin()const{return raw_ptr(spc.data());} - const_iterator end()const{return raw_ptr(spc.data()+n);} - - void clone(Node* node) - { - (spc.data()+n)->first=node; - (spc.data()+n)->second=raw_ptr(al_.allocate(1)); - BOOST_TRY{ - boost::detail::allocator::construct( - &(spc.data()+n)->second->value(),node->value()); - } - BOOST_CATCH(...){ - deallocate((spc.data()+n)->second); - BOOST_RETHROW; - } - BOOST_CATCH_END - ++n; - - if(n==size_){ - std::sort( - raw_ptr*>(spc.data()), - raw_ptr*>(spc.data())+size_); - } - } - - Node* find(Node* node)const - { - if(node==header_org_)return header_cpy_; - return std::lower_bound( - begin(),end(),copy_map_entry(node,0))->second; - } - - void release() - { - released=true; - } - -private: - typedef typename boost::detail::allocator::rebind_to< - Allocator,Node - >::type allocator_type; - typedef typename allocator_type::pointer allocator_pointer; - - allocator_type al_; - std::size_t size_; - auto_space,Allocator> spc; - std::size_t n; - Node* header_org_; - Node* header_cpy_; - bool released; - - void deallocate(Node* node) - { - al_.deallocate(static_cast(node),1); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/do_not_copy_elements_tag.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/do_not_copy_elements_tag.hpp deleted file mode 100644 index f0fa7304253..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/do_not_copy_elements_tag.hpp +++ /dev/null @@ -1,34 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_DO_NOT_COPY_ELEMENTS_TAG_HPP -#define BOOST_MULTI_INDEX_DETAIL_DO_NOT_COPY_ELEMENTS_TAG_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Used to mark a special ctor variant that copies the internal objects of - * a container but not its elements. - */ - -struct do_not_copy_elements_tag{}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/duplicates_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/duplicates_iterator.hpp deleted file mode 100644 index cbebf264045..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/duplicates_iterator.hpp +++ /dev/null @@ -1,120 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_DUPLICATES_ITERATOR_HPP -#define BOOST_MULTI_INDEX_DETAIL_DUPLICATES_ITERATOR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* duplicates_operator is given a range of ordered elements and - * passes only over those which are duplicated. - */ - -template -class duplicates_iterator -{ -public: - typedef typename Node::value_type value_type; - typedef std::ptrdiff_t difference_type; - typedef const typename Node::value_type* pointer; - typedef const typename Node::value_type& reference; - typedef std::forward_iterator_tag iterator_category; - - duplicates_iterator(Node* node_,Node* end_,Predicate pred_): - node(node_),begin_chunk(0),end(end_),pred(pred_) - { - advance(); - } - - duplicates_iterator(Node* end_,Predicate pred_): - node(end_),begin_chunk(end_),end(end_),pred(pred_) - { - } - - reference operator*()const - { - return node->value(); - } - - pointer operator->()const - { - return &node->value(); - } - - duplicates_iterator& operator++() - { - Node::increment(node); - sync(); - return *this; - } - - duplicates_iterator operator++(int) - { - duplicates_iterator tmp(*this); - ++(*this); - return tmp; - } - - Node* get_node()const{return node;} - -private: - void sync() - { - if(node!=end&&pred(begin_chunk->value(),node->value()))advance(); - } - - void advance() - { - for(Node* node2=node;node!=end;node=node2){ - Node::increment(node2); - if(node2!=end&&!pred(node->value(),node2->value()))break; - } - begin_chunk=node; - } - - Node* node; - Node* begin_chunk; - Node* end; - Predicate pred; -}; - -template -bool operator==( - const duplicates_iterator& x, - const duplicates_iterator& y) -{ - return x.get_node()==y.get_node(); -} - -template -bool operator!=( - const duplicates_iterator& x, - const duplicates_iterator& y) -{ - return !(x==y); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/has_tag.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/has_tag.hpp deleted file mode 100644 index 217b61143af..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/has_tag.hpp +++ /dev/null @@ -1,42 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_HAS_TAG_HPP -#define BOOST_MULTI_INDEX_DETAIL_HAS_TAG_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* determines whether an index type has a given tag in its tag list */ - -template -struct has_tag -{ - template - struct apply:mpl::contains - { - }; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_args.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_args.hpp deleted file mode 100644 index 81902f5a4a5..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_args.hpp +++ /dev/null @@ -1,105 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_ARGS_HPP -#define BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_ARGS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Hashed index specifiers can be instantiated in two forms: - * - * (hashed_unique|hashed_non_unique)< - * KeyFromValue, - * Hash=boost::hash, - * Pred=std::equal_to > - * (hashed_unique|hashed_non_unique)< - * TagList, - * KeyFromValue, - * Hash=boost::hash, - * Pred=std::equal_to > - * - * hashed_index_args implements the machinery to accept this - * argument-dependent polymorphism. - */ - -template -struct index_args_default_hash -{ - typedef ::boost::hash type; -}; - -template -struct index_args_default_pred -{ - typedef std::equal_to type; -}; - -template -struct hashed_index_args -{ - typedef is_tag full_form; - - typedef typename mpl::if_< - full_form, - Arg1, - tag< > >::type tag_list_type; - typedef typename mpl::if_< - full_form, - Arg2, - Arg1>::type key_from_value_type; - typedef typename mpl::if_< - full_form, - Arg3, - Arg2>::type supplied_hash_type; - typedef typename mpl::eval_if< - mpl::is_na, - index_args_default_hash, - mpl::identity - >::type hash_type; - typedef typename mpl::if_< - full_form, - Arg4, - Arg3>::type supplied_pred_type; - typedef typename mpl::eval_if< - mpl::is_na, - index_args_default_pred, - mpl::identity - >::type pred_type; - - BOOST_STATIC_ASSERT(is_tag::value); - BOOST_STATIC_ASSERT(!mpl::is_na::value); - BOOST_STATIC_ASSERT(!mpl::is_na::value); - BOOST_STATIC_ASSERT(!mpl::is_na::value); -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_iterator.hpp deleted file mode 100644 index 8d063002a1d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_iterator.hpp +++ /dev/null @@ -1,166 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_ITERATOR_HPP -#define BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_ITERATOR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Iterator class for hashed indices. - */ - -struct hashed_index_global_iterator_tag{}; -struct hashed_index_local_iterator_tag{}; - -template -class hashed_index_iterator: - public forward_iterator_helper< - hashed_index_iterator, - typename Node::value_type, - std::ptrdiff_t, - const typename Node::value_type*, - const typename Node::value_type&> -{ -public: - /* coverity[uninit_ctor]: suppress warning */ - hashed_index_iterator(){} - hashed_index_iterator(Node* node_):node(node_){} - - const typename Node::value_type& operator*()const - { - return node->value(); - } - - hashed_index_iterator& operator++() - { - this->increment(Category()); - return *this; - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* Serialization. As for why the following is public, - * see explanation in safe_mode_iterator notes in safe_mode.hpp. - */ - - BOOST_SERIALIZATION_SPLIT_MEMBER() - - typedef typename Node::base_type node_base_type; - - template - void save(Archive& ar,const unsigned int)const - { - node_base_type* bnode=node; - ar< - void load(Archive& ar,const unsigned int version) - { - load(ar,version,Category()); - } - - template - void load( - Archive& ar,const unsigned int version,hashed_index_global_iterator_tag) - { - node_base_type* bnode; - ar>>serialization::make_nvp("pointer",bnode); - node=static_cast(bnode); - if(version<1){ - BucketArray* throw_away; /* consume unused ptr */ - ar>>serialization::make_nvp("pointer",throw_away); - } - } - - template - void load( - Archive& ar,const unsigned int version,hashed_index_local_iterator_tag) - { - node_base_type* bnode; - ar>>serialization::make_nvp("pointer",bnode); - node=static_cast(bnode); - if(version<1){ - BucketArray* buckets; - ar>>serialization::make_nvp("pointer",buckets); - if(buckets&&node&&node->impl()==buckets->end()->prior()){ - /* end local_iterators used to point to end node, now they are null */ - node=0; - } - } - } -#endif - - /* get_node is not to be used by the user */ - - typedef Node node_type; - - Node* get_node()const{return node;} - -private: - - void increment(hashed_index_global_iterator_tag) - { - Node::increment(node); - } - - void increment(hashed_index_local_iterator_tag) - { - Node::increment_local(node); - } - - Node* node; -}; - -template -bool operator==( - const hashed_index_iterator& x, - const hashed_index_iterator& y) -{ - return x.get_node()==y.get_node(); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -/* class version = 1 : hashed_index_iterator does no longer serialize a bucket - * array pointer. - */ - -namespace serialization { -template -struct version< - boost::multi_index::detail::hashed_index_iterator -> -{ - BOOST_STATIC_CONSTANT(int,value=1); -}; -} /* namespace serialization */ -#endif - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_node.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_node.hpp deleted file mode 100644 index 7788e810ac9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_node.hpp +++ /dev/null @@ -1,778 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_NODE_HPP -#define BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_NODE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Certain C++ requirements on unordered associative containers (see LWG issue - * #579) imply a data structure where nodes are linked in a single list, which - * in its turn forces implementors to add additional overhed per node to - * associate each with its corresponding bucket. Others resort to storing hash - * values, we use an alternative structure providing unconditional O(1) - * manipulation, even in situations of unfair hash distribution, plus some - * lookup speedups. For unique indices we maintain a doubly linked list of - * nodes except that if N is the first node of a bucket its associated - * bucket node is embedded between N and the preceding node in the following - * manner: - * - * +---+ +---+ +---+ +---+ - * <--+ |<--+ | <--+ |<--+ | - * ... | B0| | B1| ... | B1| | B2| ... - * | |-+ | +--> | |-+ | +--> - * +-+-+ | +---+ +-+-+ | +---+ - * | ^ | ^ - * | | | | - * | +-+ | +-+ - * | | | | - * v | v | - * --+---+---+---+-- --+---+---+---+-- - * ... | | B1| | ... | | B2| | ... - * --+---+---+---+-- --+---+---+---+-- - * - * - * The fist and last nodes of buckets can be checked with - * - * first node of a bucket: Npn != N - * last node of a bucket: Nnp != N - * - * (n and p short for ->next(), ->prior(), bucket nodes have prior pointers - * only). Pure insert and erase (without lookup) can be unconditionally done - * in O(1). - * For non-unique indices we add the following additional complexity: when - * there is a group of 3 or more equivalent elements, they are linked as - * follows: - * - * +-----------------------+ - * | v - * +---+ | +---+ +---+ +---+ - * | | +-+ | | |<--+ | - * | F | | S | ... | P | | L | - * | +-->| | | +-+ | | - * +---+ +---+ +---+ | +---+ - * ^ | - * +-----------------------+ - * - * F, S, P and L are the first, second, penultimate and last node in the - * group, respectively (S and P can coincide if the group has size 3.) This - * arrangement is used to skip equivalent elements in O(1) when doing lookup, - * while preserving O(1) insert/erase. The following invariants identify - * special positions (some of the operations have to be carefully implemented - * as Xnn is not valid if Xn points to a bucket): - * - * first node of a bucket: Npnp == N - * last node of a bucket: Nnpp == N - * first node of a group: Nnp != N && Nnppn == N - * second node of a group: Npn != N && Nppnn == N - * n-1 node of a group: Nnp != N && Nnnpp == N - * last node of a group: Npn != N && Npnnp == N - * - * The memory overhead is one pointer per bucket plus two pointers per node, - * probably unbeatable. The resulting structure is bidirectonally traversable, - * though currently we are just providing forward iteration. - */ - -template -struct hashed_index_node_impl; - -/* half-header (only prior() pointer) to use for the bucket array */ - -template -struct hashed_index_base_node_impl -{ - typedef typename - boost::detail::allocator::rebind_to< - Allocator,hashed_index_base_node_impl - >::type::pointer base_pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator,hashed_index_base_node_impl - >::type::const_pointer const_base_pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator, - hashed_index_node_impl - >::type::pointer pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator, - hashed_index_node_impl - >::type::const_pointer const_pointer; - - pointer& prior(){return prior_;} - pointer prior()const{return prior_;} - -private: - pointer prior_; -}; - -/* full header (prior() and next()) for the nodes */ - -template -struct hashed_index_node_impl:hashed_index_base_node_impl -{ -private: - typedef hashed_index_base_node_impl super; - -public: - typedef typename super::base_pointer base_pointer; - typedef typename super::const_base_pointer const_base_pointer; - typedef typename super::pointer pointer; - typedef typename super::const_pointer const_pointer; - - base_pointer& next(){return next_;} - base_pointer next()const{return next_;} - - static pointer pointer_from(base_pointer x) - { - return static_cast( - static_cast( - raw_ptr(x))); - } - - static base_pointer base_pointer_from(pointer x) - { - return static_cast( - raw_ptr(x)); - } - -private: - base_pointer next_; -}; - -/* Boost.MultiIndex requires machinery to reverse unlink operations. A simple - * way to make a pointer-manipulation function undoable is to templatize - * its internal pointer assignments with a functor that, besides doing the - * assignment, keeps track of the original pointer values and can later undo - * the operations in reverse order. - */ - -struct default_assigner -{ - template void operator()(T& x,const T& val){x=val;} -}; - -template -struct unlink_undo_assigner -{ - typedef typename Node::base_pointer base_pointer; - typedef typename Node::pointer pointer; - - unlink_undo_assigner():pointer_track_count(0),base_pointer_track_count(0){} - - void operator()(pointer& x,pointer val) - { - pointer_tracks[pointer_track_count].x=&x; - pointer_tracks[pointer_track_count++].val=x; - x=val; - } - - void operator()(base_pointer& x,base_pointer val) - { - base_pointer_tracks[base_pointer_track_count].x=&x; - base_pointer_tracks[base_pointer_track_count++].val=x; - x=val; - } - - void operator()() /* undo op */ - { - /* in the absence of aliasing, restitution order is immaterial */ - - while(pointer_track_count--){ - *(pointer_tracks[pointer_track_count].x)= - pointer_tracks[pointer_track_count].val; - } - while(base_pointer_track_count--){ - *(base_pointer_tracks[base_pointer_track_count].x)= - base_pointer_tracks[base_pointer_track_count].val; - } - } - - struct pointer_track {pointer* x; pointer val;}; - struct base_pointer_track{base_pointer* x; base_pointer val;}; - - /* We know the maximum number of pointer and base pointer assignments that - * the two unlink versions do, so we can statically reserve the needed - * storage. - */ - - pointer_track pointer_tracks[3]; - int pointer_track_count; - base_pointer_track base_pointer_tracks[2]; - int base_pointer_track_count; -}; - -/* algorithmic stuff for unique and non-unique variants */ - -struct hashed_unique_tag{}; -struct hashed_non_unique_tag{}; - -template -struct hashed_index_node_alg; - -template -struct hashed_index_node_alg -{ - typedef typename Node::base_pointer base_pointer; - typedef typename Node::const_base_pointer const_base_pointer; - typedef typename Node::pointer pointer; - typedef typename Node::const_pointer const_pointer; - - static bool is_first_of_bucket(pointer x) - { - return x->prior()->next()!=base_pointer_from(x); - } - - static pointer after(pointer x) - { - return is_last_of_bucket(x)?x->next()->prior():pointer_from(x->next()); - } - - static pointer after_local(pointer x) - { - return is_last_of_bucket(x)?pointer(0):pointer_from(x->next()); - } - - static pointer next_to_inspect(pointer x) - { - return is_last_of_bucket(x)?pointer(0):pointer_from(x->next()); - } - - static void link(pointer x,base_pointer buc,pointer end) - { - if(buc->prior()==pointer(0)){ /* empty bucket */ - x->prior()=end->prior(); - x->next()=end->prior()->next(); - x->prior()->next()=buc; - buc->prior()=x; - end->prior()=x; - } - else{ - x->prior()=buc->prior()->prior(); - x->next()=base_pointer_from(buc->prior()); - buc->prior()=x; - x->next()->prior()=x; - } - } - - static void unlink(pointer x) - { - default_assigner assign; - unlink(x,assign); - } - - typedef unlink_undo_assigner unlink_undo; - - template - static void unlink(pointer x,Assigner& assign) - { - if(is_first_of_bucket(x)){ - if(is_last_of_bucket(x)){ - assign(x->prior()->next()->prior(),pointer(0)); - assign(x->prior()->next(),x->next()); - assign(x->next()->prior()->prior(),x->prior()); - } - else{ - assign(x->prior()->next()->prior(),pointer_from(x->next())); - assign(x->next()->prior(),x->prior()); - } - } - else if(is_last_of_bucket(x)){ - assign(x->prior()->next(),x->next()); - assign(x->next()->prior()->prior(),x->prior()); - } - else{ - assign(x->prior()->next(),x->next()); - assign(x->next()->prior(),x->prior()); - } - } - - /* used only at rehashing */ - - static void append(pointer x,pointer end) - { - x->prior()=end->prior(); - x->next()=end->prior()->next(); - x->prior()->next()=base_pointer_from(x); - end->prior()=x; - } - - static bool unlink_last(pointer end) - { - /* returns true iff bucket is emptied */ - - pointer x=end->prior(); - if(x->prior()->next()==base_pointer_from(x)){ - x->prior()->next()=x->next(); - end->prior()=x->prior(); - return false; - } - else{ - x->prior()->next()->prior()=pointer(0); - x->prior()->next()=x->next(); - end->prior()=x->prior(); - return true; - } - } - -private: - static pointer pointer_from(base_pointer x) - { - return Node::pointer_from(x); - } - - static base_pointer base_pointer_from(pointer x) - { - return Node::base_pointer_from(x); - } - - static bool is_last_of_bucket(pointer x) - { - return x->next()->prior()!=x; - } -}; - -template -struct hashed_index_node_alg -{ - typedef typename Node::base_pointer base_pointer; - typedef typename Node::const_base_pointer const_base_pointer; - typedef typename Node::pointer pointer; - typedef typename Node::const_pointer const_pointer; - - static bool is_first_of_bucket(pointer x) - { - return x->prior()->next()->prior()==x; - } - - static bool is_first_of_group(pointer x) - { - return - x->next()->prior()!=x&& - x->next()->prior()->prior()->next()==base_pointer_from(x); - } - - static pointer after(pointer x) - { - if(x->next()->prior()==x)return pointer_from(x->next()); - if(x->next()->prior()->prior()==x)return x->next()->prior(); - if(x->next()->prior()->prior()->next()==base_pointer_from(x)) - return pointer_from(x->next()); - return pointer_from(x->next())->next()->prior(); - } - - static pointer after_local(pointer x) - { - if(x->next()->prior()==x)return pointer_from(x->next()); - if(x->next()->prior()->prior()==x)return pointer(0); - if(x->next()->prior()->prior()->next()==base_pointer_from(x)) - return pointer_from(x->next()); - return pointer_from(x->next())->next()->prior(); - } - - static pointer next_to_inspect(pointer x) - { - if(x->next()->prior()==x)return pointer_from(x->next()); - if(x->next()->prior()->prior()==x)return pointer(0); - if(x->next()->prior()->next()->prior()!=x->next()->prior()) - return pointer(0); - return pointer_from(x->next()->prior()->next()); - } - - static void link(pointer x,base_pointer buc,pointer end) - { - if(buc->prior()==pointer(0)){ /* empty bucket */ - x->prior()=end->prior(); - x->next()=end->prior()->next(); - x->prior()->next()=buc; - buc->prior()=x; - end->prior()=x; - } - else{ - x->prior()=buc->prior()->prior(); - x->next()=base_pointer_from(buc->prior()); - buc->prior()=x; - x->next()->prior()=x; - } - }; - - static void link(pointer x,pointer first,pointer last) - { - x->prior()=first->prior(); - x->next()=base_pointer_from(first); - if(is_first_of_bucket(first)){ - x->prior()->next()->prior()=x; - } - else{ - x->prior()->next()=base_pointer_from(x); - } - - if(first==last){ - last->prior()=x; - } - else if(first->next()==base_pointer_from(last)){ - first->prior()=last; - first->next()=base_pointer_from(x); - } - else{ - pointer second=pointer_from(first->next()), - lastbutone=last->prior(); - second->prior()=first; - first->prior()=last; - lastbutone->next()=base_pointer_from(x); - } - } - - static void unlink(pointer x) - { - default_assigner assign; - unlink(x,assign); - } - - typedef unlink_undo_assigner unlink_undo; - - template - static void unlink(pointer x,Assigner& assign) - { - if(x->prior()->next()==base_pointer_from(x)){ - if(x->next()->prior()==x){ - left_unlink(x,assign); - right_unlink(x,assign); - } - else if(x->next()->prior()->prior()==x){ /* last of bucket */ - left_unlink(x,assign); - right_unlink_last_of_bucket(x,assign); - } - else if(x->next()->prior()->prior()->next()== - base_pointer_from(x)){ /* first of group size */ - left_unlink(x,assign); - right_unlink_first_of_group(x,assign); - } - else{ /* n-1 of group */ - unlink_last_but_one_of_group(x,assign); - } - } - else if(x->prior()->next()->prior()==x){ /* first of bucket */ - if(x->next()->prior()==x){ - left_unlink_first_of_bucket(x,assign); - right_unlink(x,assign); - } - else if(x->next()->prior()->prior()==x){ /* last of bucket */ - assign(x->prior()->next()->prior(),pointer(0)); - assign(x->prior()->next(),x->next()); - assign(x->next()->prior()->prior(),x->prior()); - } - else{ /* first of group */ - left_unlink_first_of_bucket(x,assign); - right_unlink_first_of_group(x,assign); - } - } - else if(x->next()->prior()->prior()==x){ /* last of group and bucket */ - left_unlink_last_of_group(x,assign); - right_unlink_last_of_bucket(x,assign); - } - else if(pointer_from(x->prior()->prior()->next()) - ->next()==base_pointer_from(x)){ /* second of group */ - unlink_second_of_group(x,assign); - } - else{ /* last of group, ~(last of bucket) */ - left_unlink_last_of_group(x,assign); - right_unlink(x,assign); - } - } - - /* used only at rehashing */ - - static void link_range( - pointer first,pointer last,base_pointer buc,pointer cend) - { - if(buc->prior()==pointer(0)){ /* empty bucket */ - first->prior()=cend->prior(); - last->next()=cend->prior()->next(); - first->prior()->next()=buc; - buc->prior()=first; - cend->prior()=last; - } - else{ - first->prior()=buc->prior()->prior(); - last->next()=base_pointer_from(buc->prior()); - buc->prior()=first; - last->next()->prior()=last; - } - } - - static void append_range(pointer first,pointer last,pointer cend) - { - first->prior()=cend->prior(); - last->next()=cend->prior()->next(); - first->prior()->next()=base_pointer_from(first); - cend->prior()=last; - } - - static std::pair unlink_last_group(pointer end) - { - /* returns first of group true iff bucket is emptied */ - - pointer x=end->prior(); - if(x->prior()->next()==base_pointer_from(x)){ - x->prior()->next()=x->next(); - end->prior()=x->prior(); - return std::make_pair(x,false); - } - else if(x->prior()->next()->prior()==x){ - x->prior()->next()->prior()=pointer(0); - x->prior()->next()=x->next(); - end->prior()=x->prior(); - return std::make_pair(x,true); - } - else{ - pointer y=pointer_from(x->prior()->next()); - - if(y->prior()->next()==base_pointer_from(y)){ - y->prior()->next()=x->next(); - end->prior()=y->prior(); - return std::make_pair(y,false); - } - else{ - y->prior()->next()->prior()=pointer(0); - y->prior()->next()=x->next(); - end->prior()=y->prior(); - return std::make_pair(y,true); - } - } - } - - static void unlink_range(pointer first,pointer last) - { - if(is_first_of_bucket(first)){ - if(is_last_of_bucket(last)){ - first->prior()->next()->prior()=pointer(0); - first->prior()->next()=last->next(); - last->next()->prior()->prior()=first->prior(); - } - else{ - first->prior()->next()->prior()=pointer_from(last->next()); - last->next()->prior()=first->prior(); - } - } - else if(is_last_of_bucket(last)){ - first->prior()->next()=last->next(); - last->next()->prior()->prior()=first->prior(); - } - else{ - first->prior()->next()=last->next(); - last->next()->prior()=first->prior(); - } - } - -private: - static pointer pointer_from(base_pointer x) - { - return Node::pointer_from(x); - } - - static base_pointer base_pointer_from(pointer x) - { - return Node::base_pointer_from(x); - } - - static bool is_last_of_bucket(pointer x) - { - return x->next()->prior()->prior()==x; - } - - template - static void left_unlink(pointer x,Assigner& assign) - { - assign(x->prior()->next(),x->next()); - } - - template - static void right_unlink(pointer x,Assigner& assign) - { - assign(x->next()->prior(),x->prior()); - } - - template - static void left_unlink_first_of_bucket(pointer x,Assigner& assign) - { - assign(x->prior()->next()->prior(),pointer_from(x->next())); - } - - template - static void right_unlink_last_of_bucket(pointer x,Assigner& assign) - { - assign(x->next()->prior()->prior(),x->prior()); - } - - template - static void right_unlink_first_of_group(pointer x,Assigner& assign) - { - pointer second=pointer_from(x->next()), - last=second->prior(), - lastbutone=last->prior(); - if(second==lastbutone){ - assign(second->next(),base_pointer_from(last)); - assign(second->prior(),x->prior()); - } - else{ - assign(lastbutone->next(),base_pointer_from(second)); - assign(second->next()->prior(),last); - assign(second->prior(),x->prior()); - } - } - - template - static void left_unlink_last_of_group(pointer x,Assigner& assign) - { - pointer lastbutone=x->prior(), - first=pointer_from(lastbutone->next()), - second=pointer_from(first->next()); - if(lastbutone==second){ - assign(lastbutone->prior(),first); - assign(lastbutone->next(),x->next()); - } - else{ - assign(second->prior(),lastbutone); - assign(lastbutone->prior()->next(),base_pointer_from(first)); - assign(lastbutone->next(),x->next()); - } - } - - template - static void unlink_last_but_one_of_group(pointer x,Assigner& assign) - { - pointer first=pointer_from(x->next()), - second=pointer_from(first->next()), - last=second->prior(); - if(second==x){ - assign(last->prior(),first); - assign(first->next(),base_pointer_from(last)); - } - else{ - assign(last->prior(),x->prior()); - assign(x->prior()->next(),base_pointer_from(first)); - } - } - - template - static void unlink_second_of_group(pointer x,Assigner& assign) - { - pointer last=x->prior(), - lastbutone=last->prior(), - first=pointer_from(lastbutone->next()); - if(lastbutone==x){ - assign(first->next(),base_pointer_from(last)); - assign(last->prior(),first); - } - else{ - assign(first->next(),x->next()); - assign(x->next()->prior(),last); - } - } -}; - -template -struct hashed_index_node_trampoline: - hashed_index_node_impl< - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > -{ - typedef typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type impl_allocator_type; - typedef hashed_index_node_impl impl_type; -}; - -template -struct hashed_index_node: - Super,hashed_index_node_trampoline -{ -private: - typedef hashed_index_node_trampoline trampoline; - -public: - typedef typename trampoline::impl_type impl_type; - typedef hashed_index_node_alg< - impl_type,Category> node_alg; - typedef typename trampoline::base_pointer impl_base_pointer; - typedef typename trampoline::const_base_pointer const_impl_base_pointer; - typedef typename trampoline::pointer impl_pointer; - typedef typename trampoline::const_pointer const_impl_pointer; - - impl_pointer& prior(){return trampoline::prior();} - impl_pointer prior()const{return trampoline::prior();} - impl_base_pointer& next(){return trampoline::next();} - impl_base_pointer next()const{return trampoline::next();} - - impl_pointer impl() - { - return static_cast( - static_cast(static_cast(this))); - } - - const_impl_pointer impl()const - { - return static_cast( - static_cast(static_cast(this))); - } - - static hashed_index_node* from_impl(impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - static const hashed_index_node* from_impl(const_impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - /* interoperability with hashed_index_iterator */ - - static void increment(hashed_index_node*& x) - { - x=from_impl(node_alg::after(x->impl())); - } - - static void increment_local(hashed_index_node*& x) - { - x=from_impl(node_alg::after_local(x->impl())); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/header_holder.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/header_holder.hpp deleted file mode 100644 index ca8a9b2edb1..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/header_holder.hpp +++ /dev/null @@ -1,50 +0,0 @@ -/* Copyright 2003-2008 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_HEADER_HOLDER_HPP -#define BOOST_MULTI_INDEX_DETAIL_HEADER_HOLDER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* A utility class used to hold a pointer to the header node. - * The base from member idiom is used because index classes, which are - * superclasses of multi_index_container, need this header in construction - * time. The allocation is made by the allocator of the multi_index_container - * class --hence, this allocator needs also be stored resorting - * to the base from member trick. - */ - -template -struct header_holder:private noncopyable -{ - header_holder():member(final().allocate_node()){} - ~header_holder(){final().deallocate_node(&*member);} - - NodeTypePtr member; - -private: - Final& final(){return *static_cast(this);} -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ignore_wstrict_aliasing.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ignore_wstrict_aliasing.hpp deleted file mode 100644 index ae398456d1f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ignore_wstrict_aliasing.hpp +++ /dev/null @@ -1,18 +0,0 @@ -/* Copyright 2003-2016 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#include - -#if defined(BOOST_GCC)&&(BOOST_GCC>=4*10000+6*100) -#if !defined(BOOST_MULTI_INDEX_DETAIL_RESTORE_WSTRICT_ALIASING) -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wstrict-aliasing" -#else -#pragma GCC diagnostic pop -#endif -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_base.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_base.hpp deleted file mode 100644 index 99000ed4813..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_base.hpp +++ /dev/null @@ -1,293 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_BASE_HPP -#define BOOST_MULTI_INDEX_DETAIL_INDEX_BASE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* The role of this class is threefold: - * - tops the linear hierarchy of indices. - * - terminates some cascading backbone function calls (insert_, etc.), - * - grants access to the backbone functions of the final - * multi_index_container class (for access restriction reasons, these - * cannot be called directly from the index classes.) - */ - -struct lvalue_tag{}; -struct rvalue_tag{}; -struct emplaced_tag{}; - -template -class index_base -{ -protected: - typedef index_node_base node_type; - typedef typename multi_index_node_type< - Value,IndexSpecifierList,Allocator>::type final_node_type; - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> final_type; - typedef tuples::null_type ctor_args_list; - typedef typename - boost::detail::allocator::rebind_to< - Allocator, - typename Allocator::value_type - >::type final_allocator_type; - typedef mpl::vector0<> index_type_list; - typedef mpl::vector0<> iterator_type_list; - typedef mpl::vector0<> const_iterator_type_list; - typedef copy_map< - final_node_type, - final_allocator_type> copy_map_type; - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - typedef index_saver< - node_type, - final_allocator_type> index_saver_type; - typedef index_loader< - node_type, - final_node_type, - final_allocator_type> index_loader_type; -#endif - -private: - typedef Value value_type; - -protected: - explicit index_base(const ctor_args_list&,const Allocator&){} - - index_base( - const index_base&, - do_not_copy_elements_tag) - {} - - void copy_( - const index_base&,const copy_map_type&) - {} - - final_node_type* insert_(const value_type& v,final_node_type*& x,lvalue_tag) - { - x=final().allocate_node(); - BOOST_TRY{ - boost::detail::allocator::construct(&x->value(),v); - } - BOOST_CATCH(...){ - final().deallocate_node(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - return x; - } - - final_node_type* insert_(const value_type& v,final_node_type*& x,rvalue_tag) - { - x=final().allocate_node(); - BOOST_TRY{ - /* This shoud have used a modified, T&&-compatible version of - * boost::detail::allocator::construct, but - * is too old and venerable to - * mess with; besides, it is a general internal utility and the imperfect - * perfect forwarding emulation of Boost.Move might break other libs. - */ - - new (&x->value()) value_type(boost::move(const_cast(v))); - } - BOOST_CATCH(...){ - final().deallocate_node(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - return x; - } - - final_node_type* insert_(const value_type&,final_node_type*& x,emplaced_tag) - { - return x; - } - - final_node_type* insert_( - const value_type& v,node_type*,final_node_type*& x,lvalue_tag) - { - return insert_(v,x,lvalue_tag()); - } - - final_node_type* insert_( - const value_type& v,node_type*,final_node_type*& x,rvalue_tag) - { - return insert_(v,x,rvalue_tag()); - } - - final_node_type* insert_( - const value_type&,node_type*,final_node_type*& x,emplaced_tag) - { - return x; - } - - void erase_(node_type* x) - { - boost::detail::allocator::destroy(&x->value()); - } - - void delete_node_(node_type* x) - { - boost::detail::allocator::destroy(&x->value()); - } - - void clear_(){} - - void swap_(index_base&){} - - void swap_elements_(index_base&){} - - bool replace_(const value_type& v,node_type* x,lvalue_tag) - { - x->value()=v; - return true; - } - - bool replace_(const value_type& v,node_type* x,rvalue_tag) - { - x->value()=boost::move(const_cast(v)); - return true; - } - - bool modify_(node_type*){return true;} - - bool modify_rollback_(node_type*){return true;} - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* serialization */ - - template - void save_(Archive&,const unsigned int,const index_saver_type&)const{} - - template - void load_(Archive&,const unsigned int,const index_loader_type&){} -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - bool invariant_()const{return true;} -#endif - - /* access to backbone memfuns of Final class */ - - final_type& final(){return *static_cast(this);} - const final_type& final()const{return *static_cast(this);} - - final_node_type* final_header()const{return final().header();} - - bool final_empty_()const{return final().empty_();} - std::size_t final_size_()const{return final().size_();} - std::size_t final_max_size_()const{return final().max_size_();} - - std::pair final_insert_(const value_type& x) - {return final().insert_(x);} - std::pair final_insert_rv_(const value_type& x) - {return final().insert_rv_(x);} - template - std::pair final_insert_ref_(const T& t) - {return final().insert_ref_(t);} - template - std::pair final_insert_ref_(T& t) - {return final().insert_ref_(t);} - - template - std::pair final_emplace_( - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - return final().emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - } - - std::pair final_insert_( - const value_type& x,final_node_type* position) - {return final().insert_(x,position);} - std::pair final_insert_rv_( - const value_type& x,final_node_type* position) - {return final().insert_rv_(x,position);} - template - std::pair final_insert_ref_( - const T& t,final_node_type* position) - {return final().insert_ref_(t,position);} - template - std::pair final_insert_ref_( - T& t,final_node_type* position) - {return final().insert_ref_(t,position);} - - template - std::pair final_emplace_hint_( - final_node_type* position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - return final().emplace_hint_( - position,BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - } - - void final_erase_(final_node_type* x){final().erase_(x);} - - void final_delete_node_(final_node_type* x){final().delete_node_(x);} - void final_delete_all_nodes_(){final().delete_all_nodes_();} - void final_clear_(){final().clear_();} - - void final_swap_(final_type& x){final().swap_(x);} - - bool final_replace_( - const value_type& k,final_node_type* x) - {return final().replace_(k,x);} - bool final_replace_rv_( - const value_type& k,final_node_type* x) - {return final().replace_rv_(k,x);} - - template - bool final_modify_(Modifier& mod,final_node_type* x) - {return final().modify_(mod,x);} - - template - bool final_modify_(Modifier& mod,Rollback& back,final_node_type* x) - {return final().modify_(mod,back,x);} - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - void final_check_invariant_()const{final().check_invariant_();} -#endif -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_loader.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_loader.hpp deleted file mode 100644 index 71418a10e19..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_loader.hpp +++ /dev/null @@ -1,139 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_LOADER_HPP -#define BOOST_MULTI_INDEX_DETAIL_INDEX_LOADER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Counterpart of index_saver (check index_saver.hpp for serialization - * details.)* multi_index_container is in charge of supplying the info about - * the base sequence, and each index can subsequently load itself using the - * const interface of index_loader. - */ - -template -class index_loader:private noncopyable -{ -public: - index_loader(const Allocator& al,std::size_t size): - spc(al,size),size_(size),n(0),sorted(false) - { - } - - template - void add(Node* node,Archive& ar,const unsigned int) - { - ar>>serialization::make_nvp("position",*node); - entries()[n++]=node; - } - - template - void add_track(Node* node,Archive& ar,const unsigned int) - { - ar>>serialization::make_nvp("position",*node); - } - - /* A rearranger is passed two nodes, and is expected to - * reposition the second after the first. - * If the first node is 0, then the second should be moved - * to the beginning of the sequence. - */ - - template - void load(Rearranger r,Archive& ar,const unsigned int)const - { - FinalNode* prev=unchecked_load_node(ar); - if(!prev)return; - - if(!sorted){ - std::sort(entries(),entries()+size_); - sorted=true; - } - - check_node(prev); - - for(;;){ - for(;;){ - FinalNode* node=load_node(ar); - if(!node)break; - - if(node==prev)prev=0; - r(prev,node); - - prev=node; - } - prev=load_node(ar); - if(!prev)break; - } - } - -private: - Node** entries()const{return raw_ptr(spc.data());} - - /* We try to delay sorting as much as possible just in case it - * is not necessary, hence this version of load_node. - */ - - template - FinalNode* unchecked_load_node(Archive& ar)const - { - Node* node=0; - ar>>serialization::make_nvp("pointer",node); - return static_cast(node); - } - - template - FinalNode* load_node(Archive& ar)const - { - Node* node=0; - ar>>serialization::make_nvp("pointer",node); - check_node(node); - return static_cast(node); - } - - void check_node(Node* node)const - { - if(node!=0&&!std::binary_search(entries(),entries()+size_,node)){ - throw_exception( - archive::archive_exception( - archive::archive_exception::other_exception)); - } - } - - auto_space spc; - std::size_t size_; - std::size_t n; - mutable bool sorted; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_matcher.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_matcher.hpp deleted file mode 100644 index 34d1f9d5a8d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_matcher.hpp +++ /dev/null @@ -1,249 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_MATCHER_HPP -#define BOOST_MULTI_INDEX_DETAIL_INDEX_MATCHER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* index_matcher compares a sequence of elements against a - * base sequence, identifying those elements that belong to the - * longest subsequence which is ordered with respect to the base. - * For instance, if the base sequence is: - * - * 0 1 2 3 4 5 6 7 8 9 - * - * and the compared sequence (not necesarilly the same length): - * - * 1 4 2 3 0 7 8 9 - * - * the elements of the longest ordered subsequence are: - * - * 1 2 3 7 8 9 - * - * The algorithm for obtaining such a subsequence is called - * Patience Sorting, described in ch. 1 of: - * Aldous, D., Diaconis, P.: "Longest increasing subsequences: from - * patience sorting to the Baik-Deift-Johansson Theorem", Bulletin - * of the American Mathematical Society, vol. 36, no 4, pp. 413-432, - * July 1999. - * http://www.ams.org/bull/1999-36-04/S0273-0979-99-00796-X/ - * S0273-0979-99-00796-X.pdf - * - * This implementation is not fully generic since it assumes that - * the sequences given are pointed to by index iterators (having a - * get_node() memfun.) - */ - -namespace index_matcher{ - -/* The algorithm stores the nodes of the base sequence and a number - * of "piles" that are dynamically updated during the calculation - * stage. From a logical point of view, nodes form an independent - * sequence from piles. They are stored together so as to minimize - * allocated memory. - */ - -struct entry -{ - entry(void* node_,std::size_t pos_=0):node(node_),pos(pos_){} - - /* node stuff */ - - void* node; - std::size_t pos; - entry* previous; - bool ordered; - - struct less_by_node - { - bool operator()( - const entry& x,const entry& y)const - { - return std::less()(x.node,y.node); - } - }; - - /* pile stuff */ - - std::size_t pile_top; - entry* pile_top_entry; - - struct less_by_pile_top - { - bool operator()( - const entry& x,const entry& y)const - { - return x.pile_top -class algorithm_base:private noncopyable -{ -protected: - algorithm_base(const Allocator& al,std::size_t size): - spc(al,size),size_(size),n_(0),sorted(false) - { - } - - void add(void* node) - { - entries()[n_]=entry(node,n_); - ++n_; - } - - void begin_algorithm()const - { - if(!sorted){ - std::sort(entries(),entries()+size_,entry::less_by_node()); - sorted=true; - } - num_piles=0; - } - - void add_node_to_algorithm(void* node)const - { - entry* ent= - std::lower_bound( - entries(),entries()+size_, - entry(node),entry::less_by_node()); /* localize entry */ - ent->ordered=false; - std::size_t n=ent->pos; /* get its position */ - - entry dummy(0); - dummy.pile_top=n; - - entry* pile_ent= /* find the first available pile */ - std::lower_bound( /* to stack the entry */ - entries(),entries()+num_piles, - dummy,entry::less_by_pile_top()); - - pile_ent->pile_top=n; /* stack the entry */ - pile_ent->pile_top_entry=ent; - - /* if not the first pile, link entry to top of the preceding pile */ - if(pile_ent>&entries()[0]){ - ent->previous=(pile_ent-1)->pile_top_entry; - } - - if(pile_ent==&entries()[num_piles]){ /* new pile? */ - ++num_piles; - } - } - - void finish_algorithm()const - { - if(num_piles>0){ - /* Mark those elements which are in their correct position, i.e. those - * belonging to the longest increasing subsequence. These are those - * elements linked from the top of the last pile. - */ - - entry* ent=entries()[num_piles-1].pile_top_entry; - for(std::size_t n=num_piles;n--;){ - ent->ordered=true; - ent=ent->previous; - } - } - } - - bool is_ordered(void * node)const - { - return std::lower_bound( - entries(),entries()+size_, - entry(node),entry::less_by_node())->ordered; - } - -private: - entry* entries()const{return raw_ptr(spc.data());} - - auto_space spc; - std::size_t size_; - std::size_t n_; - mutable bool sorted; - mutable std::size_t num_piles; -}; - -/* The algorithm has three phases: - * - Initialization, during which the nodes of the base sequence are added. - * - Execution. - * - Results querying, through the is_ordered memfun. - */ - -template -class algorithm:private algorithm_base -{ - typedef algorithm_base super; - -public: - algorithm(const Allocator& al,std::size_t size):super(al,size){} - - void add(Node* node) - { - super::add(node); - } - - template - void execute(IndexIterator first,IndexIterator last)const - { - super::begin_algorithm(); - - for(IndexIterator it=first;it!=last;++it){ - add_node_to_algorithm(get_node(it)); - } - - super::finish_algorithm(); - } - - bool is_ordered(Node* node)const - { - return super::is_ordered(node); - } - -private: - void add_node_to_algorithm(Node* node)const - { - super::add_node_to_algorithm(node); - } - - template - static Node* get_node(IndexIterator it) - { - return static_cast(it.get_node()); - } -}; - -} /* namespace multi_index::detail::index_matcher */ - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_node_base.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_node_base.hpp deleted file mode 100644 index 1a1f0cae4be..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_node_base.hpp +++ /dev/null @@ -1,135 +0,0 @@ -/* Copyright 2003-2016 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_NODE_BASE_HPP -#define BOOST_MULTI_INDEX_DETAIL_INDEX_NODE_BASE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* index_node_base tops the node hierarchy of multi_index_container. It holds - * the value of the element contained. - */ - -template -struct pod_value_holder -{ - typename aligned_storage< - sizeof(Value), - alignment_of::value - >::type space; -}; - -template -struct index_node_base:private pod_value_holder -{ - typedef index_node_base base_type; /* used for serialization purposes */ - typedef Value value_type; - typedef Allocator allocator_type; - -#include - - value_type& value() - { - return *reinterpret_cast(&this->space); - } - - const value_type& value()const - { - return *reinterpret_cast(&this->space); - } - -#include - - static index_node_base* from_value(const value_type* p) - { - return static_cast( - reinterpret_cast*>( /* std 9.2.17 */ - const_cast(p))); - } - -private: -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - friend class boost::serialization::access; - - /* nodes do not emit any kind of serialization info. They are - * fed to Boost.Serialization so that pointers to nodes are - * tracked correctly. - */ - - template - void serialize(Archive&,const unsigned int) - { - } -#endif -}; - -template -Node* node_from_value(const Value* p) -{ - typedef typename Node::allocator_type allocator_type; - return static_cast( - index_node_base::from_value(p)); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -/* Index nodes never get constructed directly by Boost.Serialization, - * as archives are always fed pointers to previously existent - * nodes. So, if this is called it means we are dealing with a - * somehow invalid archive. - */ - -#if defined(BOOST_NO_ARGUMENT_DEPENDENT_LOOKUP) -namespace serialization{ -#else -namespace multi_index{ -namespace detail{ -#endif - -template -inline void load_construct_data( - Archive&,boost::multi_index::detail::index_node_base*, - const unsigned int) -{ - throw_exception( - archive::archive_exception(archive::archive_exception::other_exception)); -} - -#if defined(BOOST_NO_ARGUMENT_DEPENDENT_LOOKUP) -} /* namespace serialization */ -#else -} /* namespace multi_index::detail */ -} /* namespace multi_index */ -#endif - -#endif - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_saver.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_saver.hpp deleted file mode 100644 index ae09d4eba4f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_saver.hpp +++ /dev/null @@ -1,135 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_SAVER_HPP -#define BOOST_MULTI_INDEX_DETAIL_INDEX_SAVER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* index_saver accepts a base sequence of previously saved elements - * and saves a possibly reordered subsequence in an efficient manner, - * serializing only the information needed to rearrange the subsequence - * based on the original order of the base. - * multi_index_container is in charge of supplying the info about the - * base sequence, and each index can subsequently save itself using the - * const interface of index_saver. - */ - -template -class index_saver:private noncopyable -{ -public: - index_saver(const Allocator& al,std::size_t size):alg(al,size){} - - template - void add(Node* node,Archive& ar,const unsigned int) - { - ar< - void add_track(Node* node,Archive& ar,const unsigned int) - { - ar< - void save( - IndexIterator first,IndexIterator last,Archive& ar, - const unsigned int)const - { - /* calculate ordered positions */ - - alg.execute(first,last); - - /* Given a consecutive subsequence of displaced elements - * x1,...,xn, the following information is serialized: - * - * p0,p1,...,pn,0 - * - * where pi is a pointer to xi and p0 is a pointer to the element - * preceding x1. Crealy, from this information is possible to - * restore the original order on loading time. If x1 is the first - * element in the sequence, the following is serialized instead: - * - * p1,p1,...,pn,0 - * - * For each subsequence of n elements, n+2 pointers are serialized. - * An optimization policy is applied: consider for instance the - * sequence - * - * a,B,c,D - * - * where B and D are displaced, but c is in its correct position. - * Applying the schema described above we would serialize 6 pointers: - * - * p(a),p(B),0 - * p(c),p(D),0 - * - * but this can be reduced to 5 pointers by treating c as a displaced - * element: - * - * p(a),p(B),p(c),p(D),0 - */ - - std::size_t last_saved=3; /* distance to last pointer saved */ - for(IndexIterator it=first,prev=first;it!=last;prev=it++,++last_saved){ - if(!alg.is_ordered(get_node(it))){ - if(last_saved>1)save_node(get_node(prev),ar); - save_node(get_node(it),ar); - last_saved=0; - } - else if(last_saved==2)save_node(null_node(),ar); - } - if(last_saved<=2)save_node(null_node(),ar); - - /* marks the end of the serialization info for [first,last) */ - - save_node(null_node(),ar); - } - -private: - template - static Node* get_node(IndexIterator it) - { - return it.get_node(); - } - - static Node* null_node(){return 0;} - - template - static void save_node(Node* node,Archive& ar) - { - ar< alg; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/invariant_assert.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/invariant_assert.hpp deleted file mode 100644 index c6c547c7c33..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/invariant_assert.hpp +++ /dev/null @@ -1,21 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_INVARIANT_ASSERT_HPP -#define BOOST_MULTI_INDEX_DETAIL_INVARIANT_ASSERT_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#if !defined(BOOST_MULTI_INDEX_INVARIANT_ASSERT) -#include -#define BOOST_MULTI_INDEX_INVARIANT_ASSERT BOOST_ASSERT -#endif - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_index_list.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_index_list.hpp deleted file mode 100644 index f6a24218b81..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_index_list.hpp +++ /dev/null @@ -1,40 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_IS_INDEX_LIST_HPP -#define BOOST_MULTI_INDEX_DETAIL_IS_INDEX_LIST_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -struct is_index_list -{ - BOOST_STATIC_CONSTANT(bool,mpl_sequence=mpl::is_sequence::value); - BOOST_STATIC_CONSTANT(bool,non_empty=!mpl::empty::value); - BOOST_STATIC_CONSTANT(bool,value=mpl_sequence&&non_empty); -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_transparent.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_transparent.hpp deleted file mode 100644 index 72036d257e2..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_transparent.hpp +++ /dev/null @@ -1,135 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_IS_TRANSPARENT_HPP -#define BOOST_MULTI_INDEX_DETAIL_IS_TRANSPARENT_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Metafunction that checks if f(arg,arg2) executes without argument type - * conversion. By default (i.e. when it cannot be determined) it evaluates to - * true. - */ - -template -struct is_transparent:mpl::true_{}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#if !defined(BOOST_NO_SFINAE)&&!defined(BOOST_NO_SFINAE_EXPR)&& \ - !defined(BOOST_NO_CXX11_DECLTYPE)&& \ - (defined(BOOST_NO_CXX11_FINAL)||defined(BOOST_IS_FINAL)) - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -struct not_is_transparent_result_type{}; - -template -struct is_transparent_class_helper:F -{ - using F::operator(); - template - not_is_transparent_result_type operator()(const T&,const Q&)const; -}; - -template -struct is_transparent_class:mpl::true_{}; - -template -struct is_transparent_class< - F,Arg1,Arg2, - typename enable_if< - is_same< - decltype( - declval >()( - declval(),declval()) - ), - not_is_transparent_result_type - > - >::type ->:mpl::false_{}; - -template -struct is_transparent< - F,Arg1,Arg2, - typename enable_if< - mpl::and_< - is_class, - mpl::not_ > /* is_transparent_class_helper derives from F */ - > - >::type ->:is_transparent_class{}; - -template -struct is_transparent_function:mpl::true_{}; - -template -struct is_transparent_function< - F,Arg1,Arg2, - typename enable_if< - mpl::or_< - mpl::not_::arg1_type,const Arg1&>, - is_same::arg1_type,Arg1> - > >, - mpl::not_::arg2_type,const Arg2&>, - is_same::arg2_type,Arg2> - > > - > - >::type ->:mpl::false_{}; - -template -struct is_transparent< - F,Arg1,Arg2, - typename enable_if< - is_function::type> - >::type ->:is_transparent_function::type,Arg1,Arg2>{}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/iter_adaptor.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/iter_adaptor.hpp deleted file mode 100644 index 7a032350b36..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/iter_adaptor.hpp +++ /dev/null @@ -1,321 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ITER_ADAPTOR_HPP -#define BOOST_MULTI_INDEX_DETAIL_ITER_ADAPTOR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Poor man's version of boost::iterator_adaptor. Used instead of the - * original as compile times for the latter are significantly higher. - * The interface is not replicated exactly, only to the extent necessary - * for internal consumption. - */ - -/* NB. The purpose of the (non-inclass) global operators ==, < and - defined - * above is to partially alleviate a problem of MSVC++ 6.0 by * which - * friend-injected operators on T are not visible if T is instantiated only - * in template code where T is a dependent type. - */ - -class iter_adaptor_access -{ -public: - template - static typename Class::reference dereference(const Class& x) - { - return x.dereference(); - } - - template - static bool equal(const Class& x,const Class& y) - { - return x.equal(y); - } - - template - static void increment(Class& x) - { - x.increment(); - } - - template - static void decrement(Class& x) - { - x.decrement(); - } - - template - static void advance(Class& x,typename Class::difference_type n) - { - x.advance(n); - } - - template - static typename Class::difference_type distance_to( - const Class& x,const Class& y) - { - return x.distance_to(y); - } -}; - -template -struct iter_adaptor_selector; - -template -class forward_iter_adaptor_base: - public forward_iterator_helper< - Derived, - typename Base::value_type, - typename Base::difference_type, - typename Base::pointer, - typename Base::reference> -{ -public: - typedef typename Base::reference reference; - - reference operator*()const - { - return iter_adaptor_access::dereference(final()); - } - - friend bool operator==(const Derived& x,const Derived& y) - { - return iter_adaptor_access::equal(x,y); - } - - Derived& operator++() - { - iter_adaptor_access::increment(final()); - return final(); - } - -private: - Derived& final(){return *static_cast(this);} - const Derived& final()const{return *static_cast(this);} -}; - -template -bool operator==( - const forward_iter_adaptor_base& x, - const forward_iter_adaptor_base& y) -{ - return iter_adaptor_access::equal( - static_cast(x),static_cast(y)); -} - -template<> -struct iter_adaptor_selector -{ - template - struct apply - { - typedef forward_iter_adaptor_base type; - }; -}; - -template -class bidirectional_iter_adaptor_base: - public bidirectional_iterator_helper< - Derived, - typename Base::value_type, - typename Base::difference_type, - typename Base::pointer, - typename Base::reference> -{ -public: - typedef typename Base::reference reference; - - reference operator*()const - { - return iter_adaptor_access::dereference(final()); - } - - friend bool operator==(const Derived& x,const Derived& y) - { - return iter_adaptor_access::equal(x,y); - } - - Derived& operator++() - { - iter_adaptor_access::increment(final()); - return final(); - } - - Derived& operator--() - { - iter_adaptor_access::decrement(final()); - return final(); - } - -private: - Derived& final(){return *static_cast(this);} - const Derived& final()const{return *static_cast(this);} -}; - -template -bool operator==( - const bidirectional_iter_adaptor_base& x, - const bidirectional_iter_adaptor_base& y) -{ - return iter_adaptor_access::equal( - static_cast(x),static_cast(y)); -} - -template<> -struct iter_adaptor_selector -{ - template - struct apply - { - typedef bidirectional_iter_adaptor_base type; - }; -}; - -template -class random_access_iter_adaptor_base: - public random_access_iterator_helper< - Derived, - typename Base::value_type, - typename Base::difference_type, - typename Base::pointer, - typename Base::reference> -{ -public: - typedef typename Base::reference reference; - typedef typename Base::difference_type difference_type; - - reference operator*()const - { - return iter_adaptor_access::dereference(final()); - } - - friend bool operator==(const Derived& x,const Derived& y) - { - return iter_adaptor_access::equal(x,y); - } - - friend bool operator<(const Derived& x,const Derived& y) - { - return iter_adaptor_access::distance_to(x,y)>0; - } - - Derived& operator++() - { - iter_adaptor_access::increment(final()); - return final(); - } - - Derived& operator--() - { - iter_adaptor_access::decrement(final()); - return final(); - } - - Derived& operator+=(difference_type n) - { - iter_adaptor_access::advance(final(),n); - return final(); - } - - Derived& operator-=(difference_type n) - { - iter_adaptor_access::advance(final(),-n); - return final(); - } - - friend difference_type operator-(const Derived& x,const Derived& y) - { - return iter_adaptor_access::distance_to(y,x); - } - -private: - Derived& final(){return *static_cast(this);} - const Derived& final()const{return *static_cast(this);} -}; - -template -bool operator==( - const random_access_iter_adaptor_base& x, - const random_access_iter_adaptor_base& y) -{ - return iter_adaptor_access::equal( - static_cast(x),static_cast(y)); -} - -template -bool operator<( - const random_access_iter_adaptor_base& x, - const random_access_iter_adaptor_base& y) -{ - return iter_adaptor_access::distance_to( - static_cast(x),static_cast(y))>0; -} - -template -typename random_access_iter_adaptor_base::difference_type -operator-( - const random_access_iter_adaptor_base& x, - const random_access_iter_adaptor_base& y) -{ - return iter_adaptor_access::distance_to( - static_cast(y),static_cast(x)); -} - -template<> -struct iter_adaptor_selector -{ - template - struct apply - { - typedef random_access_iter_adaptor_base type; - }; -}; - -template -struct iter_adaptor_base -{ - typedef iter_adaptor_selector< - typename Base::iterator_category> selector; - typedef typename mpl::apply2< - selector,Derived,Base>::type type; -}; - -template -class iter_adaptor:public iter_adaptor_base::type -{ -protected: - iter_adaptor(){} - explicit iter_adaptor(const Base& b_):b(b_){} - - const Base& base_reference()const{return b;} - Base& base_reference(){return b;} - -private: - Base b; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/modify_key_adaptor.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/modify_key_adaptor.hpp deleted file mode 100644 index 6df89b18386..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/modify_key_adaptor.hpp +++ /dev/null @@ -1,49 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_MODIFY_KEY_ADAPTOR_HPP -#define BOOST_MULTI_INDEX_DETAIL_MODIFY_KEY_ADAPTOR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Functional adaptor to resolve modify_key as a call to modify. - * Preferred over compose_f_gx and stuff cause it eliminates problems - * with references to references, dealing with function pointers, etc. - */ - -template -struct modify_key_adaptor -{ - - modify_key_adaptor(Fun f_,KeyFromValue kfv_):f(f_),kfv(kfv_){} - - void operator()(Value& x) - { - f(kfv(x)); - } - -private: - Fun f; - KeyFromValue kfv; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/no_duplicate_tags.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/no_duplicate_tags.hpp deleted file mode 100644 index ba216ed82cf..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/no_duplicate_tags.hpp +++ /dev/null @@ -1,97 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_NO_DUPLICATE_TAGS_HPP -#define BOOST_MULTI_INDEX_DETAIL_NO_DUPLICATE_TAGS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* no_duplicate_tags check at compile-time that a tag list - * has no duplicate tags. - * The algorithm deserves some explanation: tags - * are sequentially inserted into a mpl::set if they were - * not already present. Due to the magic of mpl::set - * (mpl::has_key is contant time), this operation takes linear - * time, and even MSVC++ 6.5 handles it gracefully (other obvious - * solutions are quadratic.) - */ - -struct duplicate_tag_mark{}; - -struct duplicate_tag_marker -{ - template - struct apply - { - typedef mpl::s_item< - typename mpl::if_,duplicate_tag_mark,Tag>::type, - MplSet - > type; - }; -}; - -template -struct no_duplicate_tags -{ - typedef typename mpl::fold< - TagList, - mpl::set0<>, - duplicate_tag_marker - >::type aux; - - BOOST_STATIC_CONSTANT( - bool,value=!(mpl::has_key::value)); -}; - -/* Variant for an index list: duplication is checked - * across all the indices. - */ - -struct duplicate_tag_list_marker -{ - template - struct apply:mpl::fold< - BOOST_DEDUCED_TYPENAME Index::tag_list, - MplSet, - duplicate_tag_marker> - { - }; -}; - -template -struct no_duplicate_tags_in_index_list -{ - typedef typename mpl::fold< - IndexList, - mpl::set0<>, - duplicate_tag_list_marker - >::type aux; - - BOOST_STATIC_CONSTANT( - bool,value=!(mpl::has_key::value)); -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/node_type.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/node_type.hpp deleted file mode 100644 index 7fe85cf968b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/node_type.hpp +++ /dev/null @@ -1,66 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_NODE_TYPE_HPP -#define BOOST_MULTI_INDEX_DETAIL_NODE_TYPE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* MPL machinery to construct the internal node type associated to an - * index list. - */ - -struct index_node_applier -{ - template - struct apply - { - typedef typename mpl::deref::type index_specifier; - typedef typename index_specifier:: - BOOST_NESTED_TEMPLATE node_class::type type; - }; -}; - -template -struct multi_index_node_type -{ - BOOST_STATIC_ASSERT(detail::is_index_list::value); - - typedef typename mpl::reverse_iter_fold< - IndexSpecifierList, - index_node_base, - mpl::bind2 - >::type type; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_args.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_args.hpp deleted file mode 100644 index 3e2641f2f4d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_args.hpp +++ /dev/null @@ -1,83 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_ARGS_HPP -#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_ARGS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Oredered index specifiers can be instantiated in two forms: - * - * (ordered_unique|ordered_non_unique)< - * KeyFromValue,Compare=std::less > - * (ordered_unique|ordered_non_unique)< - * TagList,KeyFromValue,Compare=std::less > - * - * index_args implements the machinery to accept this argument-dependent - * polymorphism. - */ - -template -struct index_args_default_compare -{ - typedef std::less type; -}; - -template -struct ordered_index_args -{ - typedef is_tag full_form; - - typedef typename mpl::if_< - full_form, - Arg1, - tag< > >::type tag_list_type; - typedef typename mpl::if_< - full_form, - Arg2, - Arg1>::type key_from_value_type; - typedef typename mpl::if_< - full_form, - Arg3, - Arg2>::type supplied_compare_type; - typedef typename mpl::eval_if< - mpl::is_na, - index_args_default_compare, - mpl::identity - >::type compare_type; - - BOOST_STATIC_ASSERT(is_tag::value); - BOOST_STATIC_ASSERT(!mpl::is_na::value); - BOOST_STATIC_ASSERT(!mpl::is_na::value); -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl.hpp deleted file mode 100644 index 040cb989630..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl.hpp +++ /dev/null @@ -1,1567 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - * - * The internal implementation of red-black trees is based on that of SGI STL - * stl_tree.h file: - * - * Copyright (c) 1996,1997 - * Silicon Graphics Computer Systems, Inc. - * - * Permission to use, copy, modify, distribute and sell this software - * and its documentation for any purpose is hereby granted without fee, - * provided that the above copyright notice appear in all copies and - * that both that copyright notice and this permission notice appear - * in supporting documentation. Silicon Graphics makes no - * representations about the suitability of this software for any - * purpose. It is provided "as is" without express or implied warranty. - * - * - * Copyright (c) 1994 - * Hewlett-Packard Company - * - * Permission to use, copy, modify, distribute and sell this software - * and its documentation for any purpose is hereby granted without fee, - * provided that the above copyright notice appear in all copies and - * that both that copyright notice and this permission notice appear - * in supporting documentation. Hewlett-Packard Company makes no - * representations about the suitability of this software for any - * purpose. It is provided "as is" without express or implied warranty. - * - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_IMPL_HPP -#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_IMPL_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) -#include -#endif - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#include -#include -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) -#define BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF(x) \ - detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ - detail::make_obj_guard(x,&ordered_index_impl::check_invariant_); \ - BOOST_JOIN(check_invariant_,__LINE__).touch(); -#define BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT \ - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF(*this) -#else -#define BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF(x) -#define BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* ordered_index adds a layer of ordered indexing to a given Super and accepts - * an augmenting policy for optional addition of order statistics. - */ - -/* Most of the implementation of unique and non-unique indices is - * shared. We tell from one another on instantiation time by using - * these tags. - */ - -struct ordered_unique_tag{}; -struct ordered_non_unique_tag{}; - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -class ordered_index; - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -class ordered_index_impl: - BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS SuperMeta::type - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - ,public safe_mode::safe_container< - ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy> > -#endif - -{ -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the - * lifetime of const references bound to temporaries --precisely what - * scopeguards are. - */ - -#pragma parse_mfunc_templ off -#endif - - typedef typename SuperMeta::type super; - -protected: - typedef ordered_index_node< - AugmentPolicy,typename super::node_type> node_type; - -protected: /* for the benefit of AugmentPolicy::augmented_interface */ - typedef typename node_type::impl_type node_impl_type; - typedef typename node_impl_type::pointer node_impl_pointer; - -public: - /* types */ - - typedef typename KeyFromValue::result_type key_type; - typedef typename node_type::value_type value_type; - typedef KeyFromValue key_from_value; - typedef Compare key_compare; - typedef value_comparison< - value_type,KeyFromValue,Compare> value_compare; - typedef tuple ctor_args; - typedef typename super::final_allocator_type allocator_type; - typedef typename allocator_type::reference reference; - typedef typename allocator_type::const_reference const_reference; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_iterator< - bidir_node_iterator, - ordered_index_impl> iterator; -#else - typedef bidir_node_iterator iterator; -#endif - - typedef iterator const_iterator; - - typedef std::size_t size_type; - typedef std::ptrdiff_t difference_type; - typedef typename allocator_type::pointer pointer; - typedef typename allocator_type::const_pointer const_pointer; - typedef typename - boost::reverse_iterator reverse_iterator; - typedef typename - boost::reverse_iterator const_reverse_iterator; - typedef TagList tag_list; - -protected: - typedef typename super::final_node_type final_node_type; - typedef tuples::cons< - ctor_args, - typename super::ctor_args_list> ctor_args_list; - typedef typename mpl::push_front< - typename super::index_type_list, - ordered_index< - KeyFromValue,Compare, - SuperMeta,TagList,Category,AugmentPolicy - > >::type index_type_list; - typedef typename mpl::push_front< - typename super::iterator_type_list, - iterator>::type iterator_type_list; - typedef typename mpl::push_front< - typename super::const_iterator_type_list, - const_iterator>::type const_iterator_type_list; - typedef typename super::copy_map_type copy_map_type; - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - typedef typename super::index_saver_type index_saver_type; - typedef typename super::index_loader_type index_loader_type; -#endif - -protected: -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_container< - ordered_index_impl> safe_super; -#endif - - typedef typename call_traits< - value_type>::param_type value_param_type; - typedef typename call_traits< - key_type>::param_type key_param_type; - - /* Needed to avoid commas in BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL - * expansion. - */ - - typedef std::pair emplace_return_type; - -public: - - /* construct/copy/destroy - * Default and copy ctors are in the protected section as indices are - * not supposed to be created on their own. No range ctor either. - * Assignment operators defined at ordered_index rather than here. - */ - - allocator_type get_allocator()const BOOST_NOEXCEPT - { - return this->final().get_allocator(); - } - - /* iterators */ - - iterator - begin()BOOST_NOEXCEPT{return make_iterator(leftmost());} - const_iterator - begin()const BOOST_NOEXCEPT{return make_iterator(leftmost());} - iterator - end()BOOST_NOEXCEPT{return make_iterator(header());} - const_iterator - end()const BOOST_NOEXCEPT{return make_iterator(header());} - reverse_iterator - rbegin()BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} - const_reverse_iterator - rbegin()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} - reverse_iterator - rend()BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} - const_reverse_iterator - rend()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} - const_iterator - cbegin()const BOOST_NOEXCEPT{return begin();} - const_iterator - cend()const BOOST_NOEXCEPT{return end();} - const_reverse_iterator - crbegin()const BOOST_NOEXCEPT{return rbegin();} - const_reverse_iterator - crend()const BOOST_NOEXCEPT{return rend();} - - iterator iterator_to(const value_type& x) - { - return make_iterator(node_from_value(&x)); - } - - const_iterator iterator_to(const value_type& x)const - { - return make_iterator(node_from_value(&x)); - } - - /* capacity */ - - bool empty()const BOOST_NOEXCEPT{return this->final_empty_();} - size_type size()const BOOST_NOEXCEPT{return this->final_size_();} - size_type max_size()const BOOST_NOEXCEPT{return this->final_max_size_();} - - /* modifiers */ - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( - emplace_return_type,emplace,emplace_impl) - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( - iterator,emplace_hint,emplace_hint_impl,iterator,position) - - std::pair insert(const value_type& x) - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_(x); - return std::pair(make_iterator(p.first),p.second); - } - - std::pair insert(BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_rv_(x); - return std::pair(make_iterator(p.first),p.second); - } - - iterator insert(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_( - x,static_cast(position.get_node())); - return make_iterator(p.first); - } - - iterator insert(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_rv_( - x,static_cast(position.get_node())); - return make_iterator(p.first); - } - - template - void insert(InputIterator first,InputIterator last) - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - node_type* hint=header(); /* end() */ - for(;first!=last;++first){ - hint=this->final_insert_ref_( - *first,static_cast(hint)).first; - node_type::increment(hint); - } - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - void insert(std::initializer_list list) - { - insert(list.begin(),list.end()); - } -#endif - - iterator erase(iterator position) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - this->final_erase_(static_cast(position++.get_node())); - return position; - } - - size_type erase(key_param_type x) - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pair p=equal_range(x); - size_type s=0; - while(p.first!=p.second){ - p.first=erase(p.first); - ++s; - } - return s; - } - - iterator erase(iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - while(first!=last){ - first=erase(first); - } - return first; - } - - bool replace(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - return this->final_replace_( - x,static_cast(position.get_node())); - } - - bool replace(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - return this->final_replace_rv_( - x,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod,Rollback back_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,back_,static_cast(position.get_node())); - } - - template - bool modify_key(iterator position,Modifier mod) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - return modify( - position,modify_key_adaptor(mod,key)); - } - - template - bool modify_key(iterator position,Modifier mod,Rollback back_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - return modify( - position, - modify_key_adaptor(mod,key), - modify_key_adaptor(back_,key)); - } - - void swap( - ordered_index< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x) - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF(x); - this->final_swap_(x.final()); - } - - void clear()BOOST_NOEXCEPT - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - this->final_clear_(); - } - - /* observers */ - - key_from_value key_extractor()const{return key;} - key_compare key_comp()const{return comp_;} - value_compare value_comp()const{return value_compare(key,comp_);} - - /* set operations */ - - /* Internally, these ops rely on const_iterator being the same - * type as iterator. - */ - - template - iterator find(const CompatibleKey& x)const - { - return make_iterator(ordered_index_find(root(),header(),key,x,comp_)); - } - - template - iterator find( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return make_iterator(ordered_index_find(root(),header(),key,x,comp)); - } - - template - size_type count(const CompatibleKey& x)const - { - return count(x,comp_); - } - - template - size_type count(const CompatibleKey& x,const CompatibleCompare& comp)const - { - std::pair p=equal_range(x,comp); - size_type n=std::distance(p.first,p.second); - return n; - } - - template - iterator lower_bound(const CompatibleKey& x)const - { - return make_iterator( - ordered_index_lower_bound(root(),header(),key,x,comp_)); - } - - template - iterator lower_bound( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return make_iterator( - ordered_index_lower_bound(root(),header(),key,x,comp)); - } - - template - iterator upper_bound(const CompatibleKey& x)const - { - return make_iterator( - ordered_index_upper_bound(root(),header(),key,x,comp_)); - } - - template - iterator upper_bound( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return make_iterator( - ordered_index_upper_bound(root(),header(),key,x,comp)); - } - - template - std::pair equal_range( - const CompatibleKey& x)const - { - std::pair p= - ordered_index_equal_range(root(),header(),key,x,comp_); - return std::pair( - make_iterator(p.first),make_iterator(p.second)); - } - - template - std::pair equal_range( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - std::pair p= - ordered_index_equal_range(root(),header(),key,x,comp); - return std::pair( - make_iterator(p.first),make_iterator(p.second)); - } - - /* range */ - - template - std::pair - range(LowerBounder lower,UpperBounder upper)const - { - typedef typename mpl::if_< - is_same, - BOOST_DEDUCED_TYPENAME mpl::if_< - is_same, - both_unbounded_tag, - lower_unbounded_tag - >::type, - BOOST_DEDUCED_TYPENAME mpl::if_< - is_same, - upper_unbounded_tag, - none_unbounded_tag - >::type - >::type dispatch; - - return range(lower,upper,dispatch()); - } - -BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: - ordered_index_impl(const ctor_args_list& args_list,const allocator_type& al): - super(args_list.get_tail(),al), - key(tuples::get<0>(args_list.get_head())), - comp_(tuples::get<1>(args_list.get_head())) - { - empty_initialize(); - } - - ordered_index_impl( - const ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x): - super(x), - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super(), -#endif - - key(x.key), - comp_(x.comp_) - { - /* Copy ctor just takes the key and compare objects from x. The rest is - * done in a subsequent call to copy_(). - */ - } - - ordered_index_impl( - const ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x, - do_not_copy_elements_tag): - super(x,do_not_copy_elements_tag()), - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super(), -#endif - - key(x.key), - comp_(x.comp_) - { - empty_initialize(); - } - - ~ordered_index_impl() - { - /* the container is guaranteed to be empty by now */ - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - iterator make_iterator(node_type* node){return iterator(node,this);} - const_iterator make_iterator(node_type* node)const - {return const_iterator(node,const_cast(this));} -#else - iterator make_iterator(node_type* node){return iterator(node);} - const_iterator make_iterator(node_type* node)const - {return const_iterator(node);} -#endif - - void copy_( - const ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x, - const copy_map_type& map) - { - if(!x.root()){ - empty_initialize(); - } - else{ - header()->color()=x.header()->color(); - AugmentPolicy::copy(x.header()->impl(),header()->impl()); - - node_type* root_cpy=map.find(static_cast(x.root())); - header()->parent()=root_cpy->impl(); - - node_type* leftmost_cpy=map.find( - static_cast(x.leftmost())); - header()->left()=leftmost_cpy->impl(); - - node_type* rightmost_cpy=map.find( - static_cast(x.rightmost())); - header()->right()=rightmost_cpy->impl(); - - typedef typename copy_map_type::const_iterator copy_map_iterator; - for(copy_map_iterator it=map.begin(),it_end=map.end();it!=it_end;++it){ - node_type* org=it->first; - node_type* cpy=it->second; - - cpy->color()=org->color(); - AugmentPolicy::copy(org->impl(),cpy->impl()); - - node_impl_pointer parent_org=org->parent(); - if(parent_org==node_impl_pointer(0))cpy->parent()=node_impl_pointer(0); - else{ - node_type* parent_cpy=map.find( - static_cast(node_type::from_impl(parent_org))); - cpy->parent()=parent_cpy->impl(); - if(parent_org->left()==org->impl()){ - parent_cpy->left()=cpy->impl(); - } - else if(parent_org->right()==org->impl()){ - /* header() does not satisfy this nor the previous check */ - parent_cpy->right()=cpy->impl(); - } - } - - if(org->left()==node_impl_pointer(0)) - cpy->left()=node_impl_pointer(0); - if(org->right()==node_impl_pointer(0)) - cpy->right()=node_impl_pointer(0); - } - } - - super::copy_(x,map); - } - - template - final_node_type* insert_( - value_param_type v,final_node_type*& x,Variant variant) - { - link_info inf; - if(!link_point(key(v),inf,Category())){ - return static_cast(node_type::from_impl(inf.pos)); - } - - final_node_type* res=super::insert_(v,x,variant); - if(res==x){ - node_impl_type::link( - static_cast(x)->impl(),inf.side,inf.pos,header()->impl()); - } - return res; - } - - template - final_node_type* insert_( - value_param_type v,node_type* position,final_node_type*& x,Variant variant) - { - link_info inf; - if(!hinted_link_point(key(v),position,inf,Category())){ - return static_cast(node_type::from_impl(inf.pos)); - } - - final_node_type* res=super::insert_(v,position,x,variant); - if(res==x){ - node_impl_type::link( - static_cast(x)->impl(),inf.side,inf.pos,header()->impl()); - } - return res; - } - - void erase_(node_type* x) - { - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - } - - void delete_all_nodes_() - { - delete_all_nodes(root()); - } - - void clear_() - { - super::clear_(); - empty_initialize(); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::detach_dereferenceable_iterators(); -#endif - } - - void swap_( - ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x) - { - std::swap(key,x.key); - std::swap(comp_,x.comp_); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_(x); - } - - void swap_elements_( - ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x) - { -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_elements_(x); - } - - template - bool replace_(value_param_type v,node_type* x,Variant variant) - { - if(in_place(v,x,Category())){ - return super::replace_(v,x,variant); - } - - node_type* next=x; - node_type::increment(next); - - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - - BOOST_TRY{ - link_info inf; - if(link_point(key(v),inf,Category())&&super::replace_(v,x,variant)){ - node_impl_type::link(x->impl(),inf.side,inf.pos,header()->impl()); - return true; - } - node_impl_type::restore(x->impl(),next->impl(),header()->impl()); - return false; - } - BOOST_CATCH(...){ - node_impl_type::restore(x->impl(),next->impl(),header()->impl()); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - bool modify_(node_type* x) - { - bool b; - BOOST_TRY{ - b=in_place(x->value(),x,Category()); - } - BOOST_CATCH(...){ - erase_(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - if(!b){ - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - BOOST_TRY{ - link_info inf; - if(!link_point(key(x->value()),inf,Category())){ - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - return false; - } - node_impl_type::link(x->impl(),inf.side,inf.pos,header()->impl()); - } - BOOST_CATCH(...){ - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - BOOST_TRY{ - if(!super::modify_(x)){ - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - return false; - } - else return true; - } - BOOST_CATCH(...){ - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - bool modify_rollback_(node_type* x) - { - if(in_place(x->value(),x,Category())){ - return super::modify_rollback_(x); - } - - node_type* next=x; - node_type::increment(next); - - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - - BOOST_TRY{ - link_info inf; - if(link_point(key(x->value()),inf,Category())&& - super::modify_rollback_(x)){ - node_impl_type::link(x->impl(),inf.side,inf.pos,header()->impl()); - return true; - } - node_impl_type::restore(x->impl(),next->impl(),header()->impl()); - return false; - } - BOOST_CATCH(...){ - node_impl_type::restore(x->impl(),next->impl(),header()->impl()); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* serialization */ - - template - void save_( - Archive& ar,const unsigned int version,const index_saver_type& sm)const - { - save_(ar,version,sm,Category()); - } - - template - void load_(Archive& ar,const unsigned int version,const index_loader_type& lm) - { - load_(ar,version,lm,Category()); - } -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - bool invariant_()const - { - if(size()==0||begin()==end()){ - if(size()!=0||begin()!=end()|| - header()->left()!=header()->impl()|| - header()->right()!=header()->impl())return false; - } - else{ - if((size_type)std::distance(begin(),end())!=size())return false; - - std::size_t len=node_impl_type::black_count( - leftmost()->impl(),root()->impl()); - for(const_iterator it=begin(),it_end=end();it!=it_end;++it){ - node_type* x=it.get_node(); - node_type* left_x=node_type::from_impl(x->left()); - node_type* right_x=node_type::from_impl(x->right()); - - if(x->color()==red){ - if((left_x&&left_x->color()==red)|| - (right_x&&right_x->color()==red))return false; - } - if(left_x&&comp_(key(x->value()),key(left_x->value())))return false; - if(right_x&&comp_(key(right_x->value()),key(x->value())))return false; - if(!left_x&&!right_x&& - node_impl_type::black_count(x->impl(),root()->impl())!=len) - return false; - if(!AugmentPolicy::invariant(x->impl()))return false; - } - - if(leftmost()->impl()!=node_impl_type::minimum(root()->impl())) - return false; - if(rightmost()->impl()!=node_impl_type::maximum(root()->impl())) - return false; - } - - return super::invariant_(); - } - - - /* This forwarding function eases things for the boost::mem_fn construct - * in BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT. Actually, - * final_check_invariant is already an inherited member function of - * ordered_index_impl. - */ - void check_invariant_()const{this->final_check_invariant_();} -#endif - -protected: /* for the benefit of AugmentPolicy::augmented_interface */ - node_type* header()const{return this->final_header();} - node_type* root()const{return node_type::from_impl(header()->parent());} - node_type* leftmost()const{return node_type::from_impl(header()->left());} - node_type* rightmost()const{return node_type::from_impl(header()->right());} - -private: - void empty_initialize() - { - header()->color()=red; - /* used to distinguish header() from root, in iterator.operator++ */ - - header()->parent()=node_impl_pointer(0); - header()->left()=header()->impl(); - header()->right()=header()->impl(); - } - - struct link_info - { - /* coverity[uninit_ctor]: suppress warning */ - link_info():side(to_left){} - - ordered_index_side side; - node_impl_pointer pos; - }; - - bool link_point(key_param_type k,link_info& inf,ordered_unique_tag) - { - node_type* y=header(); - node_type* x=root(); - bool c=true; - while(x){ - y=x; - c=comp_(k,key(x->value())); - x=node_type::from_impl(c?x->left():x->right()); - } - node_type* yy=y; - if(c){ - if(yy==leftmost()){ - inf.side=to_left; - inf.pos=y->impl(); - return true; - } - else node_type::decrement(yy); - } - - if(comp_(key(yy->value()),k)){ - inf.side=c?to_left:to_right; - inf.pos=y->impl(); - return true; - } - else{ - inf.pos=yy->impl(); - return false; - } - } - - bool link_point(key_param_type k,link_info& inf,ordered_non_unique_tag) - { - node_type* y=header(); - node_type* x=root(); - bool c=true; - while (x){ - y=x; - c=comp_(k,key(x->value())); - x=node_type::from_impl(c?x->left():x->right()); - } - inf.side=c?to_left:to_right; - inf.pos=y->impl(); - return true; - } - - bool lower_link_point(key_param_type k,link_info& inf,ordered_non_unique_tag) - { - node_type* y=header(); - node_type* x=root(); - bool c=false; - while (x){ - y=x; - c=comp_(key(x->value()),k); - x=node_type::from_impl(c?x->right():x->left()); - } - inf.side=c?to_right:to_left; - inf.pos=y->impl(); - return true; - } - - bool hinted_link_point( - key_param_type k,node_type* position,link_info& inf,ordered_unique_tag) - { - if(position->impl()==header()->left()){ - if(size()>0&&comp_(k,key(position->value()))){ - inf.side=to_left; - inf.pos=position->impl(); - return true; - } - else return link_point(k,inf,ordered_unique_tag()); - } - else if(position==header()){ - if(comp_(key(rightmost()->value()),k)){ - inf.side=to_right; - inf.pos=rightmost()->impl(); - return true; - } - else return link_point(k,inf,ordered_unique_tag()); - } - else{ - node_type* before=position; - node_type::decrement(before); - if(comp_(key(before->value()),k)&&comp_(k,key(position->value()))){ - if(before->right()==node_impl_pointer(0)){ - inf.side=to_right; - inf.pos=before->impl(); - return true; - } - else{ - inf.side=to_left; - inf.pos=position->impl(); - return true; - } - } - else return link_point(k,inf,ordered_unique_tag()); - } - } - - bool hinted_link_point( - key_param_type k,node_type* position,link_info& inf,ordered_non_unique_tag) - { - if(position->impl()==header()->left()){ - if(size()>0&&!comp_(key(position->value()),k)){ - inf.side=to_left; - inf.pos=position->impl(); - return true; - } - else return lower_link_point(k,inf,ordered_non_unique_tag()); - } - else if(position==header()){ - if(!comp_(k,key(rightmost()->value()))){ - inf.side=to_right; - inf.pos=rightmost()->impl(); - return true; - } - else return link_point(k,inf,ordered_non_unique_tag()); - } - else{ - node_type* before=position; - node_type::decrement(before); - if(!comp_(k,key(before->value()))){ - if(!comp_(key(position->value()),k)){ - if(before->right()==node_impl_pointer(0)){ - inf.side=to_right; - inf.pos=before->impl(); - return true; - } - else{ - inf.side=to_left; - inf.pos=position->impl(); - return true; - } - } - else return lower_link_point(k,inf,ordered_non_unique_tag()); - } - else return link_point(k,inf,ordered_non_unique_tag()); - } - } - - void delete_all_nodes(node_type* x) - { - if(!x)return; - - delete_all_nodes(node_type::from_impl(x->left())); - delete_all_nodes(node_type::from_impl(x->right())); - this->final_delete_node_(static_cast(x)); - } - - bool in_place(value_param_type v,node_type* x,ordered_unique_tag) - { - node_type* y; - if(x!=leftmost()){ - y=x; - node_type::decrement(y); - if(!comp_(key(y->value()),key(v)))return false; - } - - y=x; - node_type::increment(y); - return y==header()||comp_(key(v),key(y->value())); - } - - bool in_place(value_param_type v,node_type* x,ordered_non_unique_tag) - { - node_type* y; - if(x!=leftmost()){ - y=x; - node_type::decrement(y); - if(comp_(key(v),key(y->value())))return false; - } - - y=x; - node_type::increment(y); - return y==header()||!comp_(key(y->value()),key(v)); - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - void detach_iterators(node_type* x) - { - iterator it=make_iterator(x); - safe_mode::detach_equivalent_iterators(it); - } -#endif - - template - std::pair emplace_impl(BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pairp= - this->final_emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - return std::pair(make_iterator(p.first),p.second); - } - - template - iterator emplace_hint_impl( - iterator position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pairp= - this->final_emplace_hint_( - static_cast(position.get_node()), - BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - return make_iterator(p.first); - } - - template - std::pair - range(LowerBounder lower,UpperBounder upper,none_unbounded_tag)const - { - node_type* y=header(); - node_type* z=root(); - - while(z){ - if(!lower(key(z->value()))){ - z=node_type::from_impl(z->right()); - } - else if(!upper(key(z->value()))){ - y=z; - z=node_type::from_impl(z->left()); - } - else{ - return std::pair( - make_iterator( - lower_range(node_type::from_impl(z->left()),z,lower)), - make_iterator( - upper_range(node_type::from_impl(z->right()),y,upper))); - } - } - - return std::pair(make_iterator(y),make_iterator(y)); - } - - template - std::pair - range(LowerBounder,UpperBounder upper,lower_unbounded_tag)const - { - return std::pair( - begin(), - make_iterator(upper_range(root(),header(),upper))); - } - - template - std::pair - range(LowerBounder lower,UpperBounder,upper_unbounded_tag)const - { - return std::pair( - make_iterator(lower_range(root(),header(),lower)), - end()); - } - - template - std::pair - range(LowerBounder,UpperBounder,both_unbounded_tag)const - { - return std::pair(begin(),end()); - } - - template - node_type * lower_range(node_type* top,node_type* y,LowerBounder lower)const - { - while(top){ - if(lower(key(top->value()))){ - y=top; - top=node_type::from_impl(top->left()); - } - else top=node_type::from_impl(top->right()); - } - - return y; - } - - template - node_type * upper_range(node_type* top,node_type* y,UpperBounder upper)const - { - while(top){ - if(!upper(key(top->value()))){ - y=top; - top=node_type::from_impl(top->left()); - } - else top=node_type::from_impl(top->right()); - } - - return y; - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - template - void save_( - Archive& ar,const unsigned int version,const index_saver_type& sm, - ordered_unique_tag)const - { - super::save_(ar,version,sm); - } - - template - void load_( - Archive& ar,const unsigned int version,const index_loader_type& lm, - ordered_unique_tag) - { - super::load_(ar,version,lm); - } - - template - void save_( - Archive& ar,const unsigned int version,const index_saver_type& sm, - ordered_non_unique_tag)const - { - typedef duplicates_iterator dup_iterator; - - sm.save( - dup_iterator(begin().get_node(),end().get_node(),value_comp()), - dup_iterator(end().get_node(),value_comp()), - ar,version); - super::save_(ar,version,sm); - } - - template - void load_( - Archive& ar,const unsigned int version,const index_loader_type& lm, - ordered_non_unique_tag) - { - lm.load( - ::boost::bind( - &ordered_index_impl::rearranger,this, - ::boost::arg<1>(),::boost::arg<2>()), - ar,version); - super::load_(ar,version,lm); - } - - void rearranger(node_type* position,node_type *x) - { - if(!position||comp_(key(position->value()),key(x->value()))){ - position=lower_bound(key(x->value())).get_node(); - } - else if(comp_(key(x->value()),key(position->value()))){ - /* inconsistent rearrangement */ - throw_exception( - archive::archive_exception( - archive::archive_exception::other_exception)); - } - else node_type::increment(position); - - if(position!=x){ - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - node_impl_type::restore( - x->impl(),position->impl(),header()->impl()); - } - } -#endif /* serialization */ - -protected: /* for the benefit of AugmentPolicy::augmented_interface */ - key_from_value key; - key_compare comp_; - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -#pragma parse_mfunc_templ reset -#endif -}; - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -class ordered_index: - public AugmentPolicy::template augmented_interface< - ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy - > - >::type -{ - typedef typename AugmentPolicy::template - augmented_interface< - ordered_index_impl< - KeyFromValue,Compare, - SuperMeta,TagList,Category,AugmentPolicy - > - >::type super; -public: - typedef typename super::ctor_args_list ctor_args_list; - typedef typename super::allocator_type allocator_type; - typedef typename super::iterator iterator; - - /* construct/copy/destroy - * Default and copy ctors are in the protected section as indices are - * not supposed to be created on their own. No range ctor either. - */ - - ordered_index& operator=(const ordered_index& x) - { - this->final()=x.final(); - return *this; - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - ordered_index& operator=( - std::initializer_list list) - { - this->final()=list; - return *this; - } -#endif - -protected: - ordered_index( - const ctor_args_list& args_list,const allocator_type& al): - super(args_list,al){} - - ordered_index(const ordered_index& x):super(x){}; - - ordered_index(const ordered_index& x,do_not_copy_elements_tag): - super(x,do_not_copy_elements_tag()){}; -}; - -/* comparison */ - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator==( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) -{ - return x.size()==y.size()&&std::equal(x.begin(),x.end(),y.begin()); -} - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator<( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) -{ - return std::lexicographical_compare(x.begin(),x.end(),y.begin(),y.end()); -} - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator!=( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) -{ - return !(x==y); -} - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator>( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) -{ - return y -bool operator>=( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) -{ - return !(x -bool operator<=( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) -{ - return !(x>y); -} - -/* specialized algorithms */ - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -void swap( - ordered_index< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x, - ordered_index< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -/* Boost.Foreach compatibility */ - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -inline boost::mpl::true_* boost_foreach_is_noncopyable( - boost::multi_index::detail::ordered_index< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>*&, - boost_foreach_argument_dependent_lookup_hack) -{ - return 0; -} - -#undef BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT -#undef BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl_fwd.hpp deleted file mode 100644 index 6590ef05fdd..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl_fwd.hpp +++ /dev/null @@ -1,128 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_IMPL_FWD_HPP -#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_IMPL_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -class ordered_index; - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator==( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator<( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator!=( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator>( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator>=( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator<=( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -void swap( - ordered_index< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x, - ordered_index< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& y); - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_node.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_node.hpp deleted file mode 100644 index e7af0377fb9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_node.hpp +++ /dev/null @@ -1,658 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - * - * The internal implementation of red-black trees is based on that of SGI STL - * stl_tree.h file: - * - * Copyright (c) 1996,1997 - * Silicon Graphics Computer Systems, Inc. - * - * Permission to use, copy, modify, distribute and sell this software - * and its documentation for any purpose is hereby granted without fee, - * provided that the above copyright notice appear in all copies and - * that both that copyright notice and this permission notice appear - * in supporting documentation. Silicon Graphics makes no - * representations about the suitability of this software for any - * purpose. It is provided "as is" without express or implied warranty. - * - * - * Copyright (c) 1994 - * Hewlett-Packard Company - * - * Permission to use, copy, modify, distribute and sell this software - * and its documentation for any purpose is hereby granted without fee, - * provided that the above copyright notice appear in all copies and - * that both that copyright notice and this permission notice appear - * in supporting documentation. Hewlett-Packard Company makes no - * representations about the suitability of this software for any - * purpose. It is provided "as is" without express or implied warranty. - * - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_NODE_HPP -#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_NODE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_COMPRESSED_ORDERED_INDEX_NODES) -#include -#include -#include -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* definition of red-black nodes for ordered_index */ - -enum ordered_index_color{red=false,black=true}; -enum ordered_index_side{to_left=false,to_right=true}; - -template -struct ordered_index_node_impl; /* fwd decl. */ - -template -struct ordered_index_node_std_base -{ - typedef typename - boost::detail::allocator::rebind_to< - Allocator, - ordered_index_node_impl - >::type::pointer pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator, - ordered_index_node_impl - >::type::const_pointer const_pointer; - typedef ordered_index_color& color_ref; - typedef pointer& parent_ref; - - ordered_index_color& color(){return color_;} - ordered_index_color color()const{return color_;} - pointer& parent(){return parent_;} - pointer parent()const{return parent_;} - pointer& left(){return left_;} - pointer left()const{return left_;} - pointer& right(){return right_;} - pointer right()const{return right_;} - -private: - ordered_index_color color_; - pointer parent_; - pointer left_; - pointer right_; -}; - -#if !defined(BOOST_MULTI_INDEX_DISABLE_COMPRESSED_ORDERED_INDEX_NODES) -/* If ordered_index_node_impl has even alignment, we can use the least - * significant bit of one of the ordered_index_node_impl pointers to - * store color information. This typically reduces the size of - * ordered_index_node_impl by 25%. - */ - -#if defined(BOOST_MSVC) -/* This code casts pointers to an integer type that has been computed - * to be large enough to hold the pointer, however the metaprogramming - * logic is not always spotted by the VC++ code analyser that issues a - * long list of warnings. - */ - -#pragma warning(push) -#pragma warning(disable:4312 4311) -#endif - -template -struct ordered_index_node_compressed_base -{ - typedef ordered_index_node_impl< - AugmentPolicy,Allocator>* pointer; - typedef const ordered_index_node_impl< - AugmentPolicy,Allocator>* const_pointer; - - struct color_ref - { - color_ref(uintptr_type* r_):r(r_){} - - operator ordered_index_color()const - { - return ordered_index_color(*r&uintptr_type(1)); - } - - color_ref& operator=(ordered_index_color c) - { - *r&=~uintptr_type(1); - *r|=uintptr_type(c); - return *this; - } - - color_ref& operator=(const color_ref& x) - { - return operator=(x.operator ordered_index_color()); - } - - private: - uintptr_type* r; - }; - - struct parent_ref - { - parent_ref(uintptr_type* r_):r(r_){} - - operator pointer()const - { - return (pointer)(void*)(*r&~uintptr_type(1)); - } - - parent_ref& operator=(pointer p) - { - *r=((uintptr_type)(void*)p)|(*r&uintptr_type(1)); - return *this; - } - - parent_ref& operator=(const parent_ref& x) - { - return operator=(x.operator pointer()); - } - - pointer operator->()const - { - return operator pointer(); - } - - private: - uintptr_type* r; - }; - - color_ref color(){return color_ref(&parentcolor_);} - ordered_index_color color()const - { - return ordered_index_color(parentcolor_&uintptr_type(1)); - } - - parent_ref parent(){return parent_ref(&parentcolor_);} - pointer parent()const - { - return (pointer)(void*)(parentcolor_&~uintptr_type(1)); - } - - pointer& left(){return left_;} - pointer left()const{return left_;} - pointer& right(){return right_;} - pointer right()const{return right_;} - -private: - uintptr_type parentcolor_; - pointer left_; - pointer right_; -}; -#if defined(BOOST_MSVC) -#pragma warning(pop) -#endif -#endif - -template -struct ordered_index_node_impl_base: - -#if !defined(BOOST_MULTI_INDEX_DISABLE_COMPRESSED_ORDERED_INDEX_NODES) - AugmentPolicy::template augmented_node< - typename mpl::if_c< - !(has_uintptr_type::value)|| - (alignment_of< - ordered_index_node_compressed_base - >::value%2)|| - !(is_same< - typename boost::detail::allocator::rebind_to< - Allocator, - ordered_index_node_impl - >::type::pointer, - ordered_index_node_impl*>::value), - ordered_index_node_std_base, - ordered_index_node_compressed_base - >::type - >::type -#else - AugmentPolicy::template augmented_node< - ordered_index_node_std_base - >::type -#endif - -{}; - -template -struct ordered_index_node_impl: - ordered_index_node_impl_base -{ -private: - typedef ordered_index_node_impl_base super; - -public: - typedef typename super::color_ref color_ref; - typedef typename super::parent_ref parent_ref; - typedef typename super::pointer pointer; - typedef typename super::const_pointer const_pointer; - - /* interoperability with bidir_node_iterator */ - - static void increment(pointer& x) - { - if(x->right()!=pointer(0)){ - x=x->right(); - while(x->left()!=pointer(0))x=x->left(); - } - else{ - pointer y=x->parent(); - while(x==y->right()){ - x=y; - y=y->parent(); - } - if(x->right()!=y)x=y; - } - } - - static void decrement(pointer& x) - { - if(x->color()==red&&x->parent()->parent()==x){ - x=x->right(); - } - else if(x->left()!=pointer(0)){ - pointer y=x->left(); - while(y->right()!=pointer(0))y=y->right(); - x=y; - }else{ - pointer y=x->parent(); - while(x==y->left()){ - x=y; - y=y->parent(); - } - x=y; - } - } - - /* algorithmic stuff */ - - static void rotate_left(pointer x,parent_ref root) - { - pointer y=x->right(); - x->right()=y->left(); - if(y->left()!=pointer(0))y->left()->parent()=x; - y->parent()=x->parent(); - - if(x==root) root=y; - else if(x==x->parent()->left())x->parent()->left()=y; - else x->parent()->right()=y; - y->left()=x; - x->parent()=y; - AugmentPolicy::rotate_left(x,y); - } - - static pointer minimum(pointer x) - { - while(x->left()!=pointer(0))x=x->left(); - return x; - } - - static pointer maximum(pointer x) - { - while(x->right()!=pointer(0))x=x->right(); - return x; - } - - static void rotate_right(pointer x,parent_ref root) - { - pointer y=x->left(); - x->left()=y->right(); - if(y->right()!=pointer(0))y->right()->parent()=x; - y->parent()=x->parent(); - - if(x==root) root=y; - else if(x==x->parent()->right())x->parent()->right()=y; - else x->parent()->left()=y; - y->right()=x; - x->parent()=y; - AugmentPolicy::rotate_right(x,y); - } - - static void rebalance(pointer x,parent_ref root) - { - x->color()=red; - while(x!=root&&x->parent()->color()==red){ - if(x->parent()==x->parent()->parent()->left()){ - pointer y=x->parent()->parent()->right(); - if(y!=pointer(0)&&y->color()==red){ - x->parent()->color()=black; - y->color()=black; - x->parent()->parent()->color()=red; - x=x->parent()->parent(); - } - else{ - if(x==x->parent()->right()){ - x=x->parent(); - rotate_left(x,root); - } - x->parent()->color()=black; - x->parent()->parent()->color()=red; - rotate_right(x->parent()->parent(),root); - } - } - else{ - pointer y=x->parent()->parent()->left(); - if(y!=pointer(0)&&y->color()==red){ - x->parent()->color()=black; - y->color()=black; - x->parent()->parent()->color()=red; - x=x->parent()->parent(); - } - else{ - if(x==x->parent()->left()){ - x=x->parent(); - rotate_right(x,root); - } - x->parent()->color()=black; - x->parent()->parent()->color()=red; - rotate_left(x->parent()->parent(),root); - } - } - } - root->color()=black; - } - - static void link( - pointer x,ordered_index_side side,pointer position,pointer header) - { - if(side==to_left){ - position->left()=x; /* also makes leftmost=x when parent==header */ - if(position==header){ - header->parent()=x; - header->right()=x; - } - else if(position==header->left()){ - header->left()=x; /* maintain leftmost pointing to min node */ - } - } - else{ - position->right()=x; - if(position==header->right()){ - header->right()=x; /* maintain rightmost pointing to max node */ - } - } - x->parent()=position; - x->left()=pointer(0); - x->right()=pointer(0); - AugmentPolicy::add(x,pointer(header->parent())); - ordered_index_node_impl::rebalance(x,header->parent()); - } - - static pointer rebalance_for_erase( - pointer z,parent_ref root,pointer& leftmost,pointer& rightmost) - { - pointer y=z; - pointer x=pointer(0); - pointer x_parent=pointer(0); - if(y->left()==pointer(0)){ /* z has at most one non-null child. y==z. */ - x=y->right(); /* x might be null */ - } - else{ - if(y->right()==pointer(0)){ /* z has exactly one non-null child. y==z. */ - x=y->left(); /* x is not null */ - } - else{ /* z has two non-null children. Set y to */ - y=y->right(); /* z's successor. x might be null. */ - while(y->left()!=pointer(0))y=y->left(); - x=y->right(); - } - } - AugmentPolicy::remove(y,pointer(root)); - if(y!=z){ - AugmentPolicy::copy(z,y); - z->left()->parent()=y; /* relink y in place of z. y is z's successor */ - y->left()=z->left(); - if(y!=z->right()){ - x_parent=y->parent(); - if(x!=pointer(0))x->parent()=y->parent(); - y->parent()->left()=x; /* y must be a child of left */ - y->right()=z->right(); - z->right()->parent()=y; - } - else{ - x_parent=y; - } - - if(root==z) root=y; - else if(z->parent()->left()==z)z->parent()->left()=y; - else z->parent()->right()=y; - y->parent()=z->parent(); - ordered_index_color c=y->color(); - y->color()=z->color(); - z->color()=c; - y=z; /* y now points to node to be actually deleted */ - } - else{ /* y==z */ - x_parent=y->parent(); - if(x!=pointer(0))x->parent()=y->parent(); - if(root==z){ - root=x; - } - else{ - if(z->parent()->left()==z)z->parent()->left()=x; - else z->parent()->right()=x; - } - if(leftmost==z){ - if(z->right()==pointer(0)){ /* z->left() must be null also */ - leftmost=z->parent(); - } - else{ - leftmost=minimum(x); /* makes leftmost==header if z==root */ - } - } - if(rightmost==z){ - if(z->left()==pointer(0)){ /* z->right() must be null also */ - rightmost=z->parent(); - } - else{ /* x==z->left() */ - rightmost=maximum(x); /* makes rightmost==header if z==root */ - } - } - } - if(y->color()!=red){ - while(x!=root&&(x==pointer(0)|| x->color()==black)){ - if(x==x_parent->left()){ - pointer w=x_parent->right(); - if(w->color()==red){ - w->color()=black; - x_parent->color()=red; - rotate_left(x_parent,root); - w=x_parent->right(); - } - if((w->left()==pointer(0)||w->left()->color()==black) && - (w->right()==pointer(0)||w->right()->color()==black)){ - w->color()=red; - x=x_parent; - x_parent=x_parent->parent(); - } - else{ - if(w->right()==pointer(0 ) - || w->right()->color()==black){ - if(w->left()!=pointer(0)) w->left()->color()=black; - w->color()=red; - rotate_right(w,root); - w=x_parent->right(); - } - w->color()=x_parent->color(); - x_parent->color()=black; - if(w->right()!=pointer(0))w->right()->color()=black; - rotate_left(x_parent,root); - break; - } - } - else{ /* same as above,with right <-> left */ - pointer w=x_parent->left(); - if(w->color()==red){ - w->color()=black; - x_parent->color()=red; - rotate_right(x_parent,root); - w=x_parent->left(); - } - if((w->right()==pointer(0)||w->right()->color()==black) && - (w->left()==pointer(0)||w->left()->color()==black)){ - w->color()=red; - x=x_parent; - x_parent=x_parent->parent(); - } - else{ - if(w->left()==pointer(0)||w->left()->color()==black){ - if(w->right()!=pointer(0))w->right()->color()=black; - w->color()=red; - rotate_left(w,root); - w=x_parent->left(); - } - w->color()=x_parent->color(); - x_parent->color()=black; - if(w->left()!=pointer(0))w->left()->color()=black; - rotate_right(x_parent,root); - break; - } - } - } - if(x!=pointer(0))x->color()=black; - } - return y; - } - - static void restore(pointer x,pointer position,pointer header) - { - if(position->left()==pointer(0)||position->left()==header){ - link(x,to_left,position,header); - } - else{ - decrement(position); - link(x,to_right,position,header); - } - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - static std::size_t black_count(pointer node,pointer root) - { - if(node==pointer(0))return 0; - std::size_t sum=0; - for(;;){ - if(node->color()==black)++sum; - if(node==root)break; - node=node->parent(); - } - return sum; - } -#endif -}; - -template -struct ordered_index_node_trampoline: - ordered_index_node_impl< - AugmentPolicy, - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > -{ - typedef ordered_index_node_impl< - AugmentPolicy, - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > impl_type; -}; - -template -struct ordered_index_node: - Super,ordered_index_node_trampoline -{ -private: - typedef ordered_index_node_trampoline trampoline; - -public: - typedef typename trampoline::impl_type impl_type; - typedef typename trampoline::color_ref impl_color_ref; - typedef typename trampoline::parent_ref impl_parent_ref; - typedef typename trampoline::pointer impl_pointer; - typedef typename trampoline::const_pointer const_impl_pointer; - - impl_color_ref color(){return trampoline::color();} - ordered_index_color color()const{return trampoline::color();} - impl_parent_ref parent(){return trampoline::parent();} - impl_pointer parent()const{return trampoline::parent();} - impl_pointer& left(){return trampoline::left();} - impl_pointer left()const{return trampoline::left();} - impl_pointer& right(){return trampoline::right();} - impl_pointer right()const{return trampoline::right();} - - impl_pointer impl() - { - return static_cast( - static_cast(static_cast(this))); - } - - const_impl_pointer impl()const - { - return static_cast( - static_cast(static_cast(this))); - } - - static ordered_index_node* from_impl(impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - static const ordered_index_node* from_impl(const_impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - /* interoperability with bidir_node_iterator */ - - static void increment(ordered_index_node*& x) - { - impl_pointer xi=x->impl(); - trampoline::increment(xi); - x=from_impl(xi); - } - - static void decrement(ordered_index_node*& x) - { - impl_pointer xi=x->impl(); - trampoline::decrement(xi); - x=from_impl(xi); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_ops.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_ops.hpp deleted file mode 100644 index 84d5cacae19..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_ops.hpp +++ /dev/null @@ -1,266 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - * - * The internal implementation of red-black trees is based on that of SGI STL - * stl_tree.h file: - * - * Copyright (c) 1996,1997 - * Silicon Graphics Computer Systems, Inc. - * - * Permission to use, copy, modify, distribute and sell this software - * and its documentation for any purpose is hereby granted without fee, - * provided that the above copyright notice appear in all copies and - * that both that copyright notice and this permission notice appear - * in supporting documentation. Silicon Graphics makes no - * representations about the suitability of this software for any - * purpose. It is provided "as is" without express or implied warranty. - * - * - * Copyright (c) 1994 - * Hewlett-Packard Company - * - * Permission to use, copy, modify, distribute and sell this software - * and its documentation for any purpose is hereby granted without fee, - * provided that the above copyright notice appear in all copies and - * that both that copyright notice and this permission notice appear - * in supporting documentation. Hewlett-Packard Company makes no - * representations about the suitability of this software for any - * purpose. It is provided "as is" without express or implied warranty. - * - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_OPS_HPP -#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_OPS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Common code for index memfuns having templatized and - * non-templatized versions. - * Implementation note: When CompatibleKey is consistently promoted to - * KeyFromValue::result_type for comparison, the promotion is made once in - * advance to increase efficiency. - */ - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline Node* ordered_index_find( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ordered_index_find( - top,y,key,x,comp, - mpl::and_< - promotes_1st_arg, - promotes_2nd_arg >()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline Node* ordered_index_find( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ordered_index_find(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline Node* ordered_index_find( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - Node* y0=y; - - while (top){ - if(!comp(key(top->value()),x)){ - y=top; - top=Node::from_impl(top->left()); - } - else top=Node::from_impl(top->right()); - } - - return (y==y0||comp(x,key(y->value())))?y0:y; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline Node* ordered_index_lower_bound( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ordered_index_lower_bound( - top,y,key,x,comp, - promotes_2nd_arg()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline Node* ordered_index_lower_bound( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ordered_index_lower_bound(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline Node* ordered_index_lower_bound( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - while(top){ - if(!comp(key(top->value()),x)){ - y=top; - top=Node::from_impl(top->left()); - } - else top=Node::from_impl(top->right()); - } - - return y; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline Node* ordered_index_upper_bound( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ordered_index_upper_bound( - top,y,key,x,comp, - promotes_1st_arg()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline Node* ordered_index_upper_bound( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ordered_index_upper_bound(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline Node* ordered_index_upper_bound( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - while(top){ - if(comp(x,key(top->value()))){ - y=top; - top=Node::from_impl(top->left()); - } - else top=Node::from_impl(top->right()); - } - - return y; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::pair ordered_index_equal_range( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ordered_index_equal_range( - top,y,key,x,comp, - mpl::and_< - promotes_1st_arg, - promotes_2nd_arg >()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline std::pair ordered_index_equal_range( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ordered_index_equal_range(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::pair ordered_index_equal_range( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - while(top){ - if(comp(key(top->value()),x)){ - top=Node::from_impl(top->right()); - } - else if(comp(x,key(top->value()))){ - y=top; - top=Node::from_impl(top->left()); - } - else{ - return std::pair( - ordered_index_lower_bound( - Node::from_impl(top->left()),top,key,x,comp,mpl::false_()), - ordered_index_upper_bound( - Node::from_impl(top->right()),y,key,x,comp,mpl::false_())); - } - } - - return std::pair(y,y); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/promotes_arg.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/promotes_arg.hpp deleted file mode 100644 index 7a11b6e9fbe..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/promotes_arg.hpp +++ /dev/null @@ -1,83 +0,0 @@ -/* Copyright 2003-2017 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_PROMOTES_ARG_HPP -#define BOOST_MULTI_INDEX_DETAIL_PROMOTES_ARG_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -/* Metafunctions to check if f(arg1,arg2) promotes either arg1 to the type of - * arg2 or viceversa. By default, (i.e. if it cannot be determined), no - * promotion is assumed. - */ - -#if BOOST_WORKAROUND(BOOST_MSVC,<1400) - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -struct promotes_1st_arg:mpl::false_{}; - -template -struct promotes_2nd_arg:mpl::false_{}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#else - -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -struct promotes_1st_arg: - mpl::and_< - mpl::not_ >, - is_convertible, - is_transparent - > -{}; - -template -struct promotes_2nd_arg: - mpl::and_< - mpl::not_ >, - is_convertible, - is_transparent - > -{}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/raw_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/raw_ptr.hpp deleted file mode 100644 index c32007435c0..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/raw_ptr.hpp +++ /dev/null @@ -1,52 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RAW_PTR_HPP -#define BOOST_MULTI_INDEX_DETAIL_RAW_PTR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* gets the underlying pointer of a pointer-like value */ - -template -inline RawPointer raw_ptr(RawPointer const& p,mpl::true_) -{ - return p; -} - -template -inline RawPointer raw_ptr(Pointer const& p,mpl::false_) -{ - return p==Pointer(0)?0:&*p; -} - -template -inline RawPointer raw_ptr(Pointer const& p) -{ - return raw_ptr(p,is_same()); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/restore_wstrict_aliasing.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/restore_wstrict_aliasing.hpp deleted file mode 100644 index ee2c799d5a8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/restore_wstrict_aliasing.hpp +++ /dev/null @@ -1,11 +0,0 @@ -/* Copyright 2003-2016 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#define BOOST_MULTI_INDEX_DETAIL_RESTORE_WSTRICT_ALIASING -#include -#undef BOOST_MULTI_INDEX_DETAIL_RESTORE_WSTRICT_ALIASING diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_loader.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_loader.hpp deleted file mode 100644 index 4b00345a6d9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_loader.hpp +++ /dev/null @@ -1,173 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RND_INDEX_LOADER_HPP -#define BOOST_MULTI_INDEX_DETAIL_RND_INDEX_LOADER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* This class implements a serialization rearranger for random access - * indices. In order to achieve O(n) performance, the following strategy - * is followed: the nodes of the index are handled as if in a bidirectional - * list, where the next pointers are stored in the original - * random_access_index_ptr_array and the prev pointers are stored in - * an auxiliary array. Rearranging of nodes in such a bidirectional list - * is constant time. Once all the arrangements are performed (on destruction - * time) the list is traversed in reverse order and - * pointers are swapped and set accordingly so that they recover its - * original semantics ( *(node->up())==node ) while retaining the - * new order. - */ - -template -class random_access_index_loader_base:private noncopyable -{ -protected: - typedef random_access_index_node_impl< - typename boost::detail::allocator::rebind_to< - Allocator, - char - >::type - > node_impl_type; - typedef typename node_impl_type::pointer node_impl_pointer; - typedef random_access_index_ptr_array ptr_array; - - random_access_index_loader_base(const Allocator& al_,ptr_array& ptrs_): - al(al_), - ptrs(ptrs_), - header(*ptrs.end()), - prev_spc(al,0), - preprocessed(false) - {} - - ~random_access_index_loader_base() - { - if(preprocessed) - { - node_impl_pointer n=header; - next(n)=n; - - for(std::size_t i=ptrs.size();i--;){ - n=prev(n); - std::size_t d=position(n); - if(d!=i){ - node_impl_pointer m=prev(next_at(i)); - std::swap(m->up(),n->up()); - next_at(d)=next_at(i); - std::swap(prev_at(d),prev_at(i)); - } - next(n)=n; - } - } - } - - void rearrange(node_impl_pointer position_,node_impl_pointer x) - { - preprocess(); /* only incur this penalty if rearrange() is ever called */ - if(position_==node_impl_pointer(0))position_=header; - next(prev(x))=next(x); - prev(next(x))=prev(x); - prev(x)=position_; - next(x)=next(position_); - next(prev(x))=prev(next(x))=x; - } - -private: - void preprocess() - { - if(!preprocessed){ - /* get space for the auxiliary prev array */ - auto_space tmp(al,ptrs.size()+1); - prev_spc.swap(tmp); - - /* prev_spc elements point to the prev nodes */ - std::rotate_copy( - &*ptrs.begin(),&*ptrs.end(),&*ptrs.end()+1,&*prev_spc.data()); - - /* ptrs elements point to the next nodes */ - std::rotate(&*ptrs.begin(),&*ptrs.begin()+1,&*ptrs.end()+1); - - preprocessed=true; - } - } - - std::size_t position(node_impl_pointer x)const - { - return (std::size_t)(x->up()-ptrs.begin()); - } - - node_impl_pointer& next_at(std::size_t n)const - { - return *ptrs.at(n); - } - - node_impl_pointer& prev_at(std::size_t n)const - { - return *(prev_spc.data()+n); - } - - node_impl_pointer& next(node_impl_pointer x)const - { - return *(x->up()); - } - - node_impl_pointer& prev(node_impl_pointer x)const - { - return prev_at(position(x)); - } - - Allocator al; - ptr_array& ptrs; - node_impl_pointer header; - auto_space prev_spc; - bool preprocessed; -}; - -template -class random_access_index_loader: - private random_access_index_loader_base -{ - typedef random_access_index_loader_base super; - typedef typename super::node_impl_pointer node_impl_pointer; - typedef typename super::ptr_array ptr_array; - -public: - random_access_index_loader(const Allocator& al_,ptr_array& ptrs_): - super(al_,ptrs_) - {} - - void rearrange(Node* position_,Node *x) - { - super::rearrange( - position_?position_->impl():node_impl_pointer(0),x->impl()); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_node.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_node.hpp deleted file mode 100644 index ad61ea25dda..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_node.hpp +++ /dev/null @@ -1,273 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RND_INDEX_NODE_HPP -#define BOOST_MULTI_INDEX_DETAIL_RND_INDEX_NODE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -struct random_access_index_node_impl -{ - typedef typename - boost::detail::allocator::rebind_to< - Allocator,random_access_index_node_impl - >::type::pointer pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator,random_access_index_node_impl - >::type::const_pointer const_pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator,pointer - >::type::pointer ptr_pointer; - - ptr_pointer& up(){return up_;} - ptr_pointer up()const{return up_;} - - /* interoperability with rnd_node_iterator */ - - static void increment(pointer& x) - { - x=*(x->up()+1); - } - - static void decrement(pointer& x) - { - x=*(x->up()-1); - } - - static void advance(pointer& x,std::ptrdiff_t n) - { - x=*(x->up()+n); - } - - static std::ptrdiff_t distance(pointer x,pointer y) - { - return y->up()-x->up(); - } - - /* algorithmic stuff */ - - static void relocate(ptr_pointer pos,ptr_pointer x) - { - pointer n=*x; - if(xup()=pos-1; - } - else{ - while(x!=pos){ - *x=*(x-1); - (*x)->up()=x; - --x; - } - *pos=n; - n->up()=pos; - } - }; - - static void relocate(ptr_pointer pos,ptr_pointer first,ptr_pointer last) - { - ptr_pointer begin,middle,end; - if(posup()=begin+j; - break; - } - else{ - *(begin+j)=*(begin+k); - (*(begin+j))->up()=begin+j; - } - - if(kup()=begin+k; - break; - } - else{ - *(begin+k)=*(begin+j); - (*(begin+k))->up()=begin+k; - } - } - } - }; - - static void extract(ptr_pointer x,ptr_pointer pend) - { - --pend; - while(x!=pend){ - *x=*(x+1); - (*x)->up()=x; - ++x; - } - } - - static void transfer( - ptr_pointer pbegin0,ptr_pointer pend0,ptr_pointer pbegin1) - { - while(pbegin0!=pend0){ - *pbegin1=*pbegin0++; - (*pbegin1)->up()=pbegin1; - ++pbegin1; - } - } - - static void reverse(ptr_pointer pbegin,ptr_pointer pend) - { - std::ptrdiff_t d=(pend-pbegin)/2; - for(std::ptrdiff_t i=0;iup()=pbegin; - (*pend)->up()=pend; - ++pbegin; - } - } - -private: - ptr_pointer up_; -}; - -template -struct random_access_index_node_trampoline: - random_access_index_node_impl< - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > -{ - typedef random_access_index_node_impl< - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > impl_type; -}; - -template -struct random_access_index_node: - Super,random_access_index_node_trampoline -{ -private: - typedef random_access_index_node_trampoline trampoline; - -public: - typedef typename trampoline::impl_type impl_type; - typedef typename trampoline::pointer impl_pointer; - typedef typename trampoline::const_pointer const_impl_pointer; - typedef typename trampoline::ptr_pointer impl_ptr_pointer; - - impl_ptr_pointer& up(){return trampoline::up();} - impl_ptr_pointer up()const{return trampoline::up();} - - impl_pointer impl() - { - return static_cast( - static_cast(static_cast(this))); - } - - const_impl_pointer impl()const - { - return static_cast( - static_cast(static_cast(this))); - } - - static random_access_index_node* from_impl(impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - static const random_access_index_node* from_impl(const_impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - /* interoperability with rnd_node_iterator */ - - static void increment(random_access_index_node*& x) - { - impl_pointer xi=x->impl(); - trampoline::increment(xi); - x=from_impl(xi); - } - - static void decrement(random_access_index_node*& x) - { - impl_pointer xi=x->impl(); - trampoline::decrement(xi); - x=from_impl(xi); - } - - static void advance(random_access_index_node*& x,std::ptrdiff_t n) - { - impl_pointer xi=x->impl(); - trampoline::advance(xi,n); - x=from_impl(xi); - } - - static std::ptrdiff_t distance( - random_access_index_node* x,random_access_index_node* y) - { - return trampoline::distance(x->impl(),y->impl()); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ops.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ops.hpp deleted file mode 100644 index f5e76e4441f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ops.hpp +++ /dev/null @@ -1,203 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RND_INDEX_OPS_HPP -#define BOOST_MULTI_INDEX_DETAIL_RND_INDEX_OPS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Common code for random_access_index memfuns having templatized and - * non-templatized versions. - */ - -template -Node* random_access_index_remove( - random_access_index_ptr_array& ptrs,Predicate pred) -{ - typedef typename Node::value_type value_type; - typedef typename Node::impl_ptr_pointer impl_ptr_pointer; - - impl_ptr_pointer first=ptrs.begin(), - res=first, - last=ptrs.end(); - for(;first!=last;++first){ - if(!pred( - const_cast(Node::from_impl(*first)->value()))){ - if(first!=res){ - std::swap(*first,*res); - (*first)->up()=first; - (*res)->up()=res; - } - ++res; - } - } - return Node::from_impl(*res); -} - -template -Node* random_access_index_unique( - random_access_index_ptr_array& ptrs,BinaryPredicate binary_pred) -{ - typedef typename Node::value_type value_type; - typedef typename Node::impl_ptr_pointer impl_ptr_pointer; - - impl_ptr_pointer first=ptrs.begin(), - res=first, - last=ptrs.end(); - if(first!=last){ - for(;++first!=last;){ - if(!binary_pred( - const_cast(Node::from_impl(*res)->value()), - const_cast(Node::from_impl(*first)->value()))){ - ++res; - if(first!=res){ - std::swap(*first,*res); - (*first)->up()=first; - (*res)->up()=res; - } - } - } - ++res; - } - return Node::from_impl(*res); -} - -template -void random_access_index_inplace_merge( - const Allocator& al, - random_access_index_ptr_array& ptrs, - BOOST_DEDUCED_TYPENAME Node::impl_ptr_pointer first1,Compare comp) -{ - typedef typename Node::value_type value_type; - typedef typename Node::impl_pointer impl_pointer; - typedef typename Node::impl_ptr_pointer impl_ptr_pointer; - - auto_space spc(al,ptrs.size()); - - impl_ptr_pointer first0=ptrs.begin(), - last0=first1, - last1=ptrs.end(), - out=spc.data(); - while(first0!=last0&&first1!=last1){ - if(comp( - const_cast(Node::from_impl(*first1)->value()), - const_cast(Node::from_impl(*first0)->value()))){ - *out++=*first1++; - } - else{ - *out++=*first0++; - } - } - std::copy(&*first0,&*last0,&*out); - std::copy(&*first1,&*last1,&*out); - - first1=ptrs.begin(); - out=spc.data(); - while(first1!=last1){ - *first1=*out++; - (*first1)->up()=first1; - ++first1; - } -} - -/* sorting */ - -/* auxiliary stuff */ - -template -struct random_access_index_sort_compare -{ - typedef typename Node::impl_pointer first_argument_type; - typedef typename Node::impl_pointer second_argument_type; - typedef bool result_type; - - random_access_index_sort_compare(Compare comp_=Compare()):comp(comp_){} - - bool operator()( - typename Node::impl_pointer x,typename Node::impl_pointer y)const - { - typedef typename Node::value_type value_type; - - return comp( - const_cast(Node::from_impl(x)->value()), - const_cast(Node::from_impl(y)->value())); - } - -private: - Compare comp; -}; - -template -void random_access_index_sort( - const Allocator& al, - random_access_index_ptr_array& ptrs, - Compare comp) -{ - /* The implementation is extremely simple: an auxiliary - * array of pointers is sorted using stdlib facilities and - * then used to rearrange the index. This is suboptimal - * in space and time, but has some advantages over other - * possible approaches: - * - Use std::stable_sort() directly on ptrs using some - * special iterator in charge of maintaining pointers - * and up() pointers in sync: we cannot guarantee - * preservation of the container invariants in the face of - * exceptions, if, for instance, std::stable_sort throws - * when ptrs transitorily contains duplicate elements. - * - Rewrite the internal algorithms of std::stable_sort - * adapted for this case: besides being a fair amount of - * work, making a stable sort compatible with Boost.MultiIndex - * invariants (basically, no duplicates or missing elements - * even if an exception is thrown) is complicated, error-prone - * and possibly won't perform much better than the - * solution adopted. - */ - - if(ptrs.size()<=1)return; - - typedef typename Node::impl_pointer impl_pointer; - typedef typename Node::impl_ptr_pointer impl_ptr_pointer; - typedef random_access_index_sort_compare< - Node,Compare> ptr_compare; - - impl_ptr_pointer first=ptrs.begin(); - impl_ptr_pointer last=ptrs.end(); - auto_space< - impl_pointer, - Allocator> spc(al,ptrs.size()); - impl_ptr_pointer buf=spc.data(); - - std::copy(&*first,&*last,&*buf); - std::stable_sort(&*buf,&*buf+ptrs.size(),ptr_compare(comp)); - - while(first!=last){ - *first=*buf++; - (*first)->up()=first; - ++first; - } -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ptr_array.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ptr_array.hpp deleted file mode 100644 index bae1c851b8e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ptr_array.hpp +++ /dev/null @@ -1,144 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RND_INDEX_PTR_ARRAY_HPP -#define BOOST_MULTI_INDEX_DETAIL_RND_INDEX_PTR_ARRAY_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* pointer structure for use by random access indices */ - -template -class random_access_index_ptr_array:private noncopyable -{ - typedef random_access_index_node_impl< - typename boost::detail::allocator::rebind_to< - Allocator, - char - >::type - > node_impl_type; - -public: - typedef typename node_impl_type::pointer value_type; - typedef typename boost::detail::allocator::rebind_to< - Allocator,value_type - >::type::pointer pointer; - - random_access_index_ptr_array( - const Allocator& al,value_type end_,std::size_t sz): - size_(sz), - capacity_(sz), - spc(al,capacity_+1) - { - *end()=end_; - end_->up()=end(); - } - - std::size_t size()const{return size_;} - std::size_t capacity()const{return capacity_;} - - void room_for_one() - { - if(size_==capacity_){ - reserve(capacity_<=10?15:capacity_+capacity_/2); - } - } - - void reserve(std::size_t c) - { - if(c>capacity_)set_capacity(c); - } - - void shrink_to_fit() - { - if(capacity_>size_)set_capacity(size_); - } - - pointer begin()const{return ptrs();} - pointer end()const{return ptrs()+size_;} - pointer at(std::size_t n)const{return ptrs()+n;} - - void push_back(value_type x) - { - *(end()+1)=*end(); - (*(end()+1))->up()=end()+1; - *end()=x; - (*end())->up()=end(); - ++size_; - } - - void erase(value_type x) - { - node_impl_type::extract(x->up(),end()+1); - --size_; - } - - void clear() - { - *begin()=*end(); - (*begin())->up()=begin(); - size_=0; - } - - void swap(random_access_index_ptr_array& x) - { - std::swap(size_,x.size_); - std::swap(capacity_,x.capacity_); - spc.swap(x.spc); - } - -private: - std::size_t size_; - std::size_t capacity_; - auto_space spc; - - pointer ptrs()const - { - return spc.data(); - } - - void set_capacity(std::size_t c) - { - auto_space spc1(spc.get_allocator(),c+1); - node_impl_type::transfer(begin(),end()+1,spc1.data()); - spc.swap(spc1); - capacity_=c; - } -}; - -template -void swap( - random_access_index_ptr_array& x, - random_access_index_ptr_array& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_node_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_node_iterator.hpp deleted file mode 100644 index 48026132fb7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_node_iterator.hpp +++ /dev/null @@ -1,140 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RND_NODE_ITERATOR_HPP -#define BOOST_MULTI_INDEX_DETAIL_RND_NODE_ITERATOR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Iterator class for node-based indices with random access iterators. */ - -template -class rnd_node_iterator: - public random_access_iterator_helper< - rnd_node_iterator, - typename Node::value_type, - std::ptrdiff_t, - const typename Node::value_type*, - const typename Node::value_type&> -{ -public: - /* coverity[uninit_ctor]: suppress warning */ - rnd_node_iterator(){} - explicit rnd_node_iterator(Node* node_):node(node_){} - - const typename Node::value_type& operator*()const - { - return node->value(); - } - - rnd_node_iterator& operator++() - { - Node::increment(node); - return *this; - } - - rnd_node_iterator& operator--() - { - Node::decrement(node); - return *this; - } - - rnd_node_iterator& operator+=(std::ptrdiff_t n) - { - Node::advance(node,n); - return *this; - } - - rnd_node_iterator& operator-=(std::ptrdiff_t n) - { - Node::advance(node,-n); - return *this; - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* Serialization. As for why the following is public, - * see explanation in safe_mode_iterator notes in safe_mode.hpp. - */ - - BOOST_SERIALIZATION_SPLIT_MEMBER() - - typedef typename Node::base_type node_base_type; - - template - void save(Archive& ar,const unsigned int)const - { - node_base_type* bnode=node; - ar< - void load(Archive& ar,const unsigned int) - { - node_base_type* bnode; - ar>>serialization::make_nvp("pointer",bnode); - node=static_cast(bnode); - } -#endif - - /* get_node is not to be used by the user */ - - typedef Node node_type; - - Node* get_node()const{return node;} - -private: - Node* node; -}; - -template -bool operator==( - const rnd_node_iterator& x, - const rnd_node_iterator& y) -{ - return x.get_node()==y.get_node(); -} - -template -bool operator<( - const rnd_node_iterator& x, - const rnd_node_iterator& y) -{ - return Node::distance(x.get_node(),y.get_node())>0; -} - -template -std::ptrdiff_t operator-( - const rnd_node_iterator& x, - const rnd_node_iterator& y) -{ - return Node::distance(y.get_node(),x.get_node()); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnk_index_ops.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnk_index_ops.hpp deleted file mode 100644 index fb233cf4973..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnk_index_ops.hpp +++ /dev/null @@ -1,300 +0,0 @@ -/* Copyright 2003-2017 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RNK_INDEX_OPS_HPP -#define BOOST_MULTI_INDEX_DETAIL_RNK_INDEX_OPS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Common code for ranked_index memfuns having templatized and - * non-templatized versions. - */ - -template -inline std::size_t ranked_node_size(Pointer x) -{ - return x!=Pointer(0)?x->size:0; -} - -template -inline Pointer ranked_index_nth(std::size_t n,Pointer end_) -{ - Pointer top=end_->parent(); - if(top==Pointer(0)||n>=top->size)return end_; - - for(;;){ - std::size_t s=ranked_node_size(top->left()); - if(n==s)return top; - if(nleft(); - else{ - top=top->right(); - n-=s+1; - } - } -} - -template -inline std::size_t ranked_index_rank(Pointer x,Pointer end_) -{ - Pointer top=end_->parent(); - if(top==Pointer(0))return 0; - if(x==end_)return top->size; - - std::size_t s=ranked_node_size(x->left()); - while(x!=top){ - Pointer z=x->parent(); - if(x==z->right()){ - s+=ranked_node_size(z->left())+1; - } - x=z; - } - return s; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::size_t ranked_index_find_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ranked_index_find_rank( - top,y,key,x,comp, - mpl::and_< - promotes_1st_arg, - promotes_2nd_arg >()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline std::size_t ranked_index_find_rank( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ranked_index_find_rank(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::size_t ranked_index_find_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - if(!top)return 0; - - std::size_t s=top->impl()->size, - s0=s; - Node* y0=y; - - do{ - if(!comp(key(top->value()),x)){ - y=top; - s-=ranked_node_size(y->right())+1; - top=Node::from_impl(top->left()); - } - else top=Node::from_impl(top->right()); - }while(top); - - return (y==y0||comp(x,key(y->value())))?s0:s; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::size_t ranked_index_lower_bound_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ranked_index_lower_bound_rank( - top,y,key,x,comp, - promotes_2nd_arg()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline std::size_t ranked_index_lower_bound_rank( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ranked_index_lower_bound_rank(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::size_t ranked_index_lower_bound_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - if(!top)return 0; - - std::size_t s=top->impl()->size; - - do{ - if(!comp(key(top->value()),x)){ - y=top; - s-=ranked_node_size(y->right())+1; - top=Node::from_impl(top->left()); - } - else top=Node::from_impl(top->right()); - }while(top); - - return s; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::size_t ranked_index_upper_bound_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ranked_index_upper_bound_rank( - top,y,key,x,comp, - promotes_1st_arg()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline std::size_t ranked_index_upper_bound_rank( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ranked_index_upper_bound_rank(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::size_t ranked_index_upper_bound_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - if(!top)return 0; - - std::size_t s=top->impl()->size; - - do{ - if(comp(x,key(top->value()))){ - y=top; - s-=ranked_node_size(y->right())+1; - top=Node::from_impl(top->left()); - } - else top=Node::from_impl(top->right()); - }while(top); - - return s; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::pair ranked_index_equal_range_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ranked_index_equal_range_rank( - top,y,key,x,comp, - mpl::and_< - promotes_1st_arg, - promotes_2nd_arg >()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline std::pair ranked_index_equal_range_rank( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ranked_index_equal_range_rank(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::pair ranked_index_equal_range_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - if(!top)return std::pair(0,0); - - std::size_t s=top->impl()->size; - - do{ - if(comp(key(top->value()),x)){ - top=Node::from_impl(top->right()); - } - else if(comp(x,key(top->value()))){ - y=top; - s-=ranked_node_size(y->right())+1; - top=Node::from_impl(top->left()); - } - else{ - return std::pair( - s-top->impl()->size+ - ranked_index_lower_bound_rank( - Node::from_impl(top->left()),top,key,x,comp,mpl::false_()), - s-ranked_node_size(top->right())+ - ranked_index_upper_bound_rank( - Node::from_impl(top->right()),y,key,x,comp,mpl::false_())); - } - }while(top); - - return std::pair(s,s); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/safe_mode.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/safe_mode.hpp deleted file mode 100644 index 905270e9fb3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/safe_mode.hpp +++ /dev/null @@ -1,588 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_SAFE_MODE_HPP -#define BOOST_MULTI_INDEX_DETAIL_SAFE_MODE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -/* Safe mode machinery, in the spirit of Cay Hortmann's "Safe STL" - * (http://www.horstmann.com/safestl.html). - * In this mode, containers of type Container are derived from - * safe_container, and their corresponding iterators - * are wrapped with safe_iterator. These classes provide - * an internal record of which iterators are at a given moment associated - * to a given container, and properly mark the iterators as invalid - * when the container gets destroyed. - * Iterators are chained in a single attached list, whose header is - * kept by the container. More elaborate data structures would yield better - * performance, but I decided to keep complexity to a minimum since - * speed is not an issue here. - * Safe mode iterators automatically check that only proper operations - * are performed on them: for instance, an invalid iterator cannot be - * dereferenced. Additionally, a set of utilty macros and functions are - * provided that serve to implement preconditions and cooperate with - * the framework within the container. - * Iterators can also be unchecked, i.e. they do not have info about - * which container they belong in. This situation arises when the iterator - * is restored from a serialization archive: only information on the node - * is available, and it is not possible to determine to which container - * the iterator is associated to. The only sensible policy is to assume - * unchecked iterators are valid, though this can certainly generate false - * positive safe mode checks. - * This is not a full-fledged safe mode framework, and is only intended - * for use within the limits of Boost.MultiIndex. - */ - -/* Assertion macros. These resolve to no-ops if - * !defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE). - */ - -#if !defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) -#undef BOOST_MULTI_INDEX_SAFE_MODE_ASSERT -#define BOOST_MULTI_INDEX_SAFE_MODE_ASSERT(expr,error_code) ((void)0) -#else -#if !defined(BOOST_MULTI_INDEX_SAFE_MODE_ASSERT) -#include -#define BOOST_MULTI_INDEX_SAFE_MODE_ASSERT(expr,error_code) BOOST_ASSERT(expr) -#endif -#endif - -#define BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_valid_iterator(it), \ - safe_mode::invalid_iterator); - -#define BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(it) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_dereferenceable_iterator(it), \ - safe_mode::not_dereferenceable_iterator); - -#define BOOST_MULTI_INDEX_CHECK_INCREMENTABLE_ITERATOR(it) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_incrementable_iterator(it), \ - safe_mode::not_incrementable_iterator); - -#define BOOST_MULTI_INDEX_CHECK_DECREMENTABLE_ITERATOR(it) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_decrementable_iterator(it), \ - safe_mode::not_decrementable_iterator); - -#define BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,cont) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_is_owner(it,cont), \ - safe_mode::not_owner); - -#define BOOST_MULTI_INDEX_CHECK_SAME_OWNER(it0,it1) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_same_owner(it0,it1), \ - safe_mode::not_same_owner); - -#define BOOST_MULTI_INDEX_CHECK_VALID_RANGE(it0,it1) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_valid_range(it0,it1), \ - safe_mode::invalid_range); - -#define BOOST_MULTI_INDEX_CHECK_OUTSIDE_RANGE(it,it0,it1) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_outside_range(it,it0,it1), \ - safe_mode::inside_range); - -#define BOOST_MULTI_INDEX_CHECK_IN_BOUNDS(it,n) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_in_bounds(it,n), \ - safe_mode::out_of_bounds); - -#define BOOST_MULTI_INDEX_CHECK_DIFFERENT_CONTAINER(cont0,cont1) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_different_container(cont0,cont1), \ - safe_mode::same_container); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#endif - -#if defined(BOOST_HAS_THREADS) -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace safe_mode{ - -/* Checking routines. Assume the best for unchecked iterators - * (i.e. they pass the checking when there is not enough info - * to know.) - */ - -template -inline bool check_valid_iterator(const Iterator& it) -{ - return it.valid()||it.unchecked(); -} - -template -inline bool check_dereferenceable_iterator(const Iterator& it) -{ - return (it.valid()&&it!=it.owner()->end())||it.unchecked(); -} - -template -inline bool check_incrementable_iterator(const Iterator& it) -{ - return (it.valid()&&it!=it.owner()->end())||it.unchecked(); -} - -template -inline bool check_decrementable_iterator(const Iterator& it) -{ - return (it.valid()&&it!=it.owner()->begin())||it.unchecked(); -} - -template -inline bool check_is_owner( - const Iterator& it,const typename Iterator::container_type& cont) -{ - return (it.valid()&&it.owner()==&cont)||it.unchecked(); -} - -template -inline bool check_same_owner(const Iterator& it0,const Iterator& it1) -{ - return (it0.valid()&&it1.valid()&&it0.owner()==it1.owner())|| - it0.unchecked()||it1.unchecked(); -} - -template -inline bool check_valid_range(const Iterator& it0,const Iterator& it1) -{ - if(!check_same_owner(it0,it1))return false; - - if(it0.valid()){ - Iterator last=it0.owner()->end(); - if(it1==last)return true; - - for(Iterator first=it0;first!=last;++first){ - if(first==it1)return true; - } - return false; - } - return true; -} - -template -inline bool check_outside_range( - const Iterator& it,const Iterator& it0,const Iterator& it1) -{ - if(!check_same_owner(it0,it1))return false; - - if(it0.valid()){ - Iterator last=it0.owner()->end(); - bool found=false; - - Iterator first=it0; - for(;first!=last;++first){ - if(first==it1)break; - - /* crucial that this check goes after previous break */ - - if(first==it)found=true; - } - if(first!=it1)return false; - return !found; - } - return true; -} - -template -inline bool check_in_bounds(const Iterator& it,Difference n) -{ - if(it.unchecked())return true; - if(!it.valid()) return false; - if(n>0) return it.owner()->end()-it>=n; - else return it.owner()->begin()-it<=n; -} - -template -inline bool check_different_container( - const Container& cont0,const Container& cont1) -{ - return &cont0!=&cont1; -} - -/* Invalidates all iterators equivalent to that given. Safe containers - * must call this when deleting elements: the safe mode framework cannot - * perform this operation automatically without outside help. - */ - -template -inline void detach_equivalent_iterators(Iterator& it) -{ - if(it.valid()){ - { -#if defined(BOOST_HAS_THREADS) - boost::detail::lightweight_mutex::scoped_lock lock(it.cont->mutex); -#endif - - Iterator *prev_,*next_; - for( - prev_=static_cast(&it.cont->header); - (next_=static_cast(prev_->next))!=0;){ - if(next_!=&it&&*next_==it){ - prev_->next=next_->next; - next_->cont=0; - } - else prev_=next_; - } - } - it.detach(); - } -} - -template class safe_container; /* fwd decl. */ - -} /* namespace multi_index::safe_mode */ - -namespace detail{ - -class safe_container_base; /* fwd decl. */ - -class safe_iterator_base -{ -public: - bool valid()const{return cont!=0;} - bool unchecked()const{return unchecked_;} - - inline void detach(); - - void uncheck() - { - detach(); - unchecked_=true; - } - -protected: - safe_iterator_base():cont(0),next(0),unchecked_(false){} - - explicit safe_iterator_base(safe_container_base* cont_): - unchecked_(false) - { - attach(cont_); - } - - safe_iterator_base(const safe_iterator_base& it): - unchecked_(it.unchecked_) - { - attach(it.cont); - } - - safe_iterator_base& operator=(const safe_iterator_base& it) - { - unchecked_=it.unchecked_; - safe_container_base* new_cont=it.cont; - if(cont!=new_cont){ - detach(); - attach(new_cont); - } - return *this; - } - - ~safe_iterator_base() - { - detach(); - } - - const safe_container_base* owner()const{return cont;} - -BOOST_MULTI_INDEX_PRIVATE_IF_MEMBER_TEMPLATE_FRIENDS: - friend class safe_container_base; - -#if !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) - template friend class safe_mode::safe_container; - template friend - void safe_mode::detach_equivalent_iterators(Iterator&); -#endif - - inline void attach(safe_container_base* cont_); - - safe_container_base* cont; - safe_iterator_base* next; - bool unchecked_; -}; - -class safe_container_base:private noncopyable -{ -public: - safe_container_base(){} - -BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: - friend class safe_iterator_base; - -#if !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) - template friend - void safe_mode::detach_equivalent_iterators(Iterator&); -#endif - - ~safe_container_base() - { - /* Detaches all remaining iterators, which by now will - * be those pointing to the end of the container. - */ - - for(safe_iterator_base* it=header.next;it;it=it->next)it->cont=0; - header.next=0; - } - - void swap(safe_container_base& x) - { - for(safe_iterator_base* it0=header.next;it0;it0=it0->next)it0->cont=&x; - for(safe_iterator_base* it1=x.header.next;it1;it1=it1->next)it1->cont=this; - std::swap(header.cont,x.header.cont); - std::swap(header.next,x.header.next); - } - - safe_iterator_base header; - -#if defined(BOOST_HAS_THREADS) - boost::detail::lightweight_mutex mutex; -#endif -}; - -void safe_iterator_base::attach(safe_container_base* cont_) -{ - cont=cont_; - if(cont){ -#if defined(BOOST_HAS_THREADS) - boost::detail::lightweight_mutex::scoped_lock lock(cont->mutex); -#endif - - next=cont->header.next; - cont->header.next=this; - } -} - -void safe_iterator_base::detach() -{ - if(cont){ -#if defined(BOOST_HAS_THREADS) - boost::detail::lightweight_mutex::scoped_lock lock(cont->mutex); -#endif - - safe_iterator_base *prev_,*next_; - for(prev_=&cont->header;(next_=prev_->next)!=this;prev_=next_){} - prev_->next=next; - cont=0; - } -} - -} /* namespace multi_index::detail */ - -namespace safe_mode{ - -/* In order to enable safe mode on a container: - * - The container must derive from safe_container, - * - iterators must be generated via safe_iterator, which adapts a - * preexistent unsafe iterator class. - */ - -template -class safe_container; - -template -class safe_iterator: - public detail::iter_adaptor,Iterator>, - public detail::safe_iterator_base -{ - typedef detail::iter_adaptor super; - typedef detail::safe_iterator_base safe_super; - -public: - typedef Container container_type; - typedef typename Iterator::reference reference; - typedef typename Iterator::difference_type difference_type; - - safe_iterator(){} - explicit safe_iterator(safe_container* cont_): - safe_super(cont_){} - template - safe_iterator(const T0& t0,safe_container* cont_): - super(Iterator(t0)),safe_super(cont_){} - template - safe_iterator( - const T0& t0,const T1& t1,safe_container* cont_): - super(Iterator(t0,t1)),safe_super(cont_){} - - safe_iterator& operator=(const safe_iterator& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(x); - this->base_reference()=x.base_reference(); - safe_super::operator=(x); - return *this; - } - - const container_type* owner()const - { - return - static_cast( - static_cast*>( - this->safe_super::owner())); - } - - /* get_node is not to be used by the user */ - - typedef typename Iterator::node_type node_type; - - node_type* get_node()const{return this->base_reference().get_node();} - -private: - friend class boost::multi_index::detail::iter_adaptor_access; - - reference dereference()const - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(*this); - return *(this->base_reference()); - } - - bool equal(const safe_iterator& x)const - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(x); - BOOST_MULTI_INDEX_CHECK_SAME_OWNER(*this,x); - return this->base_reference()==x.base_reference(); - } - - void increment() - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - BOOST_MULTI_INDEX_CHECK_INCREMENTABLE_ITERATOR(*this); - ++(this->base_reference()); - } - - void decrement() - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - BOOST_MULTI_INDEX_CHECK_DECREMENTABLE_ITERATOR(*this); - --(this->base_reference()); - } - - void advance(difference_type n) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - BOOST_MULTI_INDEX_CHECK_IN_BOUNDS(*this,n); - this->base_reference()+=n; - } - - difference_type distance_to(const safe_iterator& x)const - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(x); - BOOST_MULTI_INDEX_CHECK_SAME_OWNER(*this,x); - return x.base_reference()-this->base_reference(); - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* Serialization. Note that Iterator::save and Iterator:load - * are assumed to be defined and public: at first sight it seems - * like we could have resorted to the public serialization interface - * for doing the forwarding to the adapted iterator class: - * ar<>base_reference(); - * but this would cause incompatibilities if a saving - * program is in safe mode and the loading program is not, or - * viceversa --in safe mode, the archived iterator data is one layer - * deeper, this is especially relevant with XML archives. - * It'd be nice if Boost.Serialization provided some forwarding - * facility for use by adaptor classes. - */ - - friend class boost::serialization::access; - - BOOST_SERIALIZATION_SPLIT_MEMBER() - - template - void save(Archive& ar,const unsigned int version)const - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - this->base_reference().save(ar,version); - } - - template - void load(Archive& ar,const unsigned int version) - { - this->base_reference().load(ar,version); - safe_super::uncheck(); - } -#endif -}; - -template -class safe_container:public detail::safe_container_base -{ - typedef detail::safe_container_base super; - -public: - void detach_dereferenceable_iterators() - { - typedef typename Container::iterator iterator; - - iterator end_=static_cast(this)->end(); - iterator *prev_,*next_; - for( - prev_=static_cast(&this->header); - (next_=static_cast(prev_->next))!=0;){ - if(*next_!=end_){ - prev_->next=next_->next; - next_->cont=0; - } - else prev_=next_; - } - } - - void swap(safe_container& x) - { - super::swap(x); - } -}; - -} /* namespace multi_index::safe_mode */ - -} /* namespace multi_index */ - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -namespace serialization{ -template -struct version< - boost::multi_index::safe_mode::safe_iterator -> -{ - BOOST_STATIC_CONSTANT( - int,value=boost::serialization::version::value); -}; -} /* namespace serialization */ -#endif - -} /* namespace boost */ - -#endif /* BOOST_MULTI_INDEX_ENABLE_SAFE_MODE */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/scope_guard.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/scope_guard.hpp deleted file mode 100644 index 116f8f50415..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/scope_guard.hpp +++ /dev/null @@ -1,453 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_SCOPE_GUARD_HPP -#define BOOST_MULTI_INDEX_DETAIL_SCOPE_GUARD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Until some official version of the ScopeGuard idiom makes it into Boost, - * we locally define our own. This is a merely reformated version of - * ScopeGuard.h as defined in: - * Alexandrescu, A., Marginean, P.:"Generic: Change the Way You - * Write Exception-Safe Code - Forever", C/C++ Users Jornal, Dec 2000, - * http://www.drdobbs.com/184403758 - * with the following modifications: - * - General pretty formatting (pretty to my taste at least.) - * - Naming style changed to standard C++ library requirements. - * - Added scope_guard_impl4 and obj_scope_guard_impl3, (Boost.MultiIndex - * needs them). A better design would provide guards for many more - * arguments through the Boost Preprocessor Library. - * - Added scope_guard_impl_base::touch (see below.) - * - Removed RefHolder and ByRef, whose functionality is provided - * already by Boost.Ref. - * - Removed static make_guard's and make_obj_guard's, so that the code - * will work even if BOOST_NO_MEMBER_TEMPLATES is defined. This forces - * us to move some private ctors to public, though. - * - * NB: CodeWarrior Pro 8 seems to have problems looking up safe_execute - * without an explicit qualification. - * - * We also define the following variants of the idiom: - * - * - make_guard_if_c( ... ) - * - make_guard_if( ... ) - * - make_obj_guard_if_c( ... ) - * - make_obj_guard_if( ... ) - * which may be used with a compile-time constant to yield - * a "null_guard" if the boolean compile-time parameter is false, - * or conversely, the guard is only constructed if the constant is true. - * This is useful to avoid extra tagging, because the returned - * null_guard can be optimzed comlpetely away by the compiler. - */ - -class scope_guard_impl_base -{ -public: - scope_guard_impl_base():dismissed_(false){} - void dismiss()const{dismissed_=true;} - - /* This helps prevent some "unused variable" warnings under, for instance, - * GCC 3.2. - */ - void touch()const{} - -protected: - ~scope_guard_impl_base(){} - - scope_guard_impl_base(const scope_guard_impl_base& other): - dismissed_(other.dismissed_) - { - other.dismiss(); - } - - template - static void safe_execute(J& j){ - BOOST_TRY{ - if(!j.dismissed_)j.execute(); - } - BOOST_CATCH(...){} - BOOST_CATCH_END - } - - mutable bool dismissed_; - -private: - scope_guard_impl_base& operator=(const scope_guard_impl_base&); -}; - -typedef const scope_guard_impl_base& scope_guard; - -struct null_guard : public scope_guard_impl_base -{ - template< class T1 > - null_guard( const T1& ) - { } - - template< class T1, class T2 > - null_guard( const T1&, const T2& ) - { } - - template< class T1, class T2, class T3 > - null_guard( const T1&, const T2&, const T3& ) - { } - - template< class T1, class T2, class T3, class T4 > - null_guard( const T1&, const T2&, const T3&, const T4& ) - { } - - template< class T1, class T2, class T3, class T4, class T5 > - null_guard( const T1&, const T2&, const T3&, const T4&, const T5& ) - { } -}; - -template< bool cond, class T > -struct null_guard_return -{ - typedef typename boost::mpl::if_c::type type; -}; - -template -class scope_guard_impl0:public scope_guard_impl_base -{ -public: - scope_guard_impl0(F fun):fun_(fun){} - ~scope_guard_impl0(){scope_guard_impl_base::safe_execute(*this);} - void execute(){fun_();} - -protected: - - F fun_; -}; - -template -inline scope_guard_impl0 make_guard(F fun) -{ - return scope_guard_impl0(fun); -} - -template -inline typename null_guard_return >::type -make_guard_if_c(F fun) -{ - return typename null_guard_return >::type(fun); -} - -template -inline typename null_guard_return >::type -make_guard_if(F fun) -{ - return make_guard_if(fun); -} - -template -class scope_guard_impl1:public scope_guard_impl_base -{ -public: - scope_guard_impl1(F fun,P1 p1):fun_(fun),p1_(p1){} - ~scope_guard_impl1(){scope_guard_impl_base::safe_execute(*this);} - void execute(){fun_(p1_);} - -protected: - F fun_; - const P1 p1_; -}; - -template -inline scope_guard_impl1 make_guard(F fun,P1 p1) -{ - return scope_guard_impl1(fun,p1); -} - -template -inline typename null_guard_return >::type -make_guard_if_c(F fun,P1 p1) -{ - return typename null_guard_return >::type(fun,p1); -} - -template -inline typename null_guard_return >::type -make_guard_if(F fun,P1 p1) -{ - return make_guard_if_c(fun,p1); -} - -template -class scope_guard_impl2:public scope_guard_impl_base -{ -public: - scope_guard_impl2(F fun,P1 p1,P2 p2):fun_(fun),p1_(p1),p2_(p2){} - ~scope_guard_impl2(){scope_guard_impl_base::safe_execute(*this);} - void execute(){fun_(p1_,p2_);} - -protected: - F fun_; - const P1 p1_; - const P2 p2_; -}; - -template -inline scope_guard_impl2 make_guard(F fun,P1 p1,P2 p2) -{ - return scope_guard_impl2(fun,p1,p2); -} - -template -inline typename null_guard_return >::type -make_guard_if_c(F fun,P1 p1,P2 p2) -{ - return typename null_guard_return >::type(fun,p1,p2); -} - -template -inline typename null_guard_return >::type -make_guard_if(F fun,P1 p1,P2 p2) -{ - return make_guard_if_c(fun,p1,p2); -} - -template -class scope_guard_impl3:public scope_guard_impl_base -{ -public: - scope_guard_impl3(F fun,P1 p1,P2 p2,P3 p3):fun_(fun),p1_(p1),p2_(p2),p3_(p3){} - ~scope_guard_impl3(){scope_guard_impl_base::safe_execute(*this);} - void execute(){fun_(p1_,p2_,p3_);} - -protected: - F fun_; - const P1 p1_; - const P2 p2_; - const P3 p3_; -}; - -template -inline scope_guard_impl3 make_guard(F fun,P1 p1,P2 p2,P3 p3) -{ - return scope_guard_impl3(fun,p1,p2,p3); -} - -template -inline typename null_guard_return >::type -make_guard_if_c(F fun,P1 p1,P2 p2,P3 p3) -{ - return typename null_guard_return >::type(fun,p1,p2,p3); -} - -template -inline typename null_guard_return< C::value,scope_guard_impl3 >::type -make_guard_if(F fun,P1 p1,P2 p2,P3 p3) -{ - return make_guard_if_c(fun,p1,p2,p3); -} - -template -class scope_guard_impl4:public scope_guard_impl_base -{ -public: - scope_guard_impl4(F fun,P1 p1,P2 p2,P3 p3,P4 p4): - fun_(fun),p1_(p1),p2_(p2),p3_(p3),p4_(p4){} - ~scope_guard_impl4(){scope_guard_impl_base::safe_execute(*this);} - void execute(){fun_(p1_,p2_,p3_,p4_);} - -protected: - F fun_; - const P1 p1_; - const P2 p2_; - const P3 p3_; - const P4 p4_; -}; - -template -inline scope_guard_impl4 make_guard( - F fun,P1 p1,P2 p2,P3 p3,P4 p4) -{ - return scope_guard_impl4(fun,p1,p2,p3,p4); -} - -template -inline typename null_guard_return >::type -make_guard_if_c( - F fun,P1 p1,P2 p2,P3 p3,P4 p4) -{ - return typename null_guard_return >::type(fun,p1,p2,p3,p4); -} - -template -inline typename null_guard_return >::type -make_guard_if( - F fun,P1 p1,P2 p2,P3 p3,P4 p4) -{ - return make_guard_if_c(fun,p1,p2,p3,p4); -} - -template -class obj_scope_guard_impl0:public scope_guard_impl_base -{ -public: - obj_scope_guard_impl0(Obj& obj,MemFun mem_fun):obj_(obj),mem_fun_(mem_fun){} - ~obj_scope_guard_impl0(){scope_guard_impl_base::safe_execute(*this);} - void execute(){(obj_.*mem_fun_)();} - -protected: - Obj& obj_; - MemFun mem_fun_; -}; - -template -inline obj_scope_guard_impl0 make_obj_guard(Obj& obj,MemFun mem_fun) -{ - return obj_scope_guard_impl0(obj,mem_fun); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if_c(Obj& obj,MemFun mem_fun) -{ - return typename null_guard_return >::type(obj,mem_fun); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if(Obj& obj,MemFun mem_fun) -{ - return make_obj_guard_if_c(obj,mem_fun); -} - -template -class obj_scope_guard_impl1:public scope_guard_impl_base -{ -public: - obj_scope_guard_impl1(Obj& obj,MemFun mem_fun,P1 p1): - obj_(obj),mem_fun_(mem_fun),p1_(p1){} - ~obj_scope_guard_impl1(){scope_guard_impl_base::safe_execute(*this);} - void execute(){(obj_.*mem_fun_)(p1_);} - -protected: - Obj& obj_; - MemFun mem_fun_; - const P1 p1_; -}; - -template -inline obj_scope_guard_impl1 make_obj_guard( - Obj& obj,MemFun mem_fun,P1 p1) -{ - return obj_scope_guard_impl1(obj,mem_fun,p1); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if_c( Obj& obj,MemFun mem_fun,P1 p1) -{ - return typename null_guard_return >::type(obj,mem_fun,p1); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if( Obj& obj,MemFun mem_fun,P1 p1) -{ - return make_obj_guard_if_c(obj,mem_fun,p1); -} - -template -class obj_scope_guard_impl2:public scope_guard_impl_base -{ -public: - obj_scope_guard_impl2(Obj& obj,MemFun mem_fun,P1 p1,P2 p2): - obj_(obj),mem_fun_(mem_fun),p1_(p1),p2_(p2) - {} - ~obj_scope_guard_impl2(){scope_guard_impl_base::safe_execute(*this);} - void execute(){(obj_.*mem_fun_)(p1_,p2_);} - -protected: - Obj& obj_; - MemFun mem_fun_; - const P1 p1_; - const P2 p2_; -}; - -template -inline obj_scope_guard_impl2 -make_obj_guard(Obj& obj,MemFun mem_fun,P1 p1,P2 p2) -{ - return obj_scope_guard_impl2(obj,mem_fun,p1,p2); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if_c(Obj& obj,MemFun mem_fun,P1 p1,P2 p2) -{ - return typename null_guard_return >::type(obj,mem_fun,p1,p2); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if(Obj& obj,MemFun mem_fun,P1 p1,P2 p2) -{ - return make_obj_guard_if_c(obj,mem_fun,p1,p2); -} - -template -class obj_scope_guard_impl3:public scope_guard_impl_base -{ -public: - obj_scope_guard_impl3(Obj& obj,MemFun mem_fun,P1 p1,P2 p2,P3 p3): - obj_(obj),mem_fun_(mem_fun),p1_(p1),p2_(p2),p3_(p3) - {} - ~obj_scope_guard_impl3(){scope_guard_impl_base::safe_execute(*this);} - void execute(){(obj_.*mem_fun_)(p1_,p2_,p3_);} - -protected: - Obj& obj_; - MemFun mem_fun_; - const P1 p1_; - const P2 p2_; - const P3 p3_; -}; - -template -inline obj_scope_guard_impl3 -make_obj_guard(Obj& obj,MemFun mem_fun,P1 p1,P2 p2,P3 p3) -{ - return obj_scope_guard_impl3(obj,mem_fun,p1,p2,p3); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if_c(Obj& obj,MemFun mem_fun,P1 p1,P2 p2,P3 p3) -{ - return typename null_guard_return >::type(obj,mem_fun,p1,p2,p3); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if(Obj& obj,MemFun mem_fun,P1 p1,P2 p2,P3 p3) -{ - return make_obj_guard_if_c(obj,mem_fun,p1,p2,p3); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_node.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_node.hpp deleted file mode 100644 index 85b345af938..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_node.hpp +++ /dev/null @@ -1,217 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_SEQ_INDEX_NODE_HPP -#define BOOST_MULTI_INDEX_DETAIL_SEQ_INDEX_NODE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* doubly-linked node for use by sequenced_index */ - -template -struct sequenced_index_node_impl -{ - typedef typename - boost::detail::allocator::rebind_to< - Allocator,sequenced_index_node_impl - >::type::pointer pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator,sequenced_index_node_impl - >::type::const_pointer const_pointer; - - pointer& prior(){return prior_;} - pointer prior()const{return prior_;} - pointer& next(){return next_;} - pointer next()const{return next_;} - - /* interoperability with bidir_node_iterator */ - - static void increment(pointer& x){x=x->next();} - static void decrement(pointer& x){x=x->prior();} - - /* algorithmic stuff */ - - static void link(pointer x,pointer header) - { - x->prior()=header->prior(); - x->next()=header; - x->prior()->next()=x->next()->prior()=x; - }; - - static void unlink(pointer x) - { - x->prior()->next()=x->next(); - x->next()->prior()=x->prior(); - } - - static void relink(pointer position,pointer x) - { - unlink(x); - x->prior()=position->prior(); - x->next()=position; - x->prior()->next()=x->next()->prior()=x; - } - - static void relink(pointer position,pointer x,pointer y) - { - /* position is assumed not to be in [x,y) */ - - if(x!=y){ - pointer z=y->prior(); - x->prior()->next()=y; - y->prior()=x->prior(); - x->prior()=position->prior(); - z->next()=position; - x->prior()->next()=x; - z->next()->prior()=z; - } - } - - static void reverse(pointer header) - { - pointer x=header; - do{ - pointer y=x->next(); - std::swap(x->prior(),x->next()); - x=y; - }while(x!=header); - } - - static void swap(pointer x,pointer y) - { - /* This swap function does not exchange the header nodes, - * but rather their pointers. This is *not* used for implementing - * sequenced_index::swap. - */ - - if(x->next()!=x){ - if(y->next()!=y){ - std::swap(x->next(),y->next()); - std::swap(x->prior(),y->prior()); - x->next()->prior()=x->prior()->next()=x; - y->next()->prior()=y->prior()->next()=y; - } - else{ - y->next()=x->next(); - y->prior()=x->prior(); - x->next()=x->prior()=x; - y->next()->prior()=y->prior()->next()=y; - } - } - else if(y->next()!=y){ - x->next()=y->next(); - x->prior()=y->prior(); - y->next()=y->prior()=y; - x->next()->prior()=x->prior()->next()=x; - } - } - -private: - pointer prior_; - pointer next_; -}; - -template -struct sequenced_index_node_trampoline: - sequenced_index_node_impl< - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > -{ - typedef sequenced_index_node_impl< - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > impl_type; -}; - -template -struct sequenced_index_node:Super,sequenced_index_node_trampoline -{ -private: - typedef sequenced_index_node_trampoline trampoline; - -public: - typedef typename trampoline::impl_type impl_type; - typedef typename trampoline::pointer impl_pointer; - typedef typename trampoline::const_pointer const_impl_pointer; - - impl_pointer& prior(){return trampoline::prior();} - impl_pointer prior()const{return trampoline::prior();} - impl_pointer& next(){return trampoline::next();} - impl_pointer next()const{return trampoline::next();} - - impl_pointer impl() - { - return static_cast( - static_cast(static_cast(this))); - } - - const_impl_pointer impl()const - { - return static_cast( - static_cast(static_cast(this))); - } - - static sequenced_index_node* from_impl(impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - static const sequenced_index_node* from_impl(const_impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - /* interoperability with bidir_node_iterator */ - - static void increment(sequenced_index_node*& x) - { - impl_pointer xi=x->impl(); - trampoline::increment(xi); - x=from_impl(xi); - } - - static void decrement(sequenced_index_node*& x) - { - impl_pointer xi=x->impl(); - trampoline::decrement(xi); - x=from_impl(xi); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_ops.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_ops.hpp deleted file mode 100644 index 142bdd9dd9a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_ops.hpp +++ /dev/null @@ -1,203 +0,0 @@ -/* Copyright 2003-2016 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_SEQ_INDEX_OPS_HPP -#define BOOST_MULTI_INDEX_DETAIL_SEQ_INDEX_OPS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Common code for sequenced_index memfuns having templatized and - * non-templatized versions. - */ - -template -void sequenced_index_remove(SequencedIndex& x,Predicate pred) -{ - typedef typename SequencedIndex::iterator iterator; - iterator first=x.begin(),last=x.end(); - while(first!=last){ - if(pred(*first))x.erase(first++); - else ++first; - } -} - -template -void sequenced_index_unique(SequencedIndex& x,BinaryPredicate binary_pred) -{ - typedef typename SequencedIndex::iterator iterator; - iterator first=x.begin(); - iterator last=x.end(); - if(first!=last){ - for(iterator middle=first;++middle!=last;middle=first){ - if(binary_pred(*middle,*first))x.erase(middle); - else first=middle; - } - } -} - -template -void sequenced_index_merge(SequencedIndex& x,SequencedIndex& y,Compare comp) -{ - typedef typename SequencedIndex::iterator iterator; - if(&x!=&y){ - iterator first0=x.begin(),last0=x.end(); - iterator first1=y.begin(),last1=y.end(); - while(first0!=last0&&first1!=last1){ - if(comp(*first1,*first0))x.splice(first0,y,first1++); - else ++first0; - } - x.splice(last0,y,first1,last1); - } -} - -/* sorting */ - -/* auxiliary stuff */ - -template -void sequenced_index_collate( - BOOST_DEDUCED_TYPENAME Node::impl_type* x, - BOOST_DEDUCED_TYPENAME Node::impl_type* y, - Compare comp) -{ - typedef typename Node::impl_type impl_type; - typedef typename Node::impl_pointer impl_pointer; - - impl_pointer first0=x->next(); - impl_pointer last0=x; - impl_pointer first1=y->next(); - impl_pointer last1=y; - while(first0!=last0&&first1!=last1){ - if(comp( - Node::from_impl(first1)->value(),Node::from_impl(first0)->value())){ - impl_pointer tmp=first1->next(); - impl_type::relink(first0,first1); - first1=tmp; - } - else first0=first0->next(); - } - impl_type::relink(last0,first1,last1); -} - -/* Some versions of CGG require a bogus typename in counter_spc - * inside sequenced_index_sort if the following is defined - * also inside sequenced_index_sort. - */ - -BOOST_STATIC_CONSTANT( - std::size_t, - sequenced_index_sort_max_fill= - (std::size_t)std::numeric_limits::digits+1); - -#include - -template -void sequenced_index_sort(Node* header,Compare comp) -{ - /* Musser's mergesort, see http://www.cs.rpi.edu/~musser/gp/List/lists1.html. - * The implementation is a little convoluted: in the original code - * counter elements and carry are std::lists: here we do not want - * to use multi_index instead, so we do things at a lower level, managing - * directly the internal node representation. - * Incidentally, the implementations I've seen of this algorithm (SGI, - * Dinkumware, STLPort) are not exception-safe: this is. Moreover, we do not - * use any dynamic storage. - */ - - if(header->next()==header->impl()|| - header->next()->next()==header->impl())return; - - typedef typename Node::impl_type impl_type; - typedef typename Node::impl_pointer impl_pointer; - - typedef typename aligned_storage< - sizeof(impl_type), - alignment_of::value - >::type carry_spc_type; - carry_spc_type carry_spc; - impl_type& carry= - *reinterpret_cast(&carry_spc); - typedef typename aligned_storage< - sizeof( - impl_type - [sequenced_index_sort_max_fill]), - alignment_of< - impl_type - [sequenced_index_sort_max_fill] - >::value - >::type counter_spc_type; - counter_spc_type counter_spc; - impl_type* counter= - reinterpret_cast(&counter_spc); - std::size_t fill=0; - - carry.prior()=carry.next()=static_cast(&carry); - counter[0].prior()=counter[0].next()=static_cast(&counter[0]); - - BOOST_TRY{ - while(header->next()!=header->impl()){ - impl_type::relink(carry.next(),header->next()); - std::size_t i=0; - while(i(&counter[i])){ - sequenced_index_collate(&carry,&counter[i++],comp); - } - impl_type::swap( - static_cast(&carry), - static_cast(&counter[i])); - if(i==fill){ - ++fill; - counter[fill].prior()=counter[fill].next()= - static_cast(&counter[fill]); - } - } - - for(std::size_t i=1;i(&counter[i],&counter[i-1],comp); - } - impl_type::swap( - header->impl(),static_cast(&counter[fill-1])); - } - BOOST_CATCH(...) - { - impl_type::relink( - header->impl(),carry.next(),static_cast(&carry)); - for(std::size_t i=0;i<=fill;++i){ - impl_type::relink( - header->impl(),counter[i].next(), - static_cast(&counter[i])); - } - BOOST_RETHROW; - } - BOOST_CATCH_END -} - -#include - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/serialization_version.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/serialization_version.hpp deleted file mode 100644 index ccd8bb4f791..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/serialization_version.hpp +++ /dev/null @@ -1,73 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_SERIALIZATION_VERSION_HPP -#define BOOST_MULTI_INDEX_DETAIL_SERIALIZATION_VERSION_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Helper class for storing and retrieving a given type serialization class - * version while avoiding saving the number multiple times in the same - * archive. - * Behavior undefined if template partial specialization is not supported. - */ - -template -struct serialization_version -{ - serialization_version(): - value(boost::serialization::version::value){} - - serialization_version& operator=(unsigned int x){value=x;return *this;}; - - operator unsigned int()const{return value;} - -private: - friend class boost::serialization::access; - - BOOST_SERIALIZATION_SPLIT_MEMBER() - - template - void save(Archive&,const unsigned int)const{} - - template - void load(Archive&,const unsigned int version) - { - this->value=version; - } - - unsigned int value; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -namespace serialization { -template -struct version > -{ - BOOST_STATIC_CONSTANT(int,value=version::value); -}; -} /* namespace serialization */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/uintptr_type.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/uintptr_type.hpp deleted file mode 100644 index 9c92d01d4de..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/uintptr_type.hpp +++ /dev/null @@ -1,76 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_UINTPTR_TYPE_HPP -#define BOOST_MULTI_INDEX_DETAIL_UINTPTR_TYPE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* has_uintptr_type is an MPL integral constant determining whether - * there exists an unsigned integral type with the same size as - * void *. - * uintptr_type is such a type if has_uintptr is true, or unsigned int - * otherwise. - * Note that uintptr_type is more restrictive than C99 uintptr_t, - * where an integral type with size greater than that of void * - * would be conformant. - */ - -templatestruct uintptr_candidates; -template<>struct uintptr_candidates<-1>{typedef unsigned int type;}; -template<>struct uintptr_candidates<0> {typedef unsigned int type;}; -template<>struct uintptr_candidates<1> {typedef unsigned short type;}; -template<>struct uintptr_candidates<2> {typedef unsigned long type;}; - -#if defined(BOOST_HAS_LONG_LONG) -template<>struct uintptr_candidates<3> {typedef boost::ulong_long_type type;}; -#else -template<>struct uintptr_candidates<3> {typedef unsigned int type;}; -#endif - -#if defined(BOOST_HAS_MS_INT64) -template<>struct uintptr_candidates<4> {typedef unsigned __int64 type;}; -#else -template<>struct uintptr_candidates<4> {typedef unsigned int type;}; -#endif - -struct uintptr_aux -{ - BOOST_STATIC_CONSTANT(int,index= - sizeof(void*)==sizeof(uintptr_candidates<0>::type)?0: - sizeof(void*)==sizeof(uintptr_candidates<1>::type)?1: - sizeof(void*)==sizeof(uintptr_candidates<2>::type)?2: - sizeof(void*)==sizeof(uintptr_candidates<3>::type)?3: - sizeof(void*)==sizeof(uintptr_candidates<4>::type)?4:-1); - - BOOST_STATIC_CONSTANT(bool,has_uintptr_type=(index>=0)); - - typedef uintptr_candidates::type type; -}; - -typedef mpl::bool_ has_uintptr_type; -typedef uintptr_aux::type uintptr_type; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/unbounded.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/unbounded.hpp deleted file mode 100644 index dc09be1770d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/unbounded.hpp +++ /dev/null @@ -1,66 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_UNBOUNDED_HPP -#define BOOST_MULTI_INDEX_DETAIL_UNBOUNDED_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -namespace boost{ - -namespace multi_index{ - -/* dummy type and variable for use in ordered_index::range() */ - -/* ODR-abiding technique shown at the example attached to - * http://lists.boost.org/Archives/boost/2006/07/108355.php - */ - -namespace detail{class unbounded_helper;} - -detail::unbounded_helper unbounded(detail::unbounded_helper); - -namespace detail{ - -class unbounded_helper -{ - unbounded_helper(){} - unbounded_helper(const unbounded_helper&){} - friend unbounded_helper multi_index::unbounded(unbounded_helper); -}; - -typedef unbounded_helper (*unbounded_type)(unbounded_helper); - -} /* namespace multi_index::detail */ - -inline detail::unbounded_helper unbounded(detail::unbounded_helper) -{ - return detail::unbounded_helper(); -} - -/* tags used in the implementation of range */ - -namespace detail{ - -struct none_unbounded_tag{}; -struct lower_unbounded_tag{}; -struct upper_unbounded_tag{}; -struct both_unbounded_tag{}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/value_compare.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/value_compare.hpp deleted file mode 100644 index ac42e8779aa..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/value_compare.hpp +++ /dev/null @@ -1,56 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_VALUE_COMPARE_HPP -#define BOOST_MULTI_INDEX_DETAIL_VALUE_COMPARE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -struct value_comparison -{ - typedef Value first_argument_type; - typedef Value second_argument_type; - typedef bool result_type; - - value_comparison( - const KeyFromValue& key_=KeyFromValue(),const Compare& comp_=Compare()): - key(key_),comp(comp_) - { - } - - bool operator()( - typename call_traits::param_type x, - typename call_traits::param_type y)const - { - return comp(key(x),key(y)); - } - -private: - KeyFromValue key; - Compare comp; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/vartempl_support.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/vartempl_support.hpp deleted file mode 100644 index 06ff430f4be..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/vartempl_support.hpp +++ /dev/null @@ -1,247 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_VARTEMPL_SUPPORT_HPP -#define BOOST_MULTI_INDEX_DETAIL_VARTEMPL_SUPPORT_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -/* Utilities for emulation of variadic template functions. Variadic packs are - * replaced by lists of BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS parameters: - * - * - typename... Args --> BOOST_MULTI_INDEX_TEMPLATE_PARAM_PACK - * - Args&&... args --> BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK - * - std::forward(args)... --> BOOST_MULTI_INDEX_FORWARD_PARAM_PACK - * - * Forwarding emulated with Boost.Move. A template functions foo_imp - * defined in such way accepts *exactly* BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS - * arguments: variable number of arguments is emulated by providing a set of - * overloads foo forwarding to foo_impl with - * - * BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL - * BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG (initial extra arg) - * - * which fill the extra args with boost::multi_index::detail::noarg's. - * boost::multi_index::detail::vartempl_placement_new works the opposite - * way: it acceps a full a pointer x to Value and a - * BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK and forwards to - * new(x) Value(args) where args is the argument pack after discarding - * noarg's. - * - * Emulation decays to the real thing when the compiler supports variadic - * templates and move semantics natively. - */ - -#include - -#if defined(BOOST_NO_CXX11_RVALUE_REFERENCES)||\ - defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS) -#define BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS 5 -#endif - -#define BOOST_MULTI_INDEX_TEMPLATE_PARAM_PACK \ -BOOST_PP_ENUM_PARAMS( \ - BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,typename T) - -#define BOOST_MULTI_INDEX_VARTEMPL_ARG(z,n,_) \ -BOOST_FWD_REF(BOOST_PP_CAT(T,n)) BOOST_PP_CAT(t,n) - -#define BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK \ -BOOST_PP_ENUM( \ - BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS, \ - BOOST_MULTI_INDEX_VARTEMPL_ARG,~) - -#define BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG(z,n,_) \ -boost::forward(BOOST_PP_CAT(t,n)) - -#define BOOST_MULTI_INDEX_FORWARD_PARAM_PACK \ -BOOST_PP_ENUM( \ - BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS, \ - BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG,~) - -namespace boost{namespace multi_index{namespace detail{ -struct noarg{}; -}}} - -/* call vartempl function without args */ - -#define BOOST_MULTI_INDEX_NULL_PARAM_PACK \ -BOOST_PP_ENUM_PARAMS( \ - BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS, \ - boost::multi_index::detail::noarg() BOOST_PP_INTERCEPT) - -#define BOOST_MULTI_INDEX_TEMPLATE_N(n) \ -template - -#define BOOST_MULTI_INDEX_TEMPLATE_0(n) - -#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_AUX(z,n,data) \ -BOOST_PP_IF(n, \ - BOOST_MULTI_INDEX_TEMPLATE_N, \ - BOOST_MULTI_INDEX_TEMPLATE_0)(n) \ -BOOST_PP_SEQ_ELEM(0,data) /* ret */ \ -BOOST_PP_SEQ_ELEM(1,data) /* name_from */ ( \ - BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_ARG,~)) \ -{ \ - return BOOST_PP_SEQ_ELEM(2,data) /* name_to */ ( \ - BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG,~) \ - BOOST_PP_COMMA_IF( \ - BOOST_PP_AND( \ - n,BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n))) \ - BOOST_PP_ENUM_PARAMS( \ - BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n), \ - boost::multi_index::detail::noarg() BOOST_PP_INTERCEPT) \ - ); \ -} - -#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( \ - ret,name_from,name_to) \ -BOOST_PP_REPEAT_FROM_TO( \ - 0,BOOST_PP_ADD(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,1), \ - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_AUX, \ - (ret)(name_from)(name_to)) - -#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG_AUX( \ - z,n,data) \ -BOOST_PP_IF(n, \ - BOOST_MULTI_INDEX_TEMPLATE_N, \ - BOOST_MULTI_INDEX_TEMPLATE_0)(n) \ -BOOST_PP_SEQ_ELEM(0,data) /* ret */ \ -BOOST_PP_SEQ_ELEM(1,data) /* name_from */ ( \ - BOOST_PP_SEQ_ELEM(3,data) BOOST_PP_SEQ_ELEM(4,data) /* extra arg */\ - BOOST_PP_COMMA_IF(n) \ - BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_ARG,~)) \ -{ \ - return BOOST_PP_SEQ_ELEM(2,data) /* name_to */ ( \ - BOOST_PP_SEQ_ELEM(4,data) /* extra_arg_name */ \ - BOOST_PP_COMMA_IF(n) \ - BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG,~) \ - BOOST_PP_COMMA_IF( \ - BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n)) \ - BOOST_PP_ENUM_PARAMS( \ - BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n), \ - boost::multi_index::detail::noarg() BOOST_PP_INTERCEPT) \ - ); \ -} - -#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( \ - ret,name_from,name_to,extra_arg_type,extra_arg_name) \ -BOOST_PP_REPEAT_FROM_TO( \ - 0,BOOST_PP_ADD(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,1), \ - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG_AUX, \ - (ret)(name_from)(name_to)(extra_arg_type)(extra_arg_name)) - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -#define BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW_AUX(z,n,name) \ -template< \ - typename Value \ - BOOST_PP_COMMA_IF(n) \ - BOOST_PP_ENUM_PARAMS(n,typename T) \ -> \ -Value* name( \ - Value* x \ - BOOST_PP_COMMA_IF(n) \ - BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_ARG,~) \ - BOOST_PP_COMMA_IF( \ - BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n)) \ - BOOST_PP_ENUM_PARAMS( \ - BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n), \ - BOOST_FWD_REF(noarg) BOOST_PP_INTERCEPT)) \ -{ \ - return new(x) Value( \ - BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG,~)); \ -} - -#define BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW(name) \ -BOOST_PP_REPEAT_FROM_TO( \ - 0,BOOST_PP_ADD(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,1), \ - BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW_AUX, \ - name) - -BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW(vartempl_placement_new) - -#undef BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW_AUX -#undef BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#else - -/* native variadic templates support */ - -#include - -#define BOOST_MULTI_INDEX_TEMPLATE_PARAM_PACK typename... Args -#define BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK Args&&... args -#define BOOST_MULTI_INDEX_FORWARD_PARAM_PACK std::forward(args)... -#define BOOST_MULTI_INDEX_NULL_PARAM_PACK - -#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( \ - ret,name_from,name_to) \ -template ret name_from(Args&&... args) \ -{ \ - return name_to(std::forward(args)...); \ -} - -#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( \ - ret,name_from,name_to,extra_arg_type,extra_arg_name) \ -template ret name_from( \ - extra_arg_type extra_arg_name,Args&&... args) \ -{ \ - return name_to(extra_arg_name,std::forward(args)...); \ -} - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -Value* vartempl_placement_new(Value*x,Args&&... args) -{ - return new(x) Value(std::forward(args)...); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/global_fun.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/global_fun.hpp deleted file mode 100644 index 2c13769100c..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/global_fun.hpp +++ /dev/null @@ -1,185 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_GLOBAL_FUN_HPP -#define BOOST_MULTI_INDEX_GLOBAL_FUN_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_SFINAE) -#include -#endif - -namespace boost{ - -template class reference_wrapper; /* fwd decl. */ - -namespace multi_index{ - -namespace detail{ - -/* global_fun is a read-only key extractor from Value based on a given global - * (or static member) function with signature: - * - * Type f([const] Value [&]); - * - * Additionally, global_fun and const_global_fun are overloaded to support - * referece_wrappers of Value and "chained pointers" to Value's. By chained - * pointer to T we mean a type P such that, given a p of Type P - * *...n...*x is convertible to T&, for some n>=1. - * Examples of chained pointers are raw and smart pointers, iterators and - * arbitrary combinations of these (vg. T** or unique_ptr.) - */ - -template -struct const_ref_global_fun_base -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(Value x)const - { - return PtrToFunction(x); - } - - Type operator()( - const reference_wrapper< - typename remove_reference::type>& x)const - { - return operator()(x.get()); - } - - Type operator()( - const reference_wrapper< - typename remove_const< - typename remove_reference::type>::type>& x - -#if BOOST_WORKAROUND(BOOST_MSVC,==1310) -/* http://lists.boost.org/Archives/boost/2015/10/226135.php */ - ,int=0 -#endif - - )const - { - return operator()(x.get()); - } -}; - -template -struct non_const_ref_global_fun_base -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(Value x)const - { - return PtrToFunction(x); - } - - Type operator()( - const reference_wrapper< - typename remove_reference::type>& x)const - { - return operator()(x.get()); - } -}; - -template -struct non_ref_global_fun_base -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(const Value& x)const - { - return PtrToFunction(x); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type operator()( - const reference_wrapper::type>& x)const - { - return operator()(x.get()); - } -}; - -} /* namespace multi_index::detail */ - -template -struct global_fun: - mpl::if_c< - is_reference::value, - typename mpl::if_c< - is_const::type>::value, - detail::const_ref_global_fun_base, - detail::non_const_ref_global_fun_base - >::type, - detail::non_ref_global_fun_base - >::type -{ -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index.hpp deleted file mode 100644 index 352d0c13f17..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index.hpp +++ /dev/null @@ -1,1725 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_HASHED_INDEX_HPP -#define BOOST_MULTI_INDEX_HASHED_INDEX_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) -#include -#endif - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) -#define BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF(x) \ - detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ - detail::make_obj_guard(x,&hashed_index::check_invariant_); \ - BOOST_JOIN(check_invariant_,__LINE__).touch(); -#define BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT \ - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF(*this) -#else -#define BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF(x) -#define BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* hashed_index adds a layer of hashed indexing to a given Super */ - -/* Most of the implementation of unique and non-unique indices is - * shared. We tell from one another on instantiation time by using - * Category tags defined in hash_index_node.hpp. - */ - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -class hashed_index: - BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS SuperMeta::type - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - ,public safe_mode::safe_container< - hashed_index > -#endif - -{ -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the - * lifetime of const references bound to temporaries --precisely what - * scopeguards are. - */ - -#pragma parse_mfunc_templ off -#endif - - typedef typename SuperMeta::type super; - -protected: - typedef hashed_index_node< - typename super::node_type,Category> node_type; - -private: - typedef typename node_type::node_alg node_alg; - typedef typename node_type::impl_type node_impl_type; - typedef typename node_impl_type::pointer node_impl_pointer; - typedef typename node_impl_type::base_pointer node_impl_base_pointer; - typedef bucket_array< - typename super::final_allocator_type> bucket_array_type; - -public: - /* types */ - - typedef typename KeyFromValue::result_type key_type; - typedef typename node_type::value_type value_type; - typedef KeyFromValue key_from_value; - typedef Hash hasher; - typedef Pred key_equal; - typedef tuple ctor_args; - typedef typename super::final_allocator_type allocator_type; - typedef typename allocator_type::pointer pointer; - typedef typename allocator_type::const_pointer const_pointer; - typedef typename allocator_type::reference reference; - typedef typename allocator_type::const_reference const_reference; - typedef std::size_t size_type; - typedef std::ptrdiff_t difference_type; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_iterator< - hashed_index_iterator< - node_type,bucket_array_type, - hashed_index_global_iterator_tag>, - hashed_index> iterator; -#else - typedef hashed_index_iterator< - node_type,bucket_array_type, - hashed_index_global_iterator_tag> iterator; -#endif - - typedef iterator const_iterator; - - typedef hashed_index_iterator< - node_type,bucket_array_type, - hashed_index_local_iterator_tag> local_iterator; - typedef local_iterator const_local_iterator; - - typedef TagList tag_list; - -protected: - typedef typename super::final_node_type final_node_type; - typedef tuples::cons< - ctor_args, - typename super::ctor_args_list> ctor_args_list; - typedef typename mpl::push_front< - typename super::index_type_list, - hashed_index>::type index_type_list; - typedef typename mpl::push_front< - typename super::iterator_type_list, - iterator>::type iterator_type_list; - typedef typename mpl::push_front< - typename super::const_iterator_type_list, - const_iterator>::type const_iterator_type_list; - typedef typename super::copy_map_type copy_map_type; - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - typedef typename super::index_saver_type index_saver_type; - typedef typename super::index_loader_type index_loader_type; -#endif - -private: -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_container< - hashed_index> safe_super; -#endif - - typedef typename call_traits::param_type value_param_type; - typedef typename call_traits< - key_type>::param_type key_param_type; - - /* Needed to avoid commas in BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL - * expansion. - */ - - typedef std::pair emplace_return_type; - -public: - - /* construct/destroy/copy - * Default and copy ctors are in the protected section as indices are - * not supposed to be created on their own. No range ctor either. - */ - - hashed_index& operator=( - const hashed_index& x) - { - this->final()=x.final(); - return *this; - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - hashed_index& operator=( - std::initializer_list list) - { - this->final()=list; - return *this; - } -#endif - - allocator_type get_allocator()const BOOST_NOEXCEPT - { - return this->final().get_allocator(); - } - - /* size and capacity */ - - bool empty()const BOOST_NOEXCEPT{return this->final_empty_();} - size_type size()const BOOST_NOEXCEPT{return this->final_size_();} - size_type max_size()const BOOST_NOEXCEPT{return this->final_max_size_();} - - /* iterators */ - - iterator begin()BOOST_NOEXCEPT - {return make_iterator(node_type::from_impl(header()->next()->prior()));} - const_iterator begin()const BOOST_NOEXCEPT - {return make_iterator(node_type::from_impl(header()->next()->prior()));} - iterator end()BOOST_NOEXCEPT{return make_iterator(header());} - const_iterator end()const BOOST_NOEXCEPT{return make_iterator(header());} - const_iterator cbegin()const BOOST_NOEXCEPT{return begin();} - const_iterator cend()const BOOST_NOEXCEPT{return end();} - - iterator iterator_to(const value_type& x) - { - return make_iterator(node_from_value(&x)); - } - - const_iterator iterator_to(const value_type& x)const - { - return make_iterator(node_from_value(&x)); - } - - /* modifiers */ - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( - emplace_return_type,emplace,emplace_impl) - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( - iterator,emplace_hint,emplace_hint_impl,iterator,position) - - std::pair insert(const value_type& x) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_(x); - return std::pair(make_iterator(p.first),p.second); - } - - std::pair insert(BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_rv_(x); - return std::pair(make_iterator(p.first),p.second); - } - - iterator insert(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_( - x,static_cast(position.get_node())); - return make_iterator(p.first); - } - - iterator insert(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_rv_( - x,static_cast(position.get_node())); - return make_iterator(p.first); - } - - template - void insert(InputIterator first,InputIterator last) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - for(;first!=last;++first)this->final_insert_ref_(*first); - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - void insert(std::initializer_list list) - { - insert(list.begin(),list.end()); - } -#endif - - iterator erase(iterator position) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - this->final_erase_(static_cast(position++.get_node())); - return position; - } - - size_type erase(key_param_type k) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - - std::size_t buc=buckets.position(hash_(k)); - for(node_impl_pointer x=buckets.at(buc)->prior(); - x!=node_impl_pointer(0);x=node_alg::next_to_inspect(x)){ - if(eq_(k,key(node_type::from_impl(x)->value()))){ - node_impl_pointer y=end_of_range(x); - size_type s=0; - do{ - node_impl_pointer z=node_alg::after(x); - this->final_erase_( - static_cast(node_type::from_impl(x))); - x=z; - ++s; - }while(x!=y); - return s; - } - } - return 0; - } - - iterator erase(iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - while(first!=last){ - first=erase(first); - } - return first; - } - - bool replace(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - return this->final_replace_( - x,static_cast(position.get_node())); - } - - bool replace(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - return this->final_replace_rv_( - x,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod,Rollback back_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,back_,static_cast(position.get_node())); - } - - template - bool modify_key(iterator position,Modifier mod) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - return modify( - position,modify_key_adaptor(mod,key)); - } - - template - bool modify_key(iterator position,Modifier mod,Rollback back_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - return modify( - position, - modify_key_adaptor(mod,key), - modify_key_adaptor(back_,key)); - } - - void clear()BOOST_NOEXCEPT - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - this->final_clear_(); - } - - void swap(hashed_index& x) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF(x); - this->final_swap_(x.final()); - } - - /* observers */ - - key_from_value key_extractor()const{return key;} - hasher hash_function()const{return hash_;} - key_equal key_eq()const{return eq_;} - - /* lookup */ - - /* Internally, these ops rely on const_iterator being the same - * type as iterator. - */ - - /* Implementation note: When CompatibleKey is consistently promoted to - * KeyFromValue::result_type for equality comparison, the promotion is made - * once in advance to increase efficiency. - */ - - template - iterator find(const CompatibleKey& k)const - { - return find(k,hash_,eq_); - } - - template< - typename CompatibleKey,typename CompatibleHash,typename CompatiblePred - > - iterator find( - const CompatibleKey& k, - const CompatibleHash& hash,const CompatiblePred& eq)const - { - return find( - k,hash,eq,promotes_1st_arg()); - } - - template - size_type count(const CompatibleKey& k)const - { - return count(k,hash_,eq_); - } - - template< - typename CompatibleKey,typename CompatibleHash,typename CompatiblePred - > - size_type count( - const CompatibleKey& k, - const CompatibleHash& hash,const CompatiblePred& eq)const - { - return count( - k,hash,eq,promotes_1st_arg()); - } - - template - std::pair equal_range(const CompatibleKey& k)const - { - return equal_range(k,hash_,eq_); - } - - template< - typename CompatibleKey,typename CompatibleHash,typename CompatiblePred - > - std::pair equal_range( - const CompatibleKey& k, - const CompatibleHash& hash,const CompatiblePred& eq)const - { - return equal_range( - k,hash,eq,promotes_1st_arg()); - } - - /* bucket interface */ - - size_type bucket_count()const BOOST_NOEXCEPT{return buckets.size();} - size_type max_bucket_count()const BOOST_NOEXCEPT{return static_cast(-1);} - - size_type bucket_size(size_type n)const - { - size_type res=0; - for(node_impl_pointer x=buckets.at(n)->prior(); - x!=node_impl_pointer(0);x=node_alg::after_local(x)){ - ++res; - } - return res; - } - - size_type bucket(key_param_type k)const - { - return buckets.position(hash_(k)); - } - - local_iterator begin(size_type n) - { - return const_cast(this)->begin(n); - } - - const_local_iterator begin(size_type n)const - { - node_impl_pointer x=buckets.at(n)->prior(); - if(x==node_impl_pointer(0))return end(n); - return make_local_iterator(node_type::from_impl(x)); - } - - local_iterator end(size_type n) - { - return const_cast(this)->end(n); - } - - const_local_iterator end(size_type)const - { - return make_local_iterator(0); - } - - const_local_iterator cbegin(size_type n)const{return begin(n);} - const_local_iterator cend(size_type n)const{return end(n);} - - local_iterator local_iterator_to(const value_type& x) - { - return make_local_iterator(node_from_value(&x)); - } - - const_local_iterator local_iterator_to(const value_type& x)const - { - return make_local_iterator(node_from_value(&x)); - } - - /* hash policy */ - - float load_factor()const BOOST_NOEXCEPT - {return static_cast(size())/bucket_count();} - float max_load_factor()const BOOST_NOEXCEPT{return mlf;} - void max_load_factor(float z){mlf=z;calculate_max_load();} - - void rehash(size_type n) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - if(size()<=max_load&&n<=bucket_count())return; - - size_type bc =(std::numeric_limits::max)(); - float fbc=static_cast(1+size()/mlf); - if(bc>fbc){ - bc=static_cast(fbc); - if(bc(std::ceil(static_cast(n)/mlf))); - } - -BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: - hashed_index(const ctor_args_list& args_list,const allocator_type& al): - super(args_list.get_tail(),al), - key(tuples::get<1>(args_list.get_head())), - hash_(tuples::get<2>(args_list.get_head())), - eq_(tuples::get<3>(args_list.get_head())), - buckets(al,header()->impl(),tuples::get<0>(args_list.get_head())), - mlf(1.0f) - { - calculate_max_load(); - } - - hashed_index( - const hashed_index& x): - super(x), - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super(), -#endif - - key(x.key), - hash_(x.hash_), - eq_(x.eq_), - buckets(x.get_allocator(),header()->impl(),x.buckets.size()), - mlf(x.mlf), - max_load(x.max_load) - { - /* Copy ctor just takes the internal configuration objects from x. The rest - * is done in subsequent call to copy_(). - */ - } - - hashed_index( - const hashed_index& x, - do_not_copy_elements_tag): - super(x,do_not_copy_elements_tag()), - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super(), -#endif - - key(x.key), - hash_(x.hash_), - eq_(x.eq_), - buckets(x.get_allocator(),header()->impl(),0), - mlf(1.0f) - { - calculate_max_load(); - } - - ~hashed_index() - { - /* the container is guaranteed to be empty by now */ - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - iterator make_iterator(node_type* node) - { - return iterator(node,this); - } - - const_iterator make_iterator(node_type* node)const - { - return const_iterator(node,const_cast(this)); - } -#else - iterator make_iterator(node_type* node) - { - return iterator(node); - } - - const_iterator make_iterator(node_type* node)const - { - return const_iterator(node); - } -#endif - - local_iterator make_local_iterator(node_type* node) - { - return local_iterator(node); - } - - const_local_iterator make_local_iterator(node_type* node)const - { - return const_local_iterator(node); - } - - void copy_( - const hashed_index& x, - const copy_map_type& map) - { - copy_(x,map,Category()); - } - - void copy_( - const hashed_index& x, - const copy_map_type& map,hashed_unique_tag) - { - if(x.size()!=0){ - node_impl_pointer end_org=x.header()->impl(), - org=end_org, - cpy=header()->impl(); - do{ - node_impl_pointer prev_org=org->prior(), - prev_cpy= - static_cast(map.find(static_cast( - node_type::from_impl(prev_org))))->impl(); - cpy->prior()=prev_cpy; - if(node_alg::is_first_of_bucket(org)){ - node_impl_base_pointer buc_org=prev_org->next(), - buc_cpy= - buckets.begin()+(buc_org-x.buckets.begin()); - prev_cpy->next()=buc_cpy; - buc_cpy->prior()=cpy; - } - else{ - prev_cpy->next()=node_impl_type::base_pointer_from(cpy); - } - org=prev_org; - cpy=prev_cpy; - }while(org!=end_org); - } - - super::copy_(x,map); - } - - void copy_( - const hashed_index& x, - const copy_map_type& map,hashed_non_unique_tag) - { - if(x.size()!=0){ - node_impl_pointer end_org=x.header()->impl(), - org=end_org, - cpy=header()->impl(); - do{ - node_impl_pointer next_org=node_alg::after(org), - next_cpy= - static_cast(map.find(static_cast( - node_type::from_impl(next_org))))->impl(); - if(node_alg::is_first_of_bucket(next_org)){ - node_impl_base_pointer buc_org=org->next(), - buc_cpy= - buckets.begin()+(buc_org-x.buckets.begin()); - cpy->next()=buc_cpy; - buc_cpy->prior()=next_cpy; - next_cpy->prior()=cpy; - } - else{ - if(org->next()==node_impl_type::base_pointer_from(next_org)){ - cpy->next()=node_impl_type::base_pointer_from(next_cpy); - } - else{ - cpy->next()= - node_impl_type::base_pointer_from( - static_cast(map.find(static_cast( - node_type::from_impl( - node_impl_type::pointer_from(org->next())))))->impl()); - } - - if(next_org->prior()!=org){ - next_cpy->prior()= - static_cast(map.find(static_cast( - node_type::from_impl(next_org->prior()))))->impl(); - } - else{ - next_cpy->prior()=cpy; - } - } - org=next_org; - cpy=next_cpy; - }while(org!=end_org); - } - - super::copy_(x,map); - } - - template - final_node_type* insert_( - value_param_type v,final_node_type*& x,Variant variant) - { - reserve_for_insert(size()+1); - - std::size_t buc=find_bucket(v); - link_info pos(buckets.at(buc)); - if(!link_point(v,pos)){ - return static_cast( - node_type::from_impl(node_impl_type::pointer_from(pos))); - } - - final_node_type* res=super::insert_(v,x,variant); - if(res==x)link(static_cast(x),pos); - return res; - } - - template - final_node_type* insert_( - value_param_type v,node_type* position,final_node_type*& x,Variant variant) - { - reserve_for_insert(size()+1); - - std::size_t buc=find_bucket(v); - link_info pos(buckets.at(buc)); - if(!link_point(v,pos)){ - return static_cast( - node_type::from_impl(node_impl_type::pointer_from(pos))); - } - - final_node_type* res=super::insert_(v,position,x,variant); - if(res==x)link(static_cast(x),pos); - return res; - } - - void erase_(node_type* x) - { - unlink(x); - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - } - - void delete_all_nodes_() - { - delete_all_nodes_(Category()); - } - - void delete_all_nodes_(hashed_unique_tag) - { - for(node_impl_pointer x_end=header()->impl(),x=x_end->prior();x!=x_end;){ - node_impl_pointer y=x->prior(); - this->final_delete_node_( - static_cast(node_type::from_impl(x))); - x=y; - } - } - - void delete_all_nodes_(hashed_non_unique_tag) - { - for(node_impl_pointer x_end=header()->impl(),x=x_end->prior();x!=x_end;){ - node_impl_pointer y=x->prior(); - if(y->next()!=node_impl_type::base_pointer_from(x)&& - y->next()->prior()!=x){ /* n-1 of group */ - /* Make the second node prior() pointer back-linked so that it won't - * refer to a deleted node when the time for its own destruction comes. - */ - - node_impl_pointer first=node_impl_type::pointer_from(y->next()); - first->next()->prior()=first; - } - this->final_delete_node_( - static_cast(node_type::from_impl(x))); - x=y; - } - } - - void clear_() - { - super::clear_(); - buckets.clear(header()->impl()); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::detach_dereferenceable_iterators(); -#endif - } - - void swap_( - hashed_index& x) - { - std::swap(key,x.key); - std::swap(hash_,x.hash_); - std::swap(eq_,x.eq_); - buckets.swap(x.buckets); - std::swap(mlf,x.mlf); - std::swap(max_load,x.max_load); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_(x); - } - - void swap_elements_( - hashed_index& x) - { - buckets.swap(x.buckets); - std::swap(mlf,x.mlf); - std::swap(max_load,x.max_load); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_elements_(x); - } - - template - bool replace_(value_param_type v,node_type* x,Variant variant) - { - if(eq_(key(v),key(x->value()))){ - return super::replace_(v,x,variant); - } - - unlink_undo undo; - unlink(x,undo); - - BOOST_TRY{ - std::size_t buc=find_bucket(v); - link_info pos(buckets.at(buc)); - if(link_point(v,pos)&&super::replace_(v,x,variant)){ - link(x,pos); - return true; - } - undo(); - return false; - } - BOOST_CATCH(...){ - undo(); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - bool modify_(node_type* x) - { - std::size_t buc; - bool b; - BOOST_TRY{ - buc=find_bucket(x->value()); - b=in_place(x->impl(),key(x->value()),buc); - } - BOOST_CATCH(...){ - erase_(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - if(!b){ - unlink(x); - BOOST_TRY{ - link_info pos(buckets.at(buc)); - if(!link_point(x->value(),pos)){ - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - return false; - } - link(x,pos); - } - BOOST_CATCH(...){ - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - BOOST_TRY{ - if(!super::modify_(x)){ - unlink(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - return false; - } - else return true; - } - BOOST_CATCH(...){ - unlink(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - bool modify_rollback_(node_type* x) - { - std::size_t buc=find_bucket(x->value()); - if(in_place(x->impl(),key(x->value()),buc)){ - return super::modify_rollback_(x); - } - - unlink_undo undo; - unlink(x,undo); - - BOOST_TRY{ - link_info pos(buckets.at(buc)); - if(link_point(x->value(),pos)&&super::modify_rollback_(x)){ - link(x,pos); - return true; - } - undo(); - return false; - } - BOOST_CATCH(...){ - undo(); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - /* comparison */ - -#if !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) - /* defect macro refers to class, not function, templates, but anyway */ - - template - friend bool operator==( - const hashed_index&,const hashed_index& y); -#endif - - bool equals(const hashed_index& x)const{return equals(x,Category());} - - bool equals(const hashed_index& x,hashed_unique_tag)const - { - if(size()!=x.size())return false; - for(const_iterator it=begin(),it_end=end(),it2_end=x.end(); - it!=it_end;++it){ - const_iterator it2=x.find(key(*it)); - if(it2==it2_end||!(*it==*it2))return false; - } - return true; - } - - bool equals(const hashed_index& x,hashed_non_unique_tag)const - { - if(size()!=x.size())return false; - for(const_iterator it=begin(),it_end=end();it!=it_end;){ - const_iterator it2,it2_last; - boost::tie(it2,it2_last)=x.equal_range(key(*it)); - if(it2==it2_last)return false; - - const_iterator it_last=make_iterator( - node_type::from_impl(end_of_range(it.get_node()->impl()))); - if(std::distance(it,it_last)!=std::distance(it2,it2_last))return false; - - /* From is_permutation code in - * http://www.open-std.org/jtc1/sc22/wg21/docs/papers/2010/n3068.pdf - */ - - for(;it!=it_last;++it,++it2){ - if(!(*it==*it2))break; - } - if(it!=it_last){ - for(const_iterator scan=it;scan!=it_last;++scan){ - if(std::find(it,scan,*scan)!=scan)continue; - std::ptrdiff_t matches=std::count(it2,it2_last,*scan); - if(matches==0||matches!=std::count(scan,it_last,*scan))return false; - } - it=it_last; - } - } - return true; - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* serialization */ - - template - void save_( - Archive& ar,const unsigned int version,const index_saver_type& sm)const - { - ar< - void load_(Archive& ar,const unsigned int version,const index_loader_type& lm) - { - ar>>serialization::make_nvp("position",buckets); - super::load_(ar,version,lm); - } -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - bool invariant_()const - { - if(size()==0||begin()==end()){ - if(size()!=0||begin()!=end())return false; - } - else{ - size_type s0=0; - for(const_iterator it=begin(),it_end=end();it!=it_end;++it,++s0){} - if(s0!=size())return false; - - size_type s1=0; - for(size_type buc=0;bucfinal_check_invariant_();} -#endif - -private: - node_type* header()const{return this->final_header();} - - std::size_t find_bucket(value_param_type v)const - { - return bucket(key(v)); - } - - struct link_info_non_unique - { - link_info_non_unique(node_impl_base_pointer pos): - first(pos),last(node_impl_base_pointer(0)){} - - operator const node_impl_base_pointer&()const{return this->first;} - - node_impl_base_pointer first,last; - }; - - typedef typename mpl::if_< - is_same, - node_impl_base_pointer, - link_info_non_unique - >::type link_info; - - bool link_point(value_param_type v,link_info& pos) - { - return link_point(v,pos,Category()); - } - - bool link_point( - value_param_type v,node_impl_base_pointer& pos,hashed_unique_tag) - { - for(node_impl_pointer x=pos->prior();x!=node_impl_pointer(0); - x=node_alg::after_local(x)){ - if(eq_(key(v),key(node_type::from_impl(x)->value()))){ - pos=node_impl_type::base_pointer_from(x); - return false; - } - } - return true; - } - - bool link_point( - value_param_type v,link_info_non_unique& pos,hashed_non_unique_tag) - { - for(node_impl_pointer x=pos.first->prior();x!=node_impl_pointer(0); - x=node_alg::next_to_inspect(x)){ - if(eq_(key(v),key(node_type::from_impl(x)->value()))){ - pos.first=node_impl_type::base_pointer_from(x); - pos.last=node_impl_type::base_pointer_from(last_of_range(x)); - return true; - } - } - return true; - } - - node_impl_pointer last_of_range(node_impl_pointer x)const - { - return last_of_range(x,Category()); - } - - node_impl_pointer last_of_range(node_impl_pointer x,hashed_unique_tag)const - { - return x; - } - - node_impl_pointer last_of_range( - node_impl_pointer x,hashed_non_unique_tag)const - { - node_impl_base_pointer y=x->next(); - node_impl_pointer z=y->prior(); - if(z==x){ /* range of size 1 or 2 */ - node_impl_pointer yy=node_impl_type::pointer_from(y); - return - eq_( - key(node_type::from_impl(x)->value()), - key(node_type::from_impl(yy)->value()))?yy:x; - } - else if(z->prior()==x) /* last of bucket */ - return x; - else /* group of size>2 */ - return z; - } - - node_impl_pointer end_of_range(node_impl_pointer x)const - { - return end_of_range(x,Category()); - } - - node_impl_pointer end_of_range(node_impl_pointer x,hashed_unique_tag)const - { - return node_alg::after(last_of_range(x)); - } - - node_impl_pointer end_of_range( - node_impl_pointer x,hashed_non_unique_tag)const - { - node_impl_base_pointer y=x->next(); - node_impl_pointer z=y->prior(); - if(z==x){ /* range of size 1 or 2 */ - node_impl_pointer yy=node_impl_type::pointer_from(y); - if(!eq_( - key(node_type::from_impl(x)->value()), - key(node_type::from_impl(yy)->value())))yy=x; - return yy->next()->prior()==yy? - node_impl_type::pointer_from(yy->next()): - yy->next()->prior(); - } - else if(z->prior()==x) /* last of bucket */ - return z; - else /* group of size>2 */ - return z->next()->prior()==z? - node_impl_type::pointer_from(z->next()): - z->next()->prior(); - } - - void link(node_type* x,const link_info& pos) - { - link(x,pos,Category()); - } - - void link(node_type* x,node_impl_base_pointer pos,hashed_unique_tag) - { - node_alg::link(x->impl(),pos,header()->impl()); - } - - void link(node_type* x,const link_info_non_unique& pos,hashed_non_unique_tag) - { - if(pos.last==node_impl_base_pointer(0)){ - node_alg::link(x->impl(),pos.first,header()->impl()); - } - else{ - node_alg::link( - x->impl(), - node_impl_type::pointer_from(pos.first), - node_impl_type::pointer_from(pos.last)); - } - } - - void unlink(node_type* x) - { - node_alg::unlink(x->impl()); - } - - typedef typename node_alg::unlink_undo unlink_undo; - - void unlink(node_type* x,unlink_undo& undo) - { - node_alg::unlink(x->impl(),undo); - } - - void calculate_max_load() - { - float fml=static_cast(mlf*static_cast(bucket_count())); - max_load=(std::numeric_limits::max)(); - if(max_load>fml)max_load=static_cast(fml); - } - - void reserve_for_insert(size_type n) - { - if(n>max_load){ - size_type bc =(std::numeric_limits::max)(); - float fbc=static_cast(1+static_cast(n)/mlf); - if(bc>fbc)bc =static_cast(fbc); - unchecked_rehash(bc); - } - } - - void unchecked_rehash(size_type n){unchecked_rehash(n,Category());} - - void unchecked_rehash(size_type n,hashed_unique_tag) - { - node_impl_type cpy_end_node; - node_impl_pointer cpy_end=node_impl_pointer(&cpy_end_node), - end_=header()->impl(); - bucket_array_type buckets_cpy(get_allocator(),cpy_end,n); - - if(size()!=0){ - auto_space< - std::size_t,allocator_type> hashes(get_allocator(),size()); - auto_space< - node_impl_pointer,allocator_type> node_ptrs(get_allocator(),size()); - std::size_t i=0,size_=size(); - bool within_bucket=false; - BOOST_TRY{ - for(;i!=size_;++i){ - node_impl_pointer x=end_->prior(); - - /* only this can possibly throw */ - std::size_t h=hash_(key(node_type::from_impl(x)->value())); - - hashes.data()[i]=h; - node_ptrs.data()[i]=x; - within_bucket=!node_alg::unlink_last(end_); - node_alg::link(x,buckets_cpy.at(buckets_cpy.position(h)),cpy_end); - } - } - BOOST_CATCH(...){ - if(i!=0){ - std::size_t prev_buc=buckets.position(hashes.data()[i-1]); - if(!within_bucket)prev_buc=~prev_buc; - - for(std::size_t j=i;j--;){ - std::size_t buc=buckets.position(hashes.data()[j]); - node_impl_pointer x=node_ptrs.data()[j]; - if(buc==prev_buc)node_alg::append(x,end_); - else node_alg::link(x,buckets.at(buc),end_); - prev_buc=buc; - } - } - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - end_->prior()=cpy_end->prior()!=cpy_end?cpy_end->prior():end_; - end_->next()=cpy_end->next(); - end_->prior()->next()->prior()=end_->next()->prior()->prior()=end_; - buckets.swap(buckets_cpy); - calculate_max_load(); - } - - void unchecked_rehash(size_type n,hashed_non_unique_tag) - { - node_impl_type cpy_end_node; - node_impl_pointer cpy_end=node_impl_pointer(&cpy_end_node), - end_=header()->impl(); - bucket_array_type buckets_cpy(get_allocator(),cpy_end,n); - - if(size()!=0){ - auto_space< - std::size_t,allocator_type> hashes(get_allocator(),size()); - auto_space< - node_impl_pointer,allocator_type> node_ptrs(get_allocator(),size()); - std::size_t i=0; - bool within_bucket=false; - BOOST_TRY{ - for(;;++i){ - node_impl_pointer x=end_->prior(); - if(x==end_)break; - - /* only this can possibly throw */ - std::size_t h=hash_(key(node_type::from_impl(x)->value())); - - hashes.data()[i]=h; - node_ptrs.data()[i]=x; - std::pair p= - node_alg::unlink_last_group(end_); - node_alg::link_range( - p.first,x,buckets_cpy.at(buckets_cpy.position(h)),cpy_end); - within_bucket=!(p.second); - } - } - BOOST_CATCH(...){ - if(i!=0){ - std::size_t prev_buc=buckets.position(hashes.data()[i-1]); - if(!within_bucket)prev_buc=~prev_buc; - - for(std::size_t j=i;j--;){ - std::size_t buc=buckets.position(hashes.data()[j]); - node_impl_pointer x=node_ptrs.data()[j], - y= - x->prior()->next()!=node_impl_type::base_pointer_from(x)&& - x->prior()->next()->prior()!=x? - node_impl_type::pointer_from(x->prior()->next()):x; - node_alg::unlink_range(y,x); - if(buc==prev_buc)node_alg::append_range(y,x,end_); - else node_alg::link_range(y,x,buckets.at(buc),end_); - prev_buc=buc; - } - } - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - end_->prior()=cpy_end->prior()!=cpy_end?cpy_end->prior():end_; - end_->next()=cpy_end->next(); - end_->prior()->next()->prior()=end_->next()->prior()->prior()=end_; - buckets.swap(buckets_cpy); - calculate_max_load(); - } - - bool in_place(node_impl_pointer x,key_param_type k,std::size_t buc)const - { - return in_place(x,k,buc,Category()); - } - - bool in_place( - node_impl_pointer x,key_param_type k,std::size_t buc, - hashed_unique_tag)const - { - bool found=false; - for(node_impl_pointer y=buckets.at(buc)->prior(); - y!=node_impl_pointer(0);y=node_alg::after_local(y)){ - if(y==x)found=true; - else if(eq_(k,key(node_type::from_impl(y)->value())))return false; - } - return found; - } - - bool in_place( - node_impl_pointer x,key_param_type k,std::size_t buc, - hashed_non_unique_tag)const - { - bool found=false; - int range_size=0; - for(node_impl_pointer y=buckets.at(buc)->prior();y!=node_impl_pointer(0);){ - if(node_alg::is_first_of_group(y)){ /* group of 3 or more */ - if(y==x){ - /* in place <-> equal to some other member of the group */ - return eq_( - k, - key(node_type::from_impl( - node_impl_type::pointer_from(y->next()))->value())); - } - else{ - node_impl_pointer z= - node_alg::after_local(y->next()->prior()); /* end of range */ - if(eq_(k,key(node_type::from_impl(y)->value()))){ - if(found)return false; /* x lies outside */ - do{ - if(y==x)return true; - y=node_alg::after_local(y); - }while(y!=z); - return false; /* x not found */ - } - else{ - if(range_size==1&&!found)return false; - if(range_size==2)return found; - range_size=0; - y=z; /* skip range (and potentially x, too, which is fine) */ - } - } - } - else{ /* group of 1 or 2 */ - if(y==x){ - if(range_size==1)return true; - range_size=1; - found=true; - } - else if(eq_(k,key(node_type::from_impl(y)->value()))){ - if(range_size==0&&found)return false; - if(range_size==1&&!found)return false; - if(range_size==2)return false; - ++range_size; - } - else{ - if(range_size==1&&!found)return false; - if(range_size==2)return found; - range_size=0; - } - y=node_alg::after_local(y); - } - } - return found; - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - void detach_iterators(node_type* x) - { - iterator it=make_iterator(x); - safe_mode::detach_equivalent_iterators(it); - } -#endif - - template - std::pair emplace_impl(BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - std::pairp= - this->final_emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - return std::pair(make_iterator(p.first),p.second); - } - - template - iterator emplace_hint_impl( - iterator position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - std::pairp= - this->final_emplace_hint_( - static_cast(position.get_node()), - BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - return make_iterator(p.first); - } - - template< - typename CompatibleHash,typename CompatiblePred - > - iterator find( - const key_type& k, - const CompatibleHash& hash,const CompatiblePred& eq,mpl::true_)const - { - return find(k,hash,eq,mpl::false_()); - } - - template< - typename CompatibleKey,typename CompatibleHash,typename CompatiblePred - > - iterator find( - const CompatibleKey& k, - const CompatibleHash& hash,const CompatiblePred& eq,mpl::false_)const - { - std::size_t buc=buckets.position(hash(k)); - for(node_impl_pointer x=buckets.at(buc)->prior(); - x!=node_impl_pointer(0);x=node_alg::next_to_inspect(x)){ - if(eq(k,key(node_type::from_impl(x)->value()))){ - return make_iterator(node_type::from_impl(x)); - } - } - return end(); - } - - template< - typename CompatibleHash,typename CompatiblePred - > - size_type count( - const key_type& k, - const CompatibleHash& hash,const CompatiblePred& eq,mpl::true_)const - { - return count(k,hash,eq,mpl::false_()); - } - - template< - typename CompatibleKey,typename CompatibleHash,typename CompatiblePred - > - size_type count( - const CompatibleKey& k, - const CompatibleHash& hash,const CompatiblePred& eq,mpl::false_)const - { - std::size_t buc=buckets.position(hash(k)); - for(node_impl_pointer x=buckets.at(buc)->prior(); - x!=node_impl_pointer(0);x=node_alg::next_to_inspect(x)){ - if(eq(k,key(node_type::from_impl(x)->value()))){ - size_type res=0; - node_impl_pointer y=end_of_range(x); - do{ - ++res; - x=node_alg::after(x); - }while(x!=y); - return res; - } - } - return 0; - } - - template< - typename CompatibleHash,typename CompatiblePred - > - std::pair equal_range( - const key_type& k, - const CompatibleHash& hash,const CompatiblePred& eq,mpl::true_)const - { - return equal_range(k,hash,eq,mpl::false_()); - } - - template< - typename CompatibleKey,typename CompatibleHash,typename CompatiblePred - > - std::pair equal_range( - const CompatibleKey& k, - const CompatibleHash& hash,const CompatiblePred& eq,mpl::false_)const - { - std::size_t buc=buckets.position(hash(k)); - for(node_impl_pointer x=buckets.at(buc)->prior(); - x!=node_impl_pointer(0);x=node_alg::next_to_inspect(x)){ - if(eq(k,key(node_type::from_impl(x)->value()))){ - return std::pair( - make_iterator(node_type::from_impl(x)), - make_iterator(node_type::from_impl(end_of_range(x)))); - } - } - return std::pair(end(),end()); - } - - key_from_value key; - hasher hash_; - key_equal eq_; - bucket_array_type buckets; - float mlf; - size_type max_load; - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -#pragma parse_mfunc_templ reset -#endif -}; - -/* comparison */ - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -bool operator==( - const hashed_index& x, - const hashed_index& y) -{ - return x.equals(y); -} - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -bool operator!=( - const hashed_index& x, - const hashed_index& y) -{ - return !(x==y); -} - -/* specialized algorithms */ - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -void swap( - hashed_index& x, - hashed_index& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -/* hashed index specifiers */ - -template -struct hashed_unique -{ - typedef typename detail::hashed_index_args< - Arg1,Arg2,Arg3,Arg4> index_args; - typedef typename index_args::tag_list_type::type tag_list_type; - typedef typename index_args::key_from_value_type key_from_value_type; - typedef typename index_args::hash_type hash_type; - typedef typename index_args::pred_type pred_type; - - template - struct node_class - { - typedef detail::hashed_index_node type; - }; - - template - struct index_class - { - typedef detail::hashed_index< - key_from_value_type,hash_type,pred_type, - SuperMeta,tag_list_type,detail::hashed_unique_tag> type; - }; -}; - -template -struct hashed_non_unique -{ - typedef typename detail::hashed_index_args< - Arg1,Arg2,Arg3,Arg4> index_args; - typedef typename index_args::tag_list_type::type tag_list_type; - typedef typename index_args::key_from_value_type key_from_value_type; - typedef typename index_args::hash_type hash_type; - typedef typename index_args::pred_type pred_type; - - template - struct node_class - { - typedef detail::hashed_index_node< - Super,detail::hashed_non_unique_tag> type; - }; - - template - struct index_class - { - typedef detail::hashed_index< - key_from_value_type,hash_type,pred_type, - SuperMeta,tag_list_type,detail::hashed_non_unique_tag> type; - }; -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -/* Boost.Foreach compatibility */ - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -inline boost::mpl::true_* boost_foreach_is_noncopyable( - boost::multi_index::detail::hashed_index< - KeyFromValue,Hash,Pred,SuperMeta,TagList,Category>*&, - boost_foreach_argument_dependent_lookup_hack) -{ - return 0; -} - -#undef BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT -#undef BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index_fwd.hpp deleted file mode 100644 index d77e36c321b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index_fwd.hpp +++ /dev/null @@ -1,74 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_HASHED_INDEX_FWD_HPP -#define BOOST_MULTI_INDEX_HASHED_INDEX_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -class hashed_index; - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -bool operator==( - const hashed_index& x, - const hashed_index& y); - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -bool operator!=( - const hashed_index& x, - const hashed_index& y); - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -void swap( - hashed_index& x, - hashed_index& y); - -} /* namespace multi_index::detail */ - -/* hashed_index specifiers */ - -template< - typename Arg1,typename Arg2=mpl::na, - typename Arg3=mpl::na,typename Arg4=mpl::na -> -struct hashed_unique; - -template< - typename Arg1,typename Arg2=mpl::na, - typename Arg3=mpl::na,typename Arg4=mpl::na -> -struct hashed_non_unique; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/identity.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/identity.hpp deleted file mode 100644 index 6c832ce1562..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/identity.hpp +++ /dev/null @@ -1,145 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_IDENTITY_HPP -#define BOOST_MULTI_INDEX_IDENTITY_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_SFINAE) -#include -#endif - -namespace boost{ - -template class reference_wrapper; /* fwd decl. */ - -namespace multi_index{ - -namespace detail{ - -/* identity is a do-nothing key extractor that returns the [const] Type& - * object passed. - * Additionally, identity is overloaded to support referece_wrappers - * of Type and "chained pointers" to Type's. By chained pointer to Type we - * mean a type P such that, given a p of type P - * *...n...*x is convertible to Type&, for some n>=1. - * Examples of chained pointers are raw and smart pointers, iterators and - * arbitrary combinations of these (vg. Type** or unique_ptr.) - */ - -template -struct const_identity_base -{ - typedef Type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if,Type&>::type -#else - Type& -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type& operator()(Type& x)const - { - return x; - } - - Type& operator()(const reference_wrapper& x)const - { - return x.get(); - } - - Type& operator()( - const reference_wrapper::type>& x - -#if BOOST_WORKAROUND(BOOST_MSVC,==1310) -/* http://lists.boost.org/Archives/boost/2015/10/226135.php */ - ,int=0 -#endif - - )const - { - return x.get(); - } -}; - -template -struct non_const_identity_base -{ - typedef Type result_type; - - /* templatized for pointer-like types */ - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type&>::type -#else - Type& -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - const Type& operator()(const Type& x)const - { - return x; - } - - Type& operator()(Type& x)const - { - return x; - } - - const Type& operator()(const reference_wrapper& x)const - { - return x.get(); - } - - Type& operator()(const reference_wrapper& x)const - { - return x.get(); - } -}; - -} /* namespace multi_index::detail */ - -template -struct identity: - mpl::if_c< - is_const::value, - detail::const_identity_base,detail::non_const_identity_base - >::type -{ -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/identity_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/identity_fwd.hpp deleted file mode 100644 index af6bd55ef5f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/identity_fwd.hpp +++ /dev/null @@ -1,26 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_IDENTITY_FWD_HPP -#define BOOST_MULTI_INDEX_IDENTITY_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -namespace boost{ - -namespace multi_index{ - -template struct identity; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/indexed_by.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/indexed_by.hpp deleted file mode 100644 index d2217e39166..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/indexed_by.hpp +++ /dev/null @@ -1,68 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_INDEXED_BY_HPP -#define BOOST_MULTI_INDEX_INDEXED_BY_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include - -/* An alias to mpl::vector used to hide MPL from the user. - * indexed_by contains the index specifiers for instantiation - * of a multi_index_container. - */ - -/* This user_definable macro limits the number of elements of an index list; - * useful for shortening resulting symbol names (MSVC++ 6.0, for instance, - * has problems coping with very long symbol names.) - */ - -#if !defined(BOOST_MULTI_INDEX_LIMIT_INDEXED_BY_SIZE) -#define BOOST_MULTI_INDEX_LIMIT_INDEXED_BY_SIZE BOOST_MPL_LIMIT_VECTOR_SIZE -#endif - -#if BOOST_MULTI_INDEX_LIMIT_INDEXED_BY_SIZE -struct indexed_by: - mpl::vector -{ -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#undef BOOST_MULTI_INDEX_INDEXED_BY_TEMPLATE_PARM -#undef BOOST_MULTI_INDEX_INDEXED_BY_SIZE - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/key_extractors.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/key_extractors.hpp deleted file mode 100644 index 60179ba2339..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/key_extractors.hpp +++ /dev/null @@ -1,22 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_KEY_EXTRACTORS_HPP -#define BOOST_MULTI_INDEX_KEY_EXTRACTORS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include -#include -#include -#include -#include - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/mem_fun.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/mem_fun.hpp deleted file mode 100644 index 111c386c5f5..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/mem_fun.hpp +++ /dev/null @@ -1,205 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_MEM_FUN_HPP -#define BOOST_MULTI_INDEX_MEM_FUN_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -#if !defined(BOOST_NO_SFINAE) -#include -#endif - -namespace boost{ - -template class reference_wrapper; /* fwd decl. */ - -namespace multi_index{ - -/* mem_fun implements a read-only key extractor based on a given non-const - * member function of a class. - * const_mem_fun does the same for const member functions. - * Additionally, mem_fun and const_mem_fun are overloaded to support - * referece_wrappers of T and "chained pointers" to T's. By chained pointer - * to T we mean a type P such that, given a p of Type P - * *...n...*x is convertible to T&, for some n>=1. - * Examples of chained pointers are raw and smart pointers, iterators and - * arbitrary combinations of these (vg. T** or unique_ptr.) - */ - -template -struct const_mem_fun -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(const Class& x)const - { - return (x.*PtrToMemberFunction)(); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -template -struct mem_fun -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(Class& x)const - { - return (x.*PtrToMemberFunction)(); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -/* MSVC++ 6.0 has problems with const member functions as non-type template - * parameters, somehow it takes them as non-const. const_mem_fun_explicit - * workarounds this deficiency by accepting an extra type parameter that - * specifies the signature of the member function. The workaround was found at: - * Daniel, C.:"Re: weird typedef problem in VC", - * news:microsoft.public.vc.language, 21st nov 2002, - * http://groups.google.com/groups? - * hl=en&lr=&ie=UTF-8&selm=ukwvg3O0BHA.1512%40tkmsftngp05 - * - * MSVC++ 6.0 support has been dropped and [const_]mem_fun_explicit is - * deprecated. - */ - -template< - class Class,typename Type, - typename PtrToMemberFunctionType,PtrToMemberFunctionType PtrToMemberFunction> -struct const_mem_fun_explicit -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(const Class& x)const - { - return (x.*PtrToMemberFunction)(); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -template< - class Class,typename Type, - typename PtrToMemberFunctionType,PtrToMemberFunctionType PtrToMemberFunction> -struct mem_fun_explicit -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(Class& x)const - { - return (x.*PtrToMemberFunction)(); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -/* BOOST_MULTI_INDEX_CONST_MEM_FUN and BOOST_MULTI_INDEX_MEM_FUN used to - * resolve to [const_]mem_fun_explicit for MSVC++ 6.0 and to - * [const_]mem_fun otherwise. Support for this compiler having been dropped, - * they are now just wrappers over [const_]mem_fun kept for backwards- - * compatibility reasons. - */ - -#define BOOST_MULTI_INDEX_CONST_MEM_FUN(Class,Type,MemberFunName) \ -::boost::multi_index::const_mem_fun< Class,Type,&Class::MemberFunName > -#define BOOST_MULTI_INDEX_MEM_FUN(Class,Type,MemberFunName) \ -::boost::multi_index::mem_fun< Class,Type,&Class::MemberFunName > - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/member.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/member.hpp deleted file mode 100644 index a8e645074a2..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/member.hpp +++ /dev/null @@ -1,262 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_MEMBER_HPP -#define BOOST_MULTI_INDEX_MEMBER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include - -#if !defined(BOOST_NO_SFINAE) -#include -#endif - -namespace boost{ - -template class reference_wrapper; /* fwd decl. */ - -namespace multi_index{ - -namespace detail{ - -/* member is a read/write key extractor for accessing a given - * member of a class. - * Additionally, member is overloaded to support referece_wrappers - * of T and "chained pointers" to T's. By chained pointer to T we mean - * a type P such that, given a p of Type P - * *...n...*x is convertible to T&, for some n>=1. - * Examples of chained pointers are raw and smart pointers, iterators and - * arbitrary combinations of these (vg. T** or unique_ptr.) - */ - -template -struct const_member_base -{ - typedef Type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type&>::type -#else - Type& -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type& operator()(const Class& x)const - { - return x.*PtrToMember; - } - - Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -template -struct non_const_member_base -{ - typedef Type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type&>::type -#else - Type& -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - const Type& operator()(const Class& x)const - { - return x.*PtrToMember; - } - - Type& operator()(Class& x)const - { - return x.*PtrToMember; - } - - const Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -} /* namespace multi_index::detail */ - -template -struct member: - mpl::if_c< - is_const::value, - detail::const_member_base, - detail::non_const_member_base - >::type -{ -}; - -namespace detail{ - -/* MSVC++ 6.0 does not support properly pointers to members as - * non-type template arguments, as reported in - * http://support.microsoft.com/default.aspx?scid=kb;EN-US;249045 - * A similar problem (though not identical) is shown by MSVC++ 7.0. - * We provide an alternative to member<> accepting offsets instead - * of pointers to members. This happens to work even for non-POD - * types (although the standard forbids use of offsetof on these), - * so it serves as a workaround in this compiler for all practical - * purposes. - * Surprisingly enough, other compilers, like Intel C++ 7.0/7.1 and - * Visual Age 6.0, have similar bugs. This replacement of member<> - * can be used for them too. - * - * Support for such old compilers is dropped and - * [non_]const_member_offset_base is deprecated. - */ - -template -struct const_member_offset_base -{ - typedef Type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type&>::type -#else - Type& -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type& operator()(const Class& x)const - { - return *static_cast( - static_cast( - static_cast( - static_cast(&x))+OffsetOfMember)); - } - - Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -template -struct non_const_member_offset_base -{ - typedef Type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type&>::type -#else - Type& -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - const Type& operator()(const Class& x)const - { - return *static_cast( - static_cast( - static_cast( - static_cast(&x))+OffsetOfMember)); - } - - Type& operator()(Class& x)const - { - return *static_cast( - static_cast( - static_cast(static_cast(&x))+OffsetOfMember)); - } - - const Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -} /* namespace multi_index::detail */ - -template -struct member_offset: - mpl::if_c< - is_const::value, - detail::const_member_offset_base, - detail::non_const_member_offset_base - >::type -{ -}; - -/* BOOST_MULTI_INDEX_MEMBER resolves to member in the normal cases, - * and to member_offset as a workaround in those defective compilers for - * which BOOST_NO_POINTER_TO_MEMBER_TEMPLATE_PARAMETERS is defined. - */ - -#if defined(BOOST_NO_POINTER_TO_MEMBER_TEMPLATE_PARAMETERS) -#define BOOST_MULTI_INDEX_MEMBER(Class,Type,MemberName) \ -::boost::multi_index::member_offset< Class,Type,offsetof(Class,MemberName) > -#else -#define BOOST_MULTI_INDEX_MEMBER(Class,Type,MemberName) \ -::boost::multi_index::member< Class,Type,&Class::MemberName > -#endif - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index.hpp deleted file mode 100644 index 5bcd69de8c9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index.hpp +++ /dev/null @@ -1,114 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_ORDERED_INDEX_HPP -#define BOOST_MULTI_INDEX_ORDERED_INDEX_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* no augment policy for plain ordered indices */ - -struct null_augment_policy -{ - template - struct augmented_interface - { - typedef OrderedIndexImpl type; - }; - - template - struct augmented_node - { - typedef OrderedIndexNodeImpl type; - }; - - template static void add(Pointer,Pointer){} - template static void remove(Pointer,Pointer){} - template static void copy(Pointer,Pointer){} - template static void rotate_left(Pointer,Pointer){} - template static void rotate_right(Pointer,Pointer){} - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - template static bool invariant(Pointer){return true;} - -#endif -}; - -} /* namespace multi_index::detail */ - -/* ordered_index specifiers */ - -template -struct ordered_unique -{ - typedef typename detail::ordered_index_args< - Arg1,Arg2,Arg3> index_args; - typedef typename index_args::tag_list_type::type tag_list_type; - typedef typename index_args::key_from_value_type key_from_value_type; - typedef typename index_args::compare_type compare_type; - - template - struct node_class - { - typedef detail::ordered_index_node type; - }; - - template - struct index_class - { - typedef detail::ordered_index< - key_from_value_type,compare_type, - SuperMeta,tag_list_type,detail::ordered_unique_tag, - detail::null_augment_policy> type; - }; -}; - -template -struct ordered_non_unique -{ - typedef detail::ordered_index_args< - Arg1,Arg2,Arg3> index_args; - typedef typename index_args::tag_list_type::type tag_list_type; - typedef typename index_args::key_from_value_type key_from_value_type; - typedef typename index_args::compare_type compare_type; - - template - struct node_class - { - typedef detail::ordered_index_node type; - }; - - template - struct index_class - { - typedef detail::ordered_index< - key_from_value_type,compare_type, - SuperMeta,tag_list_type,detail::ordered_non_unique_tag, - detail::null_augment_policy> type; - }; -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index_fwd.hpp deleted file mode 100644 index fe44aaf860d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index_fwd.hpp +++ /dev/null @@ -1,35 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_ORDERED_INDEX_FWD_HPP -#define BOOST_MULTI_INDEX_ORDERED_INDEX_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include -#include - -namespace boost{ - -namespace multi_index{ - -/* ordered_index specifiers */ - -template -struct ordered_unique; - -template -struct ordered_non_unique; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index.hpp deleted file mode 100644 index fe1884ddd38..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index.hpp +++ /dev/null @@ -1,1167 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_RANDOM_ACCESS_INDEX_HPP -#define BOOST_MULTI_INDEX_RANDOM_ACCESS_INDEX_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) -#include -#endif - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) -#define BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF(x) \ - detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ - detail::make_obj_guard(x,&random_access_index::check_invariant_); \ - BOOST_JOIN(check_invariant_,__LINE__).touch(); -#define BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT \ - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF(*this) -#else -#define BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF(x) -#define BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* random_access_index adds a layer of random access indexing - * to a given Super - */ - -template -class random_access_index: - BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS SuperMeta::type - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - ,public safe_mode::safe_container< - random_access_index > -#endif - -{ -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the - * lifetime of const references bound to temporaries --precisely what - * scopeguards are. - */ - -#pragma parse_mfunc_templ off -#endif - - typedef typename SuperMeta::type super; - -protected: - typedef random_access_index_node< - typename super::node_type> node_type; - -private: - typedef typename node_type::impl_type node_impl_type; - typedef random_access_index_ptr_array< - typename super::final_allocator_type> ptr_array; - typedef typename ptr_array::pointer node_impl_ptr_pointer; - -public: - /* types */ - - typedef typename node_type::value_type value_type; - typedef tuples::null_type ctor_args; - typedef typename super::final_allocator_type allocator_type; - typedef typename allocator_type::reference reference; - typedef typename allocator_type::const_reference const_reference; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_iterator< - rnd_node_iterator, - random_access_index> iterator; -#else - typedef rnd_node_iterator iterator; -#endif - - typedef iterator const_iterator; - - typedef std::size_t size_type; - typedef std::ptrdiff_t difference_type; - typedef typename allocator_type::pointer pointer; - typedef typename allocator_type::const_pointer const_pointer; - typedef typename - boost::reverse_iterator reverse_iterator; - typedef typename - boost::reverse_iterator const_reverse_iterator; - typedef TagList tag_list; - -protected: - typedef typename super::final_node_type final_node_type; - typedef tuples::cons< - ctor_args, - typename super::ctor_args_list> ctor_args_list; - typedef typename mpl::push_front< - typename super::index_type_list, - random_access_index>::type index_type_list; - typedef typename mpl::push_front< - typename super::iterator_type_list, - iterator>::type iterator_type_list; - typedef typename mpl::push_front< - typename super::const_iterator_type_list, - const_iterator>::type const_iterator_type_list; - typedef typename super::copy_map_type copy_map_type; - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - typedef typename super::index_saver_type index_saver_type; - typedef typename super::index_loader_type index_loader_type; -#endif - -private: -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_container< - random_access_index> safe_super; -#endif - - typedef typename call_traits< - value_type>::param_type value_param_type; - - /* Needed to avoid commas in BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL - * expansion. - */ - - typedef std::pair emplace_return_type; - -public: - - /* construct/copy/destroy - * Default and copy ctors are in the protected section as indices are - * not supposed to be created on their own. No range ctor either. - */ - - random_access_index& operator=( - const random_access_index& x) - { - this->final()=x.final(); - return *this; - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - random_access_index& operator=( - std::initializer_list list) - { - this->final()=list; - return *this; - } -#endif - - template - void assign(InputIterator first,InputIterator last) - { - assign_iter(first,last,mpl::not_ >()); - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - void assign(std::initializer_list list) - { - assign(list.begin(),list.end()); - } -#endif - - void assign(size_type n,value_param_type value) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - clear(); - for(size_type i=0;ifinal().get_allocator(); - } - - /* iterators */ - - iterator begin()BOOST_NOEXCEPT - {return make_iterator(node_type::from_impl(*ptrs.begin()));} - const_iterator begin()const BOOST_NOEXCEPT - {return make_iterator(node_type::from_impl(*ptrs.begin()));} - iterator - end()BOOST_NOEXCEPT{return make_iterator(header());} - const_iterator - end()const BOOST_NOEXCEPT{return make_iterator(header());} - reverse_iterator - rbegin()BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} - const_reverse_iterator - rbegin()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} - reverse_iterator - rend()BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} - const_reverse_iterator - rend()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} - const_iterator - cbegin()const BOOST_NOEXCEPT{return begin();} - const_iterator - cend()const BOOST_NOEXCEPT{return end();} - const_reverse_iterator - crbegin()const BOOST_NOEXCEPT{return rbegin();} - const_reverse_iterator - crend()const BOOST_NOEXCEPT{return rend();} - - iterator iterator_to(const value_type& x) - { - return make_iterator(node_from_value(&x)); - } - - const_iterator iterator_to(const value_type& x)const - { - return make_iterator(node_from_value(&x)); - } - - /* capacity */ - - bool empty()const BOOST_NOEXCEPT{return this->final_empty_();} - size_type size()const BOOST_NOEXCEPT{return this->final_size_();} - size_type max_size()const BOOST_NOEXCEPT{return this->final_max_size_();} - size_type capacity()const BOOST_NOEXCEPT{return ptrs.capacity();} - - void reserve(size_type n) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - ptrs.reserve(n); - } - - void shrink_to_fit() - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - ptrs.shrink_to_fit(); - } - - void resize(size_type n) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - if(n>size()) - for(size_type m=n-size();m--;) - this->final_emplace_(BOOST_MULTI_INDEX_NULL_PARAM_PACK); - else if(nsize())for(size_type m=n-size();m--;)this->final_insert_(x); - else if(nvalue(); - } - - const_reference at(size_type n)const - { - if(n>=size())throw_exception(std::out_of_range("random access index")); - return node_type::from_impl(*ptrs.at(n))->value(); - } - - const_reference front()const{return operator[](0);} - const_reference back()const{return operator[](size()-1);} - - /* modifiers */ - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( - emplace_return_type,emplace_front,emplace_front_impl) - - std::pair push_front(const value_type& x) - {return insert(begin(),x);} - std::pair push_front(BOOST_RV_REF(value_type) x) - {return insert(begin(),boost::move(x));} - void pop_front(){erase(begin());} - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( - emplace_return_type,emplace_back,emplace_back_impl) - - std::pair push_back(const value_type& x) - {return insert(end(),x);} - std::pair push_back(BOOST_RV_REF(value_type) x) - {return insert(end(),boost::move(x));} - void pop_back(){erase(--end());} - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( - emplace_return_type,emplace,emplace_impl,iterator,position) - - std::pair insert(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_(x); - if(p.second&&position.get_node()!=header()){ - relocate(position.get_node(),p.first); - } - return std::pair(make_iterator(p.first),p.second); - } - - std::pair insert(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_rv_(x); - if(p.second&&position.get_node()!=header()){ - relocate(position.get_node(),p.first); - } - return std::pair(make_iterator(p.first),p.second); - } - - void insert(iterator position,size_type n,value_param_type x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - size_type s=0; - BOOST_TRY{ - while(n--){ - if(push_back(x).second)++s; - } - } - BOOST_CATCH(...){ - relocate(position,end()-s,end()); - BOOST_RETHROW; - } - BOOST_CATCH_END - relocate(position,end()-s,end()); - } - - template - void insert(iterator position,InputIterator first,InputIterator last) - { - insert_iter(position,first,last,mpl::not_ >()); - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - void insert(iterator position,std::initializer_list list) - { - insert(position,list.begin(),list.end()); - } -#endif - - iterator erase(iterator position) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - this->final_erase_(static_cast(position++.get_node())); - return position; - } - - iterator erase(iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - difference_type n=last-first; - relocate(end(),first,last); - while(n--)pop_back(); - return last; - } - - bool replace(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - return this->final_replace_( - x,static_cast(position.get_node())); - } - - bool replace(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - return this->final_replace_rv_( - x,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod,Rollback back_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,back_,static_cast(position.get_node())); - } - - void swap(random_access_index& x) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF(x); - this->final_swap_(x.final()); - } - - void clear()BOOST_NOEXCEPT - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - this->final_clear_(); - } - - /* list operations */ - - void splice(iterator position,random_access_index& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_DIFFERENT_CONTAINER(*this,x); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - iterator first=x.begin(),last=x.end(); - size_type n=0; - BOOST_TRY{ - while(first!=last){ - if(push_back(*first).second){ - first=x.erase(first); - ++n; - } - else ++first; - } - } - BOOST_CATCH(...){ - relocate(position,end()-n,end()); - BOOST_RETHROW; - } - BOOST_CATCH_END - relocate(position,end()-n,end()); - } - - void splice( - iterator position,random_access_index& x,iterator i) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(i,x); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - if(&x==this)relocate(position,i); - else{ - if(insert(position,*i).second){ - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer has a hard time with safe mode, and the following - * workaround is needed. Left it for all compilers as it does no - * harm. - */ - i.detach(); - x.erase(x.make_iterator(i.get_node())); -#else - x.erase(i); -#endif - - } - } - } - - void splice( - iterator position,random_access_index& x, - iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,x); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,x); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - if(&x==this)relocate(position,first,last); - else{ - size_type n=0; - BOOST_TRY{ - while(first!=last){ - if(push_back(*first).second){ - first=x.erase(first); - ++n; - } - else ++first; - } - } - BOOST_CATCH(...){ - relocate(position,end()-n,end()); - BOOST_RETHROW; - } - BOOST_CATCH_END - relocate(position,end()-n,end()); - } - } - - void remove(value_param_type value) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - difference_type n= - end()-make_iterator( - random_access_index_remove( - ptrs, - ::boost::bind(std::equal_to(),::boost::arg<1>(),value))); - while(n--)pop_back(); - } - - template - void remove_if(Predicate pred) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - difference_type n= - end()-make_iterator(random_access_index_remove(ptrs,pred)); - while(n--)pop_back(); - } - - void unique() - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - difference_type n= - end()-make_iterator( - random_access_index_unique( - ptrs,std::equal_to())); - while(n--)pop_back(); - } - - template - void unique(BinaryPredicate binary_pred) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - difference_type n= - end()-make_iterator( - random_access_index_unique(ptrs,binary_pred)); - while(n--)pop_back(); - } - - void merge(random_access_index& x) - { - if(this!=&x){ - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - size_type s=size(); - splice(end(),x); - random_access_index_inplace_merge( - get_allocator(),ptrs,ptrs.at(s),std::less()); - } - } - - template - void merge(random_access_index& x,Compare comp) - { - if(this!=&x){ - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - size_type s=size(); - splice(end(),x); - random_access_index_inplace_merge( - get_allocator(),ptrs,ptrs.at(s),comp); - } - } - - void sort() - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - random_access_index_sort( - get_allocator(),ptrs,std::less()); - } - - template - void sort(Compare comp) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - random_access_index_sort( - get_allocator(),ptrs,comp); - } - - void reverse()BOOST_NOEXCEPT - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - node_impl_type::reverse(ptrs.begin(),ptrs.end()); - } - - /* rearrange operations */ - - void relocate(iterator position,iterator i) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(i,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - if(position!=i)relocate(position.get_node(),i.get_node()); - } - - void relocate(iterator position,iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_CHECK_OUTSIDE_RANGE(position,first,last); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - if(position!=last)relocate( - position.get_node(),first.get_node(),last.get_node()); - } - - template - void rearrange(InputIterator first) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - for(node_impl_ptr_pointer p0=ptrs.begin(),p0_end=ptrs.end(); - p0!=p0_end;++first,++p0){ - const value_type& v1=*first; - node_impl_ptr_pointer p1=node_from_value(&v1)->up(); - - std::swap(*p0,*p1); - (*p0)->up()=p0; - (*p1)->up()=p1; - } - } - -BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: - random_access_index( - const ctor_args_list& args_list,const allocator_type& al): - super(args_list.get_tail(),al), - ptrs(al,header()->impl(),0) - { - } - - random_access_index(const random_access_index& x): - super(x), - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super(), -#endif - - ptrs(x.get_allocator(),header()->impl(),x.size()) - { - /* The actual copying takes place in subsequent call to copy_(). - */ - } - - random_access_index( - const random_access_index& x,do_not_copy_elements_tag): - super(x,do_not_copy_elements_tag()), - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super(), -#endif - - ptrs(x.get_allocator(),header()->impl(),0) - { - } - - ~random_access_index() - { - /* the container is guaranteed to be empty by now */ - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - iterator make_iterator(node_type* node){return iterator(node,this);} - const_iterator make_iterator(node_type* node)const - {return const_iterator(node,const_cast(this));} -#else - iterator make_iterator(node_type* node){return iterator(node);} - const_iterator make_iterator(node_type* node)const - {return const_iterator(node);} -#endif - - void copy_( - const random_access_index& x,const copy_map_type& map) - { - for(node_impl_ptr_pointer begin_org=x.ptrs.begin(), - begin_cpy=ptrs.begin(), - end_org=x.ptrs.end(); - begin_org!=end_org;++begin_org,++begin_cpy){ - *begin_cpy= - static_cast( - map.find( - static_cast( - node_type::from_impl(*begin_org))))->impl(); - (*begin_cpy)->up()=begin_cpy; - } - - super::copy_(x,map); - } - - template - final_node_type* insert_( - value_param_type v,final_node_type*& x,Variant variant) - { - ptrs.room_for_one(); - final_node_type* res=super::insert_(v,x,variant); - if(res==x)ptrs.push_back(static_cast(x)->impl()); - return res; - } - - template - final_node_type* insert_( - value_param_type v,node_type* position,final_node_type*& x,Variant variant) - { - ptrs.room_for_one(); - final_node_type* res=super::insert_(v,position,x,variant); - if(res==x)ptrs.push_back(static_cast(x)->impl()); - return res; - } - - void erase_(node_type* x) - { - ptrs.erase(x->impl()); - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - } - - void delete_all_nodes_() - { - for(node_impl_ptr_pointer x=ptrs.begin(),x_end=ptrs.end();x!=x_end;++x){ - this->final_delete_node_( - static_cast(node_type::from_impl(*x))); - } - } - - void clear_() - { - super::clear_(); - ptrs.clear(); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::detach_dereferenceable_iterators(); -#endif - } - - void swap_(random_access_index& x) - { - ptrs.swap(x.ptrs); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_(x); - } - - void swap_elements_(random_access_index& x) - { - ptrs.swap(x.ptrs); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_elements_(x); - } - - template - bool replace_(value_param_type v,node_type* x,Variant variant) - { - return super::replace_(v,x,variant); - } - - bool modify_(node_type* x) - { - BOOST_TRY{ - if(!super::modify_(x)){ - ptrs.erase(x->impl()); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - return false; - } - else return true; - } - BOOST_CATCH(...){ - ptrs.erase(x->impl()); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - bool modify_rollback_(node_type* x) - { - return super::modify_rollback_(x); - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* serialization */ - - template - void save_( - Archive& ar,const unsigned int version,const index_saver_type& sm)const - { - sm.save(begin(),end(),ar,version); - super::save_(ar,version,sm); - } - - template - void load_( - Archive& ar,const unsigned int version,const index_loader_type& lm) - { - { - typedef random_access_index_loader loader; - - loader ld(get_allocator(),ptrs); - lm.load( - ::boost::bind( - &loader::rearrange,&ld,::boost::arg<1>(),::boost::arg<2>()), - ar,version); - } /* exit scope so that ld frees its resources */ - super::load_(ar,version,lm); - } -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - bool invariant_()const - { - if(size()>capacity())return false; - if(size()==0||begin()==end()){ - if(size()!=0||begin()!=end())return false; - } - else{ - size_type s=0; - for(const_iterator it=begin(),it_end=end();;++it,++s){ - if(*(it.get_node()->up())!=it.get_node()->impl())return false; - if(it==it_end)break; - } - if(s!=size())return false; - } - - return super::invariant_(); - } - - /* This forwarding function eases things for the boost::mem_fn construct - * in BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT. Actually, - * final_check_invariant is already an inherited member function of index. - */ - void check_invariant_()const{this->final_check_invariant_();} -#endif - -private: - node_type* header()const{return this->final_header();} - - static void relocate(node_type* position,node_type* x) - { - node_impl_type::relocate(position->up(),x->up()); - } - - static void relocate(node_type* position,node_type* first,node_type* last) - { - node_impl_type::relocate( - position->up(),first->up(),last->up()); - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - void detach_iterators(node_type* x) - { - iterator it=make_iterator(x); - safe_mode::detach_equivalent_iterators(it); - } -#endif - - template - void assign_iter(InputIterator first,InputIterator last,mpl::true_) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - clear(); - for(;first!=last;++first)this->final_insert_ref_(*first); - } - - void assign_iter(size_type n,value_param_type value,mpl::false_) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - clear(); - for(size_type i=0;i - void insert_iter( - iterator position,InputIterator first,InputIterator last,mpl::true_) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - size_type s=0; - BOOST_TRY{ - for(;first!=last;++first){ - if(this->final_insert_ref_(*first).second)++s; - } - } - BOOST_CATCH(...){ - relocate(position,end()-s,end()); - BOOST_RETHROW; - } - BOOST_CATCH_END - relocate(position,end()-s,end()); - } - - void insert_iter( - iterator position,size_type n,value_param_type x,mpl::false_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - size_type s=0; - BOOST_TRY{ - while(n--){ - if(push_back(x).second)++s; - } - } - BOOST_CATCH(...){ - relocate(position,end()-s,end()); - BOOST_RETHROW; - } - BOOST_CATCH_END - relocate(position,end()-s,end()); - } - - template - std::pair emplace_front_impl( - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - return emplace_impl(begin(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - } - - template - std::pair emplace_back_impl( - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - return emplace_impl(end(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - } - - template - std::pair emplace_impl( - iterator position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - std::pair p= - this->final_emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - if(p.second&&position.get_node()!=header()){ - relocate(position.get_node(),p.first); - } - return std::pair(make_iterator(p.first),p.second); - } - - ptr_array ptrs; - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -#pragma parse_mfunc_templ reset -#endif -}; - -/* comparison */ - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator==( - const random_access_index& x, - const random_access_index& y) -{ - return x.size()==y.size()&&std::equal(x.begin(),x.end(),y.begin()); -} - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator<( - const random_access_index& x, - const random_access_index& y) -{ - return std::lexicographical_compare(x.begin(),x.end(),y.begin(),y.end()); -} - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator!=( - const random_access_index& x, - const random_access_index& y) -{ - return !(x==y); -} - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator>( - const random_access_index& x, - const random_access_index& y) -{ - return y -bool operator>=( - const random_access_index& x, - const random_access_index& y) -{ - return !(x -bool operator<=( - const random_access_index& x, - const random_access_index& y) -{ - return !(x>y); -} - -/* specialized algorithms */ - -template -void swap( - random_access_index& x, - random_access_index& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -/* random access index specifier */ - -template -struct random_access -{ - BOOST_STATIC_ASSERT(detail::is_tag::value); - - template - struct node_class - { - typedef detail::random_access_index_node type; - }; - - template - struct index_class - { - typedef detail::random_access_index< - SuperMeta,typename TagList::type> type; - }; -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -/* Boost.Foreach compatibility */ - -template -inline boost::mpl::true_* boost_foreach_is_noncopyable( - boost::multi_index::detail::random_access_index*&, - boost_foreach_argument_dependent_lookup_hack) -{ - return 0; -} - -#undef BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT -#undef BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index_fwd.hpp deleted file mode 100644 index 2ea19295426..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index_fwd.hpp +++ /dev/null @@ -1,91 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_RANDOM_ACCESS_INDEX_FWD_HPP -#define BOOST_MULTI_INDEX_RANDOM_ACCESS_INDEX_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -class random_access_index; - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator==( - const random_access_index& x, - const random_access_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator<( - const random_access_index& x, - const random_access_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator!=( - const random_access_index& x, - const random_access_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator>( - const random_access_index& x, - const random_access_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator>=( - const random_access_index& x, - const random_access_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator<=( - const random_access_index& x, - const random_access_index& y); - -template -void swap( - random_access_index& x, - random_access_index& y); - -} /* namespace multi_index::detail */ - -/* index specifiers */ - -template > -struct random_access; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index.hpp deleted file mode 100644 index 4b24c4f5937..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index.hpp +++ /dev/null @@ -1,382 +0,0 @@ -/* Copyright 2003-2017 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_RANKED_INDEX_HPP -#define BOOST_MULTI_INDEX_RANKED_INDEX_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* ranked_index augments a given ordered index to provide rank operations */ - -template -struct ranked_node:OrderedIndexNodeImpl -{ - std::size_t size; -}; - -template -class ranked_index:public OrderedIndexImpl -{ - typedef OrderedIndexImpl super; - -protected: - typedef typename super::node_type node_type; - typedef typename super::node_impl_pointer node_impl_pointer; - -public: - typedef typename super::ctor_args_list ctor_args_list; - typedef typename super::allocator_type allocator_type; - typedef typename super::iterator iterator; - - /* rank operations */ - - iterator nth(std::size_t n)const - { - return this->make_iterator(node_type::from_impl( - ranked_index_nth(n,this->header()->impl()))); - } - - std::size_t rank(iterator position)const - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - - return ranked_index_rank( - position.get_node()->impl(),this->header()->impl()); - } - - template - std::size_t find_rank(const CompatibleKey& x)const - { - return ranked_index_find_rank( - this->root(),this->header(),this->key,x,this->comp_); - } - - template - std::size_t find_rank( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return ranked_index_find_rank( - this->root(),this->header(),this->key,x,comp); - } - - template - std::size_t lower_bound_rank(const CompatibleKey& x)const - { - return ranked_index_lower_bound_rank( - this->root(),this->header(),this->key,x,this->comp_); - } - - template - std::size_t lower_bound_rank( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return ranked_index_lower_bound_rank( - this->root(),this->header(),this->key,x,comp); - } - - template - std::size_t upper_bound_rank(const CompatibleKey& x)const - { - return ranked_index_upper_bound_rank( - this->root(),this->header(),this->key,x,this->comp_); - } - - template - std::size_t upper_bound_rank( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return ranked_index_upper_bound_rank( - this->root(),this->header(),this->key,x,comp); - } - - template - std::pair equal_range_rank( - const CompatibleKey& x)const - { - return ranked_index_equal_range_rank( - this->root(),this->header(),this->key,x,this->comp_); - } - - template - std::pair equal_range_rank( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return ranked_index_equal_range_rank( - this->root(),this->header(),this->key,x,comp); - } - - template - std::pair - range_rank(LowerBounder lower,UpperBounder upper)const - { - typedef typename mpl::if_< - is_same, - BOOST_DEDUCED_TYPENAME mpl::if_< - is_same, - both_unbounded_tag, - lower_unbounded_tag - >::type, - BOOST_DEDUCED_TYPENAME mpl::if_< - is_same, - upper_unbounded_tag, - none_unbounded_tag - >::type - >::type dispatch; - - return range_rank(lower,upper,dispatch()); - } - -protected: - ranked_index(const ranked_index& x):super(x){}; - - ranked_index(const ranked_index& x,do_not_copy_elements_tag): - super(x,do_not_copy_elements_tag()){}; - - ranked_index( - const ctor_args_list& args_list,const allocator_type& al): - super(args_list,al){} - -private: - template - std::pair - range_rank(LowerBounder lower,UpperBounder upper,none_unbounded_tag)const - { - node_type* y=this->header(); - node_type* z=this->root(); - - if(!z)return std::pair(0,0); - - std::size_t s=z->impl()->size; - - do{ - if(!lower(this->key(z->value()))){ - z=node_type::from_impl(z->right()); - } - else if(!upper(this->key(z->value()))){ - y=z; - s-=ranked_node_size(y->right())+1; - z=node_type::from_impl(z->left()); - } - else{ - return std::pair( - s-z->impl()->size+ - lower_range_rank(node_type::from_impl(z->left()),z,lower), - s-ranked_node_size(z->right())+ - upper_range_rank(node_type::from_impl(z->right()),y,upper)); - } - }while(z); - - return std::pair(s,s); - } - - template - std::pair - range_rank(LowerBounder,UpperBounder upper,lower_unbounded_tag)const - { - return std::pair( - 0, - upper_range_rank(this->root(),this->header(),upper)); - } - - template - std::pair - range_rank(LowerBounder lower,UpperBounder,upper_unbounded_tag)const - { - return std::pair( - lower_range_rank(this->root(),this->header(),lower), - this->size()); - } - - template - std::pair - range_rank(LowerBounder,UpperBounder,both_unbounded_tag)const - { - return std::pair(0,this->size()); - } - - template - std::size_t - lower_range_rank(node_type* top,node_type* y,LowerBounder lower)const - { - if(!top)return 0; - - std::size_t s=top->impl()->size; - - do{ - if(lower(this->key(top->value()))){ - y=top; - s-=ranked_node_size(y->right())+1; - top=node_type::from_impl(top->left()); - } - else top=node_type::from_impl(top->right()); - }while(top); - - return s; - } - - template - std::size_t - upper_range_rank(node_type* top,node_type* y,UpperBounder upper)const - { - if(!top)return 0; - - std::size_t s=top->impl()->size; - - do{ - if(!upper(this->key(top->value()))){ - y=top; - s-=ranked_node_size(y->right())+1; - top=node_type::from_impl(top->left()); - } - else top=node_type::from_impl(top->right()); - }while(top); - - return s; - } -}; - -/* augmenting policy for ordered_index */ - -struct rank_policy -{ - template - struct augmented_node - { - typedef ranked_node type; - }; - - template - struct augmented_interface - { - typedef ranked_index type; - }; - - /* algorithmic stuff */ - - template - static void add(Pointer x,Pointer root) - { - x->size=1; - while(x!=root){ - x=x->parent(); - ++(x->size); - } - } - - template - static void remove(Pointer x,Pointer root) - { - while(x!=root){ - x=x->parent(); - --(x->size); - } - } - - template - static void copy(Pointer x,Pointer y) - { - y->size=x->size; - } - - template - static void rotate_left(Pointer x,Pointer y) /* in: x==y->left() */ - { - y->size=x->size; - x->size=ranked_node_size(x->left())+ranked_node_size(x->right())+1; - } - - template - static void rotate_right(Pointer x,Pointer y) /* in: x==y->right() */ - { - rotate_left(x,y); - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - template - static bool invariant(Pointer x) - { - return x->size==ranked_node_size(x->left())+ranked_node_size(x->right())+1; - } -#endif -}; - -} /* namespace multi_index::detail */ - -/* ranked_index specifiers */ - -template -struct ranked_unique -{ - typedef typename detail::ordered_index_args< - Arg1,Arg2,Arg3> index_args; - typedef typename index_args::tag_list_type::type tag_list_type; - typedef typename index_args::key_from_value_type key_from_value_type; - typedef typename index_args::compare_type compare_type; - - template - struct node_class - { - typedef detail::ordered_index_node type; - }; - - template - struct index_class - { - typedef detail::ordered_index< - key_from_value_type,compare_type, - SuperMeta,tag_list_type,detail::ordered_unique_tag, - detail::rank_policy> type; - }; -}; - -template -struct ranked_non_unique -{ - typedef detail::ordered_index_args< - Arg1,Arg2,Arg3> index_args; - typedef typename index_args::tag_list_type::type tag_list_type; - typedef typename index_args::key_from_value_type key_from_value_type; - typedef typename index_args::compare_type compare_type; - - template - struct node_class - { - typedef detail::ordered_index_node type; - }; - - template - struct index_class - { - typedef detail::ordered_index< - key_from_value_type,compare_type, - SuperMeta,tag_list_type,detail::ordered_non_unique_tag, - detail::rank_policy> type; - }; -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index_fwd.hpp deleted file mode 100644 index 380d3480736..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index_fwd.hpp +++ /dev/null @@ -1,35 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_RANKED_INDEX_FWD_HPP -#define BOOST_MULTI_INDEX_RANKED_INDEX_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include -#include - -namespace boost{ - -namespace multi_index{ - -/* ranked_index specifiers */ - -template -struct ranked_unique; - -template -struct ranked_non_unique; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/safe_mode_errors.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/safe_mode_errors.hpp deleted file mode 100644 index 1904706edec..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/safe_mode_errors.hpp +++ /dev/null @@ -1,48 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_SAFE_MODE_ERRORS_HPP -#define BOOST_MULTI_INDEX_SAFE_MODE_ERRORS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -namespace boost{ - -namespace multi_index{ - -namespace safe_mode{ - -/* Error codes for Boost.MultiIndex safe mode. These go in a separate - * header so that the user can include it when redefining - * BOOST_MULTI_INDEX_SAFE_MODE_ASSERT prior to the inclusion of - * any other header of Boost.MultiIndex. - */ - -enum error_code -{ - invalid_iterator=0, - not_dereferenceable_iterator, - not_incrementable_iterator, - not_decrementable_iterator, - not_owner, - not_same_owner, - invalid_range, - inside_range, - out_of_bounds, - same_container -}; - -} /* namespace multi_index::safe_mode */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index.hpp deleted file mode 100644 index 424eebc376d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index.hpp +++ /dev/null @@ -1,1062 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_SEQUENCED_INDEX_HPP -#define BOOST_MULTI_INDEX_SEQUENCED_INDEX_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) -#include -#endif - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) -#define BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF(x) \ - detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ - detail::make_obj_guard(x,&sequenced_index::check_invariant_); \ - BOOST_JOIN(check_invariant_,__LINE__).touch(); -#define BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT \ - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF(*this) -#else -#define BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF(x) -#define BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* sequenced_index adds a layer of sequenced indexing to a given Super */ - -template -class sequenced_index: - BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS SuperMeta::type - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - ,public safe_mode::safe_container< - sequenced_index > -#endif - -{ -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the - * lifetime of const references bound to temporaries --precisely what - * scopeguards are. - */ - -#pragma parse_mfunc_templ off -#endif - - typedef typename SuperMeta::type super; - -protected: - typedef sequenced_index_node< - typename super::node_type> node_type; - -private: - typedef typename node_type::impl_type node_impl_type; - -public: - /* types */ - - typedef typename node_type::value_type value_type; - typedef tuples::null_type ctor_args; - typedef typename super::final_allocator_type allocator_type; - typedef typename allocator_type::reference reference; - typedef typename allocator_type::const_reference const_reference; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_iterator< - bidir_node_iterator, - sequenced_index> iterator; -#else - typedef bidir_node_iterator iterator; -#endif - - typedef iterator const_iterator; - - typedef std::size_t size_type; - typedef std::ptrdiff_t difference_type; - typedef typename allocator_type::pointer pointer; - typedef typename allocator_type::const_pointer const_pointer; - typedef typename - boost::reverse_iterator reverse_iterator; - typedef typename - boost::reverse_iterator const_reverse_iterator; - typedef TagList tag_list; - -protected: - typedef typename super::final_node_type final_node_type; - typedef tuples::cons< - ctor_args, - typename super::ctor_args_list> ctor_args_list; - typedef typename mpl::push_front< - typename super::index_type_list, - sequenced_index>::type index_type_list; - typedef typename mpl::push_front< - typename super::iterator_type_list, - iterator>::type iterator_type_list; - typedef typename mpl::push_front< - typename super::const_iterator_type_list, - const_iterator>::type const_iterator_type_list; - typedef typename super::copy_map_type copy_map_type; - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - typedef typename super::index_saver_type index_saver_type; - typedef typename super::index_loader_type index_loader_type; -#endif - -private: -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_container< - sequenced_index> safe_super; -#endif - - typedef typename call_traits::param_type value_param_type; - - /* Needed to avoid commas in BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL - * expansion. - */ - - typedef std::pair emplace_return_type; - -public: - - /* construct/copy/destroy - * Default and copy ctors are in the protected section as indices are - * not supposed to be created on their own. No range ctor either. - */ - - sequenced_index& operator=( - const sequenced_index& x) - { - this->final()=x.final(); - return *this; - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - sequenced_index& operator=( - std::initializer_list list) - { - this->final()=list; - return *this; - } -#endif - - template - void assign(InputIterator first,InputIterator last) - { - assign_iter(first,last,mpl::not_ >()); - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - void assign(std::initializer_list list) - { - assign(list.begin(),list.end()); - } -#endif - - void assign(size_type n,value_param_type value) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - clear(); - for(size_type i=0;ifinal().get_allocator(); - } - - /* iterators */ - - iterator begin()BOOST_NOEXCEPT - {return make_iterator(node_type::from_impl(header()->next()));} - const_iterator begin()const BOOST_NOEXCEPT - {return make_iterator(node_type::from_impl(header()->next()));} - iterator - end()BOOST_NOEXCEPT{return make_iterator(header());} - const_iterator - end()const BOOST_NOEXCEPT{return make_iterator(header());} - reverse_iterator - rbegin()BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} - const_reverse_iterator - rbegin()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} - reverse_iterator - rend()BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} - const_reverse_iterator - rend()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} - const_iterator - cbegin()const BOOST_NOEXCEPT{return begin();} - const_iterator - cend()const BOOST_NOEXCEPT{return end();} - const_reverse_iterator - crbegin()const BOOST_NOEXCEPT{return rbegin();} - const_reverse_iterator - crend()const BOOST_NOEXCEPT{return rend();} - - iterator iterator_to(const value_type& x) - { - return make_iterator(node_from_value(&x)); - } - - const_iterator iterator_to(const value_type& x)const - { - return make_iterator(node_from_value(&x)); - } - - /* capacity */ - - bool empty()const BOOST_NOEXCEPT{return this->final_empty_();} - size_type size()const BOOST_NOEXCEPT{return this->final_size_();} - size_type max_size()const BOOST_NOEXCEPT{return this->final_max_size_();} - - void resize(size_type n) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - if(n>size()){ - for(size_type m=n-size();m--;) - this->final_emplace_(BOOST_MULTI_INDEX_NULL_PARAM_PACK); - } - else if(nsize())insert(end(),n-size(),x); - else if(n push_front(const value_type& x) - {return insert(begin(),x);} - std::pair push_front(BOOST_RV_REF(value_type) x) - {return insert(begin(),boost::move(x));} - void pop_front(){erase(begin());} - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( - emplace_return_type,emplace_back,emplace_back_impl) - - std::pair push_back(const value_type& x) - {return insert(end(),x);} - std::pair push_back(BOOST_RV_REF(value_type) x) - {return insert(end(),boost::move(x));} - void pop_back(){erase(--end());} - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( - emplace_return_type,emplace,emplace_impl,iterator,position) - - std::pair insert(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_(x); - if(p.second&&position.get_node()!=header()){ - relink(position.get_node(),p.first); - } - return std::pair(make_iterator(p.first),p.second); - } - - std::pair insert(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_rv_(x); - if(p.second&&position.get_node()!=header()){ - relink(position.get_node(),p.first); - } - return std::pair(make_iterator(p.first),p.second); - } - - void insert(iterator position,size_type n,value_param_type x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - for(size_type i=0;i - void insert(iterator position,InputIterator first,InputIterator last) - { - insert_iter(position,first,last,mpl::not_ >()); - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - void insert(iterator position,std::initializer_list list) - { - insert(position,list.begin(),list.end()); - } -#endif - - iterator erase(iterator position) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - this->final_erase_(static_cast(position++.get_node())); - return position; - } - - iterator erase(iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - while(first!=last){ - first=erase(first); - } - return first; - } - - bool replace(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - return this->final_replace_( - x,static_cast(position.get_node())); - } - - bool replace(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - return this->final_replace_rv_( - x,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod,Rollback back_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,back_,static_cast(position.get_node())); - } - - void swap(sequenced_index& x) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF(x); - this->final_swap_(x.final()); - } - - void clear()BOOST_NOEXCEPT - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - this->final_clear_(); - } - - /* list operations */ - - void splice(iterator position,sequenced_index& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_DIFFERENT_CONTAINER(*this,x); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - iterator first=x.begin(),last=x.end(); - while(first!=last){ - if(insert(position,*first).second)first=x.erase(first); - else ++first; - } - } - - void splice(iterator position,sequenced_index& x,iterator i) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(i,x); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - if(&x==this){ - if(position!=i)relink(position.get_node(),i.get_node()); - } - else{ - if(insert(position,*i).second){ - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer has a hard time with safe mode, and the following - * workaround is needed. Left it for all compilers as it does no - * harm. - */ - i.detach(); - x.erase(x.make_iterator(i.get_node())); -#else - x.erase(i); -#endif - - } - } - } - - void splice( - iterator position,sequenced_index& x, - iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,x); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,x); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - if(&x==this){ - BOOST_MULTI_INDEX_CHECK_OUTSIDE_RANGE(position,first,last); - if(position!=last)relink( - position.get_node(),first.get_node(),last.get_node()); - } - else{ - while(first!=last){ - if(insert(position,*first).second)first=x.erase(first); - else ++first; - } - } - } - - void remove(value_param_type value) - { - sequenced_index_remove( - *this, - ::boost::bind(std::equal_to(),::boost::arg<1>(),value)); - } - - template - void remove_if(Predicate pred) - { - sequenced_index_remove(*this,pred); - } - - void unique() - { - sequenced_index_unique(*this,std::equal_to()); - } - - template - void unique(BinaryPredicate binary_pred) - { - sequenced_index_unique(*this,binary_pred); - } - - void merge(sequenced_index& x) - { - sequenced_index_merge(*this,x,std::less()); - } - - template - void merge(sequenced_index& x,Compare comp) - { - sequenced_index_merge(*this,x,comp); - } - - void sort() - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - sequenced_index_sort(header(),std::less()); - } - - template - void sort(Compare comp) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - sequenced_index_sort(header(),comp); - } - - void reverse()BOOST_NOEXCEPT - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - node_impl_type::reverse(header()->impl()); - } - - /* rearrange operations */ - - void relocate(iterator position,iterator i) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(i,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - if(position!=i)relink(position.get_node(),i.get_node()); - } - - void relocate(iterator position,iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_CHECK_OUTSIDE_RANGE(position,first,last); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - if(position!=last)relink( - position.get_node(),first.get_node(),last.get_node()); - } - - template - void rearrange(InputIterator first) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - node_type* pos=header(); - for(size_type s=size();s--;){ - const value_type& v=*first++; - relink(pos,node_from_value(&v)); - } - } - -BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: - sequenced_index(const ctor_args_list& args_list,const allocator_type& al): - super(args_list.get_tail(),al) - { - empty_initialize(); - } - - sequenced_index(const sequenced_index& x): - super(x) - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - ,safe_super() -#endif - - { - /* the actual copying takes place in subsequent call to copy_() */ - } - - sequenced_index( - const sequenced_index& x,do_not_copy_elements_tag): - super(x,do_not_copy_elements_tag()) - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - ,safe_super() -#endif - - { - empty_initialize(); - } - - ~sequenced_index() - { - /* the container is guaranteed to be empty by now */ - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - iterator make_iterator(node_type* node){return iterator(node,this);} - const_iterator make_iterator(node_type* node)const - {return const_iterator(node,const_cast(this));} -#else - iterator make_iterator(node_type* node){return iterator(node);} - const_iterator make_iterator(node_type* node)const - {return const_iterator(node);} -#endif - - void copy_( - const sequenced_index& x,const copy_map_type& map) - { - node_type* org=x.header(); - node_type* cpy=header(); - do{ - node_type* next_org=node_type::from_impl(org->next()); - node_type* next_cpy=map.find(static_cast(next_org)); - cpy->next()=next_cpy->impl(); - next_cpy->prior()=cpy->impl(); - org=next_org; - cpy=next_cpy; - }while(org!=x.header()); - - super::copy_(x,map); - } - - template - final_node_type* insert_( - value_param_type v,final_node_type*& x,Variant variant) - { - final_node_type* res=super::insert_(v,x,variant); - if(res==x)link(static_cast(x)); - return res; - } - - template - final_node_type* insert_( - value_param_type v,node_type* position,final_node_type*& x,Variant variant) - { - final_node_type* res=super::insert_(v,position,x,variant); - if(res==x)link(static_cast(x)); - return res; - } - - void erase_(node_type* x) - { - unlink(x); - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - } - - void delete_all_nodes_() - { - for(node_type* x=node_type::from_impl(header()->next());x!=header();){ - node_type* y=node_type::from_impl(x->next()); - this->final_delete_node_(static_cast(x)); - x=y; - } - } - - void clear_() - { - super::clear_(); - empty_initialize(); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::detach_dereferenceable_iterators(); -#endif - } - - void swap_(sequenced_index& x) - { -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_(x); - } - - void swap_elements_(sequenced_index& x) - { -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_elements_(x); - } - - template - bool replace_(value_param_type v,node_type* x,Variant variant) - { - return super::replace_(v,x,variant); - } - - bool modify_(node_type* x) - { - BOOST_TRY{ - if(!super::modify_(x)){ - unlink(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - return false; - } - else return true; - } - BOOST_CATCH(...){ - unlink(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - bool modify_rollback_(node_type* x) - { - return super::modify_rollback_(x); - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* serialization */ - - template - void save_( - Archive& ar,const unsigned int version,const index_saver_type& sm)const - { - sm.save(begin(),end(),ar,version); - super::save_(ar,version,sm); - } - - template - void load_( - Archive& ar,const unsigned int version,const index_loader_type& lm) - { - lm.load( - ::boost::bind( - &sequenced_index::rearranger,this,::boost::arg<1>(),::boost::arg<2>()), - ar,version); - super::load_(ar,version,lm); - } -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - bool invariant_()const - { - if(size()==0||begin()==end()){ - if(size()!=0||begin()!=end()|| - header()->next()!=header()->impl()|| - header()->prior()!=header()->impl())return false; - } - else{ - size_type s=0; - for(const_iterator it=begin(),it_end=end();it!=it_end;++it,++s){ - if(it.get_node()->next()->prior()!=it.get_node()->impl())return false; - if(it.get_node()->prior()->next()!=it.get_node()->impl())return false; - } - if(s!=size())return false; - } - - return super::invariant_(); - } - - /* This forwarding function eases things for the boost::mem_fn construct - * in BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT. Actually, - * final_check_invariant is already an inherited member function of index. - */ - void check_invariant_()const{this->final_check_invariant_();} -#endif - -private: - node_type* header()const{return this->final_header();} - - void empty_initialize() - { - header()->prior()=header()->next()=header()->impl(); - } - - void link(node_type* x) - { - node_impl_type::link(x->impl(),header()->impl()); - }; - - static void unlink(node_type* x) - { - node_impl_type::unlink(x->impl()); - } - - static void relink(node_type* position,node_type* x) - { - node_impl_type::relink(position->impl(),x->impl()); - } - - static void relink(node_type* position,node_type* first,node_type* last) - { - node_impl_type::relink( - position->impl(),first->impl(),last->impl()); - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - void rearranger(node_type* position,node_type *x) - { - if(!position)position=header(); - node_type::increment(position); - if(position!=x)relink(position,x); - } -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - void detach_iterators(node_type* x) - { - iterator it=make_iterator(x); - safe_mode::detach_equivalent_iterators(it); - } -#endif - - template - void assign_iter(InputIterator first,InputIterator last,mpl::true_) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - clear(); - for(;first!=last;++first)this->final_insert_ref_(*first); - } - - void assign_iter(size_type n,value_param_type value,mpl::false_) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - clear(); - for(size_type i=0;i - void insert_iter( - iterator position,InputIterator first,InputIterator last,mpl::true_) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - for(;first!=last;++first){ - std::pair p= - this->final_insert_ref_(*first); - if(p.second&&position.get_node()!=header()){ - relink(position.get_node(),p.first); - } - } - } - - void insert_iter( - iterator position,size_type n,value_param_type x,mpl::false_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - for(size_type i=0;i - std::pair emplace_front_impl( - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - return emplace_impl(begin(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - } - - template - std::pair emplace_back_impl( - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - return emplace_impl(end(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - } - - template - std::pair emplace_impl( - iterator position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - std::pair p= - this->final_emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - if(p.second&&position.get_node()!=header()){ - relink(position.get_node(),p.first); - } - return std::pair(make_iterator(p.first),p.second); - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -#pragma parse_mfunc_templ reset -#endif -}; - -/* comparison */ - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator==( - const sequenced_index& x, - const sequenced_index& y) -{ - return x.size()==y.size()&&std::equal(x.begin(),x.end(),y.begin()); -} - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator<( - const sequenced_index& x, - const sequenced_index& y) -{ - return std::lexicographical_compare(x.begin(),x.end(),y.begin(),y.end()); -} - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator!=( - const sequenced_index& x, - const sequenced_index& y) -{ - return !(x==y); -} - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator>( - const sequenced_index& x, - const sequenced_index& y) -{ - return y -bool operator>=( - const sequenced_index& x, - const sequenced_index& y) -{ - return !(x -bool operator<=( - const sequenced_index& x, - const sequenced_index& y) -{ - return !(x>y); -} - -/* specialized algorithms */ - -template -void swap( - sequenced_index& x, - sequenced_index& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -/* sequenced index specifier */ - -template -struct sequenced -{ - BOOST_STATIC_ASSERT(detail::is_tag::value); - - template - struct node_class - { - typedef detail::sequenced_index_node type; - }; - - template - struct index_class - { - typedef detail::sequenced_index type; - }; -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -/* Boost.Foreach compatibility */ - -template -inline boost::mpl::true_* boost_foreach_is_noncopyable( - boost::multi_index::detail::sequenced_index*&, - boost_foreach_argument_dependent_lookup_hack) -{ - return 0; -} - -#undef BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT -#undef BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index_fwd.hpp deleted file mode 100644 index a019f2a6d2f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index_fwd.hpp +++ /dev/null @@ -1,91 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_SEQUENCED_INDEX_FWD_HPP -#define BOOST_MULTI_INDEX_SEQUENCED_INDEX_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -class sequenced_index; - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator==( - const sequenced_index& x, - const sequenced_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator<( - const sequenced_index& x, - const sequenced_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator!=( - const sequenced_index& x, - const sequenced_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator>( - const sequenced_index& x, - const sequenced_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator>=( - const sequenced_index& x, - const sequenced_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator<=( - const sequenced_index& x, - const sequenced_index& y); - -template -void swap( - sequenced_index& x, - sequenced_index& y); - -} /* namespace multi_index::detail */ - -/* index specifiers */ - -template > -struct sequenced; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/tag.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/tag.hpp deleted file mode 100644 index ce51f8241ee..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/tag.hpp +++ /dev/null @@ -1,88 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_TAG_HPP -#define BOOST_MULTI_INDEX_TAG_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -/* A wrapper of mpl::vector used to hide MPL from the user. - * tag contains types used as tag names for indices in get() functions. - */ - -/* This user_definable macro limits the number of elements of a tag; - * useful for shortening resulting symbol names (MSVC++ 6.0, for instance, - * has problems coping with very long symbol names.) - */ - -#if !defined(BOOST_MULTI_INDEX_LIMIT_TAG_SIZE) -#define BOOST_MULTI_INDEX_LIMIT_TAG_SIZE BOOST_MPL_LIMIT_VECTOR_SIZE -#endif - -#if BOOST_MULTI_INDEX_LIMIT_TAG_SIZE -struct is_tag -{ - BOOST_STATIC_CONSTANT(bool,value=(is_base_and_derived::value)); -}; - -} /* namespace multi_index::detail */ - -template< - BOOST_PP_ENUM_BINARY_PARAMS( - BOOST_MULTI_INDEX_TAG_SIZE, - typename T, - =mpl::na BOOST_PP_INTERCEPT) -> -struct tag:private detail::tag_marker -{ - /* The mpl::transform pass produces shorter symbols (without - * trailing mpl::na's.) - */ - - typedef typename mpl::transform< - mpl::vector, - mpl::identity - >::type type; - - BOOST_STATIC_ASSERT(detail::no_duplicate_tags::value); -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#undef BOOST_MULTI_INDEX_TAG_SIZE - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index_container.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index_container.hpp deleted file mode 100644 index 9993a8dfa10..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index_container.hpp +++ /dev/null @@ -1,1362 +0,0 @@ -/* Multiply indexed container. - * - * Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_HPP -#define BOOST_MULTI_INDEX_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) -#include -#endif - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#include -#include -#include -#include -#include -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) -#include -#define BOOST_MULTI_INDEX_CHECK_INVARIANT_OF(x) \ - detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ - detail::make_obj_guard(x,&multi_index_container::check_invariant_); \ - BOOST_JOIN(check_invariant_,__LINE__).touch(); -#define BOOST_MULTI_INDEX_CHECK_INVARIANT \ - BOOST_MULTI_INDEX_CHECK_INVARIANT_OF(*this) -#else -#define BOOST_MULTI_INDEX_CHECK_INVARIANT_OF(x) -#define BOOST_MULTI_INDEX_CHECK_INVARIANT -#endif - -namespace boost{ - -namespace multi_index{ - -#if BOOST_WORKAROUND(BOOST_MSVC,BOOST_TESTED_AT(1500)) -#pragma warning(push) -#pragma warning(disable:4522) /* spurious warning on multiple operator=()'s */ -#endif - -template -class multi_index_container: - private ::boost::base_from_member< - typename boost::detail::allocator::rebind_to< - Allocator, - typename detail::multi_index_node_type< - Value,IndexSpecifierList,Allocator>::type - >::type>, - BOOST_MULTI_INDEX_PRIVATE_IF_MEMBER_TEMPLATE_FRIENDS detail::header_holder< - typename boost::detail::allocator::rebind_to< - Allocator, - typename detail::multi_index_node_type< - Value,IndexSpecifierList,Allocator>::type - >::type::pointer, - multi_index_container >, - public detail::multi_index_base_type< - Value,IndexSpecifierList,Allocator>::type -{ -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the - * lifetime of const references bound to temporaries --precisely what - * scopeguards are. - */ - -#pragma parse_mfunc_templ off -#endif - -private: - BOOST_COPYABLE_AND_MOVABLE(multi_index_container) - -#if !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) - template friend class detail::index_base; - template friend struct detail::header_holder; - template friend struct detail::converter; -#endif - - typedef typename detail::multi_index_base_type< - Value,IndexSpecifierList,Allocator>::type super; - typedef typename - boost::detail::allocator::rebind_to< - Allocator, - typename super::node_type - >::type node_allocator; - typedef ::boost::base_from_member< - node_allocator> bfm_allocator; - typedef detail::header_holder< - typename node_allocator::pointer, - multi_index_container> bfm_header; - - -public: - /* All types are inherited from super, a few are explicitly - * brought forward here to save us some typename's. - */ - - typedef typename super::ctor_args_list ctor_args_list; - typedef IndexSpecifierList index_specifier_type_list; - - typedef typename super::index_type_list index_type_list; - - typedef typename super::iterator_type_list iterator_type_list; - typedef typename super::const_iterator_type_list const_iterator_type_list; - typedef typename super::value_type value_type; - typedef typename super::final_allocator_type allocator_type; - typedef typename super::iterator iterator; - typedef typename super::const_iterator const_iterator; - - BOOST_STATIC_ASSERT( - detail::no_duplicate_tags_in_index_list::value); - - /* global project() needs to see this publicly */ - - typedef typename super::node_type node_type; - - /* construct/copy/destroy */ - - explicit multi_index_container( - -#if BOOST_WORKAROUND(__IBMCPP__,<=600) - /* VisualAge seems to have an ETI issue with the default values - * for arguments args_list and al. - */ - - const ctor_args_list& args_list= - typename mpl::identity::type:: - ctor_args_list(), - const allocator_type& al= - typename mpl::identity::type:: - allocator_type()): -#else - const ctor_args_list& args_list=ctor_args_list(), - const allocator_type& al=allocator_type()): -#endif - - bfm_allocator(al), - super(args_list,bfm_allocator::member), - node_count(0) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - } - - explicit multi_index_container(const allocator_type& al): - bfm_allocator(al), - super(ctor_args_list(),bfm_allocator::member), - node_count(0) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - } - - template - multi_index_container( - InputIterator first,InputIterator last, - -#if BOOST_WORKAROUND(__IBMCPP__,<=600) - /* VisualAge seems to have an ETI issue with the default values - * for arguments args_list and al. - */ - - const ctor_args_list& args_list= - typename mpl::identity::type:: - ctor_args_list(), - const allocator_type& al= - typename mpl::identity::type:: - allocator_type()): -#else - const ctor_args_list& args_list=ctor_args_list(), - const allocator_type& al=allocator_type()): -#endif - - bfm_allocator(al), - super(args_list,bfm_allocator::member), - node_count(0) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - BOOST_TRY{ - iterator hint=super::end(); - for(;first!=last;++first){ - hint=super::make_iterator( - insert_ref_(*first,hint.get_node()).first); - ++hint; - } - } - BOOST_CATCH(...){ - clear_(); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - multi_index_container( - std::initializer_list list, - const ctor_args_list& args_list=ctor_args_list(), - const allocator_type& al=allocator_type()): - bfm_allocator(al), - super(args_list,bfm_allocator::member), - node_count(0) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - BOOST_TRY{ - typedef const Value* init_iterator; - - iterator hint=super::end(); - for(init_iterator first=list.begin(),last=list.end(); - first!=last;++first){ - hint=super::make_iterator(insert_(*first,hint.get_node()).first); - ++hint; - } - } - BOOST_CATCH(...){ - clear_(); - BOOST_RETHROW; - } - BOOST_CATCH_END - } -#endif - - multi_index_container( - const multi_index_container& x): - bfm_allocator(x.bfm_allocator::member), - bfm_header(), - super(x), - node_count(0) - { - copy_map_type map(bfm_allocator::member,x.size(),x.header(),header()); - for(const_iterator it=x.begin(),it_end=x.end();it!=it_end;++it){ - map.clone(it.get_node()); - } - super::copy_(x,map); - map.release(); - node_count=x.size(); - - /* Not until this point are the indices required to be consistent, - * hence the position of the invariant checker. - */ - - BOOST_MULTI_INDEX_CHECK_INVARIANT; - } - - multi_index_container(BOOST_RV_REF(multi_index_container) x): - bfm_allocator(x.bfm_allocator::member), - bfm_header(), - super(x,detail::do_not_copy_elements_tag()), - node_count(0) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - BOOST_MULTI_INDEX_CHECK_INVARIANT_OF(x); - swap_elements_(x); - } - - ~multi_index_container() - { - delete_all_nodes_(); - } - -#if defined(BOOST_NO_CXX11_RVALUE_REFERENCES) - /* As per http://www.boost.org/doc/html/move/emulation_limitations.html - * #move.emulation_limitations.assignment_operator - */ - - multi_index_container& operator=( - const multi_index_container& x) - { - multi_index_container y(x); - this->swap(y); - return *this; - } -#endif - - multi_index_container& operator=( - BOOST_COPY_ASSIGN_REF(multi_index_container) x) - { - multi_index_container y(x); - this->swap(y); - return *this; - } - - multi_index_container& operator=( - BOOST_RV_REF(multi_index_container) x) - { - this->swap(x); - return *this; - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - multi_index_container& operator=( - std::initializer_list list) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - typedef const Value* init_iterator; - - multi_index_container x(*this,detail::do_not_copy_elements_tag()); - iterator hint=x.end(); - for(init_iterator first=list.begin(),last=list.end(); - first!=last;++first){ - hint=x.make_iterator(x.insert_(*first,hint.get_node()).first); - ++hint; - } - x.swap_elements_(*this); - return*this; - } -#endif - - allocator_type get_allocator()const BOOST_NOEXCEPT - { - return allocator_type(bfm_allocator::member); - } - - /* retrieval of indices by number */ - -#if !defined(BOOST_NO_MEMBER_TEMPLATES) - template - struct nth_index - { - BOOST_STATIC_ASSERT(N>=0&&N::type::value); - typedef typename mpl::at_c::type type; - }; - - template - typename nth_index::type& get()BOOST_NOEXCEPT - { - BOOST_STATIC_ASSERT(N>=0&&N::type::value); - return *this; - } - - template - const typename nth_index::type& get()const BOOST_NOEXCEPT - { - BOOST_STATIC_ASSERT(N>=0&&N::type::value); - return *this; - } -#endif - - /* retrieval of indices by tag */ - -#if !defined(BOOST_NO_MEMBER_TEMPLATES) - template - struct index - { - typedef typename mpl::find_if< - index_type_list, - detail::has_tag - >::type iter; - - BOOST_STATIC_CONSTANT( - bool,index_found=!(is_same::type >::value)); - BOOST_STATIC_ASSERT(index_found); - - typedef typename mpl::deref::type type; - }; - - template - typename index::type& get()BOOST_NOEXCEPT - { - return *this; - } - - template - const typename index::type& get()const BOOST_NOEXCEPT - { - return *this; - } -#endif - - /* projection of iterators by number */ - -#if !defined(BOOST_NO_MEMBER_TEMPLATES) - template - struct nth_index_iterator - { - typedef typename nth_index::type::iterator type; - }; - - template - struct nth_index_const_iterator - { - typedef typename nth_index::type::const_iterator type; - }; - - template - typename nth_index_iterator::type project(IteratorType it) - { - typedef typename nth_index::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* fails in Sun C++ 5.7 */ - BOOST_STATIC_ASSERT( - (mpl::contains::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - BOOST_MULTI_INDEX_CHECK_IS_OWNER( - it,static_cast(*this)); - - return index_type::make_iterator(static_cast(it.get_node())); - } - - template - typename nth_index_const_iterator::type project(IteratorType it)const - { - typedef typename nth_index::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* fails in Sun C++ 5.7 */ - BOOST_STATIC_ASSERT(( - mpl::contains::value|| - mpl::contains::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - BOOST_MULTI_INDEX_CHECK_IS_OWNER( - it,static_cast(*this)); - return index_type::make_iterator(static_cast(it.get_node())); - } -#endif - - /* projection of iterators by tag */ - -#if !defined(BOOST_NO_MEMBER_TEMPLATES) - template - struct index_iterator - { - typedef typename index::type::iterator type; - }; - - template - struct index_const_iterator - { - typedef typename index::type::const_iterator type; - }; - - template - typename index_iterator::type project(IteratorType it) - { - typedef typename index::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* fails in Sun C++ 5.7 */ - BOOST_STATIC_ASSERT( - (mpl::contains::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - BOOST_MULTI_INDEX_CHECK_IS_OWNER( - it,static_cast(*this)); - return index_type::make_iterator(static_cast(it.get_node())); - } - - template - typename index_const_iterator::type project(IteratorType it)const - { - typedef typename index::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* fails in Sun C++ 5.7 */ - BOOST_STATIC_ASSERT(( - mpl::contains::value|| - mpl::contains::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - BOOST_MULTI_INDEX_CHECK_IS_OWNER( - it,static_cast(*this)); - return index_type::make_iterator(static_cast(it.get_node())); - } -#endif - -BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: - typedef typename super::copy_map_type copy_map_type; - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - multi_index_container( - const multi_index_container& x, - detail::do_not_copy_elements_tag): - bfm_allocator(x.bfm_allocator::member), - bfm_header(), - super(x,detail::do_not_copy_elements_tag()), - node_count(0) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - } -#endif - - node_type* header()const - { - return &*bfm_header::member; - } - - node_type* allocate_node() - { - return &*bfm_allocator::member.allocate(1); - } - - void deallocate_node(node_type* x) - { - typedef typename node_allocator::pointer node_pointer; - bfm_allocator::member.deallocate(static_cast(x),1); - } - - bool empty_()const - { - return node_count==0; - } - - std::size_t size_()const - { - return node_count; - } - - std::size_t max_size_()const - { - return static_cast(-1); - } - - template - std::pair insert_(const Value& v,Variant variant) - { - node_type* x=0; - node_type* res=super::insert_(v,x,variant); - if(res==x){ - ++node_count; - return std::pair(res,true); - } - else{ - return std::pair(res,false); - } - } - - std::pair insert_(const Value& v) - { - return insert_(v,detail::lvalue_tag()); - } - - std::pair insert_rv_(const Value& v) - { - return insert_(v,detail::rvalue_tag()); - } - - template - std::pair insert_ref_(T& t) - { - node_type* x=allocate_node(); - BOOST_TRY{ - new(&x->value()) value_type(t); - BOOST_TRY{ - node_type* res=super::insert_(x->value(),x,detail::emplaced_tag()); - if(res==x){ - ++node_count; - return std::pair(res,true); - } - else{ - boost::detail::allocator::destroy(&x->value()); - deallocate_node(x); - return std::pair(res,false); - } - } - BOOST_CATCH(...){ - boost::detail::allocator::destroy(&x->value()); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - BOOST_CATCH(...){ - deallocate_node(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - std::pair insert_ref_(const value_type& x) - { - return insert_(x); - } - - std::pair insert_ref_(value_type& x) - { - return insert_(x); - } - - template - std::pair emplace_( - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - node_type* x=allocate_node(); - BOOST_TRY{ - detail::vartempl_placement_new( - &x->value(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - BOOST_TRY{ - node_type* res=super::insert_(x->value(),x,detail::emplaced_tag()); - if(res==x){ - ++node_count; - return std::pair(res,true); - } - else{ - boost::detail::allocator::destroy(&x->value()); - deallocate_node(x); - return std::pair(res,false); - } - } - BOOST_CATCH(...){ - boost::detail::allocator::destroy(&x->value()); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - BOOST_CATCH(...){ - deallocate_node(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - template - std::pair insert_( - const Value& v,node_type* position,Variant variant) - { - node_type* x=0; - node_type* res=super::insert_(v,position,x,variant); - if(res==x){ - ++node_count; - return std::pair(res,true); - } - else{ - return std::pair(res,false); - } - } - - std::pair insert_(const Value& v,node_type* position) - { - return insert_(v,position,detail::lvalue_tag()); - } - - std::pair insert_rv_(const Value& v,node_type* position) - { - return insert_(v,position,detail::rvalue_tag()); - } - - template - std::pair insert_ref_( - T& t,node_type* position) - { - node_type* x=allocate_node(); - BOOST_TRY{ - new(&x->value()) value_type(t); - BOOST_TRY{ - node_type* res=super::insert_( - x->value(),position,x,detail::emplaced_tag()); - if(res==x){ - ++node_count; - return std::pair(res,true); - } - else{ - boost::detail::allocator::destroy(&x->value()); - deallocate_node(x); - return std::pair(res,false); - } - } - BOOST_CATCH(...){ - boost::detail::allocator::destroy(&x->value()); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - BOOST_CATCH(...){ - deallocate_node(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - std::pair insert_ref_( - const value_type& x,node_type* position) - { - return insert_(x,position); - } - - std::pair insert_ref_( - value_type& x,node_type* position) - { - return insert_(x,position); - } - - template - std::pair emplace_hint_( - node_type* position, - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - node_type* x=allocate_node(); - BOOST_TRY{ - detail::vartempl_placement_new( - &x->value(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - BOOST_TRY{ - node_type* res=super::insert_( - x->value(),position,x,detail::emplaced_tag()); - if(res==x){ - ++node_count; - return std::pair(res,true); - } - else{ - boost::detail::allocator::destroy(&x->value()); - deallocate_node(x); - return std::pair(res,false); - } - } - BOOST_CATCH(...){ - boost::detail::allocator::destroy(&x->value()); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - BOOST_CATCH(...){ - deallocate_node(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - void erase_(node_type* x) - { - --node_count; - super::erase_(x); - deallocate_node(x); - } - - void delete_node_(node_type* x) - { - super::delete_node_(x); - deallocate_node(x); - } - - void delete_all_nodes_() - { - super::delete_all_nodes_(); - } - - void clear_() - { - delete_all_nodes_(); - super::clear_(); - node_count=0; - } - - void swap_(multi_index_container& x) - { - if(bfm_allocator::member!=x.bfm_allocator::member){ - detail::adl_swap(bfm_allocator::member,x.bfm_allocator::member); - } - std::swap(bfm_header::member,x.bfm_header::member); - super::swap_(x); - std::swap(node_count,x.node_count); - } - - void swap_elements_( - multi_index_container& x) - { - std::swap(bfm_header::member,x.bfm_header::member); - super::swap_elements_(x); - std::swap(node_count,x.node_count); - } - - bool replace_(const Value& k,node_type* x) - { - return super::replace_(k,x,detail::lvalue_tag()); - } - - bool replace_rv_(const Value& k,node_type* x) - { - return super::replace_(k,x,detail::rvalue_tag()); - } - - template - bool modify_(Modifier& mod,node_type* x) - { - mod(const_cast(x->value())); - - BOOST_TRY{ - if(!super::modify_(x)){ - deallocate_node(x); - --node_count; - return false; - } - else return true; - } - BOOST_CATCH(...){ - deallocate_node(x); - --node_count; - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - template - bool modify_(Modifier& mod,Rollback& back_,node_type* x) - { - mod(const_cast(x->value())); - - bool b; - BOOST_TRY{ - b=super::modify_rollback_(x); - } - BOOST_CATCH(...){ - BOOST_TRY{ - back_(const_cast(x->value())); - BOOST_RETHROW; - } - BOOST_CATCH(...){ - this->erase_(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - BOOST_CATCH_END - - BOOST_TRY{ - if(!b){ - back_(const_cast(x->value())); - return false; - } - else return true; - } - BOOST_CATCH(...){ - this->erase_(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* serialization */ - - friend class boost::serialization::access; - - BOOST_SERIALIZATION_SPLIT_MEMBER() - - typedef typename super::index_saver_type index_saver_type; - typedef typename super::index_loader_type index_loader_type; - - template - void save(Archive& ar,const unsigned int version)const - { - const serialization::collection_size_type s(size_()); - const detail::serialization_version value_version; - ar< - void load(Archive& ar,const unsigned int version) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - - clear_(); - serialization::collection_size_type s; - detail::serialization_version value_version; - if(version<1){ - std::size_t sz; - ar>>serialization::make_nvp("count",sz); - s=static_cast(sz); - } - else{ - ar>>serialization::make_nvp("count",s); - } - if(version<2){ - value_version=0; - } - else{ - ar>>serialization::make_nvp("value_version",value_version); - } - - index_loader_type lm(bfm_allocator::member,s); - - for(std::size_t n=0;n value("item",ar,value_version); - std::pair p=insert_( - value.get(),super::end().get_node()); - if(!p.second)throw_exception( - archive::archive_exception( - archive::archive_exception::other_exception)); - ar.reset_object_address(&p.first->value(),&value.get()); - lm.add(p.first,ar,version); - } - lm.add_track(header(),ar,version); - - super::load_(ar,version,lm); - } -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - bool invariant_()const - { - return super::invariant_(); - } - - void check_invariant_()const - { - BOOST_MULTI_INDEX_INVARIANT_ASSERT(invariant_()); - } -#endif - -private: - std::size_t node_count; - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -#pragma parse_mfunc_templ reset -#endif -}; - -#if BOOST_WORKAROUND(BOOST_MSVC,BOOST_TESTED_AT(1500)) -#pragma warning(pop) /* C4522 */ -#endif - -/* retrieval of indices by number */ - -template -struct nth_index -{ - BOOST_STATIC_CONSTANT( - int, - M=mpl::size::type::value); - BOOST_STATIC_ASSERT(N>=0&&N::type type; -}; - -template -typename nth_index< - multi_index_container,N>::type& -get( - multi_index_container& m)BOOST_NOEXCEPT -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename nth_index< - multi_index_container< - Value,IndexSpecifierList,Allocator>, - N - >::type index_type; - - BOOST_STATIC_ASSERT(N>=0&& - N< - mpl::size< - BOOST_DEDUCED_TYPENAME multi_index_type::index_type_list - >::type::value); - - return detail::converter::index(m); -} - -template -const typename nth_index< - multi_index_container,N>::type& -get( - const multi_index_container& m -)BOOST_NOEXCEPT -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename nth_index< - multi_index_container< - Value,IndexSpecifierList,Allocator>, - N - >::type index_type; - - BOOST_STATIC_ASSERT(N>=0&& - N< - mpl::size< - BOOST_DEDUCED_TYPENAME multi_index_type::index_type_list - >::type::value); - - return detail::converter::index(m); -} - -/* retrieval of indices by tag */ - -template -struct index -{ - typedef typename MultiIndexContainer::index_type_list index_type_list; - - typedef typename mpl::find_if< - index_type_list, - detail::has_tag - >::type iter; - - BOOST_STATIC_CONSTANT( - bool,index_found=!(is_same::type >::value)); - BOOST_STATIC_ASSERT(index_found); - - typedef typename mpl::deref::type type; -}; - -template< - typename Tag,typename Value,typename IndexSpecifierList,typename Allocator -> -typename ::boost::multi_index::index< - multi_index_container,Tag>::type& -get( - multi_index_container& m)BOOST_NOEXCEPT -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename ::boost::multi_index::index< - multi_index_container< - Value,IndexSpecifierList,Allocator>, - Tag - >::type index_type; - - return detail::converter::index(m); -} - -template< - typename Tag,typename Value,typename IndexSpecifierList,typename Allocator -> -const typename ::boost::multi_index::index< - multi_index_container,Tag>::type& -get( - const multi_index_container& m -)BOOST_NOEXCEPT -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename ::boost::multi_index::index< - multi_index_container< - Value,IndexSpecifierList,Allocator>, - Tag - >::type index_type; - - return detail::converter::index(m); -} - -/* projection of iterators by number */ - -template -struct nth_index_iterator -{ - typedef typename nth_index::type::iterator type; -}; - -template -struct nth_index_const_iterator -{ - typedef typename nth_index::type::const_iterator type; -}; - -template< - int N,typename IteratorType, - typename Value,typename IndexSpecifierList,typename Allocator> -typename nth_index_iterator< - multi_index_container,N>::type -project( - multi_index_container& m, - IteratorType it) -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename nth_index::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* Sun C++ 5.7 fails */ - BOOST_STATIC_ASSERT(( - mpl::contains< - BOOST_DEDUCED_TYPENAME multi_index_type::iterator_type_list, - IteratorType>::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef detail::converter< - multi_index_type, - BOOST_DEDUCED_TYPENAME IteratorType::container_type> converter; - BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,converter::index(m)); -#endif - - return detail::converter::iterator( - m,static_cast(it.get_node())); -} - -template< - int N,typename IteratorType, - typename Value,typename IndexSpecifierList,typename Allocator> -typename nth_index_const_iterator< - multi_index_container,N>::type -project( - const multi_index_container& m, - IteratorType it) -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename nth_index::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* Sun C++ 5.7 fails */ - BOOST_STATIC_ASSERT(( - mpl::contains< - BOOST_DEDUCED_TYPENAME multi_index_type::iterator_type_list, - IteratorType>::value|| - mpl::contains< - BOOST_DEDUCED_TYPENAME multi_index_type::const_iterator_type_list, - IteratorType>::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef detail::converter< - multi_index_type, - BOOST_DEDUCED_TYPENAME IteratorType::container_type> converter; - BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,converter::index(m)); -#endif - - return detail::converter::const_iterator( - m,static_cast(it.get_node())); -} - -/* projection of iterators by tag */ - -template -struct index_iterator -{ - typedef typename ::boost::multi_index::index< - MultiIndexContainer,Tag>::type::iterator type; -}; - -template -struct index_const_iterator -{ - typedef typename ::boost::multi_index::index< - MultiIndexContainer,Tag>::type::const_iterator type; -}; - -template< - typename Tag,typename IteratorType, - typename Value,typename IndexSpecifierList,typename Allocator> -typename index_iterator< - multi_index_container,Tag>::type -project( - multi_index_container& m, - IteratorType it) -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename ::boost::multi_index::index< - multi_index_type,Tag>::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* Sun C++ 5.7 fails */ - BOOST_STATIC_ASSERT(( - mpl::contains< - BOOST_DEDUCED_TYPENAME multi_index_type::iterator_type_list, - IteratorType>::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef detail::converter< - multi_index_type, - BOOST_DEDUCED_TYPENAME IteratorType::container_type> converter; - BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,converter::index(m)); -#endif - - return detail::converter::iterator( - m,static_cast(it.get_node())); -} - -template< - typename Tag,typename IteratorType, - typename Value,typename IndexSpecifierList,typename Allocator> -typename index_const_iterator< - multi_index_container,Tag>::type -project( - const multi_index_container& m, - IteratorType it) -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename ::boost::multi_index::index< - multi_index_type,Tag>::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* Sun C++ 5.7 fails */ - BOOST_STATIC_ASSERT(( - mpl::contains< - BOOST_DEDUCED_TYPENAME multi_index_type::iterator_type_list, - IteratorType>::value|| - mpl::contains< - BOOST_DEDUCED_TYPENAME multi_index_type::const_iterator_type_list, - IteratorType>::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef detail::converter< - multi_index_type, - BOOST_DEDUCED_TYPENAME IteratorType::container_type> converter; - BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,converter::index(m)); -#endif - - return detail::converter::const_iterator( - m,static_cast(it.get_node())); -} - -/* Comparison. Simple forward to first index. */ - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator==( - const multi_index_container& x, - const multi_index_container& y) -{ - return get<0>(x)==get<0>(y); -} - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator<( - const multi_index_container& x, - const multi_index_container& y) -{ - return get<0>(x)(y); -} - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator!=( - const multi_index_container& x, - const multi_index_container& y) -{ - return get<0>(x)!=get<0>(y); -} - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator>( - const multi_index_container& x, - const multi_index_container& y) -{ - return get<0>(x)>get<0>(y); -} - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator>=( - const multi_index_container& x, - const multi_index_container& y) -{ - return get<0>(x)>=get<0>(y); -} - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator<=( - const multi_index_container& x, - const multi_index_container& y) -{ - return get<0>(x)<=get<0>(y); -} - -/* specialized algorithms */ - -template -void swap( - multi_index_container& x, - multi_index_container& y) -{ - x.swap(y); -} - -} /* namespace multi_index */ - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -/* class version = 1 : we now serialize the size through - * boost::serialization::collection_size_type. - * class version = 2 : proper use of {save|load}_construct_data. - */ - -namespace serialization { -template -struct version< - boost::multi_index_container -> -{ - BOOST_STATIC_CONSTANT(int,value=2); -}; -} /* namespace serialization */ -#endif - -/* Associated global functions are promoted to namespace boost, except - * comparison operators and swap, which are meant to be Koenig looked-up. - */ - -using multi_index::get; -using multi_index::project; - -} /* namespace boost */ - -#undef BOOST_MULTI_INDEX_CHECK_INVARIANT -#undef BOOST_MULTI_INDEX_CHECK_INVARIANT_OF - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index_container_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index_container_fwd.hpp deleted file mode 100644 index b35acad407a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index_container_fwd.hpp +++ /dev/null @@ -1,121 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_FWD_HPP -#define BOOST_MULTI_INDEX_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -/* Default value for IndexSpecifierList specifies a container - * equivalent to std::set. - */ - -template< - typename Value, - typename IndexSpecifierList=indexed_by > >, - typename Allocator=std::allocator > -class multi_index_container; - -template -struct nth_index; - -template -struct index; - -template -struct nth_index_iterator; - -template -struct nth_index_const_iterator; - -template -struct index_iterator; - -template -struct index_const_iterator; - -/* get and project functions not fwd declared due to problems - * with dependent typenames - */ - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator==( - const multi_index_container& x, - const multi_index_container& y); - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator<( - const multi_index_container& x, - const multi_index_container& y); - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator!=( - const multi_index_container& x, - const multi_index_container& y); - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator>( - const multi_index_container& x, - const multi_index_container& y); - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator>=( - const multi_index_container& x, - const multi_index_container& y); - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator<=( - const multi_index_container& x, - const multi_index_container& y); - -template -void swap( - multi_index_container& x, - multi_index_container& y); - -} /* namespace multi_index */ - -/* multi_index_container, being the main type of this library, is promoted to - * namespace boost. - */ - -using multi_index::multi_index_container; - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/access.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/access.hpp deleted file mode 100644 index f6581accc91..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/access.hpp +++ /dev/null @@ -1,145 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ACCESS_HPP -#define BOOST_SERIALIZATION_ACCESS_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// access.hpp: interface for serialization system. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -namespace boost { - -namespace archive { -namespace detail { - template - class iserializer; - template - class oserializer; -} // namespace detail -} // namespace archive - -namespace serialization { - -// forward declarations -template -inline void serialize_adl(Archive &, T &, const unsigned int); -namespace detail { - template - struct member_saver; - template - struct member_loader; -} // namespace detail - -// use an "accessor class so that we can use: -// "friend class boost::serialization::access;" -// in any serialized class to permit clean, safe access to private class members -// by the serialization system - -class access { -public: - // grant access to "real" serialization defaults -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else - template - friend struct detail::member_saver; - template - friend struct detail::member_loader; - template - friend class archive::detail::iserializer; - template - friend class archive::detail::oserializer; - template - friend inline void serialize( - Archive & ar, - T & t, - const unsigned int file_version - ); - template - friend inline void save_construct_data( - Archive & ar, - const T * t, - const unsigned int file_version - ); - template - friend inline void load_construct_data( - Archive & ar, - T * t, - const unsigned int file_version - ); -#endif - - // pass calls to users's class implementation - template - static void member_save( - Archive & ar, - //const T & t, - T & t, - const unsigned int file_version - ){ - t.save(ar, file_version); - } - template - static void member_load( - Archive & ar, - T & t, - const unsigned int file_version - ){ - t.load(ar, file_version); - } - template - static void serialize( - Archive & ar, - T & t, - const unsigned int file_version - ){ - // note: if you get a compile time error here with a - // message something like: - // cannot convert parameter 1 from to - // a likely possible cause is that the class T contains a - // serialize function - but that serialize function isn't - // a template and corresponds to a file type different than - // the class Archive. To resolve this, don't include an - // archive type other than that for which the serialization - // function is defined!!! - t.serialize(ar, file_version); - } - template - static void destroy( const T * t) // const appropriate here? - { - // the const business is an MSVC 6.0 hack that should be - // benign on everything else - delete const_cast(t); - } - template - static void construct(T * t){ - // default is inplace invocation of default constructor - // Note the :: before the placement new. Required if the - // class doesn't have a class-specific placement new defined. - ::new(t)T; - } - template - static T & cast_reference(U & u){ - return static_cast(u); - } - template - static T * cast_pointer(U * u){ - return static_cast(u); - } -}; - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_ACCESS_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_map.hpp deleted file mode 100644 index ccf806b1813..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_map.hpp +++ /dev/null @@ -1,85 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_MAP_HPP -#define BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/unordered_map.hpp: -// serialization for stl unordered_map templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { -namespace stl { - -// map input -template -struct archive_input_unordered_map -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - ar >> boost::serialization::make_nvp("item", t.reference()); - std::pair result = - s.insert(boost::move(t.reference())); - // note: the following presumes that the map::value_type was NOT tracked - // in the archive. This is the usual case, but here there is no way - // to determine that. - if(result.second){ - ar.reset_object_address( - & (result.first->second), - & t.reference().second - ); - } - } -}; - -// multimap input -template -struct archive_input_unordered_multimap -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - ar >> boost::serialization::make_nvp("item", t.reference()); - typename Container::const_iterator result = - s.insert(t.reference()); - // note: the following presumes that the map::value_type was NOT tracked - // in the archive. This is the usual case, but here there is no way - // to determine that. - ar.reset_object_address( - & result->second, - & t.reference() - ); - } -}; - -} // stl -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_set.hpp deleted file mode 100644 index 7f0003cc6a4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_set.hpp +++ /dev/null @@ -1,72 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_SET_HPP -#define BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_SET_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// archive_input_unordered_set.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -namespace stl { - -// unordered_set input -template -struct archive_input_unordered_set -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - // borland fails silently w/o full namespace - ar >> boost::serialization::make_nvp("item", t.reference()); - std::pair result = - s.insert(boost::move(t.reference())); - if(result.second) - ar.reset_object_address(& (* result.first), & t.reference()); - } -}; - -// unordered_multiset input -template -struct archive_input_unordered_multiset -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - ar >> boost::serialization::make_nvp("item", t.reference()); - typename Container::const_iterator result = - s.insert(boost::move(t.reference())); - ar.reset_object_address(& (* result), & t.reference()); - } -}; - -} // stl -} // serialization -} // boost - -#endif // BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/array.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/array.hpp deleted file mode 100644 index 612d1a61985..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/array.hpp +++ /dev/null @@ -1,48 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ARRAY_HPP -#define BOOST_SERIALIZATION_ARRAY_HPP - -// (C) Copyright 2005 Matthias Troyer and Dave Abrahams -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// for serialization of . If not supported by the standard -// library - this file becomes empty. This is to avoid breaking backward -// compatibiliy for applications which used this header to support -// serialization of native arrays. Code to serialize native arrays is -// now always include by default. RR - -#include // msvc 6.0 needs this for warning suppression - -#if defined(BOOST_NO_STDC_NAMESPACE) - -#include -#include // std::size_t -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include - -#ifndef BOOST_NO_CXX11_HDR_ARRAY - -#include -#include - -namespace boost { namespace serialization { - -template -void serialize(Archive& ar, std::array& a, const unsigned int /* version */) -{ - ar & boost::serialization::make_nvp( - "elems", - *static_cast(static_cast(a.data())) - ); - -} -} } // end namespace boost::serialization - -#endif // BOOST_NO_CXX11_HDR_ARRAY - -#endif //BOOST_SERIALIZATION_ARRAY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/array_optimization.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/array_optimization.hpp deleted file mode 100644 index 40dffba871a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/array_optimization.hpp +++ /dev/null @@ -1,37 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ARRAY_OPTIMIZATON_HPP -#define BOOST_SERIALIZATION_ARRAY_OPTIMIZATON_HPP - -// (C) Copyright 2005 Matthias Troyer and Dave Abrahams -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include // msvc 6.0 needs this for warning suppression - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include - -namespace boost { namespace serialization { - -template -struct use_array_optimization : boost::mpl::always {}; - -} } // end namespace boost::serialization - -#define BOOST_SERIALIZATION_USE_ARRAY_OPTIMIZATION(Archive) \ -namespace boost { namespace serialization { \ -template <> struct use_array_optimization { \ - template \ - struct apply : boost::mpl::apply1::type \ - >::type {}; \ -}; }} - -#endif //BOOST_SERIALIZATION_ARRAY_OPTIMIZATON_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/array_wrapper.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/array_wrapper.hpp deleted file mode 100644 index adf436e15b4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/array_wrapper.hpp +++ /dev/null @@ -1,121 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ARRAY_WRAPPER_HPP -#define BOOST_SERIALIZATION_ARRAY_WRAPPER_HPP - -// (C) Copyright 2005 Matthias Troyer and Dave Abrahams -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -//#include - -#include // msvc 6.0 needs this for warning suppression - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { namespace serialization { - -template -class array_wrapper : - public wrapper_traits > -{ -private: - array_wrapper & operator=(const array_wrapper & rhs); - // note: I would like to make the copy constructor private but this breaks - // make_array. So I make make_array a friend - template - friend const boost::serialization::array_wrapper make_array(Tx * t, S s); -public: - - array_wrapper(const array_wrapper & rhs) : - m_t(rhs.m_t), - m_element_count(rhs.m_element_count) - {} -public: - array_wrapper(T * t, std::size_t s) : - m_t(t), - m_element_count(s) - {} - - // default implementation - template - void serialize_optimized(Archive &ar, const unsigned int, mpl::false_ ) const - { - // default implemention does the loop - std::size_t c = count(); - T * t = address(); - while(0 < c--) - ar & boost::serialization::make_nvp("item", *t++); - } - - // optimized implementation - template - void serialize_optimized(Archive &ar, const unsigned int version, mpl::true_ ) - { - boost::serialization::split_member(ar, *this, version); - } - - // default implementation - template - void save(Archive &ar, const unsigned int version) const - { - ar.save_array(*this,version); - } - - // default implementation - template - void load(Archive &ar, const unsigned int version) - { - ar.load_array(*this,version); - } - - // default implementation - template - void serialize(Archive &ar, const unsigned int version) - { - typedef typename - boost::serialization::use_array_optimization::template apply< - typename remove_const< T >::type - >::type use_optimized; - serialize_optimized(ar,version,use_optimized()); - } - - T * address() const - { - return m_t; - } - - std::size_t count() const - { - return m_element_count; - } - -private: - T * const m_t; - const std::size_t m_element_count; -}; - -template -inline -const array_wrapper< T > make_array(T* t, S s){ - const array_wrapper< T > a(t, s); - return a; -} - -} } // end namespace boost::serialization - - -#endif //BOOST_SERIALIZATION_ARRAY_WRAPPER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/assume_abstract.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/assume_abstract.hpp deleted file mode 100644 index 632f9312f5f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/assume_abstract.hpp +++ /dev/null @@ -1,60 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ASSUME_ABSTRACT_HPP -#define BOOST_SERIALIZATION_ASSUME_ABSTRACT_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// assume_abstract_class.hpp: - -// (C) Copyright 2008 Robert Ramey -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// this is useful for compilers which don't support the boost::is_abstract - -#include -#include - -#ifndef BOOST_NO_IS_ABSTRACT - -// if there is an intrinsic is_abstract defined, we don't have to do anything -#define BOOST_SERIALIZATION_ASSUME_ABSTRACT(T) - -// but forward to the "official" is_abstract -namespace boost { -namespace serialization { - template - struct is_abstract : boost::is_abstract< T > {} ; -} // namespace serialization -} // namespace boost - -#else -// we have to "make" one - -namespace boost { -namespace serialization { - template - struct is_abstract : boost::false_type {}; -} // namespace serialization -} // namespace boost - -// define a macro to make explicit designation of this more transparent -#define BOOST_SERIALIZATION_ASSUME_ABSTRACT(T) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct is_abstract< T > : boost::true_type {}; \ -template<> \ -struct is_abstract< const T > : boost::true_type {}; \ -}} \ -/**/ - -#endif // BOOST_NO_IS_ABSTRACT - -#endif //BOOST_SERIALIZATION_ASSUME_ABSTRACT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/base_object.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/base_object.hpp deleted file mode 100644 index 1a82cecd4b5..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/base_object.hpp +++ /dev/null @@ -1,100 +0,0 @@ -#ifndef BOOST_SERIALIZATION_BASE_OBJECT_HPP -#define BOOST_SERIALIZATION_BASE_OBJECT_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// base_object.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// if no archive headers have been included this is a no op -// this is to permit BOOST_EXPORT etc to be included in a -// file declaration header - -#include -#include - -#include -#include -#include - -#include -#include -#include -#include - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -namespace detail -{ - // get the base type for a given derived type - // preserving the const-ness - template - struct base_cast - { - typedef typename - mpl::if_< - is_const, - const B, - B - >::type type; - BOOST_STATIC_ASSERT(is_const::value == is_const::value); - }; - - // only register void casts if the types are polymorphic - template - struct base_register - { - struct polymorphic { - static void const * invoke(){ - Base const * const b = 0; - Derived const * const d = 0; - return & void_cast_register(d, b); - } - }; - struct non_polymorphic { - static void const * invoke(){ - return 0; - } - }; - static void const * invoke(){ - typedef typename mpl::eval_if< - is_polymorphic, - mpl::identity, - mpl::identity - >::type type; - return type::invoke(); - } - }; - -} // namespace detail -template -typename detail::base_cast::type & -base_object(Derived &d) -{ - BOOST_STATIC_ASSERT(( is_base_and_derived::value)); - BOOST_STATIC_ASSERT(! is_pointer::value); - typedef typename detail::base_cast::type type; - detail::base_register::invoke(); - return access::cast_reference(d); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_BASE_OBJECT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/binary_object.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/binary_object.hpp deleted file mode 100644 index 5c9038e5a9f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/binary_object.hpp +++ /dev/null @@ -1,79 +0,0 @@ -#ifndef BOOST_SERIALIZATION_BINARY_OBJECT_HPP -#define BOOST_SERIALIZATION_BINARY_OBJECT_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// nvp.hpp: interface for serialization system. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include // std::size_t -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -struct binary_object : - public wrapper_traits > -{ - void const * m_t; - std::size_t m_size; - template - void save(Archive & ar, const unsigned int /* file_version */) const { - ar.save_binary(m_t, m_size); - } - template - void load(Archive & ar, const unsigned int /* file_version */) const { - ar.load_binary(const_cast(m_t), m_size); - } - BOOST_SERIALIZATION_SPLIT_MEMBER() - binary_object & operator=(const binary_object & rhs) { - m_t = rhs.m_t; - m_size = rhs.m_size; - return *this; - } - binary_object(const void * const t, std::size_t size) : - m_t(t), - m_size(size) - {} - binary_object(const binary_object & rhs) : - m_t(rhs.m_t), - m_size(rhs.m_size) - {} -}; - -// just a little helper to support the convention that all serialization -// wrappers follow the naming convention make_xxxxx -inline -const binary_object -make_binary_object(const void * t, std::size_t size){ - return binary_object(t, size); -} - -} // namespace serialization -} // boost - -#endif // BOOST_SERIALIZATION_BINARY_OBJECT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/bitset.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/bitset.hpp deleted file mode 100644 index 78f9bd74336..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/bitset.hpp +++ /dev/null @@ -1,75 +0,0 @@ -/*! - * \file bitset.hpp - * \brief Provides Boost.Serialization support for std::bitset - * \author Brian Ravnsgaard Riis - * \author Kenneth Riddile - * \date 16.09.2004, updated 04.03.2009 - * \copyright 2004 Brian Ravnsgaard Riis - * \license Boost Software License 1.0 - */ -#ifndef BOOST_SERIALIZATION_BITSET_HPP -#define BOOST_SERIALIZATION_BITSET_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -#include -#include // size_t - -#include -#include -#include -#include - -namespace boost{ -namespace serialization{ - -template -inline void save( - Archive & ar, - std::bitset const & t, - const unsigned int /* version */ -){ - const std::string bits = t.template to_string< - std::string::value_type, - std::string::traits_type, - std::string::allocator_type - >(); - ar << BOOST_SERIALIZATION_NVP( bits ); -} - -template -inline void load( - Archive & ar, - std::bitset & t, - const unsigned int /* version */ -){ - std::string bits; - ar >> BOOST_SERIALIZATION_NVP( bits ); - t = std::bitset(bits); -} - -template -inline void serialize( - Archive & ar, - std::bitset & t, - const unsigned int version -){ - boost::serialization::split_free( ar, t, version ); -} - -// don't track bitsets since that would trigger tracking -// all over the program - which probably would be a surprise. -// also, tracking would be hard to implement since, we're -// serialization a representation of the data rather than -// the data itself. -template -struct tracking_level > - : mpl::int_ {} ; - -} //serialization -} //boost - -#endif // BOOST_SERIALIZATION_BITSET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/boost_array.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/boost_array.hpp deleted file mode 100644 index d564ff15de0..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/boost_array.hpp +++ /dev/null @@ -1,33 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ARRAY_HPP -#define BOOST_SERIALIZATION_ARRAY_HPP - -// (C) Copyright 2005 Matthias Troyer and Dave Abrahams -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -//#include - -#include // msvc 6.0 needs this for warning suppression - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include - -namespace boost { namespace serialization { -// implement serialization for boost::array -template -void serialize(Archive& ar, boost::array& a, const unsigned int /* version */) -{ - ar & boost::serialization::make_nvp("elems", a.elems); -} - -} } // end namespace boost::serialization - - -#endif //BOOST_SERIALIZATION_ARRAY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_map.hpp deleted file mode 100644 index 8913b31f9e6..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_map.hpp +++ /dev/null @@ -1,154 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UNORDERED_MAP_HPP -#define BOOST_SERIALIZATION_UNORDERED_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/unordered_map.hpp: -// serialization for stl unordered_map templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const boost::unordered_map &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - boost::unordered_map - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - boost::unordered_map &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - boost::unordered_map, - boost::serialization::stl::archive_input_unordered_map< - Archive, - boost::unordered_map - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - boost::unordered_map &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// unordered_multimap -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const boost::unordered_multimap &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - boost::unordered_multimap - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - boost::unordered_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - boost::unordered_multimap, - boost::serialization::stl::archive_input_unordered_multimap< - Archive, - boost::unordered_multimap - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - boost::unordered_multimap &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_UNORDERED_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_set.hpp deleted file mode 100644 index 307c7819cbd..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_set.hpp +++ /dev/null @@ -1,150 +0,0 @@ -#ifndef BOOST_SERIALIZATION_BOOST_UNORDERED_SET_HPP -#define BOOST_SERIALIZATION_BOOST_UNORDERED_SET_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// unordered_set.hpp: serialization for boost unordered_set templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const boost::unordered_set &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - boost::unordered_set - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - boost::unordered_set &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - boost::unordered_set, - boost::serialization::stl::archive_input_unordered_set< - Archive, - boost::unordered_set - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - boost::unordered_set &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// unordered_multiset -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const boost::unordered_multiset &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - boost::unordered_multiset - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - boost::unordered_multiset &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - boost::unordered_multiset, - boost::serialization::stl::archive_input_unordered_multiset< - Archive, - boost::unordered_multiset - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - boost::unordered_multiset &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_BOOST_UNORDERED_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/collection_size_type.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/collection_size_type.hpp deleted file mode 100644 index 2dd8fa72584..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/collection_size_type.hpp +++ /dev/null @@ -1,62 +0,0 @@ -#ifndef BOOST_SERIALIZATION_COLLECTION_SIZE_TYPE_HPP -#define BOOST_SERIALIZATION_COLLECTION_SIZE_TYPE_HPP - -// (C) Copyright 2005 Matthias Troyer -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include // size_t -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -//BOOST_STRONG_TYPEDEF(std::size_t, collection_size_type) - -class collection_size_type { -private: - typedef std::size_t base_type; - base_type t; -public: - collection_size_type(): t(0) {}; - explicit collection_size_type(const std::size_t & t_) : - t(t_) - {} - collection_size_type(const collection_size_type & t_) : - t(t_.t) - {} - collection_size_type & operator=(const collection_size_type & rhs){ - t = rhs.t; - return *this; - } - collection_size_type & operator=(const unsigned int & rhs){ - t = rhs; - return *this; - } - // used for text output - operator base_type () const { - return t; - } - // used for text input - operator base_type & () { - return t; - } - bool operator==(const collection_size_type & rhs) const { - return t == rhs.t; - } - bool operator<(const collection_size_type & rhs) const { - return t < rhs.t; - } -}; - - -} } // end namespace boost::serialization - -BOOST_CLASS_IMPLEMENTATION(collection_size_type, primitive_type) -BOOST_IS_BITWISE_SERIALIZABLE(collection_size_type) - -#endif //BOOST_SERIALIZATION_COLLECTION_SIZE_TYPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/collection_traits.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/collection_traits.hpp deleted file mode 100644 index 3ec9401eff0..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/collection_traits.hpp +++ /dev/null @@ -1,79 +0,0 @@ -#ifndef BOOST_SERIALIZATION_COLLECTION_TRAITS_HPP -#define BOOST_SERIALIZATION_COLLECTION_TRAITS_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// collection_traits.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// This header assigns a level implemenation trait to a collection type -// for all primitives. It is needed so that archives which are meant to be -// portable don't write class information in the archive. Since, not all -// compiles recognize the same set of primitive types, the possibility -// exists for archives to be non-portable if class information for primitive -// types is included. This is addressed by the following macros. -#include -//#include -#include - -#include -#include -#include // ULONG_MAX -#include - -#define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(T, C) \ -template<> \ -struct implementation_level< C < T > > { \ - typedef mpl::integral_c_tag tag; \ - typedef mpl::int_ type; \ - BOOST_STATIC_CONSTANT(int, value = object_serializable); \ -}; \ -/**/ - -#if defined(BOOST_NO_CWCHAR) || defined(BOOST_NO_INTRINSIC_WCHAR_T) - #define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_WCHAR(C) -#else - #define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_WCHAR(C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(wchar_t, C) \ - /**/ -#endif - -#if defined(BOOST_HAS_LONG_LONG) - #define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_INT64(C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(boost::long_long_type, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(boost::ulong_long_type, C) \ - /**/ -#else - #define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_INT64(C) -#endif - -#define BOOST_SERIALIZATION_COLLECTION_TRAITS(C) \ - namespace boost { namespace serialization { \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(bool, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(char, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(signed char, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(unsigned char, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(signed int, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(unsigned int, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(signed long, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(unsigned long, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(float, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(double, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(unsigned short, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(signed short, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_INT64(C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_WCHAR(C) \ - } } \ - /**/ - -#endif // BOOST_SERIALIZATION_COLLECTION_TRAITS diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/collections_load_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/collections_load_imp.hpp deleted file mode 100644 index e042c0c130d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/collections_load_imp.hpp +++ /dev/null @@ -1,106 +0,0 @@ -#ifndef BOOST_SERIALIZATION_COLLECTIONS_LOAD_IMP_HPP -#define BOOST_SERIALIZATION_COLLECTIONS_LOAD_IMP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -#if defined(_MSC_VER) && (_MSC_VER <= 1020) -# pragma warning (disable : 4786) // too long name, harmless warning -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// collections_load_imp.hpp: serialization for loading stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// helper function templates for serialization of collections - -#include -#include // size_t -#include // msvc 6.0 needs this for warning suppression -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ -namespace serialization { -namespace stl { - -////////////////////////////////////////////////////////////////////// -// implementation of serialization for STL containers -// - -template< - class Archive, - class T -> -typename boost::enable_if< - typename detail::is_default_constructible< - typename T::value_type - >, - void ->::type -collection_load_impl( - Archive & ar, - T & t, - collection_size_type count, - item_version_type /*item_version*/ -){ - t.resize(count); - typename T::iterator hint; - hint = t.begin(); - while(count-- > 0){ - ar >> boost::serialization::make_nvp("item", *hint++); - } -} - -template< - class Archive, - class T -> -typename boost::disable_if< - typename detail::is_default_constructible< - typename T::value_type - >, - void ->::type -collection_load_impl( - Archive & ar, - T & t, - collection_size_type count, - item_version_type item_version -){ - t.clear(); - while(count-- > 0){ - detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - t.push_back(boost::move(u.reference())); - ar.reset_object_address(& t.back() , & u.reference()); - } -} - -} // namespace stl -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_COLLECTIONS_LOAD_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/collections_save_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/collections_save_imp.hpp deleted file mode 100644 index f3cabfcf3f5..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/collections_save_imp.hpp +++ /dev/null @@ -1,82 +0,0 @@ -#ifndef BOOST_SERIALIZATION_COLLECTIONS_SAVE_IMP_HPP -#define BOOST_SERIALIZATION_COLLECTIONS_SAVE_IMP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// collections_save_imp.hpp: serialization for stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// helper function templates for serialization of collections - -#include -#include -#include -#include -#include -#include - -namespace boost{ -namespace serialization { -namespace stl { - -////////////////////////////////////////////////////////////////////// -// implementation of serialization for STL containers -// - -template -inline void save_collection( - Archive & ar, - const Container &s, - collection_size_type count) -{ - ar << BOOST_SERIALIZATION_NVP(count); - // record number of elements - const item_version_type item_version( - version::value - ); - #if 0 - boost::archive::library_version_type library_version( - ar.get_library_version() - ); - if(boost::archive::library_version_type(3) < library_version){ - ar << BOOST_SERIALIZATION_NVP(item_version); - } - #else - ar << BOOST_SERIALIZATION_NVP(item_version); - #endif - - typename Container::const_iterator it = s.begin(); - while(count-- > 0){ - // note borland emits a no-op without the explicit namespace - boost::serialization::save_construct_data_adl( - ar, - &(*it), - item_version - ); - ar << boost::serialization::make_nvp("item", *it++); - } -} - -template -inline void save_collection(Archive & ar, const Container &s) -{ - // record number of elements - collection_size_type count(s.size()); - save_collection(ar, s, count); -} - -} // namespace stl -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_COLLECTIONS_SAVE_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/complex.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/complex.hpp deleted file mode 100644 index b4ef44cf973..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/complex.hpp +++ /dev/null @@ -1,81 +0,0 @@ -#ifndef BOOST_SERIALIZATION_COMPLEX_HPP -#define BOOST_SERIALIZATION_COMPLEX_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/utility.hpp: -// serialization for stl utility templates - -// (C) Copyright 2007 Matthias Troyer . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include - -namespace boost { -namespace serialization { - -template -inline void serialize( - Archive & ar, - std::complex< T > & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -template -inline void save( - Archive & ar, - std::complex< T > const & t, - const unsigned int /* file_version */ -){ - const T re = t.real(); - const T im = t.imag(); - ar << boost::serialization::make_nvp("real", re); - ar << boost::serialization::make_nvp("imag", im); -} - -template -inline void load( - Archive & ar, - std::complex< T >& t, - const unsigned int /* file_version */ -){ - T re; - T im; - ar >> boost::serialization::make_nvp("real", re); - ar >> boost::serialization::make_nvp("imag", im); - t = std::complex< T >(re,im); -} - -// specialization of serialization traits for complex -template -struct is_bitwise_serializable > - : public is_bitwise_serializable< T > {}; - -template -struct implementation_level > - : mpl::int_ {} ; - -// treat complex just like builtin arithmetic types for tracking -template -struct tracking_level > - : mpl::int_ {} ; - -} // serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_COMPLEX_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/config.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/config.hpp deleted file mode 100644 index ea8cb9239ed..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/config.hpp +++ /dev/null @@ -1,74 +0,0 @@ -#ifndef BOOST_SERIALIZATION_CONFIG_HPP -#define BOOST_SERIALIZATION_CONFIG_HPP - -// config.hpp ---------------------------------------------// - -// (c) Copyright Robert Ramey 2004 -// Use, modification, and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See library home page at http://www.boost.org/libs/serialization - -//----------------------------------------------------------------------------// - -// This header implements separate compilation features as described in -// http://www.boost.org/more/separate_compilation.html - -#include -#include - -// note: this version incorporates the related code into the the -// the same library as BOOST_ARCHIVE. This could change some day in the -// future - -// if BOOST_SERIALIZATION_DECL is defined undefine it now: -#ifdef BOOST_SERIALIZATION_DECL - #undef BOOST_SERIALIZATION_DECL -#endif - -// we need to import/export our code only if the user has specifically -// asked for it by defining either BOOST_ALL_DYN_LINK if they want all boost -// libraries to be dynamically linked, or BOOST_SERIALIZATION_DYN_LINK -// if they want just this one to be dynamically liked: -#if defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) - #if !defined(BOOST_DYN_LINK) - #define BOOST_DYN_LINK - #endif - // export if this is our own source, otherwise import: - #if defined(BOOST_SERIALIZATION_SOURCE) - #define BOOST_SERIALIZATION_DECL BOOST_SYMBOL_EXPORT - #else - #define BOOST_SERIALIZATION_DECL BOOST_SYMBOL_IMPORT - #endif // defined(BOOST_SERIALIZATION_SOURCE) -#endif // defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) - -// if BOOST_SERIALIZATION_DECL isn't defined yet define it now: -#ifndef BOOST_SERIALIZATION_DECL - #define BOOST_SERIALIZATION_DECL -#endif - -// enable automatic library variant selection ------------------------------// - -#if !defined(BOOST_ALL_NO_LIB) && !defined(BOOST_SERIALIZATION_NO_LIB) \ -&& !defined(BOOST_ARCHIVE_SOURCE) && !defined(BOOST_WARCHIVE_SOURCE) \ -&& !defined(BOOST_SERIALIZATION_SOURCE) - // - // Set the name of our library, this will get undef'ed by auto_link.hpp - // once it's done with it: - // - #define BOOST_LIB_NAME boost_serialization - // - // If we're importing code from a dll, then tell auto_link.hpp about it: - // - #if defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) - # define BOOST_DYN_LINK - #endif - // - // And include the header that does the work: - // - #include - -#endif - -#endif // BOOST_SERIALIZATION_CONFIG_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/deque.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/deque.hpp deleted file mode 100644 index bba81364ce2..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/deque.hpp +++ /dev/null @@ -1,80 +0,0 @@ -#ifndef BOOST_SERIALIZATION_DEQUE_HPP -#define BOOST_SERIALIZATION_DEQUE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// deque.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include - -#include -#include -#include - -namespace boost { -namespace serialization { - -template -inline void save( - Archive & ar, - const std::deque &t, - const unsigned int /* file_version */ -){ - boost::serialization::stl::save_collection< - Archive, std::deque - >(ar, t); -} - -template -inline void load( - Archive & ar, - std::deque &t, - const unsigned int /* file_version */ -){ - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - stl::collection_load_impl(ar, t, count, item_version); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::deque &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(std::deque) - -#endif // BOOST_SERIALIZATION_DEQUE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/is_default_constructible.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/is_default_constructible.hpp deleted file mode 100644 index 4d20b13bf3e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/detail/is_default_constructible.hpp +++ /dev/null @@ -1,54 +0,0 @@ -#ifndef BOOST_SERIALIZATION_DETAIL_IS_DEFAULT_CONSTRUCTIBLE_HPP -#define BOOST_SERIALIZATION_DETAIL_IS_DEFAULT_CONSTRUCTIBLE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// is_default_constructible.hpp: serialization for loading stl collections -// -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#if ! defined(BOOST_NO_CXX11_HDR_TYPE_TRAITS) - #include - namespace boost{ - namespace serialization { - namespace detail { - - template - struct is_default_constructible : public std::is_default_constructible {}; - - } // detail - } // serializaition - } // boost -#else - // we don't have standard library support for is_default_constructible - // so we fake it by using boost::has_trivial_construtor. But this is not - // actually correct because it's possible that a default constructor - // to be non trivial. So when using this, make sure you're not using your - // own definition of of T() but are using the actual default one! - #include - namespace boost{ - namespace serialization { - namespace detail { - - template - struct is_default_constructible : public boost::has_trivial_constructor {}; - - } // detail - } // serializaition - } // boost - -#endif - - -#endif // BOOST_SERIALIZATION_DETAIL_IS_DEFAULT_CONSTRUCTIBLE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_count_132.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_count_132.hpp deleted file mode 100644 index a5872557cf2..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_count_132.hpp +++ /dev/null @@ -1,551 +0,0 @@ -#ifndef BOOST_DETAIL_SHARED_COUNT_132_HPP_INCLUDED -#define BOOST_DETAIL_SHARED_COUNT_132_HPP_INCLUDED - -// MS compatible compilers support #pragma once - -#if defined(_MSC_VER) -# pragma once -#endif - -// -// detail/shared_count.hpp -// -// Copyright (c) 2001, 2002, 2003 Peter Dimov and Multi Media Ltd. -// -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// - -#include - -#if defined(BOOST_SP_USE_STD_ALLOCATOR) && defined(BOOST_SP_USE_QUICK_ALLOCATOR) -# error BOOST_SP_USE_STD_ALLOCATOR and BOOST_SP_USE_QUICK_ALLOCATOR are incompatible. -#endif - -#include -#include -#include - -#if defined(BOOST_SP_USE_QUICK_ALLOCATOR) -#include -#endif - -#include // std::auto_ptr, std::allocator -#include // std::less -#include // std::exception -#include // std::bad_alloc -#include // std::type_info in get_deleter -#include // std::size_t - -#include // msvc 6.0 needs this for warning suppression -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -namespace boost_132 { - -// Debug hooks - -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - -void sp_scalar_constructor_hook(void * px, std::size_t size, void * pn); -void sp_array_constructor_hook(void * px); -void sp_scalar_destructor_hook(void * px, std::size_t size, void * pn); -void sp_array_destructor_hook(void * px); - -#endif - - -// The standard library that comes with Borland C++ 5.5.1 -// defines std::exception and its members as having C calling -// convention (-pc). When the definition of bad_weak_ptr -// is compiled with -ps, the compiler issues an error. -// Hence, the temporary #pragma option -pc below. The version -// check is deliberately conservative. - -class bad_weak_ptr: public std::exception -{ -public: - - virtual char const * what() const throw() - { - return "boost::bad_weak_ptr"; - } -}; - -namespace detail{ - -class sp_counted_base -{ -//private: - - typedef boost::detail::lightweight_mutex mutex_type; - -public: - - sp_counted_base(): use_count_(1), weak_count_(1) - { - } - - virtual ~sp_counted_base() // nothrow - { - } - - // dispose() is called when use_count_ drops to zero, to release - // the resources managed by *this. - - virtual void dispose() = 0; // nothrow - - // destruct() is called when weak_count_ drops to zero. - - virtual void destruct() // nothrow - { - delete this; - } - - virtual void * get_deleter(std::type_info const & ti) = 0; - - void add_ref_copy() - { -#if defined(BOOST_HAS_THREADS) - mutex_type::scoped_lock lock(mtx_); -#endif - ++use_count_; - } - - void add_ref_lock() - { -#if defined(BOOST_HAS_THREADS) - mutex_type::scoped_lock lock(mtx_); -#endif - if(use_count_ == 0) boost::serialization::throw_exception(bad_weak_ptr()); - ++use_count_; - } - - void release() // nothrow - { - { -#if defined(BOOST_HAS_THREADS) - mutex_type::scoped_lock lock(mtx_); -#endif - long new_use_count = --use_count_; - - if(new_use_count != 0) return; - } - - dispose(); - weak_release(); - } - - void weak_add_ref() // nothrow - { -#if defined(BOOST_HAS_THREADS) - mutex_type::scoped_lock lock(mtx_); -#endif - ++weak_count_; - } - - void weak_release() // nothrow - { - long new_weak_count; - - { -#if defined(BOOST_HAS_THREADS) - mutex_type::scoped_lock lock(mtx_); -#endif - new_weak_count = --weak_count_; - } - - if(new_weak_count == 0) - { - destruct(); - } - } - - long use_count() const // nothrow - { -#if defined(BOOST_HAS_THREADS) - mutex_type::scoped_lock lock(mtx_); -#endif - return use_count_; - } - -//private: -public: - sp_counted_base(sp_counted_base const &); - sp_counted_base & operator= (sp_counted_base const &); - - long use_count_; // #shared - long weak_count_; // #weak + (#shared != 0) - -#if defined(BOOST_HAS_THREADS) || defined(BOOST_LWM_WIN32) - mutable mutex_type mtx_; -#endif -}; - -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - -template void cbi_call_constructor_hook(sp_counted_base * pn, T * px, boost::checked_deleter< T > const &) -{ - boost::sp_scalar_constructor_hook(px, sizeof(T), pn); -} - -template void cbi_call_constructor_hook(sp_counted_base *, T * px, boost::checked_array_deleter< T > const &) -{ - boost::sp_array_constructor_hook(px); -} - -template void cbi_call_constructor_hook(sp_counted_base *, P const &, D const &, long) -{ -} - -template void cbi_call_destructor_hook(sp_counted_base * pn, T * px, boost::checked_deleter< T > const &) -{ - boost::sp_scalar_destructor_hook(px, sizeof(T), pn); -} - -template void cbi_call_destructor_hook(sp_counted_base *, T * px, boost::checked_array_deleter< T > const &) -{ - boost::sp_array_destructor_hook(px); -} - -template void cbi_call_destructor_hook(sp_counted_base *, P const &, D const &, long) -{ -} - -#endif - -// -// Borland's Codeguard trips up over the -Vx- option here: -// -#ifdef __CODEGUARD__ -# pragma option push -Vx- -#endif - -template class sp_counted_base_impl: public sp_counted_base -{ -//private: -public: - P ptr; // copy constructor must not throw - D del; // copy constructor must not throw - - sp_counted_base_impl(sp_counted_base_impl const &); - sp_counted_base_impl & operator= (sp_counted_base_impl const &); - - typedef sp_counted_base_impl this_type; - -public: - - // pre: initial_use_count <= initial_weak_count, d(p) must not throw - - sp_counted_base_impl(P p, D d): ptr(p), del(d) - { -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - detail::cbi_call_constructor_hook(this, p, d, 0); -#endif - } - - virtual void dispose() // nothrow - { -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - detail::cbi_call_destructor_hook(this, ptr, del, 0); -#endif - del(ptr); - } - - virtual void * get_deleter(std::type_info const & ti) - { - return ti == typeid(D)? &del: 0; - } - -#if defined(BOOST_SP_USE_STD_ALLOCATOR) - - void * operator new(std::size_t) - { - return std::allocator().allocate(1, static_cast(0)); - } - - void operator delete(void * p) - { - std::allocator().deallocate(static_cast(p), 1); - } - -#endif - -#if defined(BOOST_SP_USE_QUICK_ALLOCATOR) - - void * operator new(std::size_t) - { - return boost::detail::quick_allocator::alloc(); - } - - void operator delete(void * p) - { - boost::detail::quick_allocator::dealloc(p); - } - -#endif -}; - -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - -int const shared_count_id = 0x2C35F101; -int const weak_count_id = 0x298C38A4; - -#endif - -class weak_count; - -class shared_count -{ -//private: -public: - sp_counted_base * pi_; - -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - int id_; -#endif - - friend class weak_count; - -public: - - shared_count(): pi_(0) // nothrow -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif - { - } - - template shared_count(P p, D d): pi_(0) -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif - { -#ifndef BOOST_NO_EXCEPTIONS - - try - { - pi_ = new sp_counted_base_impl(p, d); - } - catch(...) - { - d(p); // delete p - throw; - } - -#else - - pi_ = new sp_counted_base_impl(p, d); - - if(pi_ == 0) - { - d(p); // delete p - boost::serialization::throw_exception(std::bad_alloc()); - } - -#endif - } - -#ifndef BOOST_NO_AUTO_PTR - - // auto_ptr is special cased to provide the strong guarantee - - template - explicit shared_count(std::auto_ptr & r): pi_( - new sp_counted_base_impl< - Y *, - boost::checked_deleter - >(r.get(), boost::checked_deleter())) -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif - { - r.release(); - } - -#endif - - ~shared_count() // nothrow - { - if(pi_ != 0) pi_->release(); -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - id_ = 0; -#endif - } - - shared_count(shared_count const & r): pi_(r.pi_) // nothrow -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif - { - if(pi_ != 0) pi_->add_ref_copy(); - } - - explicit shared_count(weak_count const & r); // throws bad_weak_ptr when r.use_count() == 0 - - shared_count & operator= (shared_count const & r) // nothrow - { - sp_counted_base * tmp = r.pi_; - - if(tmp != pi_) - { - if(tmp != 0) tmp->add_ref_copy(); - if(pi_ != 0) pi_->release(); - pi_ = tmp; - } - - return *this; - } - - void swap(shared_count & r) // nothrow - { - sp_counted_base * tmp = r.pi_; - r.pi_ = pi_; - pi_ = tmp; - } - - long use_count() const // nothrow - { - return pi_ != 0? pi_->use_count(): 0; - } - - bool unique() const // nothrow - { - return use_count() == 1; - } - - friend inline bool operator==(shared_count const & a, shared_count const & b) - { - return a.pi_ == b.pi_; - } - - friend inline bool operator<(shared_count const & a, shared_count const & b) - { - return std::less()(a.pi_, b.pi_); - } - - void * get_deleter(std::type_info const & ti) const - { - return pi_? pi_->get_deleter(ti): 0; - } -}; - -#ifdef __CODEGUARD__ -# pragma option pop -#endif - - -class weak_count -{ -private: - - sp_counted_base * pi_; - -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - int id_; -#endif - - friend class shared_count; - -public: - - weak_count(): pi_(0) // nothrow -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(weak_count_id) -#endif - { - } - - weak_count(shared_count const & r): pi_(r.pi_) // nothrow -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif - { - if(pi_ != 0) pi_->weak_add_ref(); - } - - weak_count(weak_count const & r): pi_(r.pi_) // nothrow -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif - { - if(pi_ != 0) pi_->weak_add_ref(); - } - - ~weak_count() // nothrow - { - if(pi_ != 0) pi_->weak_release(); -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - id_ = 0; -#endif - } - - weak_count & operator= (shared_count const & r) // nothrow - { - sp_counted_base * tmp = r.pi_; - if(tmp != 0) tmp->weak_add_ref(); - if(pi_ != 0) pi_->weak_release(); - pi_ = tmp; - - return *this; - } - - weak_count & operator= (weak_count const & r) // nothrow - { - sp_counted_base * tmp = r.pi_; - if(tmp != 0) tmp->weak_add_ref(); - if(pi_ != 0) pi_->weak_release(); - pi_ = tmp; - - return *this; - } - - void swap(weak_count & r) // nothrow - { - sp_counted_base * tmp = r.pi_; - r.pi_ = pi_; - pi_ = tmp; - } - - long use_count() const // nothrow - { - return pi_ != 0? pi_->use_count(): 0; - } - - friend inline bool operator==(weak_count const & a, weak_count const & b) - { - return a.pi_ == b.pi_; - } - - friend inline bool operator<(weak_count const & a, weak_count const & b) - { - return std::less()(a.pi_, b.pi_); - } -}; - -inline shared_count::shared_count(weak_count const & r): pi_(r.pi_) -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif -{ - if(pi_ != 0) - { - pi_->add_ref_lock(); - } - else - { - boost::serialization::throw_exception(bad_weak_ptr()); - } -} - -} // namespace detail - -} // namespace boost - -BOOST_SERIALIZATION_ASSUME_ABSTRACT(boost_132::detail::sp_counted_base) - -#endif // #ifndef BOOST_DETAIL_SHARED_COUNT_HPP_INCLUDED diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_132.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_132.hpp deleted file mode 100644 index ee98b7b9449..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_132.hpp +++ /dev/null @@ -1,443 +0,0 @@ -#ifndef BOOST_SHARED_PTR_132_HPP_INCLUDED -#define BOOST_SHARED_PTR_132_HPP_INCLUDED - -// -// shared_ptr.hpp -// -// (C) Copyright Greg Colvin and Beman Dawes 1998, 1999. -// Copyright (c) 2001, 2002, 2003 Peter Dimov -// -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// -// See http://www.boost.org/libs/smart_ptr/shared_ptr.htm for documentation. -// - -#include // for broken compiler workarounds - -#if defined(BOOST_NO_MEMBER_TEMPLATES) && !defined(BOOST_MSVC6_MEMBER_TEMPLATES) -#include -#else - -#include -#include -#include -#include - -#include -#include - -#include // for std::auto_ptr -#include // for std::swap -#include // for std::less -#include // for std::bad_cast -#include // for std::basic_ostream - -#ifdef BOOST_MSVC // moved here to work around VC++ compiler crash -# pragma warning(push) -# pragma warning(disable:4284) // odd return type for operator-> -#endif - -namespace boost_132 { - -template class weak_ptr; -template class enable_shared_from_this; - -namespace detail -{ - -struct static_cast_tag {}; -struct const_cast_tag {}; -struct dynamic_cast_tag {}; -struct polymorphic_cast_tag {}; - -template struct shared_ptr_traits -{ - typedef T & reference; -}; - -template<> struct shared_ptr_traits -{ - typedef void reference; -}; - -#if !defined(BOOST_NO_CV_VOID_SPECIALIZATIONS) - -template<> struct shared_ptr_traits -{ - typedef void reference; -}; - -template<> struct shared_ptr_traits -{ - typedef void reference; -}; - -template<> struct shared_ptr_traits -{ - typedef void reference; -}; - -#endif - -// enable_shared_from_this support - -template void sp_enable_shared_from_this( shared_count const & pn, enable_shared_from_this< T > const * pe, Y const * px ) -{ - if(pe != 0) pe->_internal_weak_this._internal_assign(const_cast(px), pn); -} - -inline void sp_enable_shared_from_this( shared_count const & /*pn*/, ... ) -{ -} - -} // namespace detail - - -// -// shared_ptr -// -// An enhanced relative of scoped_ptr with reference counted copy semantics. -// The object pointed to is deleted when the last shared_ptr pointing to it -// is destroyed or reset. -// - -template class shared_ptr -{ -private: - // Borland 5.5.1 specific workaround - typedef shared_ptr< T > this_type; - -public: - - typedef T element_type; - typedef T value_type; - typedef T * pointer; - typedef typename detail::shared_ptr_traits< T >::reference reference; - - shared_ptr(): px(0), pn() // never throws in 1.30+ - { - } - - template - explicit shared_ptr(Y * p): px(p), pn(p, boost::checked_deleter()) // Y must be complete - { - detail::sp_enable_shared_from_this( pn, p, p ); - } - - // - // Requirements: D's copy constructor must not throw - // - // shared_ptr will release p by calling d(p) - // - - template shared_ptr(Y * p, D d): px(p), pn(p, d) - { - detail::sp_enable_shared_from_this( pn, p, p ); - } - -// generated copy constructor, assignment, destructor are fine... - -// except that Borland C++ has a bug, and g++ with -Wsynth warns -#if defined(__GNUC__) - shared_ptr & operator=(shared_ptr const & r) // never throws - { - px = r.px; - pn = r.pn; // shared_count::op= doesn't throw - return *this; - } -#endif - - template - explicit shared_ptr(weak_ptr const & r): pn(r.pn) // may throw - { - // it is now safe to copy r.px, as pn(r.pn) did not throw - px = r.px; - } - - template - shared_ptr(shared_ptr const & r): px(r.px), pn(r.pn) // never throws - { - } - - template - shared_ptr(shared_ptr const & r, detail::static_cast_tag): px(static_cast(r.px)), pn(r.pn) - { - } - - template - shared_ptr(shared_ptr const & r, detail::const_cast_tag): px(const_cast(r.px)), pn(r.pn) - { - } - - template - shared_ptr(shared_ptr const & r, detail::dynamic_cast_tag): px(dynamic_cast(r.px)), pn(r.pn) - { - if(px == 0) // need to allocate new counter -- the cast failed - { - pn = detail::shared_count(); - } - } - - template - shared_ptr(shared_ptr const & r, detail::polymorphic_cast_tag): px(dynamic_cast(r.px)), pn(r.pn) - { - if(px == 0) - { - boost::serialization::throw_exception(std::bad_cast()); - } - } - -#ifndef BOOST_NO_AUTO_PTR - - template - explicit shared_ptr(std::auto_ptr & r): px(r.get()), pn() - { - Y * tmp = r.get(); - pn = detail::shared_count(r); - detail::sp_enable_shared_from_this( pn, tmp, tmp ); - } - -#endif - -#if !defined(BOOST_MSVC) || (BOOST_MSVC > 1200) - - template - shared_ptr & operator=(shared_ptr const & r) // never throws - { - px = r.px; - pn = r.pn; // shared_count::op= doesn't throw - return *this; - } - -#endif - -#ifndef BOOST_NO_AUTO_PTR - - template - shared_ptr & operator=(std::auto_ptr & r) - { - this_type(r).swap(*this); - return *this; - } - -#endif - - void reset() // never throws in 1.30+ - { - this_type().swap(*this); - } - - template void reset(Y * p) // Y must be complete - { - BOOST_ASSERT(p == 0 || p != px); // catch self-reset errors - this_type(p).swap(*this); - } - - template void reset(Y * p, D d) - { - this_type(p, d).swap(*this); - } - - reference operator* () const // never throws - { - BOOST_ASSERT(px != 0); - return *px; - } - - T * operator-> () const // never throws - { - BOOST_ASSERT(px != 0); - return px; - } - - T * get() const // never throws - { - return px; - } - - // implicit conversion to "bool" - -#if defined(__SUNPRO_CC) && BOOST_WORKAROUND(__SUNPRO_CC, <= 0x530) - - operator bool () const - { - return px != 0; - } - -#elif defined(__MWERKS__) && BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3003)) - typedef T * (this_type::*unspecified_bool_type)() const; - - operator unspecified_bool_type() const // never throws - { - return px == 0? 0: &this_type::get; - } - -#else - - typedef T * this_type::*unspecified_bool_type; - - operator unspecified_bool_type() const // never throws - { - return px == 0? 0: &this_type::px; - } - -#endif - - // operator! is redundant, but some compilers need it - - bool operator! () const // never throws - { - return px == 0; - } - - bool unique() const // never throws - { - return pn.unique(); - } - - long use_count() const // never throws - { - return pn.use_count(); - } - - void swap(shared_ptr< T > & other) // never throws - { - std::swap(px, other.px); - pn.swap(other.pn); - } - - template bool _internal_less(shared_ptr const & rhs) const - { - return pn < rhs.pn; - } - - void * _internal_get_deleter(std::type_info const & ti) const - { - return pn.get_deleter(ti); - } - -// Tasteless as this may seem, making all members public allows member templates -// to work in the absence of member template friends. (Matthew Langston) - -#ifndef BOOST_NO_MEMBER_TEMPLATE_FRIENDS - -private: - - template friend class shared_ptr; - template friend class weak_ptr; - - -#endif -public: // for serialization - T * px; // contained pointer - detail::shared_count pn; // reference counter - -}; // shared_ptr - -template inline bool operator==(shared_ptr< T > const & a, shared_ptr const & b) -{ - return a.get() == b.get(); -} - -template inline bool operator!=(shared_ptr< T > const & a, shared_ptr const & b) -{ - return a.get() != b.get(); -} - -template inline bool operator<(shared_ptr< T > const & a, shared_ptr const & b) -{ - return a._internal_less(b); -} - -template inline void swap(shared_ptr< T > & a, shared_ptr< T > & b) -{ - a.swap(b); -} - -template shared_ptr< T > static_pointer_cast(shared_ptr const & r) -{ - return shared_ptr< T >(r, detail::static_cast_tag()); -} - -template shared_ptr< T > const_pointer_cast(shared_ptr const & r) -{ - return shared_ptr< T >(r, detail::const_cast_tag()); -} - -template shared_ptr< T > dynamic_pointer_cast(shared_ptr const & r) -{ - return shared_ptr< T >(r, detail::dynamic_cast_tag()); -} - -// shared_*_cast names are deprecated. Use *_pointer_cast instead. - -template shared_ptr< T > shared_static_cast(shared_ptr const & r) -{ - return shared_ptr< T >(r, detail::static_cast_tag()); -} - -template shared_ptr< T > shared_dynamic_cast(shared_ptr const & r) -{ - return shared_ptr< T >(r, detail::dynamic_cast_tag()); -} - -template shared_ptr< T > shared_polymorphic_cast(shared_ptr const & r) -{ - return shared_ptr< T >(r, detail::polymorphic_cast_tag()); -} - -template shared_ptr< T > shared_polymorphic_downcast(shared_ptr const & r) -{ - BOOST_ASSERT(dynamic_cast(r.get()) == r.get()); - return shared_static_cast< T >(r); -} - -// get_pointer() enables boost::mem_fn to recognize shared_ptr - -template inline T * get_pointer(shared_ptr< T > const & p) -{ - return p.get(); -} - -// operator<< - - -template std::basic_ostream & operator<< (std::basic_ostream & os, shared_ptr const & p) -{ - os << p.get(); - return os; -} - -// get_deleter (experimental) - -#if defined(__EDG_VERSION__) && (__EDG_VERSION__ <= 238) - -// g++ 2.9x doesn't allow static_cast(void *) -// apparently EDG 2.38 also doesn't accept it - -template D * get_deleter(shared_ptr< T > const & p) -{ - void const * q = p._internal_get_deleter(typeid(D)); - return const_cast(static_cast(q)); -} - -#else - -template D * get_deleter(shared_ptr< T > const & p) -{ - return static_cast(p._internal_get_deleter(typeid(D))); -} - -#endif - -} // namespace boost - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -#endif // #if defined(BOOST_NO_MEMBER_TEMPLATES) && !defined(BOOST_MSVC6_MEMBER_TEMPLATES) - -#endif // #ifndef BOOST_SHARED_PTR_132_HPP_INCLUDED diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_nmt_132.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_nmt_132.hpp deleted file mode 100644 index 490e7ddd3d0..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_nmt_132.hpp +++ /dev/null @@ -1,182 +0,0 @@ -#ifndef BOOST_DETAIL_SHARED_PTR_NMT_132_HPP_INCLUDED -#define BOOST_DETAIL_SHARED_PTR_NMT_132_HPP_INCLUDED - -// -// detail/shared_ptr_nmt.hpp - shared_ptr.hpp without member templates -// -// (C) Copyright Greg Colvin and Beman Dawes 1998, 1999. -// Copyright (c) 2001, 2002 Peter Dimov -// -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// -// See http://www.boost.org/libs/smart_ptr/shared_ptr.htm for documentation. -// - -#include -#include -#include -#include - -#ifndef BOOST_NO_AUTO_PTR -# include // for std::auto_ptr -#endif - -#include // for std::swap -#include // for std::less -#include // for std::bad_alloc - -namespace boost -{ - -template class shared_ptr -{ -private: - - typedef detail::atomic_count count_type; - -public: - - typedef T element_type; - typedef T value_type; - - explicit shared_ptr(T * p = 0): px(p) - { -#ifndef BOOST_NO_EXCEPTIONS - - try // prevent leak if new throws - { - pn = new count_type(1); - } - catch(...) - { - boost::checked_delete(p); - throw; - } - -#else - - pn = new count_type(1); - - if(pn == 0) - { - boost::checked_delete(p); - boost::serialization::throw_exception(std::bad_alloc()); - } - -#endif - } - - ~shared_ptr() - { - if(--*pn == 0) - { - boost::checked_delete(px); - delete pn; - } - } - - shared_ptr(shared_ptr const & r): px(r.px) // never throws - { - pn = r.pn; - ++*pn; - } - - shared_ptr & operator=(shared_ptr const & r) - { - shared_ptr(r).swap(*this); - return *this; - } - -#ifndef BOOST_NO_AUTO_PTR - - explicit shared_ptr(std::auto_ptr< T > & r) - { - pn = new count_type(1); // may throw - px = r.release(); // fix: moved here to stop leak if new throws - } - - shared_ptr & operator=(std::auto_ptr< T > & r) - { - shared_ptr(r).swap(*this); - return *this; - } - -#endif - - void reset(T * p = 0) - { - BOOST_ASSERT(p == 0 || p != px); - shared_ptr(p).swap(*this); - } - - T & operator*() const // never throws - { - BOOST_ASSERT(px != 0); - return *px; - } - - T * operator->() const // never throws - { - BOOST_ASSERT(px != 0); - return px; - } - - T * get() const // never throws - { - return px; - } - - long use_count() const // never throws - { - return *pn; - } - - bool unique() const // never throws - { - return *pn == 1; - } - - void swap(shared_ptr< T > & other) // never throws - { - std::swap(px, other.px); - std::swap(pn, other.pn); - } - -private: - - T * px; // contained pointer - count_type * pn; // ptr to reference counter -}; - -template inline bool operator==(shared_ptr< T > const & a, shared_ptr const & b) -{ - return a.get() == b.get(); -} - -template inline bool operator!=(shared_ptr< T > const & a, shared_ptr const & b) -{ - return a.get() != b.get(); -} - -template inline bool operator<(shared_ptr< T > const & a, shared_ptr< T > const & b) -{ - return std::less()(a.get(), b.get()); -} - -template void swap(shared_ptr< T > & a, shared_ptr< T > & b) -{ - a.swap(b); -} - -// get_pointer() enables boost::mem_fn to recognize shared_ptr - -template inline T * get_pointer(shared_ptr< T > const & p) -{ - return p.get(); -} - -} // namespace boost - -#endif // #ifndef BOOST_DETAIL_SHARED_PTR_NMT_132_HPP_INCLUDED diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/stack_constructor.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/stack_constructor.hpp deleted file mode 100644 index ae14832c6db..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/detail/stack_constructor.hpp +++ /dev/null @@ -1,66 +0,0 @@ -#ifndef BOOST_SERIALIZATION_DETAIL_STACK_CONSTRUCTOR_HPP -#define BOOST_SERIALIZATION_DETAIL_STACK_CONSTRUCTOR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// stack_constructor.hpp: serialization for loading stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -namespace boost{ -namespace serialization { -namespace detail { - -// reserve space on stack for an object of type T without actually -// construction such an object -template -struct stack_allocate -{ - T * address() { - return static_cast(storage_.address()); - } - T & reference() { - return * address(); - } -private: - typedef typename boost::aligned_storage< - sizeof(T), - boost::alignment_of::value - > type; - type storage_; -}; - -// construct element on the stack -template -struct stack_construct : public stack_allocate -{ - stack_construct(Archive & ar, const unsigned int version){ - // note borland emits a no-op without the explicit namespace - boost::serialization::load_construct_data_adl( - ar, - this->address(), - version - ); - } - ~stack_construct(){ - this->address()->~T(); // undo load_construct_data above - } -}; - -} // detail -} // serializaition -} // boost - -#endif // BOOST_SERIALIZATION_DETAIL_STACH_CONSTRUCTOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/ephemeral.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/ephemeral.hpp deleted file mode 100644 index 3a422c30a35..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/ephemeral.hpp +++ /dev/null @@ -1,72 +0,0 @@ -#ifndef BOOST_SERIALIZATION_EPHEMERAL_HPP -#define BOOST_SERIALIZATION_EPHEMERAL_HPP - -// MS compatible compilers support -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// ephemeral_object.hpp: interface for serialization system. - -// (C) Copyright 2007 Matthias Troyer. -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -#include -#include - -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template -struct ephemeral_object : - public wrapper_traits > -{ - explicit ephemeral_object(T& t) : - val(t) - {} - - T & value() const { - return val; - } - - const T & const_value() const { - return val; - } - - template - void serialize(Archive &ar, const unsigned int) const - { - ar & val; - } - -private: - T & val; -}; - -template -inline -const ephemeral_object ephemeral(const char * name, T & t){ - return ephemeral_object(name, t); -} - -} // seralization -} // boost - -#endif // BOOST_SERIALIZATION_EPHEMERAL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/export.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/export.hpp deleted file mode 100644 index 9eef440df42..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/export.hpp +++ /dev/null @@ -1,225 +0,0 @@ -#ifndef BOOST_SERIALIZATION_EXPORT_HPP -#define BOOST_SERIALIZATION_EXPORT_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// export.hpp: set traits of classes to be serialized - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// (C) Copyright 2006 David Abrahams - http://www.boost.org. -// implementation of class export functionality. This is an alternative to -// "forward declaration" method to provoke instantiation of derived classes -// that are to be serialized through pointers. - -#include -#include // NULL - -#include -#include -#include -#include - -#include -#include -#include -#include - -#include // for guid_defined only -#include -#include -#include -#include - -#include - -#include - -namespace boost { -namespace archive { -namespace detail { - -class basic_pointer_iserializer; -class basic_pointer_oserializer; - -template -class pointer_iserializer; -template -class pointer_oserializer; - -template -struct export_impl -{ - static const basic_pointer_iserializer & - enable_load(mpl::true_){ - return boost::serialization::singleton< - pointer_iserializer - >::get_const_instance(); - } - - static const basic_pointer_oserializer & - enable_save(mpl::true_){ - return boost::serialization::singleton< - pointer_oserializer - >::get_const_instance(); - } - inline static void enable_load(mpl::false_) {} - inline static void enable_save(mpl::false_) {} -}; - -// On many platforms, naming a specialization of this template is -// enough to cause its argument to be instantiated. -template -struct instantiate_function {}; - -template -struct ptr_serialization_support -{ -# if defined(BOOST_MSVC) || defined(__SUNPRO_CC) - virtual BOOST_DLLEXPORT void instantiate() BOOST_USED; -# else - static BOOST_DLLEXPORT void instantiate() BOOST_USED; - typedef instantiate_function< - &ptr_serialization_support::instantiate - > x; -# endif -}; - -template -BOOST_DLLEXPORT void -ptr_serialization_support::instantiate() -{ - export_impl::enable_save( - typename Archive::is_saving() - ); - - export_impl::enable_load( - typename Archive::is_loading() - ); -} - -// Note INTENTIONAL usage of anonymous namespace in header. -// This was made this way so that export.hpp could be included -// in other headers. This is still under study. - -namespace extra_detail { - -template -struct guid_initializer -{ - void export_guid(mpl::false_) const { - // generates the statically-initialized objects whose constructors - // register the information allowing serialization of T objects - // through pointers to their base classes. - instantiate_ptr_serialization((T*)0, 0, adl_tag()); - } - void export_guid(mpl::true_) const { - } - guid_initializer const & export_guid() const { - BOOST_STATIC_WARNING(boost::is_polymorphic< T >::value); - // note: exporting an abstract base class will have no effect - // and cannot be used to instantitiate serialization code - // (one might be using this in a DLL to instantiate code) - //BOOST_STATIC_WARNING(! boost::serialization::is_abstract< T >::value); - export_guid(boost::serialization::is_abstract< T >()); - return *this; - } -}; - -template -struct init_guid; - -} // anonymous -} // namespace detail -} // namespace archive -} // namespace boost - -#define BOOST_CLASS_EXPORT_IMPLEMENT(T) \ - namespace boost { \ - namespace archive { \ - namespace detail { \ - namespace extra_detail { \ - template<> \ - struct init_guid< T > { \ - static guid_initializer< T > const & g; \ - }; \ - guid_initializer< T > const & init_guid< T >::g = \ - ::boost::serialization::singleton< \ - guid_initializer< T > \ - >::get_mutable_instance().export_guid(); \ - }}}} \ -/**/ - -#define BOOST_CLASS_EXPORT_KEY2(T, K) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct guid_defined< T > : boost::mpl::true_ {}; \ -template<> \ -inline const char * guid< T >(){ \ - return K; \ -} \ -} /* serialization */ \ -} /* boost */ \ -/**/ - -#define BOOST_CLASS_EXPORT_KEY(T) \ - BOOST_CLASS_EXPORT_KEY2(T, BOOST_PP_STRINGIZE(T)) \ -/**/ - -#define BOOST_CLASS_EXPORT_GUID(T, K) \ -BOOST_CLASS_EXPORT_KEY2(T, K) \ -BOOST_CLASS_EXPORT_IMPLEMENT(T) \ -/**/ - -#if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) - -// CodeWarrior fails to construct static members of class templates -// when they are instantiated from within templates, so on that -// compiler we ask users to specifically register base/derived class -// relationships for exported classes. On all other compilers, use of -// this macro is entirely optional. -# define BOOST_SERIALIZATION_MWERKS_BASE_AND_DERIVED(Base,Derived) \ -namespace { \ - static int BOOST_PP_CAT(boost_serialization_mwerks_init_, __LINE__) = \ - (::boost::archive::detail::instantiate_ptr_serialization((Derived*)0,0), 3); \ - static int BOOST_PP_CAT(boost_serialization_mwerks_init2_, __LINE__) = ( \ - ::boost::serialization::void_cast_register((Derived*)0,(Base*)0) \ - , 3); \ -} - -#else - -# define BOOST_SERIALIZATION_MWERKS_BASE_AND_DERIVED(Base,Derived) - -#endif - -// check for unnecessary export. T isn't polymorphic so there is no -// need to export it. -#define BOOST_CLASS_EXPORT_CHECK(T) \ - BOOST_STATIC_WARNING( \ - boost::is_polymorphic::value \ - ); \ - /**/ - -// the default exportable class identifier is the class name -// the default list of archives types for which code id generated -// are the originally included with this serialization system -#define BOOST_CLASS_EXPORT(T) \ - BOOST_CLASS_EXPORT_GUID( \ - T, \ - BOOST_PP_STRINGIZE(T) \ - ) \ - /**/ - -#endif // BOOST_SERIALIZATION_EXPORT_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info.hpp deleted file mode 100644 index bb2a190d465..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info.hpp +++ /dev/null @@ -1,116 +0,0 @@ -#ifndef BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_HPP -#define BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// extended_type_info.hpp: interface for portable version of type_info - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// for now, extended type info is part of the serialization libraries -// this could change in the future. -#include -#include -#include // NULL -#include -#include -#include - -#include -#include // must be the last header -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4251 4231 4660 4275) -#endif - -#define BOOST_SERIALIZATION_MAX_KEY_SIZE 128 - -namespace boost { -namespace serialization { - -namespace void_cast_detail{ - class void_caster; -} - -class BOOST_SYMBOL_VISIBLE extended_type_info : - private boost::noncopyable -{ -private: - friend class boost::serialization::void_cast_detail::void_caster; - - // used to uniquely identify the type of class derived from this one - // so that different derivations of this class can be simultaneously - // included in implementation of sets and maps. - const unsigned int m_type_info_key; - virtual bool is_less_than(const extended_type_info & /*rhs*/) const = 0; - virtual bool is_equal(const extended_type_info & /*rhs*/) const = 0; - const char * m_key; - -protected: - BOOST_SERIALIZATION_DECL void key_unregister() const; - BOOST_SERIALIZATION_DECL void key_register() const; - // this class can't be used as is. It's just the - // common functionality for all type_info replacement - // systems. Hence, make these protected - BOOST_SERIALIZATION_DECL extended_type_info( - const unsigned int type_info_key, - const char * key - ); - virtual BOOST_SERIALIZATION_DECL ~extended_type_info(); -public: - const char * get_key() const { - return m_key; - } - virtual const char * get_debug_info() const = 0; - BOOST_SERIALIZATION_DECL bool operator<(const extended_type_info &rhs) const; - BOOST_SERIALIZATION_DECL bool operator==(const extended_type_info &rhs) const; - bool operator!=(const extended_type_info &rhs) const { - return !(operator==(rhs)); - } - // note explicit "export" of static function to work around - // gcc 4.5 mingw error - static BOOST_SERIALIZATION_DECL const extended_type_info * - find(const char *key); - // for plugins - virtual void * construct(unsigned int /*count*/ = 0, ...) const = 0; - virtual void destroy(void const * const /*p*/) const = 0; -}; - -template -struct guid_defined : boost::mpl::false_ {}; - -namespace ext { - template - struct guid_impl - { - static inline const char * call() - { - return NULL; - } - }; -} - -template -inline const char * guid(){ - return ext::guid_impl::call(); -} - -} // namespace serialization -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_no_rtti.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_no_rtti.hpp deleted file mode 100644 index aaa8b44459b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_no_rtti.hpp +++ /dev/null @@ -1,182 +0,0 @@ -#ifndef BOOST_EXTENDED_TYPE_INFO_NO_RTTI_HPP -#define BOOST_EXTENDED_TYPE_INFO_NO_RTTI_HPP - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -// extended_type_info_no_rtti.hpp: implementation for version that depends -// on runtime typing (rtti - typeid) but uses a user specified string -// as the portable class identifier. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include - -#include -#include - -#include -#include -#include - -#include -#include -#include -#include -#include - -#include -// hijack serialization access -#include - -#include // must be the last header -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4251 4231 4660 4275 4511 4512) -#endif - -namespace boost { -namespace serialization { -/////////////////////////////////////////////////////////////////////// -// define a special type_info that doesn't depend on rtti which is not -// available in all situations. - -namespace no_rtti_system { - -// common base class to share type_info_key. This is used to -// identify the method used to keep track of the extended type -class BOOST_SYMBOL_VISIBLE extended_type_info_no_rtti_0 : - public extended_type_info -{ -protected: - BOOST_SERIALIZATION_DECL extended_type_info_no_rtti_0(const char * key); - BOOST_SERIALIZATION_DECL ~extended_type_info_no_rtti_0(); -public: - virtual BOOST_SERIALIZATION_DECL bool - is_less_than(const boost::serialization::extended_type_info &rhs) const ; - virtual BOOST_SERIALIZATION_DECL bool - is_equal(const boost::serialization::extended_type_info &rhs) const ; -}; - -} // no_rtti_system - -template -class extended_type_info_no_rtti : - public no_rtti_system::extended_type_info_no_rtti_0, - public singleton > -{ - template - struct action { - struct defined { - static const char * invoke(){ - return guid< T >(); - } - }; - struct undefined { - // if your program traps here - you failed to - // export a guid for this type. the no_rtti - // system requires export for types serialized - // as pointers. - BOOST_STATIC_ASSERT(0 == sizeof(T)); - static const char * invoke(); - }; - static const char * invoke(){ - typedef - typename boost::mpl::if_c< - tf, - defined, - undefined - >::type type; - return type::invoke(); - } - }; -public: - extended_type_info_no_rtti() : - no_rtti_system::extended_type_info_no_rtti_0(get_key()) - { - key_register(); - } - ~extended_type_info_no_rtti(){ - key_unregister(); - } - const extended_type_info * - get_derived_extended_type_info(const T & t) const { - // find the type that corresponds to the most derived type. - // this implementation doesn't depend on typeid() but assumes - // that the specified type has a function of the following signature. - // A common implemention of such a function is to define as a virtual - // function. So if the is not a polymporphic type it's likely an error - BOOST_STATIC_WARNING(boost::is_polymorphic< T >::value); - const char * derived_key = t.get_key(); - BOOST_ASSERT(NULL != derived_key); - return boost::serialization::extended_type_info::find(derived_key); - } - const char * get_key() const{ - return action::value >::invoke(); - } - virtual const char * get_debug_info() const{ - return action::value >::invoke(); - } - virtual void * construct(unsigned int count, ...) const{ - // count up the arguments - std::va_list ap; - va_start(ap, count); - switch(count){ - case 0: - return factory::type, 0>(ap); - case 1: - return factory::type, 1>(ap); - case 2: - return factory::type, 2>(ap); - case 3: - return factory::type, 3>(ap); - case 4: - return factory::type, 4>(ap); - default: - BOOST_ASSERT(false); // too many arguments - // throw exception here? - return NULL; - } - } - virtual void destroy(void const * const p) const{ - boost::serialization::access::destroy( - static_cast(p) - ); - //delete static_cast(p) ; - } -}; - -} // namespace serialization -} // namespace boost - -/////////////////////////////////////////////////////////////////////////////// -// If no other implementation has been designated as default, -// use this one. To use this implementation as the default, specify it -// before any of the other headers. - -#ifndef BOOST_SERIALIZATION_DEFAULT_TYPE_INFO - #define BOOST_SERIALIZATION_DEFAULT_TYPE_INFO - namespace boost { - namespace serialization { - template - struct extended_type_info_impl { - typedef typename - boost::serialization::extended_type_info_no_rtti< T > type; - }; - } // namespace serialization - } // namespace boost -#endif - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_EXTENDED_TYPE_INFO_NO_RTTI_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_typeid.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_typeid.hpp deleted file mode 100644 index 8ee591b3169..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_typeid.hpp +++ /dev/null @@ -1,167 +0,0 @@ -#ifndef BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_TYPEID_HPP -#define BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_TYPEID_HPP - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -// extended_type_info_typeid.hpp: implementation for version that depends -// on runtime typing (rtti - typeid) but uses a user specified string -// as the portable class identifier. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -#include -#include -#include -#include - -#include -#include -#include -#include - -// hijack serialization access -#include - -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4251 4231 4660 4275 4511 4512) -#endif - -namespace boost { -namespace serialization { -namespace typeid_system { - -class BOOST_SYMBOL_VISIBLE extended_type_info_typeid_0 : - public extended_type_info -{ - virtual const char * get_debug_info() const { - if(static_cast(0) == m_ti) - return static_cast(0); - return m_ti->name(); - } -protected: - const std::type_info * m_ti; - BOOST_SERIALIZATION_DECL extended_type_info_typeid_0(const char * key); - BOOST_SERIALIZATION_DECL ~extended_type_info_typeid_0(); - BOOST_SERIALIZATION_DECL void type_register(const std::type_info & ti); - BOOST_SERIALIZATION_DECL void type_unregister(); - BOOST_SERIALIZATION_DECL const extended_type_info * - get_extended_type_info(const std::type_info & ti) const; -public: - virtual BOOST_SERIALIZATION_DECL bool - is_less_than(const extended_type_info &rhs) const; - virtual BOOST_SERIALIZATION_DECL bool - is_equal(const extended_type_info &rhs) const; - const std::type_info & get_typeid() const { - return *m_ti; - } -}; - -} // typeid_system - -template -class extended_type_info_typeid : - public typeid_system::extended_type_info_typeid_0, - public singleton > -{ -public: - extended_type_info_typeid() : - typeid_system::extended_type_info_typeid_0( - boost::serialization::guid< T >() - ) - { - type_register(typeid(T)); - key_register(); - } - ~extended_type_info_typeid(){ - key_unregister(); - type_unregister(); - } - // get the eti record for the true type of this record - // relying upon standard type info implemenation (rtti) - const extended_type_info * - get_derived_extended_type_info(const T & t) const { - // note: this implementation - based on usage of typeid (rtti) - // only does something if the class has at least one virtual function. - BOOST_STATIC_WARNING(boost::is_polymorphic< T >::value); - return - typeid_system::extended_type_info_typeid_0::get_extended_type_info( - typeid(t) - ); - } - const char * get_key() const { - return boost::serialization::guid< T >(); - } - virtual void * construct(unsigned int count, ...) const{ - // count up the arguments - std::va_list ap; - va_start(ap, count); - switch(count){ - case 0: - return factory::type, 0>(ap); - case 1: - return factory::type, 1>(ap); - case 2: - return factory::type, 2>(ap); - case 3: - return factory::type, 3>(ap); - case 4: - return factory::type, 4>(ap); - default: - BOOST_ASSERT(false); // too many arguments - // throw exception here? - return NULL; - } - } - virtual void destroy(void const * const p) const { - boost::serialization::access::destroy( - static_cast(p) - ); - //delete static_cast(p); - } -}; - -} // namespace serialization -} // namespace boost - -/////////////////////////////////////////////////////////////////////////////// -// If no other implementation has been designated as default, -// use this one. To use this implementation as the default, specify it -// before any of the other headers. -#ifndef BOOST_SERIALIZATION_DEFAULT_TYPE_INFO - #define BOOST_SERIALIZATION_DEFAULT_TYPE_INFO - namespace boost { - namespace serialization { - template - struct extended_type_info_impl { - typedef typename - boost::serialization::extended_type_info_typeid< T > type; - }; - } // namespace serialization - } // namespace boost -#endif - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_TYPEID_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/factory.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/factory.hpp deleted file mode 100644 index 2db7e7e36c3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/factory.hpp +++ /dev/null @@ -1,102 +0,0 @@ -#ifndef BOOST_SERIALIZATION_FACTORY_HPP -#define BOOST_SERIALIZATION_FACTORY_HPP - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -// factory.hpp: create an instance from an extended_type_info instance. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // valist -#include // NULL - -#include -#include -#include - -namespace std{ - #if defined(__LIBCOMO__) - using ::va_list; - #endif -} // namespace std - -namespace boost { -namespace serialization { - -// default implementation does nothing. -template -T * factory(std::va_list){ - BOOST_ASSERT(false); - // throw exception here? - return NULL; -} - -} // namespace serialization -} // namespace boost - -#define BOOST_SERIALIZATION_FACTORY(N, T, A0, A1, A2, A3) \ -namespace boost { \ -namespace serialization { \ - template<> \ - T * factory(std::va_list ap){ \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 0) \ - , A0 a0 = va_arg(ap, A0);, BOOST_PP_EMPTY()) \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 1) \ - , A1 a1 = va_arg(ap, A1);, BOOST_PP_EMPTY()) \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 2) \ - , A2 a2 = va_arg(ap, A2);, BOOST_PP_EMPTY()) \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 3) \ - , A3 a3 = va_arg(ap, A3);, BOOST_PP_EMPTY()) \ - return new T( \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 0) \ - , a0, BOOST_PP_EMPTY()) \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 1)) \ - , BOOST_PP_COMMA, BOOST_PP_EMPTY)() \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 1) \ - , a1, BOOST_PP_EMPTY()) \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 2)) \ - , BOOST_PP_COMMA, BOOST_PP_EMPTY)() \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 2) \ - , a2, BOOST_PP_EMPTY()) \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 3)) \ - , BOOST_PP_COMMA, BOOST_PP_EMPTY)() \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 3) \ - , a3, BOOST_PP_EMPTY()) \ - ); \ - } \ -} \ -} /**/ - -#define BOOST_SERIALIZATION_FACTORY_4(T, A0, A1, A2, A3) \ - BOOST_SERIALIZATION_FACTORY(4, T, A0, A1, A2, A3) - -#define BOOST_SERIALIZATION_FACTORY_3(T, A0, A1, A2) \ - BOOST_SERIALIZATION_FACTORY(3, T, A0, A1, A2, 0) - -#define BOOST_SERIALIZATION_FACTORY_2(T, A0, A1) \ - BOOST_SERIALIZATION_FACTORY(2, T, A0, A1, 0, 0) - -#define BOOST_SERIALIZATION_FACTORY_1(T, A0) \ - BOOST_SERIALIZATION_FACTORY(1, T, A0, 0, 0, 0) - -#define BOOST_SERIALIZATION_FACTORY_0(T) \ -namespace boost { \ -namespace serialization { \ - template<> \ - T * factory(std::va_list){ \ - return new T(); \ - } \ -} \ -} \ -/**/ - -#endif // BOOST_SERIALIZATION_FACTORY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/force_include.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/force_include.hpp deleted file mode 100644 index 55ab79d0d58..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/force_include.hpp +++ /dev/null @@ -1,55 +0,0 @@ -#ifndef BOOST_SERIALIZATION_FORCE_INCLUDE_HPP -#define BOOST_SERIALIZATION_FORCE_INCLUDE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// force_include.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -// the following help macro is to guarentee that certain coded -// is not removed by over-eager linker optimiser. In certain cases -// we create static objects must be created but are actually never -// referenced - creation has a side-effect such as global registration -// which is important to us. We make an effort to refer these objects -// so that a smart linker won't remove them as being unreferenced. -// In microsoft compilers, inlining the code that does the referring -// means the code gets lost and the static object is not included -// in the library and hence never registered. This manifests itself -// in an ungraceful crash at runtime when (and only when) built in -// release mode. - -#if defined(BOOST_HAS_DECLSPEC) && !defined(__COMO__) -# define BOOST_DLLEXPORT __declspec(dllexport) -#elif ! defined(_WIN32) && ! defined(_WIN64) -# if defined(__MWERKS__) -# define BOOST_DLLEXPORT __declspec(dllexport) -# elif defined(__GNUC__) && (__GNUC__ >= 3) -# define BOOST_USED __attribute__ ((__used__)) -# elif defined(__IBMCPP__) && (__IBMCPP__ >= 1110) -# define BOOST_USED __attribute__ ((__used__)) -# elif defined(__INTEL_COMPILER) && (BOOST_INTEL_CXX_VERSION >= 800) -# define BOOST_USED __attribute__ ((__used__)) -# endif -#endif - -#ifndef BOOST_USED -# define BOOST_USED -#endif - -#ifndef BOOST_DLLEXPORT -# define BOOST_DLLEXPORT -#endif - -#endif // BOOST_SERIALIZATION_FORCE_INCLUDE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/forward_list.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/forward_list.hpp deleted file mode 100644 index b8a3c20a6ea..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/forward_list.hpp +++ /dev/null @@ -1,124 +0,0 @@ -#ifndef BOOST_SERIALIZATION_FORWARD_LIST_HPP -#define BOOST_SERIALIZATION_FORWARD_LIST_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// forward_list.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include // distance - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template -inline void save( - Archive & ar, - const std::forward_list &t, - const unsigned int /*file_version*/ -){ - const collection_size_type count(std::distance(t.cbegin(), t.cend())); - boost::serialization::stl::save_collection< - Archive, - std::forward_list - >(ar, t, count); -} - -namespace stl { - -template< - class Archive, - class T, - class Allocator -> -typename boost::disable_if< - typename detail::is_default_constructible< - typename std::forward_list::value_type - >, - void ->::type -collection_load_impl( - Archive & ar, - std::forward_list &t, - collection_size_type count, - item_version_type item_version -){ - t.clear(); - boost::serialization::detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - t.push_front(boost::move(u.reference())); - typename std::forward_list::iterator last; - last = t.begin(); - ar.reset_object_address(&(*t.begin()) , & u.reference()); - while(--count > 0){ - detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - last = t.insert_after(last, boost::move(u.reference())); - ar.reset_object_address(&(*last) , & u.reference()); - } -} - -} // stl - -template -inline void load( - Archive & ar, - std::forward_list &t, - const unsigned int /*file_version*/ -){ - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - stl::collection_load_impl(ar, t, count, item_version); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::forward_list &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(std::forward_list) - -#endif // BOOST_SERIALIZATION_FORWARD_LIST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_load_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_load_imp.hpp deleted file mode 100644 index 88def8f1aa4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_load_imp.hpp +++ /dev/null @@ -1,77 +0,0 @@ -#ifndef BOOST_SERIALIZATION_HASH_COLLECTIONS_LOAD_IMP_HPP -#define BOOST_SERIALIZATION_HASH_COLLECTIONS_LOAD_IMP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -# pragma warning (disable : 4786) // too long name, harmless warning -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// hash_collections_load_imp.hpp: serialization for loading stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// helper function templates for serialization of hashed collections -#include -#include -#include -#include -#include - -namespace boost{ -namespace serialization { -namespace stl { - -////////////////////////////////////////////////////////////////////// -// implementation of serialization for STL containers -// -template -inline void load_hash_collection(Archive & ar, Container &s) -{ - collection_size_type count; - collection_size_type bucket_count; - boost::serialization::item_version_type item_version(0); - boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - if(boost::archive::library_version_type(6) != library_version){ - ar >> BOOST_SERIALIZATION_NVP(count); - ar >> BOOST_SERIALIZATION_NVP(bucket_count); - } - else{ - // note: fixup for error in version 6. collection size was - // changed to size_t BUT for hashed collections it was implemented - // as an unsigned int. This should be a problem only on win64 machines - // but I'll leave it for everyone just in case. - unsigned int c; - unsigned int bc; - ar >> BOOST_SERIALIZATION_NVP(c); - count = c; - ar >> BOOST_SERIALIZATION_NVP(bc); - bucket_count = bc; - } - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - s.clear(); - #if ! defined(__MWERKS__) - s.resize(bucket_count); - #endif - InputFunction ifunc; - while(count-- > 0){ - ifunc(ar, s, item_version); - } -} - -} // namespace stl -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_HASH_COLLECTIONS_LOAD_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_save_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_save_imp.hpp deleted file mode 100644 index 65dfe83f16e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_save_imp.hpp +++ /dev/null @@ -1,97 +0,0 @@ -#ifndef BOOST_SERIALIZATION_HASH_COLLECTIONS_SAVE_IMP_HPP -#define BOOST_SERIALIZATION_HASH_COLLECTIONS_SAVE_IMP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// hash_collections_save_imp.hpp: serialization for stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// helper function templates for serialization of collections - -#include -#include -#include -#include -#include -#include - -namespace boost{ -namespace serialization { -namespace stl { - -////////////////////////////////////////////////////////////////////// -// implementation of serialization for STL containers -// - -template -inline void save_hash_collection(Archive & ar, const Container &s) -{ - collection_size_type count(s.size()); - const collection_size_type bucket_count(s.bucket_count()); - const item_version_type item_version( - version::value - ); - - #if 0 - /* should only be necessary to create archives of previous versions - * which is not currently supported. So for now comment this out - */ - boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - if(boost::archive::library_version_type(6) != library_version){ - ar << BOOST_SERIALIZATION_NVP(count); - ar << BOOST_SERIALIZATION_NVP(bucket_count); - } - else{ - // note: fixup for error in version 6. collection size was - // changed to size_t BUT for hashed collections it was implemented - // as an unsigned int. This should be a problem only on win64 machines - // but I'll leave it for everyone just in case. - const unsigned int c = count; - const unsigned int bc = bucket_count; - ar << BOOST_SERIALIZATION_NVP(c); - ar << BOOST_SERIALIZATION_NVP(bc); - } - if(boost::archive::library_version_type(3) < library_version){ - // record number of elements - // make sure the target type is registered so we can retrieve - // the version when we load - ar << BOOST_SERIALIZATION_NVP(item_version); - } - #else - ar << BOOST_SERIALIZATION_NVP(count); - ar << BOOST_SERIALIZATION_NVP(bucket_count); - ar << BOOST_SERIALIZATION_NVP(item_version); - #endif - - typename Container::const_iterator it = s.begin(); - while(count-- > 0){ - // note borland emits a no-op without the explicit namespace - boost::serialization::save_construct_data_adl( - ar, - &(*it), - boost::serialization::version< - typename Container::value_type - >::value - ); - ar << boost::serialization::make_nvp("item", *it++); - } -} - -} // namespace stl -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_HASH_COLLECTIONS_SAVE_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/hash_map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/hash_map.hpp deleted file mode 100644 index 22626db6838..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/hash_map.hpp +++ /dev/null @@ -1,232 +0,0 @@ -#ifndef BOOST_SERIALIZATION_HASH_MAP_HPP -#define BOOST_SERIALIZATION_HASH_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/hash_map.hpp: -// serialization for stl hash_map templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_HAS_HASH -#include BOOST_HASH_MAP_HEADER - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -namespace stl { - -// map input -template -struct archive_input_hash_map -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - // borland fails silently w/o full namespace - ar >> boost::serialization::make_nvp("item", t.reference()); - std::pair result = - s.insert(boost::move(t.reference())); - // note: the following presumes that the map::value_type was NOT tracked - // in the archive. This is the usual case, but here there is no way - // to determine that. - if(result.second){ - ar.reset_object_address( - & (result.first->second), - & t.reference().second - ); - } - } -}; - -// multimap input -template -struct archive_input_hash_multimap -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - // borland fails silently w/o full namespace - ar >> boost::serialization::make_nvp("item", t.reference()); - typename Container::const_iterator result - = s.insert(boost::move(t.reference())); - // note: the following presumes that the map::value_type was NOT tracked - // in the archive. This is the usual case, but here there is no way - // to determine that. - ar.reset_object_address( - & result->second, - & t.reference() - ); - } -}; - -} // stl - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const BOOST_STD_EXTENSION_NAMESPACE::hash_map< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::save_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_map< - Key, HashFcn, EqualKey, Allocator - > - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_map< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::load_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_map< - Key, HashFcn, EqualKey, Allocator - >, - boost::serialization::stl::archive_input_hash_map< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_map< - Key, HashFcn, EqualKey, Allocator - > - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_map< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// hash_multimap -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::save_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< - Key, HashFcn, EqualKey, Allocator - > - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::load_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< - Key, HashFcn, EqualKey, Allocator - >, - boost::serialization::stl::archive_input_hash_multimap< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< - Key, HashFcn, EqualKey, Allocator - > - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_HAS_HASH -#endif // BOOST_SERIALIZATION_HASH_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/hash_set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/hash_set.hpp deleted file mode 100644 index 0c72c18457e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/hash_set.hpp +++ /dev/null @@ -1,222 +0,0 @@ -#ifndef BOOST_SERIALIZATION_HASH_SET_HPP -#define BOOST_SERIALIZATION_HASH_SET_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// hash_set.hpp: serialization for stl hash_set templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_HAS_HASH -#include BOOST_HASH_SET_HEADER - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -namespace stl { - -// hash_set input -template -struct archive_input_hash_set -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - // borland fails silently w/o full namespace - ar >> boost::serialization::make_nvp("item", t.reference()); - std::pair result = - s.insert(boost::move(t.reference())); - if(result.second) - ar.reset_object_address(& (* result.first), & t.reference()); - } -}; - -// hash_multiset input -template -struct archive_input_hash_multiset -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - // borland fails silently w/o full namespace - ar >> boost::serialization::make_nvp("item", t.reference()); - typename Container::const_iterator result - = s.insert(boost::move(t.reference())); - ar.reset_object_address(& (* result), & t.reference()); - } -}; - -} // stl - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const BOOST_STD_EXTENSION_NAMESPACE::hash_set< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::save_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_set< - Key, HashFcn, EqualKey, Allocator - > - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_set< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::load_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_set< - Key, HashFcn, EqualKey, Allocator - >, - boost::serialization::stl::archive_input_hash_set< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_set< - Key, HashFcn, EqualKey, Allocator - > - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_set< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// hash_multiset -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::save_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< - Key, HashFcn, EqualKey, Allocator - > - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::load_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< - Key, HashFcn, EqualKey, Allocator - >, - boost::serialization::stl::archive_input_hash_multiset< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< - Key, HashFcn, EqualKey, Allocator - > - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< - Key, HashFcn, EqualKey, Allocator - > & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(BOOST_STD_EXTENSION_NAMESPACE::hash_set) -BOOST_SERIALIZATION_COLLECTION_TRAITS(BOOST_STD_EXTENSION_NAMESPACE::hash_multiset) - -#endif // BOOST_HAS_HASH -#endif // BOOST_SERIALIZATION_HASH_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/is_bitwise_serializable.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/is_bitwise_serializable.hpp deleted file mode 100644 index 7e24a2cb6d8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/is_bitwise_serializable.hpp +++ /dev/null @@ -1,46 +0,0 @@ -// (C) Copyright 2007 Matthias Troyer - -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// Authors: Matthias Troyer - -/** @file is_bitwise_serializable.hpp - * - * This header provides a traits class for determining whether a class - * can be serialized (in a non-portable way) just by copying the bits. - */ - - -#ifndef BOOST_SERIALIZATION_IS_BITWISE_SERIALIZABLE_HPP -#define BOOST_SERIALIZATION_IS_BITWISE_SERIALIZABLE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -#include -#include - -namespace boost { -namespace serialization { - template - struct is_bitwise_serializable - : public is_arithmetic< T > - {}; -} // namespace serialization -} // namespace boost - - -// define a macro to make explicit designation of this more transparent -#define BOOST_IS_BITWISE_SERIALIZABLE(T) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct is_bitwise_serializable< T > : mpl::true_ {}; \ -}} \ -/**/ - -#endif //BOOST_SERIALIZATION_IS_BITWISE_SERIALIZABLE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/item_version_type.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/item_version_type.hpp deleted file mode 100644 index f3e5adac6f8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/item_version_type.hpp +++ /dev/null @@ -1,68 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ITEM_VERSION_TYPE_HPP -#define BOOST_SERIALIZATION_ITEM_VERSION_TYPE_HPP - -// (C) Copyright 2010 Robert Ramey -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include // uint_least8_t -#include -#include -#include - -// fixes broken example build on x86_64-linux-gnu-gcc-4.6.0 -#include - -namespace boost { -namespace serialization { - -#if defined(_MSC_VER) -#pragma warning( push ) -#pragma warning( disable : 4244 4267 ) -#endif - -class item_version_type { -private: - typedef unsigned int base_type; - base_type t; -public: - // should be private - but MPI fails if it's not!!! - item_version_type(): t(0) {}; - explicit item_version_type(const unsigned int t_) : t(t_){ - BOOST_ASSERT(t_ <= boost::integer_traits::const_max); - } - item_version_type(const item_version_type & t_) : - t(t_.t) - {} - item_version_type & operator=(item_version_type rhs){ - t = rhs.t; - return *this; - } - // used for text output - operator base_type () const { - return t; - } - // used for text input - operator base_type & () { - return t; - } - bool operator==(const item_version_type & rhs) const { - return t == rhs.t; - } - bool operator<(const item_version_type & rhs) const { - return t < rhs.t; - } -}; - -#if defined(_MSC_VER) -#pragma warning( pop ) -#endif - -} } // end namespace boost::serialization - -BOOST_IS_BITWISE_SERIALIZABLE(item_version_type) - -BOOST_CLASS_IMPLEMENTATION(item_version_type, primitive_type) - -#endif //BOOST_SERIALIZATION_ITEM_VERSION_TYPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/level.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/level.hpp deleted file mode 100644 index f6a84d10422..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/level.hpp +++ /dev/null @@ -1,116 +0,0 @@ -#ifndef BOOST_SERIALIZATION_LEVEL_HPP -#define BOOST_SERIALIZATION_LEVEL_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// level.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include - -#include - -namespace boost { -namespace serialization { - -struct basic_traits; - -// default serialization implementation level -template -struct implementation_level_impl { - template - struct traits_class_level { - typedef typename U::level type; - }; - - typedef mpl::integral_c_tag tag; - // note: at least one compiler complained w/o the full qualification - // on basic traits below - typedef - typename mpl::eval_if< - is_base_and_derived, - traits_class_level< T >, - //else - typename mpl::eval_if< - is_fundamental< T >, - mpl::int_, - //else - typename mpl::eval_if< - is_class< T >, - mpl::int_, - //else - typename mpl::eval_if< - is_array< T >, - mpl::int_, - //else - typename mpl::eval_if< - is_enum< T >, - mpl::int_, - //else - mpl::int_ - > - > - > - > - >::type type; - // vc 7.1 doesn't like enums here - BOOST_STATIC_CONSTANT(int, value = type::value); -}; - -template -struct implementation_level : - public implementation_level_impl -{ -}; - -template -inline bool operator>=(implementation_level< T > t, enum level_type l) -{ - return t.value >= (int)l; -} - -} // namespace serialization -} // namespace boost - -// specify the level of serialization implementation for the class -// require that class info saved when versioning is used -#define BOOST_CLASS_IMPLEMENTATION(T, E) \ - namespace boost { \ - namespace serialization { \ - template <> \ - struct implementation_level_impl< const T > \ - { \ - typedef mpl::integral_c_tag tag; \ - typedef mpl::int_< E > type; \ - BOOST_STATIC_CONSTANT( \ - int, \ - value = implementation_level_impl::type::value \ - ); \ - }; \ - } \ - } - /**/ - -#endif // BOOST_SERIALIZATION_LEVEL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/level_enum.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/level_enum.hpp deleted file mode 100644 index baf64e04f31..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/level_enum.hpp +++ /dev/null @@ -1,55 +0,0 @@ -#ifndef BOOST_SERIALIZATION_LEVEL_ENUM_HPP -#define BOOST_SERIALIZATION_LEVEL_ENUM_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// level_enum.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -namespace boost { -namespace serialization { - -// for each class used in the program, specify which level -// of serialization should be implemented - -// names for each level -enum level_type -{ - // Don't serialize this type. An attempt to do so should - // invoke a compile time assertion. - not_serializable = 0, - // write/read this type directly to the archive. In this case - // serialization code won't be called. This is the default - // case for fundamental types. It presumes a member function or - // template in the archive class that can handle this type. - // there is no runtime overhead associated reading/writing - // instances of this level - primitive_type = 1, - // Serialize the objects of this type using the objects "serialize" - // function or template. This permits values to be written/read - // to/from archives but includes no class or version information. - object_serializable = 2, - /////////////////////////////////////////////////////////////////// - // once an object is serialized at one of the above levels, the - // corresponding archives cannot be read if the implementation level - // for the archive object is changed. - /////////////////////////////////////////////////////////////////// - // Add class information to the archive. Class information includes - // implementation level, class version and class name if available - object_class_info = 3 -}; - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_LEVEL_ENUM_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/list.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/list.hpp deleted file mode 100644 index 5fdc114d7ed..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/list.hpp +++ /dev/null @@ -1,85 +0,0 @@ -#ifndef BOOST_SERIALIZATION_LIST_HPP -#define BOOST_SERIALIZATION_LIST_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// list.hpp: serialization for stl list templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include - -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template -inline void save( - Archive & ar, - const std::list &t, - const unsigned int /* file_version */ -){ - boost::serialization::stl::save_collection< - Archive, - std::list - >(ar, t); -} - -template -inline void load( - Archive & ar, - std::list &t, - const unsigned int /* file_version */ -){ - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - stl::collection_load_impl(ar, t, count, item_version); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::list & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(std::list) - -#endif // BOOST_SERIALIZATION_LIST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/map.hpp deleted file mode 100644 index 9209864c8cf..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/map.hpp +++ /dev/null @@ -1,139 +0,0 @@ -#ifndef BOOST_SERIALIZATION_MAP_HPP -#define BOOST_SERIALIZATION_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/map.hpp: -// serialization for stl map templates - -// (C) Copyright 2002-2014 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implementation of serialization for map and mult-map STL containers - -template -inline void load_map_collection(Archive & ar, Container &s) -{ - s.clear(); - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - typename Container::iterator hint; - hint = s.begin(); - while(count-- > 0){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, item_version); - ar >> boost::serialization::make_nvp("item", t.reference()); - typename Container::iterator result = - s.insert(hint, boost::move(t.reference())); - ar.reset_object_address(& (result->second), & t.reference().second); - hint = result; - ++hint; - } -} - -// map -template -inline void save( - Archive & ar, - const std::map &t, - const unsigned int /* file_version */ -){ - boost::serialization::stl::save_collection< - Archive, - std::map - >(ar, t); -} - -template -inline void load( - Archive & ar, - std::map &t, - const unsigned int /* file_version */ -){ - load_map_collection(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::map &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// multimap -template -inline void save( - Archive & ar, - const std::multimap &t, - const unsigned int /* file_version */ -){ - boost::serialization::stl::save_collection< - Archive, - std::multimap - >(ar, t); -} - -template -inline void load( - Archive & ar, - std::multimap &t, - const unsigned int /* file_version */ -){ - load_map_collection(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::multimap &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/nvp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/nvp.hpp deleted file mode 100644 index 4e2297b3cc9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/nvp.hpp +++ /dev/null @@ -1,123 +0,0 @@ -#ifndef BOOST_SERIALIZATION_NVP_HPP -#define BOOST_SERIALIZATION_NVP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// nvp.hpp: interface for serialization system. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template -struct nvp : - public std::pair, - public wrapper_traits > -{ -//private: - nvp(const nvp & rhs) : - std::pair(rhs.first, rhs.second) - {} -public: - explicit nvp(const char * name_, T & t) : - // note: added _ to suppress useless gcc warning - std::pair(name_, & t) - {} - - const char * name() const { - return this->first; - } - T & value() const { - return *(this->second); - } - - const T & const_value() const { - return *(this->second); - } - - template - void save( - Archive & ar, - const unsigned int /* file_version */ - ) const { - ar.operator<<(const_value()); - } - template - void load( - Archive & ar, - const unsigned int /* file_version */ - ){ - ar.operator>>(value()); - } - BOOST_SERIALIZATION_SPLIT_MEMBER() -}; - -template -inline -const nvp< T > make_nvp(const char * name, T & t){ - return nvp< T >(name, t); -} - -// to maintain efficiency and portability, we want to assign -// specific serialization traits to all instances of this wrappers. -// we can't strait forward method below as it depends upon -// Partial Template Specialization and doing so would mean that wrappers -// wouldn't be treated the same on different platforms. This would -// break archive portability. Leave this here as reminder not to use it !!! - -template -struct implementation_level > -{ - typedef mpl::integral_c_tag tag; - typedef mpl::int_ type; - BOOST_STATIC_CONSTANT(int, value = implementation_level::type::value); -}; - -// nvp objects are generally created on the stack and are never tracked -template -struct tracking_level > -{ - typedef mpl::integral_c_tag tag; - typedef mpl::int_ type; - BOOST_STATIC_CONSTANT(int, value = tracking_level::type::value); -}; - -} // seralization -} // boost - -#include - -#define BOOST_SERIALIZATION_NVP(name) \ - boost::serialization::make_nvp(BOOST_PP_STRINGIZE(name), name) -/**/ - -#define BOOST_SERIALIZATION_BASE_OBJECT_NVP(name) \ - boost::serialization::make_nvp( \ - BOOST_PP_STRINGIZE(name), \ - boost::serialization::base_object(*this) \ - ) -/**/ - -#endif // BOOST_SERIALIZATION_NVP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/optional.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/optional.hpp deleted file mode 100644 index d6ff830a8c3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/optional.hpp +++ /dev/null @@ -1,107 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 - -// (C) Copyright 2002-4 Pavel Vozenilek . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// Provides non-intrusive serialization for boost::optional. - -#ifndef BOOST_SERIALIZATION_OPTIONAL_HPP_ -#define BOOST_SERIALIZATION_OPTIONAL_HPP_ - -#if defined(_MSC_VER) -# pragma once -#endif - -#include - -#include - -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -// function specializations must be defined in the appropriate -// namespace - boost::serialization -namespace boost { -namespace serialization { - -template -void save( - Archive & ar, - const boost::optional< T > & t, - const unsigned int /*version*/ -){ - // It is an inherent limitation to the serialization of optional.hpp - // that the underlying type must be either a pointer or must have a - // default constructor. It's possible that this could change sometime - // in the future, but for now, one will have to work around it. This can - // be done by serialization the optional as optional - #if ! defined(BOOST_NO_CXX11_HDR_TYPE_TRAITS) - BOOST_STATIC_ASSERT( - boost::serialization::detail::is_default_constructible::value - || boost::is_pointer::value - ); - #endif - const bool tflag = t.is_initialized(); - ar << boost::serialization::make_nvp("initialized", tflag); - if (tflag){ - ar << boost::serialization::make_nvp("value", *t); - } -} - -template -void load( - Archive & ar, - boost::optional< T > & t, - const unsigned int version -){ - bool tflag; - ar >> boost::serialization::make_nvp("initialized", tflag); - if(! tflag){ - t.reset(); - return; - } - - if(0 == version){ - boost::serialization::item_version_type item_version(0); - boost::archive::library_version_type library_version( - ar.get_library_version() - ); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - } - if(! t.is_initialized()) - t = T(); - ar >> boost::serialization::make_nvp("value", *t); -} - -template -void serialize( - Archive & ar, - boost::optional< T > & t, - const unsigned int version -){ - boost::serialization::split_free(ar, t, version); -} - -template -struct version > { - BOOST_STATIC_CONSTANT(int, value = 1); -}; - -} // serialization -} // boost - -#endif // BOOST_SERIALIZATION_OPTIONAL_HPP_ diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/priority_queue.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/priority_queue.hpp deleted file mode 100644 index 5b08ffd1e82..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/priority_queue.hpp +++ /dev/null @@ -1,76 +0,0 @@ -#ifndef BOOST_SERIALIZATION_PRIORITY_QUEUE_HPP -#define BOOST_SERIALIZATION_PRIORITY_QUEUE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// priority_queue.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -// function specializations must be defined in the appropriate -// namespace - boost::serialization -#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) -#define STD _STLP_STD -#else -#define STD std -#endif - -namespace boost { -namespace serialization { -namespace detail{ - -template -struct priority_queue_save : public STD::priority_queue { - template - void operator()(Archive & ar, const unsigned int file_version) const { - save(ar, STD::priority_queue::c, file_version); - } -}; -template -struct priority_queue_load : public STD::priority_queue { - template - void operator()(Archive & ar, const unsigned int file_version) { - load(ar, STD::priority_queue::c, file_version); - } -}; - -} // detail - -template -inline void serialize( - Archive & ar, - std::priority_queue< T, Container, Compare> & t, - const unsigned int file_version -){ - typedef typename mpl::eval_if< - typename Archive::is_saving, - mpl::identity >, - mpl::identity > - >::type typex; - static_cast(t)(ar, file_version); -} - -} // namespace serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(STD::priority_queue) - -#undef STD - -#endif // BOOST_SERIALIZATION_PRIORITY_QUEUE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/queue.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/queue.hpp deleted file mode 100644 index b22745215d9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/queue.hpp +++ /dev/null @@ -1,76 +0,0 @@ -#ifndef BOOST_SERIALIZATION_QUEUE_HPP -#define BOOST_SERIALIZATION_QUEUE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// queue.hpp - -// (C) Copyright 2014 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -// function specializations must be defined in the appropriate -// namespace - boost::serialization -#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) -#define STD _STLP_STD -#else -#define STD std -#endif - -namespace boost { -namespace serialization { -namespace detail { - -template -struct queue_save : public STD::queue { - template - void operator()(Archive & ar, const unsigned int file_version) const { - save(ar, STD::queue::c, file_version); - } -}; -template -struct queue_load : public STD::queue { - template - void operator()(Archive & ar, const unsigned int file_version) { - load(ar, STD::queue::c, file_version); - } -}; - -} // detail - -template -inline void serialize( - Archive & ar, - std::queue< T, C> & t, - const unsigned int file_version -){ - typedef typename mpl::eval_if< - typename Archive::is_saving, - mpl::identity >, - mpl::identity > - >::type typex; - static_cast(t)(ar, file_version); -} - -} // namespace serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(STD::queue) - -#undef STD - -#endif // BOOST_SERIALIZATION_QUEUE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/scoped_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/scoped_ptr.hpp deleted file mode 100644 index 0d11f8436e0..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/scoped_ptr.hpp +++ /dev/null @@ -1,58 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SCOPED_PTR_HPP_VP_2003_10_30 -#define BOOST_SERIALIZATION_SCOPED_PTR_HPP_VP_2003_10_30 - -#if defined(_MSC_VER) -# pragma once -#endif - -// Copyright (c) 2003 Vladimir Prus. -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// Provides non-intrusive serialization for boost::scoped_ptr -// Does not allow to serialize scoped_ptr's to builtin types. - -#include - -#include -#include -#include - -namespace boost { -namespace serialization { - - template - void save( - Archive & ar, - const boost::scoped_ptr< T > & t, - const unsigned int /* version */ - ){ - T* r = t.get(); - ar << boost::serialization::make_nvp("scoped_ptr", r); - } - - template - void load( - Archive & ar, - boost::scoped_ptr< T > & t, - const unsigned int /* version */ - ){ - T* r; - ar >> boost::serialization::make_nvp("scoped_ptr", r); - t.reset(r); - } - - template - void serialize( - Archive& ar, - boost::scoped_ptr< T >& t, - const unsigned int version - ){ - boost::serialization::split_free(ar, t, version); - } - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_SCOPED_PTR_HPP_VP_2003_10_30 diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/serialization.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/serialization.hpp deleted file mode 100644 index a4d04723c75..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/serialization.hpp +++ /dev/null @@ -1,154 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SERIALIZATION_HPP -#define BOOST_SERIALIZATION_SERIALIZATION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -#if defined(_MSC_VER) -# pragma warning (disable : 4675) // suppress ADL warning -#endif - -#include -#include - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization.hpp: interface for serialization system. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -////////////////////////////////////////////////////////////////////// -// public interface to serialization. - -///////////////////////////////////////////////////////////////////////////// -// layer 0 - intrusive verison -// declared and implemented for each user defined class to be serialized -// -// template -// serialize(Archive &ar, const unsigned int file_version){ -// ar & base_object(*this) & member1 & member2 ... ; -// } - -///////////////////////////////////////////////////////////////////////////// -// layer 1 - layer that routes member access through the access class. -// this is what permits us to grant access to private class member functions -// by specifying friend class boost::serialization::access - -#include - -///////////////////////////////////////////////////////////////////////////// -// layer 2 - default implementation of non-intrusive serialization. -// -// note the usage of function overloading to compensate that C++ does not -// currently support Partial Template Specialization for function templates -// We have declared the version number as "const unsigned long". -// Overriding templates for specific data types should declare the version -// number as "const unsigned int". Template matching will first be applied -// to functions with the same version types - that is the overloads. -// If there is no declared function prototype that matches, the second argument -// will be converted to "const unsigned long" and a match will be made with -// one of the default template functions below. - -namespace boost { -namespace serialization { - -BOOST_STRONG_TYPEDEF(unsigned int, version_type) - -// default implementation - call the member function "serialize" -template -inline void serialize( - Archive & ar, T & t, const unsigned int file_version -){ - access::serialize(ar, t, static_cast(file_version)); -} - -// save data required for construction -template -inline void save_construct_data( - Archive & /*ar*/, - const T * /*t*/, - const unsigned int /*file_version */ -){ - // default is to save no data because default constructor - // requires no arguments. -} - -// load data required for construction and invoke constructor in place -template -inline void load_construct_data( - Archive & /*ar*/, - T * t, - const unsigned int /*file_version*/ -){ - // default just uses the default constructor. going - // through access permits usage of otherwise private default - // constructor - access::construct(t); -} - -///////////////////////////////////////////////////////////////////////////// -// layer 3 - move call into serialization namespace so that ADL will function -// in the manner we desire. -// -// on compilers which don't implement ADL. only the current namespace -// i.e. boost::serialization will be searched. -// -// on compilers which DO implement ADL -// serialize overrides can be in any of the following -// -// 1) same namepace as Archive -// 2) same namespace as T -// 3) boost::serialization -// -// Due to Martin Ecker - -template -inline void serialize_adl( - Archive & ar, - T & t, - const unsigned int file_version -){ - // note usage of function overloading to delay final resolution - // until the point of instantiation. This works around the two-phase - // lookup "feature" which inhibits redefintion of a default function - // template implementation. Due to Robert Ramey - // - // Note that this trick generates problems for compiles which don't support - // PFTO, suppress it here. As far as we know, there are no compilers - // which fail to support PFTO while supporting two-phase lookup. - const version_type v(file_version); - serialize(ar, t, v); -} - -template -inline void save_construct_data_adl( - Archive & ar, - const T * t, - const unsigned int file_version -){ - // see above - const version_type v(file_version); - save_construct_data(ar, t, v); -} - -template -inline void load_construct_data_adl( - Archive & ar, - T * t, - const unsigned int file_version -){ - // see above comment - const version_type v(file_version); - load_construct_data(ar, t, v); -} - -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_SERIALIZATION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/set.hpp deleted file mode 100644 index 643906c5aac..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/set.hpp +++ /dev/null @@ -1,137 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SET_HPP -#define BOOST_SERIALIZATION_SET_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// set.hpp: serialization for stl set templates - -// (C) Copyright 2002-2014 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -namespace boost { -namespace serialization { - -template -inline void load_set_collection(Archive & ar, Container &s) -{ - s.clear(); - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - typename Container::iterator hint; - hint = s.begin(); - while(count-- > 0){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, item_version); - // borland fails silently w/o full namespace - ar >> boost::serialization::make_nvp("item", t.reference()); - typename Container::iterator result = - s.insert(hint, boost::move(t.reference())); - ar.reset_object_address(& (* result), & t.reference()); - hint = result; - } -} - -template -inline void save( - Archive & ar, - const std::set &t, - const unsigned int /* file_version */ -){ - boost::serialization::stl::save_collection< - Archive, std::set - >(ar, t); -} - -template -inline void load( - Archive & ar, - std::set &t, - const unsigned int /* file_version */ -){ - load_set_collection(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::set & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// multiset -template -inline void save( - Archive & ar, - const std::multiset &t, - const unsigned int /* file_version */ -){ - boost::serialization::stl::save_collection< - Archive, - std::multiset - >(ar, t); -} - -template -inline void load( - Archive & ar, - std::multiset &t, - const unsigned int /* file_version */ -){ - load_set_collection(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::multiset & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(std::set) -BOOST_SERIALIZATION_COLLECTION_TRAITS(std::multiset) - -#endif // BOOST_SERIALIZATION_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr.hpp deleted file mode 100644 index 0d4c5ae6056..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr.hpp +++ /dev/null @@ -1,281 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SHARED_PTR_HPP -#define BOOST_SERIALIZATION_SHARED_PTR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// shared_ptr.hpp: serialization for boost shared pointer - -// (C) Copyright 2004 Robert Ramey and Martin Ecker -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include - -#include -#include -#include - -#include -#include - -#include -#include -#include -#include -#include - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// boost:: shared_ptr serialization traits -// version 1 to distinguish from boost 1.32 version. Note: we can only do this -// for a template when the compiler supports partial template specialization - -#ifndef BOOST_NO_TEMPLATE_PARTIAL_SPECIALIZATION - namespace boost { - namespace serialization{ - template - struct version< ::boost::shared_ptr< T > > { - typedef mpl::integral_c_tag tag; - #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3206)) - typedef typename mpl::int_<1> type; - #else - typedef mpl::int_<1> type; - #endif - BOOST_STATIC_CONSTANT(int, value = type::value); - }; - // don't track shared pointers - template - struct tracking_level< ::boost::shared_ptr< T > > { - typedef mpl::integral_c_tag tag; - #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3206)) - typedef typename mpl::int_< ::boost::serialization::track_never> type; - #else - typedef mpl::int_< ::boost::serialization::track_never> type; - #endif - BOOST_STATIC_CONSTANT(int, value = type::value); - }; - }} - #define BOOST_SERIALIZATION_SHARED_PTR(T) -#else - // define macro to let users of these compilers do this - #define BOOST_SERIALIZATION_SHARED_PTR(T) \ - BOOST_CLASS_VERSION( \ - ::boost::shared_ptr< T >, \ - 1 \ - ) \ - BOOST_CLASS_TRACKING( \ - ::boost::shared_ptr< T >, \ - ::boost::serialization::track_never \ - ) \ - /**/ -#endif - -namespace boost { -namespace serialization{ - -struct null_deleter { - void operator()(void const *) const {} -}; - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization for boost::shared_ptr - -// Using a constant means that all shared pointers are held in the same set. -// Thus we detect handle multiple pointers to the same value instances -// in the archive. -void * const shared_ptr_helper_id = 0; - -template -inline void save( - Archive & ar, - const boost::shared_ptr< T > &t, - const unsigned int /* file_version */ -){ - // The most common cause of trapping here would be serializing - // something like shared_ptr. This occurs because int - // is never tracked by default. Wrap int in a trackable type - BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); - const T * t_ptr = t.get(); - ar << boost::serialization::make_nvp("px", t_ptr); -} - -#ifdef BOOST_SERIALIZATION_SHARED_PTR_132_HPP -template -inline void load( - Archive & ar, - boost::shared_ptr< T > &t, - const unsigned int file_version -){ - // something like shared_ptr. This occurs because int - // is never tracked by default. Wrap int in a trackable type - BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); - T* r; - if(file_version < 1){ - ar.register_type(static_cast< - boost_132::detail::sp_counted_base_impl * - >(NULL)); - boost_132::shared_ptr< T > sp; - ar >> boost::serialization::make_nvp("px", sp.px); - ar >> boost::serialization::make_nvp("pn", sp.pn); - // got to keep the sps around so the sp.pns don't disappear - boost::serialization::shared_ptr_helper & h = - ar.template get_helper< shared_ptr_helper >( - shared_ptr_helper_id - ); - h.append(sp); - r = sp.get(); - } - else{ - ar >> boost::serialization::make_nvp("px", r); - } - shared_ptr_helper & h = - ar.template get_helper >( - shared_ptr_helper_id - ); - h.reset(t,r); -} -#else - -template -inline void load( - Archive & ar, - boost::shared_ptr< T > &t, - const unsigned int /*file_version*/ -){ - // The most common cause of trapping here would be serializing - // something like shared_ptr. This occurs because int - // is never tracked by default. Wrap int in a trackable type - BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); - T* r; - ar >> boost::serialization::make_nvp("px", r); - - boost::serialization::shared_ptr_helper & h = - ar.template get_helper >( - shared_ptr_helper_id - ); - h.reset(t,r); -} -#endif - -template -inline void serialize( - Archive & ar, - boost::shared_ptr< T > &t, - const unsigned int file_version -){ - // correct shared_ptr serialization depends upon object tracking - // being used. - BOOST_STATIC_ASSERT( - boost::serialization::tracking_level< T >::value - != boost::serialization::track_never - ); - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// std::shared_ptr serialization traits -// version 1 to distinguish from boost 1.32 version. Note: we can only do this -// for a template when the compiler supports partial template specialization - -#ifndef BOOST_NO_CXX11_SMART_PTR -#include - -// note: we presume that any compiler/library which supports C++11 -// std::pointers also supports template partial specialization -// trap here if such presumption were to turn out to wrong!!! -#ifdef BOOST_NO_TEMPLATE_PARTIAL_SPECIALIZATION - BOOST_STATIC_ASSERT(false); -#endif - -namespace boost { -namespace serialization{ - template - struct version< ::std::shared_ptr< T > > { - typedef mpl::integral_c_tag tag; - typedef mpl::int_<1> type; - BOOST_STATIC_CONSTANT(int, value = type::value); - }; - // don't track shared pointers - template - struct tracking_level< ::std::shared_ptr< T > > { - typedef mpl::integral_c_tag tag; - typedef mpl::int_< ::boost::serialization::track_never> type; - BOOST_STATIC_CONSTANT(int, value = type::value); - }; -}} -// the following just keeps older programs from breaking -#define BOOST_SERIALIZATION_SHARED_PTR(T) - -namespace boost { -namespace serialization{ - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization for std::shared_ptr - -template -inline void save( - Archive & ar, - const std::shared_ptr< T > &t, - const unsigned int /* file_version */ -){ - // The most common cause of trapping here would be serializing - // something like shared_ptr. This occurs because int - // is never tracked by default. Wrap int in a trackable type - BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); - const T * t_ptr = t.get(); - ar << boost::serialization::make_nvp("px", t_ptr); -} - -template -inline void load( - Archive & ar, - std::shared_ptr< T > &t, - const unsigned int /*file_version*/ -){ - // The most common cause of trapping here would be serializing - // something like shared_ptr. This occurs because int - // is never tracked by default. Wrap int in a trackable type - BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); - T* r; - ar >> boost::serialization::make_nvp("px", r); - //void (* const id)(Archive &, std::shared_ptr< T > &, const unsigned int) = & load; - boost::serialization::shared_ptr_helper & h = - ar.template get_helper< - shared_ptr_helper - >( - shared_ptr_helper_id - ); - h.reset(t,r); -} - -template -inline void serialize( - Archive & ar, - std::shared_ptr< T > &t, - const unsigned int file_version -){ - // correct shared_ptr serialization depends upon object tracking - // being used. - BOOST_STATIC_ASSERT( - boost::serialization::tracking_level< T >::value - != boost::serialization::track_never - ); - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_NO_CXX11_SMART_PTR - -#endif // BOOST_SERIALIZATION_SHARED_PTR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_132.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_132.hpp deleted file mode 100644 index 3dfaba4d69a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_132.hpp +++ /dev/null @@ -1,222 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SHARED_PTR_132_HPP -#define BOOST_SERIALIZATION_SHARED_PTR_132_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// shared_ptr.hpp: serialization for boost shared pointer - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// note: totally unadvised hack to gain access to private variables -// in shared_ptr and shared_count. Unfortunately its the only way to -// do this without changing shared_ptr and shared_count -// the best we can do is to detect a conflict here -#include - -#include -#include // NULL - -#include -#include -#include -#include -#include - -// mark base class as an (uncreatable) base class -#include - -///////////////////////////////////////////////////////////// -// Maintain a couple of lists of loaded shared pointers of the old previous -// version (1.32) - -namespace boost_132 { -namespace serialization { -namespace detail { - -struct null_deleter { - void operator()(void const *) const {} -}; - -} // namespace detail -} // namespace serialization -} // namespace boost_132 - -///////////////////////////////////////////////////////////// -// sp_counted_base_impl serialization - -namespace boost { -namespace serialization { - -template -inline void serialize( - Archive & /* ar */, - boost_132::detail::sp_counted_base_impl & /* t */, - const unsigned int /*file_version*/ -){ - // register the relationship between each derived class - // its polymorphic base - boost::serialization::void_cast_register< - boost_132::detail::sp_counted_base_impl, - boost_132::detail::sp_counted_base - >( - static_cast *>(NULL), - static_cast(NULL) - ); -} - -template -inline void save_construct_data( - Archive & ar, - const - boost_132::detail::sp_counted_base_impl *t, - const unsigned int /* file_version */ -){ - // variables used for construction - ar << boost::serialization::make_nvp("ptr", t->ptr); -} - -template -inline void load_construct_data( - Archive & ar, - boost_132::detail::sp_counted_base_impl * t, - const unsigned int /* file_version */ -){ - P ptr_; - ar >> boost::serialization::make_nvp("ptr", ptr_); - // ::new(t)boost_132::detail::sp_counted_base_impl(ptr_, D()); - // placement - // note: the original ::new... above is replaced by the one here. This one - // creates all new objects with a null_deleter so that after the archive - // is finished loading and the shared_ptrs are destroyed - the underlying - // raw pointers are NOT deleted. This is necessary as they are used by the - // new system as well. - ::new(t)boost_132::detail::sp_counted_base_impl< - P, - boost_132::serialization::detail::null_deleter - >( - ptr_, boost_132::serialization::detail::null_deleter() - ); // placement new - // compensate for that fact that a new shared count always is - // initialized with one. the add_ref_copy below will increment it - // every time its serialized so without this adjustment - // the use and weak counts will be off by one. - t->use_count_ = 0; -} - -} // serialization -} // namespace boost - -///////////////////////////////////////////////////////////// -// shared_count serialization - -namespace boost { -namespace serialization { - -template -inline void save( - Archive & ar, - const boost_132::detail::shared_count &t, - const unsigned int /* file_version */ -){ - ar << boost::serialization::make_nvp("pi", t.pi_); -} - -template -inline void load( - Archive & ar, - boost_132::detail::shared_count &t, - const unsigned int /* file_version */ -){ - ar >> boost::serialization::make_nvp("pi", t.pi_); - if(NULL != t.pi_) - t.pi_->add_ref_copy(); -} - -} // serialization -} // namespace boost - -BOOST_SERIALIZATION_SPLIT_FREE(boost_132::detail::shared_count) - -///////////////////////////////////////////////////////////// -// implement serialization for shared_ptr< T > - -namespace boost { -namespace serialization { - -template -inline void save( - Archive & ar, - const boost_132::shared_ptr< T > &t, - const unsigned int /* file_version */ -){ - // only the raw pointer has to be saved - // the ref count is maintained automatically as shared pointers are loaded - ar.register_type(static_cast< - boost_132::detail::sp_counted_base_impl > * - >(NULL)); - ar << boost::serialization::make_nvp("px", t.px); - ar << boost::serialization::make_nvp("pn", t.pn); -} - -template -inline void load( - Archive & ar, - boost_132::shared_ptr< T > &t, - const unsigned int /* file_version */ -){ - // only the raw pointer has to be saved - // the ref count is maintained automatically as shared pointers are loaded - ar.register_type(static_cast< - boost_132::detail::sp_counted_base_impl > * - >(NULL)); - ar >> boost::serialization::make_nvp("px", t.px); - ar >> boost::serialization::make_nvp("pn", t.pn); -} - -template -inline void serialize( - Archive & ar, - boost_132::shared_ptr< T > &t, - const unsigned int file_version -){ - // correct shared_ptr serialization depends upon object tracking - // being used. - BOOST_STATIC_ASSERT( - boost::serialization::tracking_level< T >::value - != boost::serialization::track_never - ); - boost::serialization::split_free(ar, t, file_version); -} - -} // serialization -} // namespace boost - -// note: change below uses null_deleter -// This macro is used to export GUIDS for shared pointers to allow -// the serialization system to export them properly. David Tonge -#define BOOST_SHARED_POINTER_EXPORT_GUID(T, K) \ - typedef boost_132::detail::sp_counted_base_impl< \ - T *, \ - boost::checked_deleter< T > \ - > __shared_ptr_ ## T; \ - BOOST_CLASS_EXPORT_GUID(__shared_ptr_ ## T, "__shared_ptr_" K) \ - BOOST_CLASS_EXPORT_GUID(T, K) \ - /**/ - -#define BOOST_SHARED_POINTER_EXPORT(T) \ - BOOST_SHARED_POINTER_EXPORT_GUID( \ - T, \ - BOOST_PP_STRINGIZE(T) \ - ) \ - /**/ - -#endif // BOOST_SERIALIZATION_SHARED_PTR_132_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_helper.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_helper.hpp deleted file mode 100644 index 37c34d6b2c4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_helper.hpp +++ /dev/null @@ -1,209 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SHARED_PTR_HELPER_HPP -#define BOOST_SERIALIZATION_SHARED_PTR_HELPER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// shared_ptr_helper.hpp: serialization for boost shared pointern - -// (C) Copyright 2004-2009 Robert Ramey, Martin Ecker and Takatoshi Kondo -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include // NULL - -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -namespace boost_132 { - template class shared_ptr; -} -namespace boost { -namespace serialization { - -#ifndef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -template class SPT > -void load( - Archive & ar, - SPT< class U > &t, - const unsigned int file_version -); -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// a common class for holding various types of shared pointers - -template class SPT> -class shared_ptr_helper { - typedef std::map< - const void *, // address of object - SPT // address shared ptr to single instance - > object_shared_pointer_map; - - // list of shared_pointers create accessable by raw pointer. This - // is used to "match up" shared pointers loaded at different - // points in the archive. Note, we delay construction until - // it is actually used since this is by default included as - // a "mix-in" even if shared_ptr isn't used. - object_shared_pointer_map * m_o_sp; - - struct null_deleter { - void operator()(void const *) const {} - }; - -#if defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) \ -|| defined(BOOST_MSVC) \ -|| defined(__SUNPRO_CC) -public: -#else - template - friend void boost::serialization::load( - Archive & ar, - SPT< U > &t, - const unsigned int file_version - ); -#endif - - #ifdef BOOST_SERIALIZATION_SHARED_PTR_132_HPP - // list of loaded pointers. This is used to be sure that the pointers - // stay around long enough to be "matched" with other pointers loaded - // by the same archive. These are created with a "null_deleter" so that - // when this list is destroyed - the underlaying raw pointers are not - // destroyed. This has to be done because the pointers are also held by - // new system which is disjoint from this set. This is implemented - // by a change in load_construct_data below. It makes this file suitable - // only for loading pointers into a 1.33 or later boost system. - std::list > * m_pointers_132; - void - append(const boost_132::shared_ptr & t){ - if(NULL == m_pointers_132) - m_pointers_132 = new std::list >; - m_pointers_132->push_back(t); - } - #endif - - struct non_polymorphic { - template - static const boost::serialization::extended_type_info * - get_object_type(U & ){ - return & boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< U >::type - >::get_const_instance(); - } - }; - struct polymorphic { - template - static const boost::serialization::extended_type_info * - get_object_type(U & u){ - return boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< U >::type - >::get_const_instance().get_derived_extended_type_info(u); - } - }; - -public: - template - void reset(SPT< T > & s, T * t){ - if(NULL == t){ - s.reset(); - return; - } - const boost::serialization::extended_type_info * this_type - = & boost::serialization::type_info_implementation< T >::type - ::get_const_instance(); - - // get pointer to the most derived object's eti. This is effectively - // the object type identifer - typedef typename mpl::if_< - is_polymorphic< T >, - polymorphic, - non_polymorphic - >::type type; - - const boost::serialization::extended_type_info * true_type - = type::get_object_type(*t); - - // note:if this exception is thrown, be sure that derived pointern - // is either registered or exported. - if(NULL == true_type) - boost::serialization::throw_exception( - boost::archive::archive_exception( - boost::archive::archive_exception::unregistered_class, - this_type->get_debug_info() - ) - ); - // get void pointer to the most derived type - // this uniquely identifies the object referred to - // oid = "object identifier" - const void * oid = void_downcast( - *true_type, - *this_type, - t - ); - if(NULL == oid) - boost::serialization::throw_exception( - boost::archive::archive_exception( - boost::archive::archive_exception::unregistered_cast, - true_type->get_debug_info(), - this_type->get_debug_info() - ) - ); - - // make tracking array if necessary - if(NULL == m_o_sp) - m_o_sp = new object_shared_pointer_map; - - typename object_shared_pointer_map::iterator i = m_o_sp->find(oid); - - // if it's a new object - if(i == m_o_sp->end()){ - s.reset(t); - std::pair result; - result = m_o_sp->insert(std::make_pair(oid, s)); - BOOST_ASSERT(result.second); - } - // if the object has already been seen - else{ - s = SPT(i->second, t); - } - } - - shared_ptr_helper() : - m_o_sp(NULL) - #ifdef BOOST_SERIALIZATION_SHARED_PTR_132_HPP - , m_pointers_132(NULL) - #endif - {} - virtual ~shared_ptr_helper(){ - if(NULL != m_o_sp) - delete m_o_sp; - #ifdef BOOST_SERIALIZATION_SHARED_PTR_132_HPP - if(NULL != m_pointers_132) - delete m_pointers_132; - #endif - } -}; - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_SHARED_PTR_HELPER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/singleton.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/singleton.hpp deleted file mode 100644 index b50afedbb92..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/singleton.hpp +++ /dev/null @@ -1,166 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SINGLETON_HPP -#define BOOST_SERIALIZATION_SINGLETON_HPP - -/////////1/////////2///////// 3/////////4/////////5/////////6/////////7/////////8 -// singleton.hpp -// -// Copyright David Abrahams 2006. Original version -// -// Copyright Robert Ramey 2007. Changes made to permit -// application throughout the serialization library. -// -// Distributed under the Boost -// Software License, Version 1.0. (See accompanying -// file LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) -// -// The intention here is to define a template which will convert -// any class into a singleton with the following features: -// -// a) initialized before first use. -// b) thread-safe for const access to the class -// c) non-locking -// -// In order to do this, -// a) Initialize dynamically when used. -// b) Require that all singletons be initialized before main -// is called or any entry point into the shared library is invoked. -// This guarentees no race condition for initialization. -// In debug mode, we assert that no non-const functions are called -// after main is invoked. -// - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -#include -#include -#include -#include - -#include -#include -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace serialization { - -////////////////////////////////////////////////////////////////////// -// Provides a dynamically-initialized (singleton) instance of T in a -// way that avoids LNK1179 on vc6. See http://tinyurl.com/ljdp8 or -// http://lists.boost.org/Archives/boost/2006/05/105286.php for -// details. -// - -// singletons created by this code are guarenteed to be unique -// within the executable or shared library which creates them. -// This is sufficient and in fact ideal for the serialization library. -// The singleton is created when the module is loaded and destroyed -// when the module is unloaded. - -// This base class has two functions. - -// First it provides a module handle for each singleton indicating -// the executable or shared library in which it was created. This -// turns out to be necessary and sufficient to implement the tables -// used by serialization library. - -// Second, it provides a mechanism to detect when a non-const function -// is called after initialization. - -// make a singleton to lock/unlock all singletons for alteration. -// The intent is that all singletons created/used by this code -// are to be initialized before main is called. A test program -// can lock all the singletons when main is entereed. This any -// attempt to retieve a mutable instances while locked will -// generate a assertion if compiled for debug. - -// note usage of BOOST_DLLEXPORT. These functions are in danger of -// being eliminated by the optimizer when building an application in -// release mode. Usage of the macro is meant to signal the compiler/linker -// to avoid dropping these functions which seem to be unreferenced. -// This usage is not related to autolinking. - -class BOOST_SYMBOL_VISIBLE singleton_module : - public boost::noncopyable -{ -private: - BOOST_SERIALIZATION_DECL BOOST_DLLEXPORT static bool & get_lock() BOOST_USED; -public: - BOOST_DLLEXPORT static void lock(){ - get_lock() = true; - } - BOOST_DLLEXPORT static void unlock(){ - get_lock() = false; - } - BOOST_DLLEXPORT static bool is_locked(){ - return get_lock(); - } -}; - -template -class singleton : public singleton_module -{ -private: - static T & m_instance; - // include this to provoke instantiation at pre-execution time - static void use(T const *) {} - static T & get_instance() { - // use a wrapper so that types T with protected constructors - // can be used - class singleton_wrapper : public T {}; - static singleton_wrapper t; - // refer to instance, causing it to be instantiated (and - // initialized at startup on working compilers) - BOOST_ASSERT(! is_destroyed()); - // note that the following is absolutely essential. - // commenting out this statement will cause compilers to fail to - // construct the instance at pre-execution time. This would prevent - // our usage/implementation of "locking" and introduce uncertainty into - // the sequence of object initializaition. - use(& m_instance); - return static_cast(t); - } - static bool & get_is_destroyed(){ - static bool is_destroyed; - return is_destroyed; - } - -public: - BOOST_DLLEXPORT static T & get_mutable_instance(){ - BOOST_ASSERT(! is_locked()); - return get_instance(); - } - BOOST_DLLEXPORT static const T & get_const_instance(){ - return get_instance(); - } - BOOST_DLLEXPORT static bool is_destroyed(){ - return get_is_destroyed(); - } - BOOST_DLLEXPORT singleton(){ - get_is_destroyed() = false; - } - BOOST_DLLEXPORT ~singleton() { - get_is_destroyed() = true; - } -}; - -template -T & singleton< T >::m_instance = singleton< T >::get_instance(); - -} // namespace serialization -} // namespace boost - -#include // pops abi_suffix.hpp pragmas - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_SERIALIZATION_SINGLETON_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/slist.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/slist.hpp deleted file mode 100644 index d9b971bc4f1..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/slist.hpp +++ /dev/null @@ -1,145 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SLIST_HPP -#define BOOST_SERIALIZATION_SLIST_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// slist.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_HAS_SLIST -#include BOOST_SLIST_HEADER - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template -inline void save( - Archive & ar, - const BOOST_STD_EXTENSION_NAMESPACE::slist &t, - const unsigned int file_version -){ - boost::serialization::stl::save_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::slist - >(ar, t); -} - -namespace stl { - -template< - class Archive, - class T, - class Allocator -> -typename boost::disable_if< - typename detail::is_default_constructible< - typename BOOST_STD_EXTENSION_NAMESPACE::slist::value_type - >, - void ->::type -collection_load_impl( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::slist &t, - collection_size_type count, - item_version_type item_version -){ - t.clear(); - boost::serialization::detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - t.push_front(boost::move(u.reference())); - typename BOOST_STD_EXTENSION_NAMESPACE::slist::iterator last; - last = t.begin(); - ar.reset_object_address(&(*t.begin()) , & u.reference()); - while(--count > 0){ - detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - last = t.insert_after(last, boost::move(u.reference())); - ar.reset_object_address(&(*last) , & u.reference()); - } -} - -} // stl - -template -inline void load( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::slist &t, - const unsigned int file_version -){ - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - if(detail::is_default_constructible()){ - t.resize(count); - typename BOOST_STD_EXTENSION_NAMESPACE::slist::iterator hint; - hint = t.begin(); - while(count-- > 0){ - ar >> boost::serialization::make_nvp("item", *hint++); - } - } - else{ - t.clear(); - boost::serialization::detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - t.push_front(boost::move(u.reference())); - typename BOOST_STD_EXTENSION_NAMESPACE::slist::iterator last; - last = t.begin(); - ar.reset_object_address(&(*t.begin()) , & u.reference()); - while(--count > 0){ - detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - last = t.insert_after(last, boost::move(u.reference())); - ar.reset_object_address(&(*last) , & u.reference()); - } - } -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::slist &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(BOOST_STD_EXTENSION_NAMESPACE::slist) - -#endif // BOOST_HAS_SLIST -#endif // BOOST_SERIALIZATION_SLIST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/smart_cast.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/smart_cast.hpp deleted file mode 100644 index 563f36aa20b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/smart_cast.hpp +++ /dev/null @@ -1,275 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SMART_CAST_HPP -#define BOOST_SERIALIZATION_SMART_CAST_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// smart_cast.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org/libs/serialization for updates, documentation, and revision history. - -// casting of pointers and references. - -// In casting between different C++ classes, there are a number of -// rules that have to be kept in mind in deciding whether to use -// static_cast or dynamic_cast. - -// a) dynamic casting can only be applied when one of the types is polymorphic -// Otherwise static_cast must be used. -// b) only dynamic casting can do runtime error checking -// use of static_cast is generally un checked even when compiled for debug -// c) static_cast would be considered faster than dynamic_cast. - -// If casting is applied to a template parameter, there is no apriori way -// to know which of the two casting methods will be permitted or convenient. - -// smart_cast uses C++ type_traits, and program debug mode to select the -// most convenient cast to use. - -#include -#include -#include // NULL - -#include -#include - -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace serialization { -namespace smart_cast_impl { - - template - struct reference { - - struct polymorphic { - - struct linear { - template - static T cast(U & u){ - return static_cast< T >(u); - } - }; - - struct cross { - template - static T cast(U & u){ - return dynamic_cast< T >(u); - } - }; - - template - static T cast(U & u){ - // if we're in debug mode - #if ! defined(NDEBUG) \ - || defined(__MWERKS__) - // do a checked dynamic cast - return cross::cast(u); - #else - // borland 5.51 chokes here so we can't use it - // note: if remove_reference isn't function for these types - // cross casting will be selected this will work but will - // not be the most efficient method. This will conflict with - // the original smart_cast motivation. - typedef typename mpl::eval_if< - typename mpl::and_< - mpl::not_::type, - U - > >, - mpl::not_::type - > > - >, - // borland chokes w/o full qualification here - mpl::identity, - mpl::identity - >::type typex; - // typex works around gcc 2.95 issue - return typex::cast(u); - #endif - } - }; - - struct non_polymorphic { - template - static T cast(U & u){ - return static_cast< T >(u); - } - }; - template - static T cast(U & u){ - typedef typename mpl::eval_if< - boost::is_polymorphic, - mpl::identity, - mpl::identity - >::type typex; - return typex::cast(u); - } - }; - - template - struct pointer { - - struct polymorphic { - // unfortunately, this below fails to work for virtual base - // classes. need has_virtual_base to do this. - // Subject for further study - #if 0 - struct linear { - template - static T cast(U * u){ - return static_cast< T >(u); - } - }; - - struct cross { - template - static T cast(U * u){ - T tmp = dynamic_cast< T >(u); - #ifndef NDEBUG - if ( tmp == 0 ) throw_exception(std::bad_cast()); - #endif - return tmp; - } - }; - - template - static T cast(U * u){ - typedef - typename mpl::eval_if< - typename mpl::and_< - mpl::not_::type, - U - > >, - mpl::not_::type - > > - >, - // borland chokes w/o full qualification here - mpl::identity, - mpl::identity - >::type typex; - return typex::cast(u); - } - #else - template - static T cast(U * u){ - T tmp = dynamic_cast< T >(u); - #ifndef NDEBUG - if ( tmp == 0 ) throw_exception(std::bad_cast()); - #endif - return tmp; - } - #endif - }; - - struct non_polymorphic { - template - static T cast(U * u){ - return static_cast< T >(u); - } - }; - - template - static T cast(U * u){ - typedef typename mpl::eval_if< - boost::is_polymorphic, - mpl::identity, - mpl::identity - >::type typex; - return typex::cast(u); - } - - }; - - template - struct void_pointer { - template - static TPtr cast(UPtr uptr){ - return static_cast(uptr); - } - }; - - template - struct error { - // if we get here, its because we are using one argument in the - // cast on a system which doesn't support partial template - // specialization - template - static T cast(U){ - BOOST_STATIC_ASSERT(sizeof(T)==0); - return * static_cast(NULL); - } - }; - -} // smart_cast_impl - -// this implements: -// smart_cast(Source * s) -// smart_cast(s) -// note that it will fail with -// smart_cast(s) -template -T smart_cast(U u) { - typedef - typename mpl::eval_if< - typename mpl::or_< - boost::is_same, - boost::is_same, - boost::is_same, - boost::is_same - >, - mpl::identity >, - // else - typename mpl::eval_if, - mpl::identity >, - // else - typename mpl::eval_if, - mpl::identity >, - // else - mpl::identity - > - > - > - >::type typex; - return typex::cast(u); -} - -// this implements: -// smart_cast_reference(Source & s) -template -T smart_cast_reference(U & u) { - return smart_cast_impl::reference< T >::cast(u); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_SMART_CAST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/split_free.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/split_free.hpp deleted file mode 100644 index 85e2f590fe4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/split_free.hpp +++ /dev/null @@ -1,93 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SPLIT_FREE_HPP -#define BOOST_SERIALIZATION_SPLIT_FREE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// split_free.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -namespace boost { -namespace archive { - namespace detail { - template class interface_oarchive; - template class interface_iarchive; - } // namespace detail -} // namespace archive - -namespace serialization { - -//namespace detail { -template -struct free_saver { - static void invoke( - Archive & ar, - const T & t, - const unsigned int file_version - ){ - // use function overload (version_type) to workaround - // two-phase lookup issue - const version_type v(file_version); - save(ar, t, v); - } -}; -template -struct free_loader { - static void invoke( - Archive & ar, - T & t, - const unsigned int file_version - ){ - // use function overload (version_type) to workaround - // two-phase lookup issue - const version_type v(file_version); - load(ar, t, v); - } -}; -//} // namespace detail - -template -inline void split_free( - Archive & ar, - T & t, - const unsigned int file_version -){ - typedef typename mpl::eval_if< - typename Archive::is_saving, - mpl::identity >, - mpl::identity > - >::type typex; - typex::invoke(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#define BOOST_SERIALIZATION_SPLIT_FREE(T) \ -namespace boost { namespace serialization { \ -template \ -inline void serialize( \ - Archive & ar, \ - T & t, \ - const unsigned int file_version \ -){ \ - split_free(ar, t, file_version); \ -} \ -}} -/**/ - -#endif // BOOST_SERIALIZATION_SPLIT_FREE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/split_member.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/split_member.hpp deleted file mode 100644 index 5f32520559e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/split_member.hpp +++ /dev/null @@ -1,86 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SPLIT_MEMBER_HPP -#define BOOST_SERIALIZATION_SPLIT_MEMBER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// split_member.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#include - -namespace boost { -namespace archive { - namespace detail { - template class interface_oarchive; - template class interface_iarchive; - } // namespace detail -} // namespace archive - -namespace serialization { -namespace detail { - - template - struct member_saver { - static void invoke( - Archive & ar, - const T & t, - const unsigned int file_version - ){ - access::member_save(ar, t, file_version); - } - }; - - template - struct member_loader { - static void invoke( - Archive & ar, - T & t, - const unsigned int file_version - ){ - access::member_load(ar, t, file_version); - } - }; - -} // detail - -template -inline void split_member( - Archive & ar, T & t, const unsigned int file_version -){ - typedef typename mpl::eval_if< - typename Archive::is_saving, - mpl::identity >, - mpl::identity > - >::type typex; - typex::invoke(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -// split member function serialize funcition into save/load -#define BOOST_SERIALIZATION_SPLIT_MEMBER() \ -template \ -void serialize( \ - Archive &ar, \ - const unsigned int file_version \ -){ \ - boost::serialization::split_member(ar, *this, file_version); \ -} \ -/**/ - -#endif // BOOST_SERIALIZATION_SPLIT_MEMBER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/stack.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/stack.hpp deleted file mode 100644 index 96f90fe8767..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/stack.hpp +++ /dev/null @@ -1,76 +0,0 @@ -#ifndef BOOST_SERIALIZATION_STACK_HPP -#define BOOST_SERIALIZATION_STACK_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// stack.hpp - -// (C) Copyright 2014 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -// function specializations must be defined in the appropriate -// namespace - boost::serialization -#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) -#define STD _STLP_STD -#else -#define STD std -#endif - -namespace boost { -namespace serialization { -namespace detail{ - -template -struct stack_save : public STD::stack { - template - void operator()(Archive & ar, const unsigned int file_version) const { - save(ar, STD::stack::c, file_version); - } -}; -template -struct stack_load : public STD::stack { - template - void operator()(Archive & ar, const unsigned int file_version) { - load(ar, STD::stack::c, file_version); - } -}; - -} // detail - -template -inline void serialize( - Archive & ar, - std::stack< T, C> & t, - const unsigned int file_version -){ - typedef typename mpl::eval_if< - typename Archive::is_saving, - mpl::identity >, - mpl::identity > - >::type typex; - static_cast(t)(ar, file_version); -} - -} // namespace serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(STD::stack) - -#undef STD - -#endif // BOOST_SERIALIZATION_DEQUE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/state_saver.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/state_saver.hpp deleted file mode 100644 index 248b8d91556..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/state_saver.hpp +++ /dev/null @@ -1,96 +0,0 @@ -#ifndef BOOST_SERIALIZATION_STATE_SAVER_HPP -#define BOOST_SERIALIZATION_STATE_SAVER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// state_saver.hpp: - -// (C) Copyright 2003-4 Pavel Vozenilek and Robert Ramey - http://www.rrsd.com. -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org/libs/serialization for updates, documentation, and revision history. - -// Inspired by Daryle Walker's iostate_saver concept. This saves the original -// value of a variable when a state_saver is constructed and restores -// upon destruction. Useful for being sure that state is restored to -// variables upon exit from scope. - - -#include -#ifndef BOOST_NO_EXCEPTIONS - #include -#endif - -#include -#include -#include -#include - -#include -#include - -namespace boost { -namespace serialization { - -template -// T requirements: -// - POD or object semantic (cannot be reference, function, ...) -// - copy constructor -// - operator = (no-throw one preferred) -class state_saver : private boost::noncopyable -{ -private: - const T previous_value; - T & previous_ref; - - struct restore { - static void invoke(T & previous_ref, const T & previous_value){ - previous_ref = previous_value; // won't throw - } - }; - - struct restore_with_exception { - static void invoke(T & previous_ref, const T & previous_value){ - BOOST_TRY{ - previous_ref = previous_value; - } - BOOST_CATCH(::std::exception &) { - // we must ignore it - we are in destructor - } - BOOST_CATCH_END - } - }; - -public: - state_saver( - T & object - ) : - previous_value(object), - previous_ref(object) - {} - - ~state_saver() { - #ifndef BOOST_NO_EXCEPTIONS - typedef typename mpl::eval_if< - has_nothrow_copy< T >, - mpl::identity, - mpl::identity - >::type typex; - typex::invoke(previous_ref, previous_value); - #else - previous_ref = previous_value; - #endif - } - -}; // state_saver<> - -} // serialization -} // boost - -#endif //BOOST_SERIALIZATION_STATE_SAVER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/static_warning.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/static_warning.hpp deleted file mode 100644 index 1d9238fc4d9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/static_warning.hpp +++ /dev/null @@ -1,103 +0,0 @@ -#ifndef BOOST_SERIALIZATION_STATIC_WARNING_HPP -#define BOOST_SERIALIZATION_STATIC_WARNING_HPP - -// (C) Copyright Robert Ramey 2003. Jonathan Turkanis 2004. -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org/libs/static_assert for documentation. - -/* - Revision history: - 15 June 2003 - Initial version. - 31 March 2004 - improved diagnostic messages and portability - (Jonathan Turkanis) - 03 April 2004 - works on VC6 at class and namespace scope - - ported to DigitalMars - - static warnings disabled by default; when enabled, - uses pragmas to enable required compiler warnings - on MSVC, Intel, Metrowerks and Borland 5.x. - (Jonathan Turkanis) - 30 May 2004 - tweaked for msvc 7.1 and gcc 3.3 - - static warnings ENabled by default; when enabled, - (Robert Ramey) -*/ - -#include - -// -// Implementation -// Makes use of the following warnings: -// 1. GCC prior to 3.3: division by zero. -// 2. BCC 6.0 preview: unreferenced local variable. -// 3. DigitalMars: returning address of local automatic variable. -// 4. VC6: class previously seen as struct (as in 'boost/mpl/print.hpp') -// 5. All others: deletion of pointer to incomplete type. -// -// The trick is to find code which produces warnings containing the name of -// a structure or variable. Details, with same numbering as above: -// 1. static_warning_impl::value is zero iff B is false, so diving an int -// by this value generates a warning iff B is false. -// 2. static_warning_impl::type has a constructor iff B is true, so an -// unreferenced variable of this type generates a warning iff B is false. -// 3. static_warning_impl::type overloads operator& to return a dynamically -// allocated int pointer only is B is true, so returning the address of an -// automatic variable of this type generates a warning iff B is fasle. -// 4. static_warning_impl::STATIC_WARNING is decalred as a struct iff B is -// false. -// 5. static_warning_impl::type is incomplete iff B is false, so deleting a -// pointer to this type generates a warning iff B is false. -// - -//------------------Enable selected warnings----------------------------------// - -// Enable the warnings relied on by BOOST_STATIC_WARNING, where possible. - -// 6. replaced implementation with one which depends solely on -// mpl::print<>. The previous one was found to fail for functions -// under recent versions of gcc and intel compilers - Robert Ramey - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template -struct BOOST_SERIALIZATION_STATIC_WARNING_LINE{}; - -template -struct static_warning_test{ - typename boost::mpl::eval_if_c< - B, - boost::mpl::true_, - typename boost::mpl::identity< - boost::mpl::print< - BOOST_SERIALIZATION_STATIC_WARNING_LINE - > - > - >::type type; -}; - -template -struct BOOST_SERIALIZATION_SS {}; - -} // serialization -} // boost - -#define BOOST_SERIALIZATION_BSW(B, L) \ - typedef boost::serialization::BOOST_SERIALIZATION_SS< \ - sizeof( boost::serialization::static_warning_test< B, L > ) \ - > BOOST_JOIN(STATIC_WARNING_LINE, L) BOOST_ATTRIBUTE_UNUSED; -#define BOOST_STATIC_WARNING(B) BOOST_SERIALIZATION_BSW(B, __LINE__) - -#endif // BOOST_SERIALIZATION_STATIC_WARNING_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/string.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/string.hpp deleted file mode 100644 index 76e695d4f3c..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/string.hpp +++ /dev/null @@ -1,30 +0,0 @@ -#ifndef BOOST_SERIALIZATION_STRING_HPP -#define BOOST_SERIALIZATION_STRING_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/string.hpp: -// serialization for stl string templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -BOOST_CLASS_IMPLEMENTATION(std::string, boost::serialization::primitive_type) -#ifndef BOOST_NO_STD_WSTRING -BOOST_CLASS_IMPLEMENTATION(std::wstring, boost::serialization::primitive_type) -#endif - -#endif // BOOST_SERIALIZATION_STRING_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/strong_typedef.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/strong_typedef.hpp deleted file mode 100644 index fdd1b24c9cb..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/strong_typedef.hpp +++ /dev/null @@ -1,50 +0,0 @@ -#ifndef BOOST_SERIALIZATION_STRONG_TYPEDEF_HPP -#define BOOST_SERIALIZATION_STRONG_TYPEDEF_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// strong_typedef.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2016 Ashish Sadanandan -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org/libs/serialization for updates, documentation, and revision history. - -// macro used to implement a strong typedef. strong typedef -// guarentees that two types are distinguised even though the -// share the same underlying implementation. typedef does not create -// a new type. BOOST_STRONG_TYPEDEF(T, D) creates a new type named D -// that operates as a type T. - -#include -#include -#include -#include -#include - -#define BOOST_STRONG_TYPEDEF(T, D) \ -struct D \ - : boost::totally_ordered1< D \ - , boost::totally_ordered2< D, T \ - > > \ -{ \ - T t; \ - explicit D(const T& t_) BOOST_NOEXCEPT_IF(boost::has_nothrow_copy_constructor::value) : t(t_) {} \ - D() BOOST_NOEXCEPT_IF(boost::has_nothrow_default_constructor::value) : t() {} \ - D(const D & t_) BOOST_NOEXCEPT_IF(boost::has_nothrow_copy_constructor::value) : t(t_.t) {} \ - D& operator=(const D& rhs) BOOST_NOEXCEPT_IF(boost::has_nothrow_assign::value) {t = rhs.t; return *this;} \ - D& operator=(const T& rhs) BOOST_NOEXCEPT_IF(boost::has_nothrow_assign::value) {t = rhs; return *this;} \ - operator const T&() const {return t;} \ - operator T&() {return t;} \ - bool operator==(const D& rhs) const {return t == rhs.t;} \ - bool operator<(const D& rhs) const {return t < rhs.t;} \ -}; - -#endif // BOOST_SERIALIZATION_STRONG_TYPEDEF_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/throw_exception.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/throw_exception.hpp deleted file mode 100644 index b67618adc92..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/throw_exception.hpp +++ /dev/null @@ -1,44 +0,0 @@ -#ifndef BOOST_SERIALIZATION_THROW_EXCEPTION_HPP_INCLUDED -#define BOOST_SERIALIZATION_THROW_EXCEPTION_HPP_INCLUDED - -// MS compatible compilers support #pragma once - -#if defined(_MSC_VER) -# pragma once -#endif - -// boost/throw_exception.hpp -// -// Copyright (c) 2002 Peter Dimov and Multi Media Ltd. -// -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include - -#ifndef BOOST_NO_EXCEPTIONS -#include -#endif - -namespace boost { -namespace serialization { - -#ifdef BOOST_NO_EXCEPTIONS - -inline void throw_exception(std::exception const & e) { - ::boost::throw_exception(e); -} - -#else - -template inline void throw_exception(E const & e){ - throw e; -} - -#endif - -} // namespace serialization -} // namespace boost - -#endif // #ifndef BOOST_SERIALIZATION_THROW_EXCEPTION_HPP_INCLUDED diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/tracking.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/tracking.hpp deleted file mode 100644 index d5c79b8409d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/tracking.hpp +++ /dev/null @@ -1,118 +0,0 @@ -#ifndef BOOST_SERIALIZATION_TRACKING_HPP -#define BOOST_SERIALIZATION_TRACKING_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// tracking.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -struct basic_traits; - -// default tracking level -template -struct tracking_level_impl { - template - struct traits_class_tracking { - typedef typename U::tracking type; - }; - typedef mpl::integral_c_tag tag; - // note: at least one compiler complained w/o the full qualification - // on basic traits below - typedef - typename mpl::eval_if< - is_base_and_derived, - traits_class_tracking< T >, - //else - typename mpl::eval_if< - is_pointer< T >, - // pointers are not tracked by default - mpl::int_, - //else - typename mpl::eval_if< - // for primitives - typename mpl::equal_to< - implementation_level< T >, - mpl::int_ - >, - // is never - mpl::int_, - // otherwise its selective - mpl::int_ - > > >::type type; - BOOST_STATIC_CONSTANT(int, value = type::value); -}; - -template -struct tracking_level : - public tracking_level_impl -{ -}; - -template -inline bool operator>=(tracking_level< T > t, enum tracking_type l) -{ - return t.value >= (int)l; -} - -} // namespace serialization -} // namespace boost - - -// The STATIC_ASSERT is prevents one from setting tracking for a primitive type. -// This almost HAS to be an error. Doing this will effect serialization of all -// char's in your program which is almost certainly what you don't want to do. -// If you want to track all instances of a given primitive type, You'll have to -// wrap it in your own type so its not a primitive anymore. Then it will compile -// without problem. -#define BOOST_CLASS_TRACKING(T, E) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct tracking_level< T > \ -{ \ - typedef mpl::integral_c_tag tag; \ - typedef mpl::int_< E> type; \ - BOOST_STATIC_CONSTANT( \ - int, \ - value = tracking_level::type::value \ - ); \ - /* tracking for a class */ \ - BOOST_STATIC_ASSERT(( \ - mpl::greater< \ - /* that is a prmitive */ \ - implementation_level< T >, \ - mpl::int_ \ - >::value \ - )); \ -}; \ -}} - -#endif // BOOST_SERIALIZATION_TRACKING_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/tracking_enum.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/tracking_enum.hpp deleted file mode 100644 index 278051e1baf..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/tracking_enum.hpp +++ /dev/null @@ -1,41 +0,0 @@ -#ifndef BOOST_SERIALIZATION_TRACKING_ENUM_HPP -#define BOOST_SERIALIZATION_TRACKING_ENUM_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// tracking_enum.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -namespace boost { -namespace serialization { - -// addresses of serialized objects may be tracked to avoid saving/loading -// redundant copies. This header defines a class trait that can be used -// to specify when objects should be tracked - -// names for each tracking level -enum tracking_type -{ - // never track this type - track_never = 0, - // track objects of this type if the object is serialized through a - // pointer. - track_selectively = 1, - // always track this type - track_always = 2 -}; - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_TRACKING_ENUM_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/traits.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/traits.hpp deleted file mode 100644 index 9e114fdd3df..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/traits.hpp +++ /dev/null @@ -1,65 +0,0 @@ -#ifndef BOOST_SERIALIZATION_TRAITS_HPP -#define BOOST_SERIALIZATION_TRAITS_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// traits.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// This header is used to apply serialization traits to templates. The -// standard system can't be used for platforms which don't support -// Partial Templlate Specialization. - -// The motivation for this is the Name-Value Pair (NVP) template. -// it has to work the same on all platforms in order for archives -// to be portable accross platforms. - -#include -#include - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -// common base class used to detect appended traits class -struct basic_traits {}; - -template -struct extended_type_info_impl; - -template< - class T, - int Level, - int Tracking, - unsigned int Version = 0, - class ETII = extended_type_info_impl< T >, - class Wrapper = mpl::false_ -> -struct traits : public basic_traits { - BOOST_STATIC_ASSERT(Version == 0 || Level >= object_class_info); - BOOST_STATIC_ASSERT(Tracking == track_never || Level >= object_serializable); - typedef typename mpl::int_ level; - typedef typename mpl::int_ tracking; - typedef typename mpl::int_ version; - typedef ETII type_info_implementation; - typedef Wrapper is_wrapper; -}; - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_TRAITS_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/type_info_implementation.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/type_info_implementation.hpp deleted file mode 100644 index 24637a8dbb3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/type_info_implementation.hpp +++ /dev/null @@ -1,73 +0,0 @@ -#ifndef BOOST_SERIALIZATION_TYPE_INFO_IMPLEMENTATION_HPP -#define BOOST_SERIALIZATION_TYPE_INFO_IMPLEMENTATION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// type_info_implementation.hpp: interface for portable version of type_info - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - - -#include -#include - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -// note that T and const T are folded into const T so that -// there is only one table entry per type -template -struct type_info_implementation { - template - struct traits_class_typeinfo_implementation { - typedef typename U::type_info_implementation::type type; - }; - // note: at least one compiler complained w/o the full qualification - // on basic traits below - typedef - typename mpl::eval_if< - is_base_and_derived, - traits_class_typeinfo_implementation< T >, - //else - mpl::identity< - typename extended_type_info_impl< T >::type - > - >::type type; -}; - -} // namespace serialization -} // namespace boost - -// define a macro to assign a particular derivation of extended_type_info -// to a specified a class. -#define BOOST_CLASS_TYPE_INFO(T, ETI) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct type_info_implementation< T > { \ - typedef ETI type; \ -}; \ -template<> \ -struct type_info_implementation< const T > { \ - typedef ETI type; \ -}; \ -} \ -} \ -/**/ - -#endif /// BOOST_SERIALIZATION_TYPE_INFO_IMPLEMENTATION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unique_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unique_ptr.hpp deleted file mode 100644 index 8d8703ef4f7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/unique_ptr.hpp +++ /dev/null @@ -1,68 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UNIQUE_PTR_HPP -#define BOOST_SERIALIZATION_UNIQUE_PTR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// unique_ptr.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include - -namespace boost { -namespace serialization { - -///////////////////////////////////////////////////////////// -// implement serialization for unique_ptr< T > -// note: this must be added to the boost namespace in order to -// be called by the library -template -inline void save( - Archive & ar, - const std::unique_ptr< T > &t, - const unsigned int /*file_version*/ -){ - // only the raw pointer has to be saved - // the ref count is rebuilt automatically on load - const T * const tx = t.get(); - ar << BOOST_SERIALIZATION_NVP(tx); -} - -template -inline void load( - Archive & ar, - std::unique_ptr< T > &t, - const unsigned int /*file_version*/ -){ - T *tx; - ar >> BOOST_SERIALIZATION_NVP(tx); - // note that the reset automagically maintains the reference count - t.reset(tx); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::unique_ptr< T > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - - -#endif // BOOST_SERIALIZATION_UNIQUE_PTR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_load_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_load_imp.hpp deleted file mode 100644 index d56a423d180..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_load_imp.hpp +++ /dev/null @@ -1,73 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_LOAD_IMP_HPP -#define BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_LOAD_IMP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -# pragma warning (disable : 4786) // too long name, harmless warning -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// unordered_collections_load_imp.hpp: serialization for loading stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// helper function templates for serialization of collections - -#include -#include // size_t -#include // msvc 6.0 needs this for warning suppression -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif -#include - -#include -#include -#include -#include -#include - -namespace boost{ -namespace serialization { -namespace stl { - -////////////////////////////////////////////////////////////////////// -// implementation of serialization for STL containers -// -template -inline void load_unordered_collection(Archive & ar, Container &s) -{ - collection_size_type count; - collection_size_type bucket_count; - boost::serialization::item_version_type item_version(0); - boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - ar >> BOOST_SERIALIZATION_NVP(count); - ar >> BOOST_SERIALIZATION_NVP(bucket_count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - s.clear(); - s.rehash(bucket_count); - InputFunction ifunc; - while(count-- > 0){ - ifunc(ar, s, item_version); - } -} - -} // namespace stl -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_LOAD_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_save_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_save_imp.hpp deleted file mode 100644 index 56746ebeaa3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_save_imp.hpp +++ /dev/null @@ -1,86 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_SAVE_IMP_HPP -#define BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_SAVE_IMP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// hash_collections_save_imp.hpp: serialization for stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// helper function templates for serialization of collections - -#include -#include -#include -#include -#include -#include - -namespace boost{ -namespace serialization { -namespace stl { - -////////////////////////////////////////////////////////////////////// -// implementation of serialization for STL containers -// - -template -inline void save_unordered_collection(Archive & ar, const Container &s) -{ - collection_size_type count(s.size()); - const collection_size_type bucket_count(s.bucket_count()); - const item_version_type item_version( - version::value - ); - - #if 0 - /* should only be necessary to create archives of previous versions - * which is not currently supported. So for now comment this out - */ - boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - ar << BOOST_SERIALIZATION_NVP(count); - ar << BOOST_SERIALIZATION_NVP(bucket_count); - if(boost::archive::library_version_type(3) < library_version){ - // record number of elements - // make sure the target type is registered so we can retrieve - // the version when we load - ar << BOOST_SERIALIZATION_NVP(item_version); - } - #else - ar << BOOST_SERIALIZATION_NVP(count); - ar << BOOST_SERIALIZATION_NVP(bucket_count); - ar << BOOST_SERIALIZATION_NVP(item_version); - #endif - - typename Container::const_iterator it = s.begin(); - while(count-- > 0){ - // note borland emits a no-op without the explicit namespace - boost::serialization::save_construct_data_adl( - ar, - &(*it), - boost::serialization::version< - typename Container::value_type - >::value - ); - ar << boost::serialization::make_nvp("item", *it++); - } -} - -} // namespace stl -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_SAVE_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_map.hpp deleted file mode 100644 index 4fdbddd7b65..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_map.hpp +++ /dev/null @@ -1,160 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UNORDERED_MAP_HPP -#define BOOST_SERIALIZATION_UNORDERED_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/unordered_map.hpp: -// serialization for stl unordered_map templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const std::unordered_map &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - std::unordered_map - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - std::unordered_map &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - std::unordered_map, - boost::serialization::stl::archive_input_unordered_map< - Archive, - std::unordered_map - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - std::unordered_map &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// unordered_multimap -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const std::unordered_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - std::unordered_multimap - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - std::unordered_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - std::unordered_multimap< - Key, HashFcn, EqualKey, Allocator - >, - boost::serialization::stl::archive_input_unordered_multimap< - Archive, - std::unordered_multimap - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - std::unordered_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_UNORDERED_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_set.hpp deleted file mode 100644 index adfee609cbe..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_set.hpp +++ /dev/null @@ -1,162 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UNORDERED_SET_HPP -#define BOOST_SERIALIZATION_UNORDERED_SET_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// unordered_set.hpp: serialization for stl unordered_set templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const std::unordered_set< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - std::unordered_set - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - std::unordered_set< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - std::unordered_set, - stl::archive_input_unordered_set< - Archive, - std::unordered_set< - Key, HashFcn, EqualKey, Allocator - > - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - std::unordered_set< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - split_free(ar, t, file_version); -} - -// unordered_multiset -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const std::unordered_multiset< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - stl::save_unordered_collection< - Archive, - std::unordered_multiset - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - std::unordered_multiset< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - std::unordered_multiset, - boost::serialization::stl::archive_input_unordered_multiset< - Archive, - std::unordered_multiset - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - std::unordered_multiset &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_UNORDERED_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/utility.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/utility.hpp deleted file mode 100644 index 4867a4a12d2..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/utility.hpp +++ /dev/null @@ -1,56 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UTILITY_HPP -#define BOOST_SERIALIZATION_UTILITY_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/utility.hpp: -// serialization for stl utility templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -// pair -template -inline void serialize( - Archive & ar, - std::pair & p, - const unsigned int /* file_version */ -){ - // note: we remove any const-ness on the first argument. The reason is that - // for stl maps, the type saved is pair::type typef; - ar & boost::serialization::make_nvp("first", const_cast(p.first)); - ar & boost::serialization::make_nvp("second", p.second); -} - -/// specialization of is_bitwise_serializable for pairs -template -struct is_bitwise_serializable > - : public mpl::and_,is_bitwise_serializable > -{ -}; - -} // serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_UTILITY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/valarray.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/valarray.hpp deleted file mode 100644 index 9eece5c1737..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/valarray.hpp +++ /dev/null @@ -1,86 +0,0 @@ -#ifndef BOOST_SERIALIZATION_VALARAY_HPP -#define BOOST_SERIALIZATION_VALARAY_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// valarray.hpp: serialization for stl vector templates - -// (C) Copyright 2005 Matthias Troyer . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include -#include -#include - -// function specializations must be defined in the appropriate -// namespace - boost::serialization -#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) -#define STD _STLP_STD -#else -#define STD std -#endif - -namespace boost { -namespace serialization { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// valarray< T > - -template -void save( Archive & ar, const STD::valarray &t, const unsigned int /*file_version*/ ) -{ - const collection_size_type count(t.size()); - ar << BOOST_SERIALIZATION_NVP(count); - if (t.size()){ - // explict template arguments to pass intel C++ compiler - ar << serialization::make_array( - static_cast(&t[0]), - count - ); - } -} - -template -void load( Archive & ar, STD::valarray &t, const unsigned int /*file_version*/ ) -{ - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - t.resize(count); - if (t.size()){ - // explict template arguments to pass intel C++ compiler - ar >> serialization::make_array( - static_cast(&t[0]), - count - ); - } -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( Archive & ar, STD::valarray & t, const unsigned int file_version) -{ - boost::serialization::split_free(ar, t, file_version); -} - -} } // end namespace boost::serialization - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(STD::valarray) -#undef STD - -#endif // BOOST_SERIALIZATION_VALARAY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/variant.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/variant.hpp deleted file mode 100644 index dce6f3d49e7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/variant.hpp +++ /dev/null @@ -1,158 +0,0 @@ -#ifndef BOOST_SERIALIZATION_VARIANT_HPP -#define BOOST_SERIALIZATION_VARIANT_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// variant.hpp - non-intrusive serialization of variant types -// -// copyright (c) 2005 -// troy d. straszheim -// http://www.resophonic.com -// -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// -// See http://www.boost.org for updates, documentation, and revision history. -// -// thanks to Robert Ramey, Peter Dimov, and Richard Crossley. -// - -#include -#include -#include -#include -#include -#include - -#include - -#include - -#include - -#include -#include -#include - -namespace boost { -namespace serialization { - -template -struct variant_save_visitor : - boost::static_visitor<> -{ - variant_save_visitor(Archive& ar) : - m_ar(ar) - {} - template - void operator()(T const & value) const - { - m_ar << BOOST_SERIALIZATION_NVP(value); - } -private: - Archive & m_ar; -}; - -template -void save( - Archive & ar, - boost::variant const & v, - unsigned int /*version*/ -){ - int which = v.which(); - ar << BOOST_SERIALIZATION_NVP(which); - variant_save_visitor visitor(ar); - v.apply_visitor(visitor); -} - -template -struct variant_impl { - - struct load_null { - template - static void invoke( - Archive & /*ar*/, - int /*which*/, - V & /*v*/, - const unsigned int /*version*/ - ){} - }; - - struct load_impl { - template - static void invoke( - Archive & ar, - int which, - V & v, - const unsigned int version - ){ - if(which == 0){ - // note: A non-intrusive implementation (such as this one) - // necessary has to copy the value. This wouldn't be necessary - // with an implementation that de-serialized to the address of the - // aligned storage included in the variant. - typedef typename mpl::front::type head_type; - head_type value; - ar >> BOOST_SERIALIZATION_NVP(value); - v = value; - ar.reset_object_address(& boost::get(v), & value); - return; - } - typedef typename mpl::pop_front::type type; - variant_impl::load(ar, which - 1, v, version); - } - }; - - template - static void load( - Archive & ar, - int which, - V & v, - const unsigned int version - ){ - typedef typename mpl::eval_if, - mpl::identity, - mpl::identity - >::type typex; - typex::invoke(ar, which, v, version); - } - -}; - -template -void load( - Archive & ar, - boost::variant& v, - const unsigned int version -){ - int which; - typedef typename boost::variant::types types; - ar >> BOOST_SERIALIZATION_NVP(which); - if(which >= mpl::size::value) - // this might happen if a type was removed from the list of variant types - boost::serialization::throw_exception( - boost::archive::archive_exception( - boost::archive::archive_exception::unsupported_version - ) - ); - variant_impl::load(ar, which, v, version); -} - -template -inline void serialize( - Archive & ar, - boost::variant & v, - const unsigned int file_version -){ - split_free(ar,v,file_version); -} - -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_VARIANT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/vector.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/vector.hpp deleted file mode 100644 index 9a114c00e20..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/vector.hpp +++ /dev/null @@ -1,233 +0,0 @@ -#ifndef BOOST_SERIALIZATION_VECTOR_HPP -#define BOOST_SERIALIZATION_VECTOR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// vector.hpp: serialization for stl vector templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// fast array serialization (C) Copyright 2005 Matthias Troyer -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - -// default is being compatible with version 1.34.1 files, not 1.35 files -#ifndef BOOST_SERIALIZATION_VECTOR_VERSIONED -#define BOOST_SERIALIZATION_VECTOR_VERSIONED(V) (V==4 || V==5) -#endif - -// function specializations must be defined in the appropriate -// namespace - boost::serialization -#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) -#define STD _STLP_STD -#else -#define STD std -#endif - -namespace boost { -namespace serialization { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// vector< T > - -// the default versions - -template -inline void save( - Archive & ar, - const std::vector &t, - const unsigned int /* file_version */, - mpl::false_ -){ - boost::serialization::stl::save_collection >( - ar, t - ); -} - -template -inline void load( - Archive & ar, - std::vector &t, - const unsigned int /* file_version */, - mpl::false_ -){ - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - t.reserve(count); - stl::collection_load_impl(ar, t, count, item_version); -} - -// the optimized versions - -template -inline void save( - Archive & ar, - const std::vector &t, - const unsigned int /* file_version */, - mpl::true_ -){ - const collection_size_type count(t.size()); - ar << BOOST_SERIALIZATION_NVP(count); - if (!t.empty()) - // explict template arguments to pass intel C++ compiler - ar << serialization::make_array( - static_cast(&t[0]), - count - ); -} - -template -inline void load( - Archive & ar, - std::vector &t, - const unsigned int /* file_version */, - mpl::true_ -){ - collection_size_type count(t.size()); - ar >> BOOST_SERIALIZATION_NVP(count); - t.resize(count); - unsigned int item_version=0; - if(BOOST_SERIALIZATION_VECTOR_VERSIONED(ar.get_library_version())) { - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - if (!t.empty()) - // explict template arguments to pass intel C++ compiler - ar >> serialization::make_array( - static_cast(&t[0]), - count - ); - } - -// dispatch to either default or optimized versions - -template -inline void save( - Archive & ar, - const std::vector &t, - const unsigned int file_version -){ - typedef typename - boost::serialization::use_array_optimization::template apply< - typename remove_const::type - >::type use_optimized; - save(ar,t,file_version, use_optimized()); -} - -template -inline void load( - Archive & ar, - std::vector &t, - const unsigned int file_version -){ -#ifdef BOOST_SERIALIZATION_VECTOR_135_HPP - if (ar.get_library_version()==boost::archive::library_version_type(5)) - { - load(ar,t,file_version, boost::is_arithmetic()); - return; - } -#endif - typedef typename - boost::serialization::use_array_optimization::template apply< - typename remove_const::type - >::type use_optimized; - load(ar,t,file_version, use_optimized()); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::vector & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// vector -template -inline void save( - Archive & ar, - const std::vector &t, - const unsigned int /* file_version */ -){ - // record number of elements - collection_size_type count (t.size()); - ar << BOOST_SERIALIZATION_NVP(count); - std::vector::const_iterator it = t.begin(); - while(count-- > 0){ - bool tb = *it++; - ar << boost::serialization::make_nvp("item", tb); - } -} - -template -inline void load( - Archive & ar, - std::vector &t, - const unsigned int /* file_version */ -){ - // retrieve number of elements - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - t.resize(count); - for(collection_size_type i = collection_size_type(0); i < count; ++i){ - bool b; - ar >> boost::serialization::make_nvp("item", b); - t[i] = b; - } -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::vector & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(std::vector) -#undef STD - -#endif // BOOST_SERIALIZATION_VECTOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/vector_135.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/vector_135.hpp deleted file mode 100644 index fd1a7393d1b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/vector_135.hpp +++ /dev/null @@ -1,26 +0,0 @@ -////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// vector_135.hpp: serialization for stl vector templates for compatibility -// with release 1.35, which had a bug - -// (C) Copyright 2008 Matthias Troyer -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - - -#ifndef BOOST_SERIALIZATION_VECTOR_135_HPP -#define BOOST_SERIALIZATION_VECTOR_135_HPP - -#ifdef BOOST_SERIALIZATION_VECTOR_VERSIONED -#if BOOST_SERIALIZATION_VECTOR_VERSION != 4 -#error "Boost.Serialization cannot be compatible with both 1.35 and 1.36-1.40 files" -#endif -#else -#define BOOST_SERIALIZATION_VECTOR_VERSIONED(V) (V>4) -#endif - -#include - -#endif // BOOST_SERIALIZATION_VECTOR_135_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/version.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/version.hpp deleted file mode 100644 index 21a74d73daa..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/version.hpp +++ /dev/null @@ -1,107 +0,0 @@ -#ifndef BOOST_SERIALIZATION_VERSION_HPP -#define BOOST_SERIALIZATION_VERSION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// version.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace serialization { - -struct basic_traits; - -// default version number is 0. Override with higher version -// when class definition changes. -template -struct version -{ - template - struct traits_class_version { - typedef typename U::version type; - }; - - typedef mpl::integral_c_tag tag; - // note: at least one compiler complained w/o the full qualification - // on basic traits below - typedef - typename mpl::eval_if< - is_base_and_derived, - traits_class_version< T >, - mpl::int_<0> - >::type type; - BOOST_STATIC_CONSTANT(int, value = version::type::value); -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -template -const int version::value; -#endif - -} // namespace serialization -} // namespace boost - -/* note: at first it seemed that this would be a good place to trap - * as an error an attempt to set a version # for a class which doesn't - * save its class information (including version #) in the archive. - * However, this imposes a requirement that the version be set after - * the implemention level which would be pretty confusing. If this - * is to be done, do this check in the input or output operators when - * ALL the serialization traits are available. Included the implementation - * here with this comment as a reminder not to do this! - */ -//#include -//#include - -#include -#include - -// specify the current version number for the class -// version numbers limited to 8 bits !!! -#define BOOST_CLASS_VERSION(T, N) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct version \ -{ \ - typedef mpl::int_ type; \ - typedef mpl::integral_c_tag tag; \ - BOOST_STATIC_CONSTANT(int, value = version::type::value); \ - BOOST_MPL_ASSERT(( \ - boost::mpl::less< \ - boost::mpl::int_, \ - boost::mpl::int_<256> \ - > \ - )); \ - /* \ - BOOST_MPL_ASSERT(( \ - mpl::equal_to< \ - :implementation_level, \ - mpl::int_ \ - >::value \ - )); \ - */ \ -}; \ -} \ -} - -#endif // BOOST_SERIALIZATION_VERSION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/void_cast.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/void_cast.hpp deleted file mode 100644 index f1b38286115..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/void_cast.hpp +++ /dev/null @@ -1,298 +0,0 @@ -#ifndef BOOST_SERIALIZATION_VOID_CAST_HPP -#define BOOST_SERIALIZATION_VOID_CAST_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// void_cast.hpp: interface for run-time casting of void pointers. - -// (C) Copyright 2002-2009 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// gennadiy.rozental@tfn.com - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // for ptrdiff_t -#include -#include - -#include -#include -#include -#include -#include -#include -#include - -#include -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4251 4231 4660 4275) -#endif - -namespace boost { -namespace serialization { - -class extended_type_info; - -// Given a void *, assume that it really points to an instance of one type -// and alter it so that it would point to an instance of a related type. -// Return the altered pointer. If there exists no sequence of casts that -// can transform from_type to to_type, return a NULL. - -BOOST_SERIALIZATION_DECL void const * -void_upcast( - extended_type_info const & derived, - extended_type_info const & base, - void const * const t -); - -inline void * -void_upcast( - extended_type_info const & derived, - extended_type_info const & base, - void * const t -){ - return const_cast(void_upcast( - derived, - base, - const_cast(t) - )); -} - -BOOST_SERIALIZATION_DECL void const * -void_downcast( - extended_type_info const & derived, - extended_type_info const & base, - void const * const t -); - -inline void * -void_downcast( - extended_type_info const & derived, - extended_type_info const & base, - void * const t -){ - return const_cast(void_downcast( - derived, - base, - const_cast(t) - )); -} - -namespace void_cast_detail { - -class BOOST_SYMBOL_VISIBLE void_caster : - private boost::noncopyable -{ - friend - BOOST_SERIALIZATION_DECL void const * - boost::serialization::void_upcast( - extended_type_info const & derived, - extended_type_info const & base, - void const * const - ); - friend - BOOST_SERIALIZATION_DECL void const * - boost::serialization::void_downcast( - extended_type_info const & derived, - extended_type_info const & base, - void const * const - ); -protected: - BOOST_SERIALIZATION_DECL void recursive_register(bool includes_virtual_base = false) const; - BOOST_SERIALIZATION_DECL void recursive_unregister() const; - virtual bool has_virtual_base() const = 0; -public: - // Data members - const extended_type_info * m_derived; - const extended_type_info * m_base; - /*const*/ std::ptrdiff_t m_difference; - void_caster const * const m_parent; - - // note that void_casters are keyed on value of - // member extended type info records - NOT their - // addresses. This is necessary in order for the - // void cast operations to work across dll and exe - // module boundries. - bool operator<(const void_caster & rhs) const; - - const void_caster & operator*(){ - return *this; - } - // each derived class must re-implement these; - virtual void const * upcast(void const * const t) const = 0; - virtual void const * downcast(void const * const t) const = 0; - // Constructor - void_caster( - extended_type_info const * derived, - extended_type_info const * base, - std::ptrdiff_t difference = 0, - void_caster const * const parent = 0 - ) : - m_derived(derived), - m_base(base), - m_difference(difference), - m_parent(parent) - {} - virtual ~void_caster(){} -}; - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4251 4231 4660 4275 4511 4512) -#endif - -template -class BOOST_SYMBOL_VISIBLE void_caster_primitive : - public void_caster -{ - virtual void const * downcast(void const * const t) const { - const Derived * d = - boost::serialization::smart_cast( - static_cast(t) - ); - return d; - } - virtual void const * upcast(void const * const t) const { - const Base * b = - boost::serialization::smart_cast( - static_cast(t) - ); - return b; - } - virtual bool has_virtual_base() const { - return false; - } -public: - void_caster_primitive(); - virtual ~void_caster_primitive(); -}; - -template -void_caster_primitive::void_caster_primitive() : - void_caster( - & type_info_implementation::type::get_const_instance(), - & type_info_implementation::type::get_const_instance(), - // note:I wanted to displace from 0 here, but at least one compiler - // treated 0 by not shifting it at all. - reinterpret_cast( - static_cast( - reinterpret_cast(8) - ) - ) - 8 - ) -{ - recursive_register(); -} - -template -void_caster_primitive::~void_caster_primitive(){ - recursive_unregister(); -} - -template -class BOOST_SYMBOL_VISIBLE void_caster_virtual_base : - public void_caster -{ - virtual bool has_virtual_base() const { - return true; - } -public: - virtual void const * downcast(void const * const t) const { - const Derived * d = - dynamic_cast( - static_cast(t) - ); - return d; - } - virtual void const * upcast(void const * const t) const { - const Base * b = - dynamic_cast( - static_cast(t) - ); - return b; - } - void_caster_virtual_base(); - virtual ~void_caster_virtual_base(); -}; - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -template -void_caster_virtual_base::void_caster_virtual_base() : - void_caster( - & (type_info_implementation::type::get_const_instance()), - & (type_info_implementation::type::get_const_instance()) - ) -{ - recursive_register(true); -} - -template -void_caster_virtual_base::~void_caster_virtual_base(){ - recursive_unregister(); -} - -template -struct BOOST_SYMBOL_VISIBLE void_caster_base : - public void_caster -{ - typedef - typename mpl::eval_if, - mpl::identity< - void_cast_detail::void_caster_virtual_base - > - ,// else - mpl::identity< - void_cast_detail::void_caster_primitive - > - >::type type; -}; - -} // void_cast_detail - -template -BOOST_DLLEXPORT -inline const void_cast_detail::void_caster & void_cast_register( - Derived const * /* dnull = NULL */, - Base const * /* bnull = NULL */ -){ - typedef - typename mpl::eval_if, - mpl::identity< - void_cast_detail::void_caster_virtual_base - > - ,// else - mpl::identity< - void_cast_detail::void_caster_primitive - > - >::type typex; - return singleton::get_const_instance(); -} - -template -class BOOST_SYMBOL_VISIBLE void_caster : - public void_cast_detail::void_caster_base::type -{ -}; - -} // namespace serialization -} // namespace boost - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_SERIALIZATION_VOID_CAST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/void_cast_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/void_cast_fwd.hpp deleted file mode 100644 index def61d52bb7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/void_cast_fwd.hpp +++ /dev/null @@ -1,37 +0,0 @@ -#ifndef BOOST_SERIALIZATION_VOID_CAST_FWD_HPP -#define BOOST_SERIALIZATION_VOID_CAST_FWD_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// void_cast_fwd.hpp: interface for run-time casting of void pointers. - -// (C) Copyright 2005 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// gennadiy.rozental@tfn.com - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include - -namespace boost { -namespace serialization { -namespace void_cast_detail{ -class void_caster; -} // namespace void_cast_detail -template -BOOST_DLLEXPORT -inline const void_cast_detail::void_caster & void_cast_register( - const Derived * dnull = NULL, - const Base * bnull = NULL -) BOOST_USED; -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_VOID_CAST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/weak_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/weak_ptr.hpp deleted file mode 100644 index 6952d24cb37..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/weak_ptr.hpp +++ /dev/null @@ -1,99 +0,0 @@ -#ifndef BOOST_SERIALIZATION_WEAK_PTR_HPP -#define BOOST_SERIALIZATION_WEAK_PTR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// weak_ptr.hpp: serialization for boost weak pointer - -// (C) Copyright 2004 Robert Ramey and Martin Ecker -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -namespace boost { -namespace serialization{ - -template -inline void save( - Archive & ar, - const boost::weak_ptr< T > &t, - const unsigned int /* file_version */ -){ - const boost::shared_ptr< T > sp = t.lock(); - ar << boost::serialization::make_nvp("weak_ptr", sp); -} - -template -inline void load( - Archive & ar, - boost::weak_ptr< T > &t, - const unsigned int /* file_version */ -){ - boost::shared_ptr< T > sp; - ar >> boost::serialization::make_nvp("weak_ptr", sp); - t = sp; -} - -template -inline void serialize( - Archive & ar, - boost::weak_ptr< T > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#ifndef BOOST_NO_CXX11_SMART_PTR -#include - -namespace boost { -namespace serialization{ - -template -inline void save( - Archive & ar, - const std::weak_ptr< T > &t, - const unsigned int /* file_version */ -){ - const std::shared_ptr< T > sp = t.lock(); - ar << boost::serialization::make_nvp("weak_ptr", sp); -} - -template -inline void load( - Archive & ar, - std::weak_ptr< T > &t, - const unsigned int /* file_version */ -){ - std::shared_ptr< T > sp; - ar >> boost::serialization::make_nvp("weak_ptr", sp); - t = sp; -} - -template -inline void serialize( - Archive & ar, - std::weak_ptr< T > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_NO_CXX11_SMART_PTR - -#endif // BOOST_SERIALIZATION_WEAK_PTR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/wrapper.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/wrapper.hpp deleted file mode 100644 index 60d7910b17a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/wrapper.hpp +++ /dev/null @@ -1,60 +0,0 @@ -#ifndef BOOST_SERIALIZATION_WRAPPER_HPP -#define BOOST_SERIALIZATION_WRAPPER_HPP - -// (C) Copyright 2005-2006 Matthias Troyer -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include -#include -#include -#include - -namespace boost { namespace serialization { - -/// the base class for serialization wrappers -/// -/// wrappers need to be treated differently at various places in the serialization library, -/// e.g. saving of non-const wrappers has to be possible. Since partial specialization -// is not supported by all compilers, we derive all wrappers from wrapper_traits. - -template< - class T, - int Level = object_serializable, - int Tracking = track_never, - unsigned int Version = 0, - class ETII = extended_type_info_impl< T > -> -struct wrapper_traits : - public traits -{}; - -template -struct is_wrapper_impl : - boost::mpl::eval_if< - boost::is_base_and_derived, - boost::mpl::true_, - boost::mpl::false_ - >::type -{}; - -template -struct is_wrapper { - typedef typename is_wrapper_impl::type type; -}; - -} // serialization -} // boost - -// A macro to define that a class is a wrapper -#define BOOST_CLASS_IS_WRAPPER(T) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct is_wrapper_impl : boost::mpl::true_ {}; \ -} \ -} \ -/**/ - -#endif //BOOST_SERIALIZATION_WRAPPER_HPP diff --git a/contrib/poco b/contrib/poco new file mode 160000 index 00000000000..29439cf7fa3 --- /dev/null +++ b/contrib/poco @@ -0,0 +1 @@ +Subproject commit 29439cf7fa32c1a2d62d925bb6d6a3f14668a4a2 diff --git a/contrib/zlib-ng b/contrib/zlib-ng new file mode 160000 index 00000000000..9173b89d467 --- /dev/null +++ b/contrib/zlib-ng @@ -0,0 +1 @@ +Subproject commit 9173b89d46799582d20a30578e0aa9788bc7d6e1 diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 08b4bb752e5..d9313ac8442 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -2266,8 +2266,6 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart( { auto committed_parts_range = getDataPartsStateRange(state); - auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed); - /// The part can be covered only by the previous or the next one in data_parts. auto it = data_parts_by_state_and_info.lower_bound(DataPartStateAndInfo{state, part_info}); @@ -2600,7 +2598,6 @@ MergeTreeData::getDetachedParts() const part.prefix = dir_name.substr(0, first_separator); } - return res; } From 2930965938f595933fbd88080c8c7f4a593d1970 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 6 Jun 2019 21:10:56 -0400 Subject: [PATCH 058/509] * Renaming live_view_heartbeat_delay setting to live_view_heartbeat_interval and making the setting to be in seconds instead of usec * Updated LIVE VIEW table to keep version consistent as long as the server is up * Removed calculation of hash in writeIntoLiveView method as it is recalculated in the LiveViewBlockOutputStream * Changed to using values from the local context for live_view_heartbeat_interval and temporary_live_view_timeout instead of the global context * Updated LIVE VIEW tests to match the changes --- dbms/src/Core/Defines.h | 2 +- dbms/src/Core/Settings.h | 2 +- .../DataStreams/LiveViewBlockInputStream.h | 15 +++-- .../LiveViewEventsBlockInputStream.h | 13 ++-- dbms/src/Storages/StorageLiveView.cpp | 18 +++--- dbms/src/Storages/StorageLiveView.h | 62 +++++++++---------- .../00956_live_view_watch_events.reference | 4 +- .../00957_live_view_watch.reference | 4 +- ...00959_create_temporary_live_view.reference | 2 +- .../00959_create_temporary_live_view.sql | 5 +- .../00961_temporary_live_view_watch.reference | 4 +- ..._temporary_live_view_watch_live_timeout.py | 14 ++--- .../00964_live_view_watch_events_heartbeat.py | 4 +- .../00965_live_view_watch_heartbeat.py | 4 +- 14 files changed, 82 insertions(+), 71 deletions(-) diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 82e882d29f2..49d9d1e3a74 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -36,7 +36,7 @@ #define DEFAULT_TEMPORARY_LIVE_CHANNEL_TIMEOUT_SEC 15 #define DEFAULT_ALTER_LIVE_CHANNEL_WAIT_MS 10000 #define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160) -#define DEFAULT_HEARTBEAT_DELAY 15000000 +#define DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC 15 #define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024 #define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES 3 /// each period reduces the error counter by 2 times diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index ae7f891a17d..3d9a06e5984 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -326,7 +326,7 @@ struct Settings : public SettingsCollection M(SettingBool, allow_simdjson, 1, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \ \ M(SettingUInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.") \ - M(SettingUInt64, heartbeat_delay, DEFAULT_HEARTBEAT_DELAY, "The interval in microseconds to indicate live query is alive.") \ + M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.") \ M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.") \ M(SettingSeconds, temporary_live_channel_timeout, DEFAULT_TEMPORARY_LIVE_CHANNEL_TIMEOUT_SEC, "Timeout after which temporary live channel is deleted.") \ M(SettingMilliseconds, alter_channel_wait_ms, DEFAULT_ALTER_LIVE_CHANNEL_WAIT_MS, "The wait time for alter channel request.") diff --git a/dbms/src/DataStreams/LiveViewBlockInputStream.h b/dbms/src/DataStreams/LiveViewBlockInputStream.h index a6c9c3533ee..93eae976177 100644 --- a/dbms/src/DataStreams/LiveViewBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewBlockInputStream.h @@ -37,15 +37,17 @@ public: /// Start storage no users thread /// if we are the last active user if (!storage.is_dropped && blocks_ptr.use_count() < 3) - storage.startNoUsersThread(); + storage.startNoUsersThread(temporary_live_view_timeout); } /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update /// and LIMIT 0 just returns data without waiting for any updates LiveViewBlockInputStream(StorageLiveView & storage_, std::shared_ptr blocks_ptr_, std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, Poco::Condition & condition_, Poco::FastMutex & mutex_, - int64_t length_, const UInt64 & heartbeat_delay_) - : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_delay(heartbeat_delay_), blocks_hash("") + int64_t length_, const UInt64 & heartbeat_interval_, + const UInt64 & temporary_live_view_timeout_) + : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_interval(heartbeat_interval_), temporary_live_view_timeout(temporary_live_view_timeout_), + blocks_hash("") { /// grab active pointer active = active_ptr.lock(); @@ -158,7 +160,9 @@ protected: } while (true) { - bool signaled = condition.tryWait(mutex, std::max((UInt64)0, heartbeat_delay - ((UInt64)timestamp.epochMicroseconds() - last_event_timestamp)) / 1000); + bool signaled = condition.tryWait(mutex, + std::max((UInt64)0, (heartbeat_interval * 1000000) - + ((UInt64)timestamp.epochMicroseconds() - last_event_timestamp)) / 1000); if (isCancelled() || storage.is_dropped) { @@ -211,7 +215,8 @@ private: /// Length specifies number of updates to send, default -1 (no limit) int64_t length; bool end_of_blocks{0}; - UInt64 heartbeat_delay; + UInt64 heartbeat_interval; + UInt64 temporary_live_view_timeout; String blocks_hash; UInt64 last_event_timestamp{0}; Poco::Timestamp timestamp; diff --git a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h index 32290c3333b..d0f92806d74 100644 --- a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h @@ -41,13 +41,13 @@ public: /// Start storage no users thread /// if we are the last active user if (!storage.is_dropped && blocks_ptr.use_count() < 3) - storage.startNoUsersThread(); + storage.startNoUsersThread(temporary_live_view_timeout); } /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update /// and LIMIT 0 just returns data without waiting for any updates LiveViewEventsBlockInputStream(StorageLiveView & storage_, std::shared_ptr blocks_ptr_, std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, Poco::Condition & condition_, Poco::FastMutex & mutex_, - int64_t length_, const UInt64 & heartbeat_delay_) - : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_delay(heartbeat_delay_) + int64_t length_, const UInt64 & heartbeat_interval_, const UInt64 & temporary_live_view_timeout_) + : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_interval(heartbeat_interval_), temporary_live_view_timeout(temporary_live_view_timeout_) { /// grab active pointer active = active_ptr.lock(); @@ -188,7 +188,9 @@ protected: } while (true) { - bool signaled = condition.tryWait(mutex, std::max((UInt64)0, heartbeat_delay - ((UInt64)timestamp.epochMicroseconds() - last_event_timestamp)) / 1000); + bool signaled = condition.tryWait(mutex, + std::max((UInt64)0, (heartbeat_interval * 1000000) - + ((UInt64)timestamp.epochMicroseconds() - last_event_timestamp)) / 1000); if (isCancelled() || storage.is_dropped) { @@ -241,7 +243,8 @@ private: /// Length specifies number of updates to send, default -1 (no limit) int64_t length; bool end_of_blocks{0}; - UInt64 heartbeat_delay; + UInt64 heartbeat_interval; + UInt64 temporary_live_view_timeout; UInt64 last_event_timestamp{0}; Poco::Timestamp timestamp; }; diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index 77578bc386d..108e146dc73 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -132,6 +132,7 @@ StorageLiveView::StorageLiveView( DatabaseAndTableName(database_name, table_name)); is_temporary = query.temporary; + temporary_live_view_timeout = local_context.getSettingsRef().temporary_live_view_timeout.totalSeconds(); blocks_ptr = std::make_shared(); blocks_metadata_ptr = std::make_shared(); @@ -235,7 +236,7 @@ void StorageLiveView::checkTableCanBeDropped() const } } -void StorageLiveView::noUsersThread() +void StorageLiveView::noUsersThread(const UInt64 & timeout) { if (shutdown_called) return; @@ -246,7 +247,8 @@ void StorageLiveView::noUsersThread() while (1) { Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); - if (!noUsersThreadWakeUp && !noUsersThreadCondition.tryWait(noUsersThreadMutex, global_context.getSettingsRef().temporary_live_view_timeout.totalSeconds() * 1000)) + if (!noUsersThreadWakeUp && !noUsersThreadCondition.tryWait(noUsersThreadMutex, + timeout * 1000)) { noUsersThreadWakeUp = false; if (shutdown_called) @@ -283,7 +285,7 @@ void StorageLiveView::noUsersThread() } } -void StorageLiveView::startNoUsersThread() +void StorageLiveView::startNoUsersThread(const UInt64 & timeout) { bool expected = false; if (!startnousersthread_called.compare_exchange_strong(expected, true)) @@ -308,14 +310,14 @@ void StorageLiveView::startNoUsersThread() noUsersThreadWakeUp = false; } if (!is_dropped) - no_users_thread = std::thread(&StorageLiveView::noUsersThread, this); + no_users_thread = std::thread(&StorageLiveView::noUsersThread, this, timeout); } startnousersthread_called = false; } void StorageLiveView::startup() { - startNoUsersThread(); + startNoUsersThread(temporary_live_view_timeout); } void StorageLiveView::shutdown() @@ -400,7 +402,8 @@ BlockInputStreams StorageLiveView::watch( if (query.is_watch_events) { - auto reader = std::make_shared(*this, blocks_ptr, blocks_metadata_ptr, active_ptr, condition, mutex, length, context.getSettingsRef().heartbeat_delay); + auto reader = std::make_shared(*this, blocks_ptr, blocks_metadata_ptr, active_ptr, condition, mutex, length, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), + context.getSettingsRef().temporary_live_view_timeout.totalSeconds()); if (no_users_thread.joinable()) { @@ -424,7 +427,8 @@ BlockInputStreams StorageLiveView::watch( } else { - auto reader = std::make_shared(*this, blocks_ptr, blocks_metadata_ptr, active_ptr, condition, mutex, length, context.getSettingsRef().heartbeat_delay); + auto reader = std::make_shared(*this, blocks_ptr, blocks_metadata_ptr, active_ptr, condition, mutex, length, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), + context.getSettingsRef().temporary_live_view_timeout.totalSeconds()); if (no_users_thread.joinable()) { diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index bb4377fbed6..37faee4b266 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -84,7 +84,7 @@ public: } /// Background thread for temporary tables /// which drops this table if there are no users - void startNoUsersThread(); + void startNoUsersThread(const UInt64 & timeout); Poco::FastMutex noUsersThreadMutex; bool noUsersThreadWakeUp{false}; Poco::Condition noUsersThreadCondition; @@ -110,7 +110,8 @@ public: void reset() { (*blocks_ptr).reset(); - (*blocks_metadata_ptr).reset(); + if (*blocks_metadata_ptr) + (*blocks_metadata_ptr)->hash.clear(); mergeable_blocks.reset(); } @@ -163,8 +164,6 @@ public: } } - SipHash hash; - UInt128 key; BlockInputStreams from; BlocksPtr blocks = std::make_shared(); BlocksPtrs mergeable_blocks; @@ -234,22 +233,13 @@ public: while (Block this_block = data->read()) { - this_block.updateHash(hash); blocks->push_back(this_block); } - /// get hash key - hash.get128(key.low, key.high); - /// Update blocks only if hash keys do not match - /// NOTE: hash could be different for the same result - /// if blocks are not in the same order - if (live_view.getBlocksHashKey() != key.toHexString()) - { - auto sample_block = blocks->front().cloneEmpty(); - BlockInputStreamPtr new_data = std::make_shared(std::make_shared(blocks), sample_block); - { - copyData(*new_data, *output); - } - } + + auto sample_block = blocks->front().cloneEmpty(); + BlockInputStreamPtr new_data = std::make_shared(std::make_shared(blocks), sample_block); + + copyData(*new_data, *output); } private: @@ -272,10 +262,11 @@ private: BlocksMetadataPtr new_blocks_metadata; BlocksPtrs mergeable_blocks; - void noUsersThread(); + void noUsersThread(const UInt64 & timeout); std::thread no_users_thread; std::atomic shutdown_called{false}; std::atomic startnousersthread_called{false}; + UInt64 temporary_live_view_timeout; StorageLiveView( const String & table_name_, @@ -300,29 +291,36 @@ public: void writeSuffix() override { - Poco::FastMutex::ScopedLock lock(storage.mutex); UInt128 key; + String key_str; new_hash->get128(key.low, key.high); - new_blocks_metadata->hash = key.toHexString(); - new_blocks_metadata->version = storage.getBlocksVersion() + 1; + key_str = key.toHexString(); - for (auto & block : *new_blocks) + Poco::FastMutex::ScopedLock lock(storage.mutex); + + if (storage.getBlocksHashKey() != key_str) { - block.insert({DataTypeUInt64().createColumnConst( - block.rows(), new_blocks_metadata->version)->convertToFullColumnIfConst(), - std::make_shared(), - "_version"}); - } + new_blocks_metadata->hash = key_str; + new_blocks_metadata->version = storage.getBlocksVersion() + 1; - (*storage.blocks_ptr) = new_blocks; - (*storage.blocks_metadata_ptr) = new_blocks_metadata; + for (auto & block : *new_blocks) + { + block.insert({DataTypeUInt64().createColumnConst( + block.rows(), new_blocks_metadata->version)->convertToFullColumnIfConst(), + std::make_shared(), + "_version"}); + } + + (*storage.blocks_ptr) = new_blocks; + (*storage.blocks_metadata_ptr) = new_blocks_metadata; + + storage.condition.broadcast(); + } new_blocks.reset(); new_blocks_metadata.reset(); new_hash.reset(); - - storage.condition.broadcast(); } void write(const Block & block) override diff --git a/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference b/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference index 86a72083821..b67c49182e9 100644 --- a/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference +++ b/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference @@ -1,4 +1,4 @@ lv 1 c9d39b11cce79112219a73aaa319b475 -1 4cd0592103888d4682de9a32a23602e3 -1 2186dbea325ee4c56b67e9b792e993a3 +2 4cd0592103888d4682de9a32a23602e3 +3 2186dbea325ee4c56b67e9b792e993a3 diff --git a/dbms/tests/queries/0_stateless/00957_live_view_watch.reference b/dbms/tests/queries/0_stateless/00957_live_view_watch.reference index 69a461b3fc4..65500578a69 100644 --- a/dbms/tests/queries/0_stateless/00957_live_view_watch.reference +++ b/dbms/tests/queries/0_stateless/00957_live_view_watch.reference @@ -1,4 +1,4 @@ lv 0 1 -6 1 -21 1 +6 2 +21 3 diff --git a/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.reference b/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.reference index 4bd209d4424..49d86fc2fbf 100644 --- a/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.reference +++ b/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.reference @@ -1,4 +1,4 @@ temporary_live_view_timeout 5 +live_view_heartbeat_interval 15 lv 0 -0 diff --git a/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.sql b/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.sql index 4a4a7ae5eed..c0625324bb6 100644 --- a/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.sql +++ b/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.sql @@ -2,13 +2,14 @@ DROP TABLE IF EXISTS test.lv; DROP TABLE IF EXISTS test.mt; SELECT name, value from system.settings WHERE name = 'temporary_live_view_timeout'; +SELECT name, value from system.settings WHERE name = 'live_view_heartbeat_interval'; +SET temporary_live_view_timeout=1; CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; SHOW TABLES LIKE 'lv'; -SELECT sleep(3); -SELECT sleep(2); +SELECT sleep(1); SHOW TABLES LIKE 'lv'; DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference index 69a461b3fc4..65500578a69 100644 --- a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference +++ b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference @@ -1,4 +1,4 @@ lv 0 1 -6 1 -21 1 +6 2 +21 3 diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py index 43bf748dc2b..29ec01c724a 100755 --- a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py +++ b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py @@ -26,30 +26,26 @@ client2.expect(prompt) client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) -client1.send(' DROP TABLE IF EXISTS test.mt') +client1.send('DROP TABLE IF EXISTS test.mt') +client1.expect(prompt) +client1.send('SET temporary_live_view_timeout=1') client1.expect(prompt) client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') client1.expect(prompt) client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) -client1.send('SELECT 4') -client1.expect(prompt, timeout=4) client1.send('WATCH test.lv') client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') client2.expect(prompt) client1.expect(r'6.*2' + end_of_block) -client2.send('SELECT sleep(3)') -client2.expect(prompt, timeout=4) client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') client2.expect(prompt) client1.expect(r'21.*3' + end_of_block) # send Ctrl-C os.kill(client1.process.pid,signal.SIGINT) client1.expect(prompt) -client1.send('SELECT sleep(3)') -client1.expect(prompt, timeout=4) -client1.send('SELECT sleep(3)') -client1.expect(prompt, timeout=4) +client1.send('SELECT sleep(1)') +client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect('Table test.lv doesn\'t exist') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py index 9d4f1d0716c..cbb659f1292 100755 --- a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -28,6 +28,8 @@ client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.mt') client1.expect(prompt) +client1.send('SET live_view_heartbeat_interval=1') +client1.expect(prompt) client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') client1.expect(prompt) client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') @@ -37,7 +39,7 @@ client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') client1.expect('2.*4cd0592103888d4682de9a32a23602e3' + end_of_block) client1.expect('Progress: 2.00 rows.*\)') # wait for heartbeat -client1.expect('Progress: 2.00 rows.*\)', timeout=15) +client1.expect('Progress: 2.00 rows.*\)') # send Ctrl-C os.kill(client1.process.pid,signal.SIGINT) client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py index cafbcd833b1..c596d169e1c 100755 --- a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py @@ -28,6 +28,8 @@ client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.mt') client1.expect(prompt) +client1.send('SET live_view_heartbeat_interval=1') +client1.expect(prompt) client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') client1.expect(prompt) client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') @@ -37,7 +39,7 @@ client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') client1.expect(r'6.*2' + end_of_block) client1.expect('Progress: 2.00 rows.*\)') # wait for heartbeat -client1.expect('Progress: 2.00 rows.*\)', timeout=15) +client1.expect('Progress: 2.00 rows.*\)') # send Ctrl-C os.kill(client1.process.pid,signal.SIGINT) client1.expect(prompt) From aa3ef47aab0a3fa2546604da2dec6801b230d055 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Fri, 7 Jun 2019 09:05:14 -0400 Subject: [PATCH 059/509] * Adding JSONEachRowWithProgress format that can be used with WATCH queries when HTTP connection is used in this case progress events are used as a heartbeat * Adding new tests that use JSONEachRowWithProgress format * Adding tests for WATCH query heartbeats when HTTP connection is used * Small fixes to uexpect.py --- dbms/src/Formats/FormatFactory.cpp | 2 + dbms/src/Formats/JSONEachRowRowOutputStream.h | 2 +- ...JSONEachRowWithProgressRowOutputStream.cpp | 47 +++++++++++++++ .../JSONEachRowWithProgressRowOutputStream.h | 27 +++++++++ .../00959_create_temporary_live_view.sql | 2 +- ...t_format_jsoneachrowwithprogress.reference | 5 ++ ..._select_format_jsoneachrowwithprogress.sql | 14 +++++ ...h_format_jsoneachrowwithprogress.reference | 7 +++ ...w_watch_format_jsoneachrowwithprogress.sql | 20 +++++++ ...70_live_view_watch_events_http_hearbeat.py | 57 +++++++++++++++++++ ..._view_watch_events_http_hearbeat.reference | 0 .../00971_live_view_watch_http_hearbeat.py | 57 +++++++++++++++++++ ...71_live_view_watch_http_hearbeat.reference | 0 dbms/tests/queries/0_stateless/uexpect.py | 10 +++- 14 files changed, 245 insertions(+), 5 deletions(-) create mode 100644 dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.cpp create mode 100644 dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.h create mode 100644 dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference create mode 100644 dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql create mode 100644 dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference create mode 100644 dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql create mode 100755 dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_hearbeat.py create mode 100644 dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_hearbeat.reference create mode 100755 dbms/tests/queries/0_stateless/00971_live_view_watch_http_hearbeat.py create mode 100644 dbms/tests/queries/0_stateless/00971_live_view_watch_http_hearbeat.reference diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index d5718278554..4357b324ac7 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -113,6 +113,7 @@ void registerInputFormatTSKV(FormatFactory & factory); void registerOutputFormatTSKV(FormatFactory & factory); void registerInputFormatJSONEachRow(FormatFactory & factory); void registerOutputFormatJSONEachRow(FormatFactory & factory); +void registerOutputFormatJSONEachRowWithProgress(FormatFactory & factory); void registerInputFormatParquet(FormatFactory & factory); void registerOutputFormatParquet(FormatFactory & factory); void registerInputFormatProtobuf(FormatFactory & factory); @@ -153,6 +154,7 @@ FormatFactory::FormatFactory() registerOutputFormatTSKV(*this); registerInputFormatJSONEachRow(*this); registerOutputFormatJSONEachRow(*this); + registerOutputFormatJSONEachRowWithProgress(*this); registerInputFormatProtobuf(*this); registerOutputFormatProtobuf(*this); registerInputFormatCapnProto(*this); diff --git a/dbms/src/Formats/JSONEachRowRowOutputStream.h b/dbms/src/Formats/JSONEachRowRowOutputStream.h index 4f2dc690aed..7d0d410b219 100644 --- a/dbms/src/Formats/JSONEachRowRowOutputStream.h +++ b/dbms/src/Formats/JSONEachRowRowOutputStream.h @@ -27,7 +27,7 @@ public: ostr.next(); } -private: +protected: WriteBuffer & ostr; size_t field_number = 0; Names fields; diff --git a/dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.cpp b/dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.cpp new file mode 100644 index 00000000000..4f59e99ff6d --- /dev/null +++ b/dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.cpp @@ -0,0 +1,47 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + + +void JSONEachRowWithProgressRowOutputStream::writeRowStartDelimiter() +{ + writeCString("{\"row\":{", ostr); +} + + +void JSONEachRowWithProgressRowOutputStream::writeRowEndDelimiter() +{ + writeCString("}}\n", ostr); + field_number = 0; +} + + +void JSONEachRowWithProgressRowOutputStream::onProgress(const Progress & value) +{ + progress.incrementPiecewiseAtomically(value); + writeCString("{\"progress\":", ostr); + progress.writeJSON(ostr); + writeCString("}\n", ostr); +} + + +void registerOutputFormatJSONEachRowWithProgress(FormatFactory & factory) +{ + factory.registerOutputFormat("JSONEachRowWithProgress", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & format_settings) + { + return std::make_shared( + std::make_shared(buf, sample, format_settings), sample); + }); +} + +} diff --git a/dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.h b/dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.h new file mode 100644 index 00000000000..e8cef8e147b --- /dev/null +++ b/dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/** The stream for outputting data in JSON format, by object per line + * that includes progress rows. Does not validate UTF-8. + */ +class JSONEachRowWithProgressRowOutputStream : public JSONEachRowRowOutputStream +{ +public: + using JSONEachRowRowOutputStream::JSONEachRowRowOutputStream; + + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + void onProgress(const Progress & value) override; + +private: + Progress progress; +}; + +} + diff --git a/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.sql b/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.sql index c0625324bb6..8cd6ee06ace 100644 --- a/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.sql +++ b/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.sql @@ -9,7 +9,7 @@ CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; SHOW TABLES LIKE 'lv'; -SELECT sleep(1); +SELECT sleep(2); SHOW TABLES LIKE 'lv'; DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference new file mode 100644 index 00000000000..0f6a0405cda --- /dev/null +++ b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference @@ -0,0 +1,5 @@ +lv +{"row":{"a":1}} +{"row":{"a":2}} +{"row":{"a":3}} +{"progress":{"read_rows":"3","read_bytes":"36","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} diff --git a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql new file mode 100644 index 00000000000..748d901b3bf --- /dev/null +++ b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; + +SHOW TABLES LIKE 'lv'; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT * FROM test.lv FORMAT JSONEachRowWithProgress; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference new file mode 100644 index 00000000000..8510bebad77 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference @@ -0,0 +1,7 @@ +lv +{"row":{"sum(a)":"0","_version":"1"}} +{"progress":{"read_rows":"1","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} +{"row":{"sum(a)":"6","_version":"2"}} +{"progress":{"read_rows":"1","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} +{"row":{"sum(a)":"21","_version":"3"}} +{"progress":{"read_rows":"1","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} diff --git a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql new file mode 100644 index 00000000000..9f2e0384dd8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +SHOW TABLES LIKE 'lv'; + +WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; + +INSERT INTO test.mt VALUES (1),(2),(3); + +WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; + +INSERT INTO test.mt VALUES (4),(5),(6); + +WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_hearbeat.py b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_hearbeat.py new file mode 100755 index 00000000000..1021edd7769 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_hearbeat.py @@ -0,0 +1,57 @@ +#!/usr/bin/env python +import imp +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) + +def client(name='', command=None): + if command is None: + client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) + else: + client = uexpect.spawn(command) + client.eol('\r') + # Note: uncomment this line for debugging + #client.logger(sys.stdout, prefix=name) + client.timeout(2) + return client + +prompt = ':\) ' +end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +client1 = client('client1>') +client1.expect(prompt) + +client1.send('DROP TABLE IF EXISTS test.lv') +client1.expect(prompt) +client1.send(' DROP TABLE IF EXISTS test.mt') +client1.expect(prompt) +client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') +client1.expect(prompt) +client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') +client1.expect(prompt) + +client2 = client('client2>', ['bash', '--noediting']) +client2.expect('\$ ') +client2.send('wget -O- -q "http://localhost:8123/?live_view_heartbeat_interval=1&query=WATCH test.lv EVENTS FORMAT JSONEachRowWithProgress"') +client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}\r\n', escape=True) +client2.expect('{"row":{"version":"1","hash":"c9d39b11cce79112219a73aaa319b475"}}', escape=True) +client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) +# heartbeat is provided by progress message +client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) + +client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') +client1.expect(prompt) + +client2.expect('{"row":{"version":"2","hash":"4cd0592103888d4682de9a32a23602e3"}}\r\n', escape=True) + +client2.expect('.*2\t.*\r\n') +## send Ctrl-C +os.kill(client2.process.pid,signal.SIGINT) + +client1.send('DROP TABLE test.lv') +client1.expect(prompt) +client1.send('DROP TABLE test.mt') +client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_hearbeat.reference b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_hearbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_hearbeat.py b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_hearbeat.py new file mode 100755 index 00000000000..33c937de649 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_hearbeat.py @@ -0,0 +1,57 @@ +#!/usr/bin/env python +import imp +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) + +def client(name='', command=None): + if command is None: + client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) + else: + client = uexpect.spawn(command) + client.eol('\r') + # Note: uncomment this line for debugging + #client.logger(sys.stdout, prefix=name) + client.timeout(2) + return client + +prompt = ':\) ' +end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +client1 = client('client1>') +client1.expect(prompt) + +client1.send('DROP TABLE IF EXISTS test.lv') +client1.expect(prompt) +client1.send(' DROP TABLE IF EXISTS test.mt') +client1.expect(prompt) +client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') +client1.expect(prompt) +client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') +client1.expect(prompt) + +client2 = client('client2>', ['bash', '--noediting']) +client2.expect('\$ ') +client2.send('wget -O- -q "http://localhost:8123/?live_view_heartbeat_interval=1&query=WATCH test.lv FORMAT JSONEachRowWithProgress"') +client2.expect('"progress".*',) +client2.expect('{"row":{"sum(a)":"0","_version":"1"}}\r\n', escape=True) +client2.expect('"progress".*\r\n') +# heartbeat is provided by progress message +client2.expect('"progress".*\r\n') + +client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') +client1.expect(prompt) + +client2.expect('"progress".*"read_rows":"2".*\r\n') +client2.expect('{"row":{"sum(a)":"6","_version":"2"}}\r\n', escape=True) + +## send Ctrl-C +os.kill(client2.process.pid,signal.SIGINT) + +client1.send('DROP TABLE test.lv') +client1.expect(prompt) +client1.send('DROP TABLE test.mt') +client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_hearbeat.reference b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_hearbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/uexpect.py b/dbms/tests/queries/0_stateless/uexpect.py index 412facb8389..f94a97addcc 100644 --- a/dbms/tests/queries/0_stateless/uexpect.py +++ b/dbms/tests/queries/0_stateless/uexpect.py @@ -132,7 +132,7 @@ class IO(object): self.buffer = self.buffer[self.match.end():] break if self._logger: - self._logger.write(self.before + self.after) + self._logger.write((self.before or '') + (self.after or '')) self._logger.flush() return self.match @@ -159,14 +159,18 @@ def spawn(command): os.close(slave) queue = Queue() - thread = Thread(target=reader, args=(master, queue)) + thread = Thread(target=reader, args=(process, master, queue)) thread.daemon = True thread.start() return IO(process, master, queue) -def reader(out, queue): +def reader(process, out, queue): while True: + if process.poll() is not None: + data = os.read(out) + queue.put(data) + break data = os.read(out, 65536) queue.put(data) From 4b68ffb15b0714372666bb46e86306227db3c436 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Fri, 7 Jun 2019 12:43:37 -0400 Subject: [PATCH 060/509] * Fixing typos in test names --- ...hearbeat.py => 00970_live_view_watch_events_http_heartbeat.py} | 0 ...ence => 00970_live_view_watch_events_http_heartbeat.reference} | 0 ...h_http_hearbeat.py => 00971_live_view_watch_http_heartbeat.py} | 0 ...t.reference => 00971_live_view_watch_http_heartbeat.reference} | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{00970_live_view_watch_events_http_hearbeat.py => 00970_live_view_watch_events_http_heartbeat.py} (100%) rename dbms/tests/queries/0_stateless/{00970_live_view_watch_events_http_hearbeat.reference => 00970_live_view_watch_events_http_heartbeat.reference} (100%) rename dbms/tests/queries/0_stateless/{00971_live_view_watch_http_hearbeat.py => 00971_live_view_watch_http_heartbeat.py} (100%) rename dbms/tests/queries/0_stateless/{00971_live_view_watch_http_hearbeat.reference => 00971_live_view_watch_http_heartbeat.reference} (100%) diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_hearbeat.py b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py similarity index 100% rename from dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_hearbeat.py rename to dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_hearbeat.reference b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_hearbeat.reference rename to dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.reference diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_hearbeat.py b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py similarity index 100% rename from dbms/tests/queries/0_stateless/00971_live_view_watch_http_hearbeat.py rename to dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_hearbeat.reference b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00971_live_view_watch_http_hearbeat.reference rename to dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.reference From 5079180595678bf5a92e35a16e292db99b80a285 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sat, 8 Jun 2019 09:26:02 -0400 Subject: [PATCH 061/509] * Fixing a bug in LIVE VIEW tables. Before when a table is created with "AS SELECT 1" the SELECT and WATCH queries would fail * Adding and updating LIVE VIEW tests to increase coverage --- dbms/src/Storages/IStorage.h | 4 ++-- dbms/src/Storages/ProxyStorage.h | 2 ++ .../0_stateless/00953_live_view_select.reference | 7 ++++--- .../queries/0_stateless/00953_live_view_select.sql | 10 ++++++++-- .../0_stateless/00972_live_view_select_1.reference | 2 ++ .../queries/0_stateless/00972_live_view_select_1.sql | 5 +++++ 6 files changed, 23 insertions(+), 7 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00972_live_view_select_1.reference create mode 100644 dbms/tests/queries/0_stateless/00972_live_view_select_1.sql diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 96b0dbd63ab..a617db6a8ad 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -82,8 +82,8 @@ public: public: /// thread-unsafe part. lockStructure must be acquired - const ColumnsDescription & getColumns() const; - void setColumns(ColumnsDescription columns_); + virtual const ColumnsDescription & getColumns() const; + virtual void setColumns(ColumnsDescription columns_); const IndicesDescription & getIndices() const; void setIndices(IndicesDescription indices_); diff --git a/dbms/src/Storages/ProxyStorage.h b/dbms/src/Storages/ProxyStorage.h index a4196b7d6f2..3e6926eb323 100644 --- a/dbms/src/Storages/ProxyStorage.h +++ b/dbms/src/Storages/ProxyStorage.h @@ -47,6 +47,8 @@ public: Names getColumnsRequiredForSampling() const override { return storage->getColumnsRequiredForSampling(); } Names getColumnsRequiredForFinal() const override { return storage->getColumnsRequiredForFinal(); } + const ColumnsDescription & getColumns() const override { return storage->getColumns(); } + void setColumns(ColumnsDescription columns_) override { return storage->setColumns(columns_); } NameAndTypePair getColumn(const String & column_name) const override { return storage->getColumn(column_name); } bool hasColumn(const String & column_name) const override { return storage->hasColumn(column_name); } static StoragePtr createProxyStorage(StoragePtr storage, BlockInputStreams streams, QueryProcessingStage::Enum to_stage) diff --git a/dbms/tests/queries/0_stateless/00953_live_view_select.reference b/dbms/tests/queries/0_stateless/00953_live_view_select.reference index 9c44fe02ff1..7beda18cd83 100644 --- a/dbms/tests/queries/0_stateless/00953_live_view_select.reference +++ b/dbms/tests/queries/0_stateless/00953_live_view_select.reference @@ -1,4 +1,5 @@ lv -1 -2 -3 +6 1 +6 1 +12 2 +12 2 diff --git a/dbms/tests/queries/0_stateless/00953_live_view_select.sql b/dbms/tests/queries/0_stateless/00953_live_view_select.sql index a51dfb112c1..492fa265b7d 100644 --- a/dbms/tests/queries/0_stateless/00953_live_view_select.sql +++ b/dbms/tests/queries/0_stateless/00953_live_view_select.sql @@ -2,13 +2,19 @@ DROP TABLE IF EXISTS test.lv; DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; +CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; SHOW TABLES LIKE 'lv'; INSERT INTO test.mt VALUES (1),(2),(3); -SELECT * FROM test.lv; +SELECT *,_version FROM test.lv; +SELECT *,_version FROM test.lv; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT *,_version FROM test.lv; +SELECT *,_version FROM test.lv; DROP TABLE test.lv; DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference b/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference new file mode 100644 index 00000000000..f2f8c69d020 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference @@ -0,0 +1,2 @@ +lv +1 diff --git a/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql b/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql new file mode 100644 index 00000000000..01bfeb08fd5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql @@ -0,0 +1,5 @@ +DROP TABLE IF EXISTS test.lv; +CREATE LIVE VIEW test.lv AS SELECT 1; +SHOW TABLES LIKE 'lv'; +SELECT * FROM test.lv; +DROP TABLE test.lv; From 137f5127ac3fc83b1322dfbe7fd71d0a6f8c0465 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sat, 8 Jun 2019 09:54:48 -0400 Subject: [PATCH 062/509] * Renaming 00953_live_view_select test to avoid test number conflict --- ...ive_view_select.reference => 00973_live_view_select.reference} | 0 .../{00953_live_view_select.sql => 00973_live_view_select.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{00953_live_view_select.reference => 00973_live_view_select.reference} (100%) rename dbms/tests/queries/0_stateless/{00953_live_view_select.sql => 00973_live_view_select.sql} (100%) diff --git a/dbms/tests/queries/0_stateless/00953_live_view_select.reference b/dbms/tests/queries/0_stateless/00973_live_view_select.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00953_live_view_select.reference rename to dbms/tests/queries/0_stateless/00973_live_view_select.reference diff --git a/dbms/tests/queries/0_stateless/00953_live_view_select.sql b/dbms/tests/queries/0_stateless/00973_live_view_select.sql similarity index 100% rename from dbms/tests/queries/0_stateless/00953_live_view_select.sql rename to dbms/tests/queries/0_stateless/00973_live_view_select.sql From 87b58e4150b086c100b6baee099664a19b726ca1 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Mon, 10 Jun 2019 07:18:33 -0400 Subject: [PATCH 063/509] * Fixing clang errors * Fixing LIVE VIEW tests --- .../PushingToViewsBlockOutputStream.cpp | 2 +- dbms/src/Storages/IStorage.h | 2 +- dbms/src/Storages/StorageLiveView.h | 8 +++--- .../00966_live_view_watch_events_http.py | 2 +- .../0_stateless/00967_live_view_watch_http.py | 2 +- ...0_live_view_watch_events_http_heartbeat.py | 3 +-- .../00971_live_view_watch_http_heartbeat.py | 2 +- dbms/tests/queries/0_stateless/uexpect.py | 25 +++++++++++++------ 8 files changed, 27 insertions(+), 19 deletions(-) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 06f9a1edacf..c023670d4e1 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -86,7 +86,7 @@ void PushingToViewsBlockOutputStream::write(const Block & block) if (auto * live_view = dynamic_cast(storage.get())) { - BlockOutputStreamPtr output = std::make_shared(*live_view); + output = std::make_shared(*live_view); StorageLiveView::writeIntoLiveView(*live_view, block, context, output); } else diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index a617db6a8ad..4a63c1ef34a 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -60,7 +60,7 @@ public: /// The name of the table. virtual std::string getTableName() const = 0; - virtual std::string getDatabaseName() const { return {}; } // FIXME: should be an abstract method! + virtual std::string getDatabaseName() const { return {}; } /// Returns true if the storage receives data from a remote server or servers. virtual bool isRemote() const { return false; } diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index 37faee4b266..56b48fe1154 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -47,7 +47,7 @@ public: ~StorageLiveView() override; String getName() const override { return "LiveView"; } String getTableName() const override { return table_name; } - String getDatabaseName() const { return database_name; } + String getDatabaseName() const override { return database_name; } String getSelectDatabaseName() const { return select_database_name; } String getSelectTableName() const { return select_table_name; } @@ -55,7 +55,7 @@ public: bool hasColumn(const String & column_name) const override; // const NamesAndTypesList & getColumnsListImpl() const override { return *columns; } - ASTPtr getInnerQuery() const { return inner_query->clone(); }; + ASTPtr getInnerQuery() const { return inner_query->clone(); } /// It is passed inside the query and solved at its level. bool supportsSampling() const override { return true; } @@ -218,10 +218,10 @@ public: mergeable_blocks->push_back(new_mergeable_blocks); /// Create from blocks streams - for (auto & blocks : *mergeable_blocks) + for (auto & blocks_ : *mergeable_blocks) { auto sample_block = mergeable_blocks->front()->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks), sample_block); + BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); from.push_back(std::move(stream)); } } diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py index be1cd220679..904d5214d5b 100755 --- a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py +++ b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py @@ -34,7 +34,7 @@ client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) client2 = client('client2>', ['bash', '--noediting']) -client2.expect('\$ ') +client2.expect('[\$#] ') client2.send('wget -O- -q "http://localhost:8123/?query=WATCH test.lv EVENTS"') client2.expect('.*1\tc9d39b11cce79112219a73aaa319b475\r\n') diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py index 0a844579f5d..4363b453852 100755 --- a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py +++ b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py @@ -34,7 +34,7 @@ client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) client2 = client('client2>', ['bash', '--noediting']) -client2.expect('\$ ') +client2.expect('[\$#] ') client2.send('wget -O- -q "http://localhost:8123/?query=WATCH test.lv"') client2.expect('.*0\t1\r\n') diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py index 1021edd7769..f96cac1e1c7 100755 --- a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py @@ -34,7 +34,7 @@ client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) client2 = client('client2>', ['bash', '--noediting']) -client2.expect('\$ ') +client2.expect('[\$#] ') client2.send('wget -O- -q "http://localhost:8123/?live_view_heartbeat_interval=1&query=WATCH test.lv EVENTS FORMAT JSONEachRowWithProgress"') client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}\r\n', escape=True) client2.expect('{"row":{"version":"1","hash":"c9d39b11cce79112219a73aaa319b475"}}', escape=True) @@ -47,7 +47,6 @@ client1.expect(prompt) client2.expect('{"row":{"version":"2","hash":"4cd0592103888d4682de9a32a23602e3"}}\r\n', escape=True) -client2.expect('.*2\t.*\r\n') ## send Ctrl-C os.kill(client2.process.pid,signal.SIGINT) diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py index 33c937de649..cb8cf8f5582 100755 --- a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py @@ -34,7 +34,7 @@ client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) client2 = client('client2>', ['bash', '--noediting']) -client2.expect('\$ ') +client2.expect('[\$#] ') client2.send('wget -O- -q "http://localhost:8123/?live_view_heartbeat_interval=1&query=WATCH test.lv FORMAT JSONEachRowWithProgress"') client2.expect('"progress".*',) client2.expect('{"row":{"sum(a)":"0","_version":"1"}}\r\n', escape=True) diff --git a/dbms/tests/queries/0_stateless/uexpect.py b/dbms/tests/queries/0_stateless/uexpect.py index f94a97addcc..dc77161d72f 100644 --- a/dbms/tests/queries/0_stateless/uexpect.py +++ b/dbms/tests/queries/0_stateless/uexpect.py @@ -37,8 +37,8 @@ class ExpectTimeoutError(Exception): def __str__(self): return ('Timeout %.3fs ' % float(self.timeout) + 'for %s ' % repr(self.pattern.pattern) + - 'buffer ends with %s ' % repr(self.buffer[-80:]) + - 'or \'%s\'' % ','.join(['%x' % ord(c) for c in self.buffer[-80:]])) + 'buffer %s ' % repr(self.buffer[:]) + + 'or \'%s\'' % ','.join(['%x' % ord(c) for c in self.buffer[:]])) class IO(object): class EOF(object): @@ -111,10 +111,11 @@ class IO(object): pattern = re.compile(pattern) if timeout is None: timeout = self._timeout - while timeout >= 0: + timeleft = timeout + while timeleft >= 0: start_time = time.time() try: - data = self.read(timeout=timeout, raise_exception=True) + data = self.read(timeout=timeleft, raise_exception=True) except TimeoutError: if self._logger: self._logger.write(self.buffer + '\n') @@ -122,7 +123,7 @@ class IO(object): exception = ExpectTimeoutError(pattern, timeout, self.buffer) self.buffer = None raise exception - timeout -= (time.time() - start_time) + timeleft -= (time.time() - start_time) if data: self.buffer = self.buffer + data if self.buffer else data self.match = pattern.search(self.buffer, 0) @@ -134,23 +135,31 @@ class IO(object): if self._logger: self._logger.write((self.before or '') + (self.after or '')) self._logger.flush() + if self.match is None: + exception = ExpectTimeoutError(pattern, timeout, self.buffer) + self.buffer = None + raise exception return self.match def read(self, timeout=0, raise_exception=False): data = '' + timeleft = timeout try: - while timeout >= 0 : + while timeleft >= 0 : start_time = time.time() - data += self.queue.get(timeout=timeout) + data += self.queue.get(timeout=timeleft) if data: break - timeout -= (time.time() - start_time) + timeleft -= (time.time() - start_time) except Empty: if data: return data if raise_exception: raise TimeoutError(timeout) pass + if not data and raise_exception: + raise TimeoutError(timeout) + return data def spawn(command): From a68980ab385d61dcb1e744a855e3053581b9fb90 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Mon, 10 Jun 2019 07:41:33 -0400 Subject: [PATCH 064/509] * Style check fixes --- .../DataStreams/LiveViewBlockInputStream.h | 7 +++---- .../LiveViewEventsBlockInputStream.h | 13 ++++++------ .../Interpreters/InterpreterAlterQuery.cpp | 20 +++++++++---------- dbms/src/Parsers/ParserAlterQuery.cpp | 4 +++- dbms/src/Storages/StorageLiveView.cpp | 4 ++-- dbms/src/Storages/StorageLiveView.h | 3 ++- 6 files changed, 27 insertions(+), 24 deletions(-) diff --git a/dbms/src/DataStreams/LiveViewBlockInputStream.h b/dbms/src/DataStreams/LiveViewBlockInputStream.h index 93eae976177..6129c836597 100644 --- a/dbms/src/DataStreams/LiveViewBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewBlockInputStream.h @@ -46,7 +46,7 @@ public: std::shared_ptr active_ptr_, Poco::Condition & condition_, Poco::FastMutex & mutex_, int64_t length_, const UInt64 & heartbeat_interval_, const UInt64 & temporary_live_view_timeout_) - : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_interval(heartbeat_interval_), temporary_live_view_timeout(temporary_live_view_timeout_), + : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_interval(heartbeat_interval_ * 1000000), temporary_live_view_timeout(temporary_live_view_timeout_), blocks_hash("") { /// grab active pointer @@ -160,9 +160,8 @@ protected: } while (true) { - bool signaled = condition.tryWait(mutex, - std::max((UInt64)0, (heartbeat_interval * 1000000) - - ((UInt64)timestamp.epochMicroseconds() - last_event_timestamp)) / 1000); + UInt64 timestamp_usec = (UInt64)timestamp.epochMicroseconds(); + bool signaled = condition.tryWait(mutex, std::max((UInt64)0, heartbeat_interval - (timestamp_usec - last_event_timestamp)) / 1000); if (isCancelled() || storage.is_dropped) { diff --git a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h index d0f92806d74..35d2545419b 100644 --- a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h @@ -47,7 +47,7 @@ public: /// and LIMIT 0 just returns data without waiting for any updates LiveViewEventsBlockInputStream(StorageLiveView & storage_, std::shared_ptr blocks_ptr_, std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, Poco::Condition & condition_, Poco::FastMutex & mutex_, int64_t length_, const UInt64 & heartbeat_interval_, const UInt64 & temporary_live_view_timeout_) - : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_interval(heartbeat_interval_), temporary_live_view_timeout(temporary_live_view_timeout_) + : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_interval(heartbeat_interval_ * 1000000), temporary_live_view_timeout(temporary_live_view_timeout_) { /// grab active pointer active = active_ptr.lock(); @@ -64,7 +64,8 @@ public: condition.broadcast(); } - Block getHeader() const override { + Block getHeader() const override + { return { ColumnWithTypeAndName( ColumnUInt64::create(), @@ -92,7 +93,8 @@ public: { active = active_ptr.lock(); { - if (!blocks || blocks.get() != (*blocks_ptr).get()) { + if (!blocks || blocks.get() != (*blocks_ptr).get()) + { blocks = (*blocks_ptr); blocks_metadata = (*blocks_metadata_ptr); } @@ -188,9 +190,8 @@ protected: } while (true) { - bool signaled = condition.tryWait(mutex, - std::max((UInt64)0, (heartbeat_interval * 1000000) - - ((UInt64)timestamp.epochMicroseconds() - last_event_timestamp)) / 1000); + UInt64 timestamp_usec = (UInt64)timestamp.epochMicroseconds(); + bool signaled = condition.tryWait(mutex, std::max((UInt64)0, heartbeat_interval - (timestamp_usec - last_event_timestamp)) / 1000); if (isCancelled() || storage.is_dropped) { diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index 6d2fc61f080..e8f3e4f4553 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -79,17 +79,17 @@ BlockIO InterpreterAlterQuery::execute() if (!live_view_commands.empty()) { - live_view_commands.validate(*table); - for (const LiveViewCommand & command : live_view_commands) - { - auto live_view = std::dynamic_pointer_cast(table); - switch (command.type) - { - case LiveViewCommand::REFRESH: - live_view->refresh(context); - break; - } + live_view_commands.validate(*table); + for (const LiveViewCommand & command : live_view_commands) + { + auto live_view = std::dynamic_pointer_cast(table); + switch (command.type) + { + case LiveViewCommand::REFRESH: + live_view->refresh(context); + break; } + } } if (!alter_commands.empty()) diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index f8803cc3bc8..58a3b3a388b 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -446,10 +446,12 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!s_alter_table.ignore(pos, expected)) { if (!s_alter_live_view.ignore(pos, expected)) - if (!s_alter_live_channel.ignore(pos, expected)) + { + if (!s_alter_live_channel.ignore(pos, expected)) return false; else is_live_channel = true; + } else is_live_view = true; } diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index 108e146dc73..dd6e0179410 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -141,7 +141,7 @@ StorageLiveView::StorageLiveView( NameAndTypePair StorageLiveView::getColumn(const String & column_name) const { - if ( column_name == "_version" ) + if (column_name == "_version") return NameAndTypePair("_version", std::make_shared()); return IStorage::getColumn(column_name); @@ -149,7 +149,7 @@ NameAndTypePair StorageLiveView::getColumn(const String & column_name) const bool StorageLiveView::hasColumn(const String & column_name) const { - if ( column_name == "_version" ) + if (column_name == "_version") return true; return IStorage::hasColumn(column_name); diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index 56b48fe1154..fb250aec859 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -30,7 +30,8 @@ namespace DB class IAST; using ASTPtr = std::shared_ptr; -struct BlocksMetadata { +struct BlocksMetadata +{ String hash; UInt64 version; }; From fc4ea55b98ebd1f4c9a2d6fb0f1fed5a00deade6 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Mon, 10 Jun 2019 08:11:43 -0400 Subject: [PATCH 065/509] * Fixing bug in dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp that was introduced when trying to fix a clang error. --- dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index c023670d4e1..f361420de95 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -86,8 +86,8 @@ void PushingToViewsBlockOutputStream::write(const Block & block) if (auto * live_view = dynamic_cast(storage.get())) { - output = std::make_shared(*live_view); - StorageLiveView::writeIntoLiveView(*live_view, block, context, output); + BlockOutputStreamPtr output_ = std::make_shared(*live_view); + StorageLiveView::writeIntoLiveView(*live_view, block, context, output_); } else { From 1ebe3047ffc6719b4f817a25ea66a8d2f9ee47ac Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Mon, 10 Jun 2019 18:12:06 -0400 Subject: [PATCH 066/509] * Moving uexpect.py to dbms/tests/integration/helpers folder * Updating uexpect.py to support close() method and context manager for proper resource cleanup * Updating tests to use uexpect context manager --- .../helpers}/uexpect.py | 10 +++ .../0_stateless/00958_live_view_watch_live.py | 49 +++++++-------- .../00960_live_view_watch_events_live.py | 49 +++++++-------- .../00962_temporary_live_view_watch_live.py | 49 +++++++-------- ..._temporary_live_view_watch_live_timeout.py | 63 +++++++++---------- .../00964_live_view_watch_events_heartbeat.py | 55 ++++++++-------- .../00965_live_view_watch_heartbeat.py | 55 ++++++++-------- .../00966_live_view_watch_events_http.py | 48 +++++++------- .../0_stateless/00967_live_view_watch_http.py | 48 +++++++------- ...0_live_view_watch_events_http_heartbeat.py | 56 ++++++++--------- .../00971_live_view_watch_http_heartbeat.py | 58 ++++++++--------- 11 files changed, 272 insertions(+), 268 deletions(-) rename dbms/tests/{queries/0_stateless => integration/helpers}/uexpect.py (95%) diff --git a/dbms/tests/queries/0_stateless/uexpect.py b/dbms/tests/integration/helpers/uexpect.py similarity index 95% rename from dbms/tests/queries/0_stateless/uexpect.py rename to dbms/tests/integration/helpers/uexpect.py index dc77161d72f..58be8f60ff8 100644 --- a/dbms/tests/queries/0_stateless/uexpect.py +++ b/dbms/tests/integration/helpers/uexpect.py @@ -74,6 +74,12 @@ class IO(object): self._logger = None self._eol = '' + def __enter__(self): + return self + + def __exit__(self, type, value, traceback): + self.close() + def logger(self, logger=None, prefix=''): if logger: self._logger = self.Logger(logger, prefix=prefix) @@ -90,6 +96,8 @@ class IO(object): return self._eol def close(self): + self.process.terminate() + os.close(self.master) if self._logger: self._logger.write('\n') self._logger.flush() @@ -155,9 +163,11 @@ class IO(object): if data: return data if raise_exception: + print 'DEBUG...', timeleft, repr(data), raise TimeoutError(timeout) pass if not data and raise_exception: + print 'DEBUG.... here' raise TimeoutError(timeout) return data diff --git a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py index 9fb58cb556c..b60e82299f7 100755 --- a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) def client(name=''): client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) @@ -18,29 +18,28 @@ def client(name=''): prompt = ':\) ' end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' -client1 = client('client1>') -client2 = client('client2>') -client1.expect(prompt) -client2.expect(prompt) +with client('client1>') as client1, client('client2>') as client2: + client1.expect(prompt) + client2.expect(prompt) -client1.send('DROP TABLE IF EXISTS test.lv') -client1.expect(prompt) -client1.send(' DROP TABLE IF EXISTS test.mt') -client1.expect(prompt) -client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') -client1.expect(prompt) -client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') -client1.expect(prompt) -client1.send('WATCH test.lv') -client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') -client1.expect(r'6.*2' + end_of_block) -client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') -client1.expect(r'21.*3' + end_of_block) -# send Ctrl-C -os.kill(client1.process.pid,signal.SIGINT) -client1.expect(prompt) -client1.send('DROP TABLE test.lv') -client1.expect(prompt) -client1.send('DROP TABLE test.mt') -client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv') + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(r'6.*2' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') + client1.expect(r'21.*3' + end_of_block) + # send Ctrl-C + os.kill(client1.process.pid,signal.SIGINT) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py index fa96331e3fb..9fdabd499d0 100755 --- a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py +++ b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) def client(name=''): client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) @@ -18,29 +18,28 @@ def client(name=''): prompt = ':\) ' end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' -client1 = client('client1>') -client2 = client('client2>') -client1.expect(prompt) -client2.expect(prompt) +with client('client1>') as client1, client('client2>') as client2: + client1.expect(prompt) + client2.expect(prompt) -client1.send('DROP TABLE IF EXISTS test.lv') -client1.expect(prompt) -client1.send(' DROP TABLE IF EXISTS test.mt') -client1.expect(prompt) -client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') -client1.expect(prompt) -client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') -client1.expect(prompt) -client1.send('WATCH test.lv EVENTS') -client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') -client1.expect('2.*4cd0592103888d4682de9a32a23602e3' + end_of_block) -client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') -client1.expect('3.*2186dbea325ee4c56b67e9b792e993a3' + end_of_block) -# send Ctrl-C -os.kill(client1.process.pid,signal.SIGINT) -client1.expect(prompt) -client1.send('DROP TABLE test.lv') -client1.expect(prompt) -client1.send('DROP TABLE test.mt') -client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv EVENTS') + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect('2.*4cd0592103888d4682de9a32a23602e3' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') + client1.expect('3.*2186dbea325ee4c56b67e9b792e993a3' + end_of_block) + # send Ctrl-C + os.kill(client1.process.pid,signal.SIGINT) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py index 0b0260adeed..0e7c3c43254 100755 --- a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) def client(name=''): client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) @@ -18,29 +18,28 @@ def client(name=''): prompt = ':\) ' end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' -client1 = client('client1>') -client2 = client('client2>') -client1.expect(prompt) -client2.expect(prompt) +with client('client1>') as client1, client('client2>') as client2: + client1.expect(prompt) + client2.expect(prompt) -client1.send('DROP TABLE IF EXISTS test.lv') -client1.expect(prompt) -client1.send(' DROP TABLE IF EXISTS test.mt') -client1.expect(prompt) -client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') -client1.expect(prompt) -client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') -client1.expect(prompt) -client1.send('WATCH test.lv') -client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') -client1.expect(r'6.*2' + end_of_block) -client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') -client1.expect(r'21.*3' + end_of_block) -# send Ctrl-C -os.kill(client1.process.pid,signal.SIGINT) -client1.expect(prompt) -client1.send('DROP TABLE test.lv') -client1.expect(prompt) -client1.send('DROP TABLE test.mt') -client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv') + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(r'6.*2' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') + client1.expect(r'21.*3' + end_of_block) + # send Ctrl-C + os.kill(client1.process.pid,signal.SIGINT) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py index 29ec01c724a..b3f6809ded0 100755 --- a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py +++ b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) def client(name=''): client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) @@ -18,36 +18,35 @@ def client(name=''): prompt = ':\) ' end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' -client1 = client('client1>') -client2 = client('client2>') -client1.expect(prompt) -client2.expect(prompt) +with client('client1>') as client1, client('client2>') as client2: + client1.expect(prompt) + client2.expect(prompt) -client1.send('DROP TABLE IF EXISTS test.lv') -client1.expect(prompt) -client1.send('DROP TABLE IF EXISTS test.mt') -client1.expect(prompt) -client1.send('SET temporary_live_view_timeout=1') -client1.expect(prompt) -client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') -client1.expect(prompt) -client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') -client1.expect(prompt) -client1.send('WATCH test.lv') -client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') -client2.expect(prompt) -client1.expect(r'6.*2' + end_of_block) -client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') -client2.expect(prompt) -client1.expect(r'21.*3' + end_of_block) -# send Ctrl-C -os.kill(client1.process.pid,signal.SIGINT) -client1.expect(prompt) -client1.send('SELECT sleep(1)') -client1.expect(prompt) -client1.send('DROP TABLE test.lv') -client1.expect('Table test.lv doesn\'t exist') -client1.expect(prompt) -client1.send('DROP TABLE test.mt') -client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('SET temporary_live_view_timeout=1') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv') + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client2.expect(prompt) + client1.expect(r'6.*2' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') + client2.expect(prompt) + client1.expect(r'21.*3' + end_of_block) + # send Ctrl-C + os.kill(client1.process.pid,signal.SIGINT) + client1.expect(prompt, timeout=5) + client1.send('SELECT sleep(1)') + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect('Table test.lv doesn\'t exist') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py index cbb659f1292..1e66daef953 100755 --- a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) def client(name=''): client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) @@ -18,32 +18,31 @@ def client(name=''): prompt = ':\) ' end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' -client1 = client('client1>') -client2 = client('client2>') -client1.expect(prompt) -client2.expect(prompt) +with client('client1>') as client1, client('client2>') as client2: + client1.expect(prompt) + client2.expect(prompt) -client1.send('DROP TABLE IF EXISTS test.lv') -client1.expect(prompt) -client1.send(' DROP TABLE IF EXISTS test.mt') -client1.expect(prompt) -client1.send('SET live_view_heartbeat_interval=1') -client1.expect(prompt) -client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') -client1.expect(prompt) -client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') -client1.expect(prompt) -client1.send('WATCH test.lv EVENTS') -client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') -client1.expect('2.*4cd0592103888d4682de9a32a23602e3' + end_of_block) -client1.expect('Progress: 2.00 rows.*\)') -# wait for heartbeat -client1.expect('Progress: 2.00 rows.*\)') -# send Ctrl-C -os.kill(client1.process.pid,signal.SIGINT) -client1.expect(prompt) -client1.send('DROP TABLE test.lv') -client1.expect(prompt) -client1.send('DROP TABLE test.mt') -client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('SET live_view_heartbeat_interval=1') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv EVENTS') + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect('2.*4cd0592103888d4682de9a32a23602e3' + end_of_block) + client1.expect('Progress: 2.00 rows.*\)') + # wait for heartbeat + client1.expect('Progress: 2.00 rows.*\)') + # send Ctrl-C + os.kill(client1.process.pid,signal.SIGINT) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py index c596d169e1c..7f8ed6036ac 100755 --- a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) def client(name=''): client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) @@ -18,32 +18,31 @@ def client(name=''): prompt = ':\) ' end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' -client1 = client('client1>') -client2 = client('client2>') -client1.expect(prompt) -client2.expect(prompt) +with client('client1>') as client1, client('client2>') as client2: + client1.expect(prompt) + client2.expect(prompt) -client1.send('DROP TABLE IF EXISTS test.lv') -client1.expect(prompt) -client1.send(' DROP TABLE IF EXISTS test.mt') -client1.expect(prompt) -client1.send('SET live_view_heartbeat_interval=1') -client1.expect(prompt) -client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') -client1.expect(prompt) -client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') -client1.expect(prompt) -client1.send('WATCH test.lv') -client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') -client1.expect(r'6.*2' + end_of_block) -client1.expect('Progress: 2.00 rows.*\)') -# wait for heartbeat -client1.expect('Progress: 2.00 rows.*\)') -# send Ctrl-C -os.kill(client1.process.pid,signal.SIGINT) -client1.expect(prompt) -client1.send('DROP TABLE test.lv') -client1.expect(prompt) -client1.send('DROP TABLE test.mt') -client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('SET live_view_heartbeat_interval=1') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv') + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(r'6.*2' + end_of_block) + client1.expect('Progress: 2.00 rows.*\)') + # wait for heartbeat + client1.expect('Progress: 2.00 rows.*\)') + # send Ctrl-C + os.kill(client1.process.pid,signal.SIGINT) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py index 904d5214d5b..e7dc54ee4c0 100755 --- a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py +++ b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) def client(name='', command=None): if command is None: @@ -21,31 +21,31 @@ def client(name='', command=None): prompt = ':\) ' end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' -client1 = client('client1>') -client1.expect(prompt) -client1.send('DROP TABLE IF EXISTS test.lv') -client1.expect(prompt) -client1.send(' DROP TABLE IF EXISTS test.mt') -client1.expect(prompt) -client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') -client1.expect(prompt) -client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') -client1.expect(prompt) +with client('client1>') as client1, client('client2>', ['bash', '--noediting']) as client2: + client1.expect(prompt) -client2 = client('client2>', ['bash', '--noediting']) -client2.expect('[\$#] ') -client2.send('wget -O- -q "http://localhost:8123/?query=WATCH test.lv EVENTS"') -client2.expect('.*1\tc9d39b11cce79112219a73aaa319b475\r\n') + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) -client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') -client1.expect(prompt) + client2.expect('[\$#] ') + client2.send('wget -O- -q "http://localhost:8123/?query=WATCH test.lv EVENTS"') + client2.expect('.*1\tc9d39b11cce79112219a73aaa319b475\r\n') -client2.expect('.*2\t.*\r\n') -## send Ctrl-C -os.kill(client2.process.pid,signal.SIGINT) + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) -client1.send('DROP TABLE test.lv') -client1.expect(prompt) -client1.send('DROP TABLE test.mt') -client1.expect(prompt) + client2.expect('.*2\t.*\r\n') + ## send Ctrl-C + os.kill(client2.process.pid,signal.SIGINT) + + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py index 4363b453852..6874ae376a3 100755 --- a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py +++ b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) def client(name='', command=None): if command is None: @@ -21,31 +21,31 @@ def client(name='', command=None): prompt = ':\) ' end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' -client1 = client('client1>') -client1.expect(prompt) -client1.send('DROP TABLE IF EXISTS test.lv') -client1.expect(prompt) -client1.send(' DROP TABLE IF EXISTS test.mt') -client1.expect(prompt) -client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') -client1.expect(prompt) -client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') -client1.expect(prompt) +with client('client1>') as client1, client('client2>', ['bash', '--noediting']) as client2: + client1.expect(prompt) -client2 = client('client2>', ['bash', '--noediting']) -client2.expect('[\$#] ') -client2.send('wget -O- -q "http://localhost:8123/?query=WATCH test.lv"') -client2.expect('.*0\t1\r\n') + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) -client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') -client1.expect(prompt) + client2.expect('[\$#] ') + client2.send('wget -O- -q "http://localhost:8123/?query=WATCH test.lv"') + client2.expect('.*0\t1\r\n') -client2.expect('.*6\t2\r\n') -## send Ctrl-C -os.kill(client2.process.pid,signal.SIGINT) + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) -client1.send('DROP TABLE test.lv') -client1.expect(prompt) -client1.send('DROP TABLE test.mt') -client1.expect(prompt) + client2.expect('.*6\t2\r\n') + ## send Ctrl-C + os.kill(client2.process.pid,signal.SIGINT) + + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py index f96cac1e1c7..4730caf5f57 100755 --- a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) def client(name='', command=None): if command is None: @@ -21,36 +21,36 @@ def client(name='', command=None): prompt = ':\) ' end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' -client1 = client('client1>') -client1.expect(prompt) -client1.send('DROP TABLE IF EXISTS test.lv') -client1.expect(prompt) -client1.send(' DROP TABLE IF EXISTS test.mt') -client1.expect(prompt) -client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') -client1.expect(prompt) -client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') -client1.expect(prompt) +with client('client1>') as client1, client('client2>', ['bash', '--noediting']) as client2: + client1.expect(prompt) -client2 = client('client2>', ['bash', '--noediting']) -client2.expect('[\$#] ') -client2.send('wget -O- -q "http://localhost:8123/?live_view_heartbeat_interval=1&query=WATCH test.lv EVENTS FORMAT JSONEachRowWithProgress"') -client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}\r\n', escape=True) -client2.expect('{"row":{"version":"1","hash":"c9d39b11cce79112219a73aaa319b475"}}', escape=True) -client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) -# heartbeat is provided by progress message -client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) -client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') -client1.expect(prompt) + client2.expect('[\$#] ') + client2.send('wget -O- -q "http://localhost:8123/?live_view_heartbeat_interval=1&query=WATCH test.lv EVENTS FORMAT JSONEachRowWithProgress"') + client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}\r\n', escape=True) + client2.expect('{"row":{"version":"1","hash":"c9d39b11cce79112219a73aaa319b475"}}', escape=True) + client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) + # heartbeat is provided by progress message + client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) -client2.expect('{"row":{"version":"2","hash":"4cd0592103888d4682de9a32a23602e3"}}\r\n', escape=True) + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) -## send Ctrl-C -os.kill(client2.process.pid,signal.SIGINT) + client2.expect('{"row":{"version":"2","hash":"4cd0592103888d4682de9a32a23602e3"}}\r\n', escape=True) -client1.send('DROP TABLE test.lv') -client1.expect(prompt) -client1.send('DROP TABLE test.mt') -client1.expect(prompt) + ## send Ctrl-C + os.kill(client2.process.pid,signal.SIGINT) + + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py index cb8cf8f5582..5884bc1e2a5 100755 --- a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) def client(name='', command=None): if command is None: @@ -21,37 +21,37 @@ def client(name='', command=None): prompt = ':\) ' end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' -client1 = client('client1>') -client1.expect(prompt) -client1.send('DROP TABLE IF EXISTS test.lv') -client1.expect(prompt) -client1.send(' DROP TABLE IF EXISTS test.mt') -client1.expect(prompt) -client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') -client1.expect(prompt) -client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') -client1.expect(prompt) +with client('client1>') as client1, client('client2>', ['bash', '--noediting']) as client2: + client1.expect(prompt) -client2 = client('client2>', ['bash', '--noediting']) -client2.expect('[\$#] ') -client2.send('wget -O- -q "http://localhost:8123/?live_view_heartbeat_interval=1&query=WATCH test.lv FORMAT JSONEachRowWithProgress"') -client2.expect('"progress".*',) -client2.expect('{"row":{"sum(a)":"0","_version":"1"}}\r\n', escape=True) -client2.expect('"progress".*\r\n') -# heartbeat is provided by progress message -client2.expect('"progress".*\r\n') + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) -client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') -client1.expect(prompt) + client2.expect('[\$#] ') + client2.send('wget -O- -q "http://localhost:8123/?live_view_heartbeat_interval=1&query=WATCH test.lv FORMAT JSONEachRowWithProgress"') + client2.expect('"progress".*',) + client2.expect('{"row":{"sum(a)":"0","_version":"1"}}\r\n', escape=True) + client2.expect('"progress".*\r\n') + # heartbeat is provided by progress message + client2.expect('"progress".*\r\n') -client2.expect('"progress".*"read_rows":"2".*\r\n') -client2.expect('{"row":{"sum(a)":"6","_version":"2"}}\r\n', escape=True) + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) -## send Ctrl-C -os.kill(client2.process.pid,signal.SIGINT) + client2.expect('"progress".*"read_rows":"2".*\r\n') + client2.expect('{"row":{"sum(a)":"6","_version":"2"}}\r\n', escape=True) -client1.send('DROP TABLE test.lv') -client1.expect(prompt) -client1.send('DROP TABLE test.mt') -client1.expect(prompt) + ## send Ctrl-C + os.kill(client2.process.pid,signal.SIGINT) + + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) From 79989a766c54e6e316fa1cff39739d26f9021309 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Mon, 10 Jun 2019 19:56:12 -0400 Subject: [PATCH 067/509] * Fixing uexpect.py close() method and forcing to close the process by default * Updating tests to be more robust --- dbms/tests/integration/helpers/uexpect.py | 38 ++++++++++--------- .../0_stateless/00958_live_view_watch_live.py | 2 +- .../00960_live_view_watch_events_live.py | 2 +- .../00962_temporary_live_view_watch_live.py | 2 +- ..._temporary_live_view_watch_live_timeout.py | 2 +- .../00964_live_view_watch_events_heartbeat.py | 2 +- .../00965_live_view_watch_heartbeat.py | 2 +- .../00966_live_view_watch_events_http.py | 2 +- .../0_stateless/00967_live_view_watch_http.py | 2 +- ...0_live_view_watch_events_http_heartbeat.py | 2 +- .../00971_live_view_watch_http_heartbeat.py | 4 +- 11 files changed, 32 insertions(+), 28 deletions(-) diff --git a/dbms/tests/integration/helpers/uexpect.py b/dbms/tests/integration/helpers/uexpect.py index 58be8f60ff8..2a42b8a5de0 100644 --- a/dbms/tests/integration/helpers/uexpect.py +++ b/dbms/tests/integration/helpers/uexpect.py @@ -17,7 +17,7 @@ import time import sys import re -from threading import Thread +from threading import Thread, Event from subprocess import Popen from Queue import Queue, Empty @@ -61,7 +61,7 @@ class IO(object): def flush(self): self._logger.flush() - def __init__(self, process, master, queue): + def __init__(self, process, master, queue, reader): self.process = process self.master = master self.queue = queue @@ -70,6 +70,7 @@ class IO(object): self.after = None self.match = None self.pattern = None + self.reader = reader self._timeout = None self._logger = None self._eol = '' @@ -95,8 +96,12 @@ class IO(object): self._eol = eol return self._eol - def close(self): - self.process.terminate() + def close(self, force=True): + self.reader['kill_event'].set() + if force: + self.process.kill() + else: + self.process.terminate() os.close(self.master) if self._logger: self._logger.write('\n') @@ -133,9 +138,9 @@ class IO(object): raise exception timeleft -= (time.time() - start_time) if data: - self.buffer = self.buffer + data if self.buffer else data + self.buffer = (self.buffer + data) if self.buffer else data self.match = pattern.search(self.buffer, 0) - if self.match: + if self.match is not None: self.after = self.buffer[self.match.start():self.match.end()] self.before = self.buffer[:self.match.start()] self.buffer = self.buffer[self.match.end():] @@ -163,11 +168,9 @@ class IO(object): if data: return data if raise_exception: - print 'DEBUG...', timeleft, repr(data), raise TimeoutError(timeout) pass if not data and raise_exception: - print 'DEBUG.... here' raise TimeoutError(timeout) return data @@ -178,20 +181,21 @@ def spawn(command): os.close(slave) queue = Queue() - thread = Thread(target=reader, args=(process, master, queue)) - thread.daemon = True + reader_kill_event = Event() + thread = Thread(target=reader, args=(process, master, queue, reader_kill_event)) thread.start() - return IO(process, master, queue) + return IO(process, master, queue, reader={'thread':thread, 'kill_event':reader_kill_event}) -def reader(process, out, queue): +def reader(process, out, queue, kill_event): while True: - if process.poll() is not None: - data = os.read(out) + try: + data = os.read(out, 65536) queue.put(data) - break - data = os.read(out, 65536) - queue.put(data) + except OSError, e: + if e.errno == 5 and kill_event.is_set(): + break + raise if __name__ == '__main__': io = spawn(['/bin/bash','--noediting']) diff --git a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py index b60e82299f7..51bdca1a9b6 100755 --- a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py @@ -17,7 +17,7 @@ def client(name=''): return client prompt = ':\) ' -end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +end_of_block = r'.*\r\n.*\r\n' with client('client1>') as client1, client('client2>') as client2: client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py index 9fdabd499d0..0b441b72805 100755 --- a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py +++ b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py @@ -17,7 +17,7 @@ def client(name=''): return client prompt = ':\) ' -end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +end_of_block = r'.*\r\n.*\r\n' with client('client1>') as client1, client('client2>') as client2: client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py index 0e7c3c43254..5dfa6ac841e 100755 --- a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py @@ -17,7 +17,7 @@ def client(name=''): return client prompt = ':\) ' -end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +end_of_block = r'.*\r\n.*\r\n' with client('client1>') as client1, client('client2>') as client2: client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py index b3f6809ded0..09301aa6d59 100755 --- a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py +++ b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py @@ -17,7 +17,7 @@ def client(name=''): return client prompt = ':\) ' -end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +end_of_block = r'.*\r\n.*\r\n' with client('client1>') as client1, client('client2>') as client2: client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py index 1e66daef953..f3e5b1ee435 100755 --- a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -17,7 +17,7 @@ def client(name=''): return client prompt = ':\) ' -end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +end_of_block = r'.*\r\n.*\r\n' with client('client1>') as client1, client('client2>') as client2: client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py index 7f8ed6036ac..9f8a3c84af0 100755 --- a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py @@ -17,7 +17,7 @@ def client(name=''): return client prompt = ':\) ' -end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +end_of_block = r'.*\r\n.*\r\n' with client('client1>') as client1, client('client2>') as client2: client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py index e7dc54ee4c0..65999f4c55f 100755 --- a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py +++ b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py @@ -20,7 +20,7 @@ def client(name='', command=None): return client prompt = ':\) ' -end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +end_of_block = r'.*\r\n.*\r\n' with client('client1>') as client1, client('client2>', ['bash', '--noediting']) as client2: client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py index 6874ae376a3..a79d6882aa9 100755 --- a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py +++ b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py @@ -20,7 +20,7 @@ def client(name='', command=None): return client prompt = ':\) ' -end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +end_of_block = r'.*\r\n.*\r\n' with client('client1>') as client1, client('client2>', ['bash', '--noediting']) as client2: client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py index 4730caf5f57..95587ba9355 100755 --- a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py @@ -20,7 +20,7 @@ def client(name='', command=None): return client prompt = ':\) ' -end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +end_of_block = r'.*\r\n.*\r\n' with client('client1>') as client1, client('client2>', ['bash', '--noediting']) as client2: client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py index 5884bc1e2a5..8c40fc6d39d 100755 --- a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py @@ -20,7 +20,7 @@ def client(name='', command=None): return client prompt = ':\) ' -end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n' +end_of_block = r'.*\r\n.*\r\n' with client('client1>') as client1, client('client2>', ['bash', '--noediting']) as client2: client1.expect(prompt) @@ -49,7 +49,7 @@ with client('client1>') as client1, client('client2>', ['bash', '--noediting']) client2.expect('{"row":{"sum(a)":"6","_version":"2"}}\r\n', escape=True) ## send Ctrl-C - os.kill(client2.process.pid,signal.SIGINT) + os.kill(client2.process.pid, signal.SIGINT) client1.send('DROP TABLE test.lv') client1.expect(prompt) From a467e7c1054217d6b3751009cb03b850ae6e459c Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Mon, 10 Jun 2019 22:22:53 -0400 Subject: [PATCH 068/509] * Moving uexpect.py so that it can be found * Trying to fix failed style checks * Trying to fix clang build fail --- .../DataStreams/LiveViewEventsBlockInputStream.h | 15 +++++---------- dbms/src/Interpreters/InterpreterAlterQuery.cpp | 2 +- dbms/src/Parsers/ASTWatchQuery.h | 2 +- dbms/src/Storages/StorageLiveView.h | 2 +- .../0_stateless/00958_live_view_watch_live.py | 2 +- .../00960_live_view_watch_events_live.py | 2 +- .../00962_temporary_live_view_watch_live.py | 2 +- ...0963_temporary_live_view_watch_live_timeout.py | 2 +- .../00964_live_view_watch_events_heartbeat.py | 2 +- .../00965_live_view_watch_heartbeat.py | 2 +- .../00966_live_view_watch_events_http.py | 2 +- .../0_stateless/00967_live_view_watch_http.py | 2 +- ...00970_live_view_watch_events_http_heartbeat.py | 2 +- .../00971_live_view_watch_http_heartbeat.py | 2 +- .../0_stateless}/helpers/uexpect.py | 0 15 files changed, 18 insertions(+), 23 deletions(-) rename dbms/tests/{integration => queries/0_stateless}/helpers/uexpect.py (100%) diff --git a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h index 35d2545419b..be21c5c71f6 100644 --- a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h @@ -66,15 +66,10 @@ public: Block getHeader() const override { - return { - ColumnWithTypeAndName( - ColumnUInt64::create(), - std::make_shared(), - "version"), - ColumnWithTypeAndName( - ColumnString::create(), - std::make_shared(), - "hash") + return + { + ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "version"), + ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "hash") }; } @@ -93,7 +88,7 @@ public: { active = active_ptr.lock(); { - if (!blocks || blocks.get() != (*blocks_ptr).get()) + if (!blocks || (blocks.get() != (*blocks_ptr).get())) { blocks = (*blocks_ptr); blocks_metadata = (*blocks_metadata_ptr); diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index e8f3e4f4553..507b7a873d2 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -80,7 +80,7 @@ BlockIO InterpreterAlterQuery::execute() if (!live_view_commands.empty()) { live_view_commands.validate(*table); - for (const LiveViewCommand & command : live_view_commands) + for (const LiveViewCommand & command : live_view_commands) { auto live_view = std::dynamic_pointer_cast(table); switch (command.type) diff --git a/dbms/src/Parsers/ASTWatchQuery.h b/dbms/src/Parsers/ASTWatchQuery.h index 7e75d62a629..06d1460f038 100644 --- a/dbms/src/Parsers/ASTWatchQuery.h +++ b/dbms/src/Parsers/ASTWatchQuery.h @@ -25,7 +25,7 @@ public: bool is_watch_events; ASTWatchQuery() = default; - String getID(char) const override { return "WatchQuery_" + database + "_" + table; }; + String getID(char) const override { return "WatchQuery_" + database + "_" + table; } ASTPtr clone() const override { diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index fb250aec859..1114da7049d 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -28,7 +28,6 @@ namespace DB { class IAST; -using ASTPtr = std::shared_ptr; struct BlocksMetadata { @@ -36,6 +35,7 @@ struct BlocksMetadata UInt64 version; }; +using ASTPtr = std::shared_ptr; using BlocksMetadataPtr = std::shared_ptr; using SipHashPtr = std::shared_ptr; diff --git a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py index 51bdca1a9b6..96537d0d5c1 100755 --- a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) def client(name=''): client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py index 0b441b72805..a775854de9e 100755 --- a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py +++ b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) def client(name=''): client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py index 5dfa6ac841e..fb75134fc4c 100755 --- a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) def client(name=''): client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py index 09301aa6d59..4e12cd2234b 100755 --- a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py +++ b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) def client(name=''): client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py index f3e5b1ee435..c7b2b1a6906 100755 --- a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) def client(name=''): client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py index 9f8a3c84af0..b501c6e5889 100755 --- a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) def client(name=''): client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py index 65999f4c55f..4424bac3e54 100755 --- a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py +++ b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) def client(name='', command=None): if command is None: diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py index a79d6882aa9..3c7e8cdd489 100755 --- a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py +++ b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) def client(name='', command=None): if command is None: diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py index 95587ba9355..b7ab164c67a 100755 --- a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) def client(name='', command=None): if command is None: diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py index 8c40fc6d39d..2102b890132 100755 --- a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py @@ -6,7 +6,7 @@ import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, '..', '..', 'integration', 'helpers', 'uexpect.py')) +uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) def client(name='', command=None): if command is None: diff --git a/dbms/tests/integration/helpers/uexpect.py b/dbms/tests/queries/0_stateless/helpers/uexpect.py similarity index 100% rename from dbms/tests/integration/helpers/uexpect.py rename to dbms/tests/queries/0_stateless/helpers/uexpect.py From b6efd8599fee992e1f54337fe376918f123b3520 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Tue, 11 Jun 2019 06:30:57 -0400 Subject: [PATCH 069/509] * Trying again to fix style checks. Now running ./utils/check-style/check-style reports no errors. --- dbms/src/DataStreams/LiveViewEventsBlockInputStream.h | 11 ++++------- dbms/src/Storages/StorageLiveView.h | 2 +- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h index be21c5c71f6..7841550b5cb 100644 --- a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h @@ -64,13 +64,10 @@ public: condition.broadcast(); } - Block getHeader() const override + Block getHeader() const override { - return - { - ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "version"), - ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "hash") - }; + Block header(ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "version"), ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "hash")); + return header; } void refresh() @@ -88,7 +85,7 @@ public: { active = active_ptr.lock(); { - if (!blocks || (blocks.get() != (*blocks_ptr).get())) + if (!blocks || blocks.get() != (*blocks_ptr).get()) { blocks = (*blocks_ptr); blocks_metadata = (*blocks_metadata_ptr); diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index 1114da7049d..6899067c9f9 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -29,7 +29,7 @@ namespace DB class IAST; -struct BlocksMetadata +struct BlocksMetadata { String hash; UInt64 version; From 12060d887973daea23821c8ab8bc8d76f48549b6 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Tue, 11 Jun 2019 08:27:47 -0400 Subject: [PATCH 070/509] * Fixing errors that prevented clang build. --- dbms/src/DataStreams/LiveViewBlockInputStream.h | 8 ++++---- dbms/src/DataStreams/LiveViewEventsBlockInputStream.h | 11 +++++------ dbms/src/Parsers/ParserCreateQuery.cpp | 1 - dbms/src/Storages/StorageLiveView.cpp | 3 +-- dbms/src/Storages/StorageLiveView.h | 2 -- 5 files changed, 10 insertions(+), 15 deletions(-) diff --git a/dbms/src/DataStreams/LiveViewBlockInputStream.h b/dbms/src/DataStreams/LiveViewBlockInputStream.h index 6129c836597..9de503fd6f7 100644 --- a/dbms/src/DataStreams/LiveViewBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewBlockInputStream.h @@ -160,8 +160,8 @@ protected: } while (true) { - UInt64 timestamp_usec = (UInt64)timestamp.epochMicroseconds(); - bool signaled = condition.tryWait(mutex, std::max((UInt64)0, heartbeat_interval - (timestamp_usec - last_event_timestamp)) / 1000); + UInt64 timestamp_usec = static_cast(timestamp.epochMicroseconds()); + bool signaled = condition.tryWait(mutex, std::max(static_cast(0), heartbeat_interval - (timestamp_usec - last_event_timestamp)) / 1000); if (isCancelled() || storage.is_dropped) { @@ -174,7 +174,7 @@ protected: else { // heartbeat - last_event_timestamp = (UInt64)timestamp.epochMicroseconds(); + last_event_timestamp = static_cast(timestamp.epochMicroseconds()); return { getHeader(), true }; } } @@ -194,7 +194,7 @@ protected: --length; } - last_event_timestamp = (UInt64)timestamp.epochMicroseconds(); + last_event_timestamp = static_cast(timestamp.epochMicroseconds()); return { res, true }; } diff --git a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h index 7841550b5cb..215d5a17380 100644 --- a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h @@ -66,8 +66,7 @@ public: Block getHeader() const override { - Block header(ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "version"), ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "hash")); - return header; + return {ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "version"), ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "hash")}; } void refresh() @@ -182,8 +181,8 @@ protected: } while (true) { - UInt64 timestamp_usec = (UInt64)timestamp.epochMicroseconds(); - bool signaled = condition.tryWait(mutex, std::max((UInt64)0, heartbeat_interval - (timestamp_usec - last_event_timestamp)) / 1000); + UInt64 timestamp_usec = static_cast(timestamp.epochMicroseconds()); + bool signaled = condition.tryWait(mutex, std::max(static_cast(0), heartbeat_interval - (timestamp_usec - last_event_timestamp)) / 1000); if (isCancelled() || storage.is_dropped) { @@ -196,7 +195,7 @@ protected: else { // repeat the event block as a heartbeat - last_event_timestamp = (UInt64)timestamp.epochMicroseconds(); + last_event_timestamp = static_cast(timestamp.epochMicroseconds()); return { getHeader(), true }; } } @@ -215,7 +214,7 @@ protected: --length; } - last_event_timestamp = (UInt64)timestamp.epochMicroseconds(); + last_event_timestamp = static_cast(timestamp.epochMicroseconds()); return { getEventBlock(), true }; } diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 4064dde5213..c8341f81770 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -509,7 +509,6 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!s_as.ignore(pos, expected)) return false; - ParserSelectWithUnionQuery select_p; if (!select_p.parse(pos, select, expected)) return false; } diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index dd6e0179410..28e6717611f 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -172,7 +172,6 @@ Block StorageLiveView::getHeader() const bool StorageLiveView::getNewBlocks() { - Block block; SipHash hash; UInt128 key; BlocksPtr new_blocks = std::make_shared(); @@ -398,7 +397,7 @@ BlockInputStreams StorageLiveView::watch( int64_t length = -2; if (query.limit_length) - length = (int64_t)safeGet(typeid_cast(*query.limit_length).value); + length = static_cast(safeGet(typeid_cast(*query.limit_length).value)); if (query.is_watch_events) { diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index 6899067c9f9..01dd82ff1fd 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -257,10 +257,8 @@ private: std::shared_ptr active_ptr; /// Current data blocks that store query result std::shared_ptr blocks_ptr; - BlocksPtr new_blocks; /// Current data blocks metadata std::shared_ptr blocks_metadata_ptr; - BlocksMetadataPtr new_blocks_metadata; BlocksPtrs mergeable_blocks; void noUsersThread(const UInt64 & timeout); From 536bdb588da53bc90f89933783dc94f04e1c307a Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Tue, 11 Jun 2019 21:28:20 -0400 Subject: [PATCH 071/509] * Fixing and reorganazing LIVE VIEW tests --- .../0_stateless/00958_live_view_watch_live.py | 20 ++--- .../00960_live_view_watch_events_live.py | 20 ++--- .../00962_temporary_live_view_watch_live.py | 20 ++--- ..._temporary_live_view_watch_live_timeout.py | 20 ++--- .../00964_live_view_watch_events_heartbeat.py | 19 ++--- .../00965_live_view_watch_heartbeat.py | 20 ++--- .../00966_live_view_watch_events_http.py | 38 +++------- .../0_stateless/00967_live_view_watch_http.py | 38 +++------- ...0_live_view_watch_events_http_heartbeat.py | 45 ++++-------- .../00971_live_view_watch_http_heartbeat.py | 48 +++++------- .../queries/0_stateless/helpers/client.py | 21 ++++++ .../queries/0_stateless/helpers/httpclient.py | 14 ++++ .../queries/0_stateless/helpers/httpexpect.py | 73 +++++++++++++++++++ .../queries/0_stateless/helpers/uexpect.py | 30 +++++--- 14 files changed, 224 insertions(+), 202 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/helpers/client.py create mode 100644 dbms/tests/queries/0_stateless/helpers/httpclient.py create mode 100644 dbms/tests/queries/0_stateless/helpers/httpexpect.py diff --git a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py index 96537d0d5c1..cb6f1e95f7e 100755 --- a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py @@ -1,25 +1,18 @@ #!/usr/bin/env python -import imp import os import sys import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) +from client import client, prompt, end_of_block -def client(name=''): - client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) - client.eol('\r') - # Note: uncomment this line for debugging - #client.logger(sys.stdout, prefix=name) - client.timeout(2) - return client +log = None +# uncomment the line below for debugging +#log=sys.stdout -prompt = ':\) ' -end_of_block = r'.*\r\n.*\r\n' - -with client('client1>') as client1, client('client2>') as client2: +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: client1.expect(prompt) client2.expect(prompt) @@ -32,6 +25,7 @@ with client('client1>') as client1, client('client2>') as client2: client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) client1.send('WATCH test.lv') + client1.expect(r'0.*1' + end_of_block) client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') client1.expect(r'6.*2' + end_of_block) client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py index a775854de9e..414f9c1ad96 100755 --- a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py +++ b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py @@ -1,25 +1,18 @@ #!/usr/bin/env python -import imp import os import sys import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) +from client import client, prompt, end_of_block -def client(name=''): - client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) - client.eol('\r') - # Note: uncomment this line for debugging - #client.logger(sys.stdout, prefix=name) - client.timeout(2) - return client +log = None +# uncomment the line below for debugging +#log=sys.stdout -prompt = ':\) ' -end_of_block = r'.*\r\n.*\r\n' - -with client('client1>') as client1, client('client2>') as client2: +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: client1.expect(prompt) client2.expect(prompt) @@ -32,6 +25,7 @@ with client('client1>') as client1, client('client2>') as client2: client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) client1.send('WATCH test.lv EVENTS') + client1.expect('1.*' + end_of_block) client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') client1.expect('2.*4cd0592103888d4682de9a32a23602e3' + end_of_block) client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py index fb75134fc4c..48311d890c5 100755 --- a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py @@ -1,25 +1,18 @@ #!/usr/bin/env python -import imp import os import sys import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) +from client import client, prompt, end_of_block -def client(name=''): - client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) - client.eol('\r') - # Note: uncomment this line for debugging - #client.logger(sys.stdout, prefix=name) - client.timeout(2) - return client +log = None +# uncomment the line below for debugging +#log=sys.stdout -prompt = ':\) ' -end_of_block = r'.*\r\n.*\r\n' - -with client('client1>') as client1, client('client2>') as client2: +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: client1.expect(prompt) client2.expect(prompt) @@ -32,6 +25,7 @@ with client('client1>') as client1, client('client2>') as client2: client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) client1.send('WATCH test.lv') + client1.expect(r'0.*1' + end_of_block) client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') client1.expect(r'6.*2' + end_of_block) client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py index 4e12cd2234b..581363cd796 100755 --- a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py +++ b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py @@ -1,25 +1,18 @@ #!/usr/bin/env python -import imp import os import sys import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) +from client import client, prompt, end_of_block -def client(name=''): - client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) - client.eol('\r') - # Note: uncomment this line for debugging - #client.logger(sys.stdout, prefix=name) - client.timeout(2) - return client +log = None +# uncomment the line below for debugging +#log=sys.stdout -prompt = ':\) ' -end_of_block = r'.*\r\n.*\r\n' - -with client('client1>') as client1, client('client2>') as client2: +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: client1.expect(prompt) client2.expect(prompt) @@ -34,6 +27,7 @@ with client('client1>') as client1, client('client2>') as client2: client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) client1.send('WATCH test.lv') + client1.expect(r'0.*1' + end_of_block) client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') client2.expect(prompt) client1.expect(r'6.*2' + end_of_block) diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py index c7b2b1a6906..b624e0b7080 100755 --- a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -1,25 +1,18 @@ #!/usr/bin/env python -import imp import os import sys import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) +from client import client, prompt, end_of_block -def client(name=''): - client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) - client.eol('\r') - # Note: uncomment this line for debugging - #client.logger(sys.stdout, prefix=name) - client.timeout(2) - return client +log = None +# uncomment the line below for debugging +#log=sys.stdout -prompt = ':\) ' -end_of_block = r'.*\r\n.*\r\n' - -with client('client1>') as client1, client('client2>') as client2: +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: client1.expect(prompt) client2.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py index b501c6e5889..dfb46273f7c 100755 --- a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py @@ -1,25 +1,18 @@ #!/usr/bin/env python -import imp import os import sys import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) +from client import client, prompt, end_of_block -def client(name=''): - client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) - client.eol('\r') - # Note: uncomment this line for debugging - #client.logger(sys.stdout, prefix=name) - client.timeout(2) - return client +log = None +# uncomment the line below for debugging +#log=sys.stdout -prompt = ':\) ' -end_of_block = r'.*\r\n.*\r\n' - -with client('client1>') as client1, client('client2>') as client2: +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: client1.expect(prompt) client2.expect(prompt) @@ -34,6 +27,7 @@ with client('client1>') as client1, client('client2>') as client2: client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) client1.send('WATCH test.lv') + client1.expect(r'0.*1' + end_of_block) client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') client1.expect(r'6.*2' + end_of_block) client1.expect('Progress: 2.00 rows.*\)') diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py index 4424bac3e54..a1b6f2418ea 100755 --- a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py +++ b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py @@ -1,28 +1,18 @@ #!/usr/bin/env python -import imp import os import sys -import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) +from client import client, prompt, end_of_block +from httpclient import client as http_client -def client(name='', command=None): - if command is None: - client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) - else: - client = uexpect.spawn(command) - client.eol('\r') - # Note: uncomment this line for debugging - #client.logger(sys.stdout, prefix=name) - client.timeout(2) - return client +log = None +# uncomment the line below for debugging +#log=sys.stdout -prompt = ':\) ' -end_of_block = r'.*\r\n.*\r\n' - -with client('client1>') as client1, client('client2>', ['bash', '--noediting']) as client2: +with client(name='client1>', log=log) as client1: client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.lv') @@ -34,16 +24,12 @@ with client('client1>') as client1, client('client2>', ['bash', '--noediting']) client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) - client2.expect('[\$#] ') - client2.send('wget -O- -q "http://localhost:8123/?query=WATCH test.lv EVENTS"') - client2.expect('.*1\tc9d39b11cce79112219a73aaa319b475\r\n') - client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect(prompt) - - client2.expect('.*2\t.*\r\n') - ## send Ctrl-C - os.kill(client2.process.pid,signal.SIGINT) + with http_client({'method':'GET', 'url': '/?query=WATCH%20test.lv%20EVENTS'}, name='client2>', log=log) as client2: + client2.expect('.*1\tc9d39b11cce79112219a73aaa319b475\n') + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) + client2.expect('.*2\t.*\n') client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py index 3c7e8cdd489..d3439431eb3 100755 --- a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py +++ b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py @@ -1,28 +1,18 @@ #!/usr/bin/env python -import imp import os import sys -import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) +from client import client, prompt, end_of_block +from httpclient import client as http_client -def client(name='', command=None): - if command is None: - client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) - else: - client = uexpect.spawn(command) - client.eol('\r') - # Note: uncomment this line for debugging - #client.logger(sys.stdout, prefix=name) - client.timeout(2) - return client +log = None +# uncomment the line below for debugging +#log=sys.stdout -prompt = ':\) ' -end_of_block = r'.*\r\n.*\r\n' - -with client('client1>') as client1, client('client2>', ['bash', '--noediting']) as client2: +with client(name='client1>', log=log) as client1: client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.lv') @@ -34,16 +24,12 @@ with client('client1>') as client1, client('client2>', ['bash', '--noediting']) client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) - client2.expect('[\$#] ') - client2.send('wget -O- -q "http://localhost:8123/?query=WATCH test.lv"') - client2.expect('.*0\t1\r\n') - client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect(prompt) - - client2.expect('.*6\t2\r\n') - ## send Ctrl-C - os.kill(client2.process.pid,signal.SIGINT) + with http_client({'method':'GET', 'url':'/?query=WATCH%20test.lv'}, name='client2>', log=log) as client2: + client2.expect('.*0\t1\n') + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) + client2.expect('.*6\t2\n') client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py index b7ab164c67a..29ea2142d5c 100755 --- a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py @@ -1,28 +1,18 @@ #!/usr/bin/env python -import imp import os import sys -import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) +from client import client, prompt, end_of_block +from httpclient import client as http_client -def client(name='', command=None): - if command is None: - client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) - else: - client = uexpect.spawn(command) - client.eol('\r') - # Note: uncomment this line for debugging - #client.logger(sys.stdout, prefix=name) - client.timeout(2) - return client +log = None +# uncomment the line below for debugging +#log=sys.stdout -prompt = ':\) ' -end_of_block = r'.*\r\n.*\r\n' - -with client('client1>') as client1, client('client2>', ['bash', '--noediting']) as client2: +with client(name='client1>', log=log) as client1: client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.lv') @@ -34,21 +24,18 @@ with client('client1>') as client1, client('client2>', ['bash', '--noediting']) client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) - client2.expect('[\$#] ') - client2.send('wget -O- -q "http://localhost:8123/?live_view_heartbeat_interval=1&query=WATCH test.lv EVENTS FORMAT JSONEachRowWithProgress"') - client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}\r\n', escape=True) - client2.expect('{"row":{"version":"1","hash":"c9d39b11cce79112219a73aaa319b475"}}', escape=True) - client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) - # heartbeat is provided by progress message - client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) - client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect(prompt) + with http_client({'method':'GET', 'url': '/?live_view_heartbeat_interval=1&query=WATCH%20test.lv%20EVENTS%20FORMAT%20JSONEachRowWithProgress'}, name='client2>', log=log) as client2: + client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}\n', escape=True) + client2.expect('{"row":{"version":"1","hash":"c9d39b11cce79112219a73aaa319b475"}}', escape=True) + client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) + # heartbeat is provided by progress message + client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) - client2.expect('{"row":{"version":"2","hash":"4cd0592103888d4682de9a32a23602e3"}}\r\n', escape=True) + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) - ## send Ctrl-C - os.kill(client2.process.pid,signal.SIGINT) + client2.expect('{"row":{"version":"2","hash":"4cd0592103888d4682de9a32a23602e3"}}\n', escape=True) client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py index 2102b890132..7bdb47b7caa 100755 --- a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py @@ -1,28 +1,18 @@ #!/usr/bin/env python -import imp import os import sys -import signal CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) -uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'helpers', 'uexpect.py')) +from client import client, prompt, end_of_block +from httpclient import client as http_client -def client(name='', command=None): - if command is None: - client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) - else: - client = uexpect.spawn(command) - client.eol('\r') - # Note: uncomment this line for debugging - #client.logger(sys.stdout, prefix=name) - client.timeout(2) - return client +log = None +# uncomment the line below for debugging +#log=sys.stdout -prompt = ':\) ' -end_of_block = r'.*\r\n.*\r\n' - -with client('client1>') as client1, client('client2>', ['bash', '--noediting']) as client2: +with client(name='client1>', log=log) as client1: client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.lv') @@ -34,22 +24,18 @@ with client('client1>') as client1, client('client2>', ['bash', '--noediting']) client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) - client2.expect('[\$#] ') - client2.send('wget -O- -q "http://localhost:8123/?live_view_heartbeat_interval=1&query=WATCH test.lv FORMAT JSONEachRowWithProgress"') - client2.expect('"progress".*',) - client2.expect('{"row":{"sum(a)":"0","_version":"1"}}\r\n', escape=True) - client2.expect('"progress".*\r\n') - # heartbeat is provided by progress message - client2.expect('"progress".*\r\n') + with http_client({'method':'GET', 'url':'/?live_view_heartbeat_interval=1&query=WATCH%20test.lv%20FORMAT%20JSONEachRowWithProgress'}, name='client2>', log=log) as client2: + client2.expect('"progress".*',) + client2.expect('{"row":{"sum(a)":"0","_version":"1"}}\n', escape=True) + client2.expect('"progress".*\n') + # heartbeat is provided by progress message + client2.expect('"progress".*\n') - client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect(prompt) + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) - client2.expect('"progress".*"read_rows":"2".*\r\n') - client2.expect('{"row":{"sum(a)":"6","_version":"2"}}\r\n', escape=True) - - ## send Ctrl-C - os.kill(client2.process.pid, signal.SIGINT) + client2.expect('"progress".*"read_rows":"2".*\n') + client2.expect('{"row":{"sum(a)":"6","_version":"2"}}\n', escape=True) client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/helpers/client.py b/dbms/tests/queries/0_stateless/helpers/client.py new file mode 100644 index 00000000000..b509a321cee --- /dev/null +++ b/dbms/tests/queries/0_stateless/helpers/client.py @@ -0,0 +1,21 @@ +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +sys.path.insert(0, os.path.join(CURDIR)) + +import uexpect + +prompt = ':\) ' +end_of_block = r'.*\r\n.*\r\n' + +def client(command=None, name='', log=None): + if command is None: + client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) + else: + client = uexpect.spawn(command) + client.eol('\r') + client.logger(log, prefix=name) + client.timeout(2) + return client diff --git a/dbms/tests/queries/0_stateless/helpers/httpclient.py b/dbms/tests/queries/0_stateless/helpers/httpclient.py new file mode 100644 index 00000000000..111841ba708 --- /dev/null +++ b/dbms/tests/queries/0_stateless/helpers/httpclient.py @@ -0,0 +1,14 @@ +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +sys.path.insert(0, os.path.join(CURDIR)) + +import httpexpect + +def client(request, name='', log=None): + client = httpexpect.spawn({'host':'localhost','port':8123}, request) + client.logger(log, prefix=name) + client.timeout(2) + return client diff --git a/dbms/tests/queries/0_stateless/helpers/httpexpect.py b/dbms/tests/queries/0_stateless/helpers/httpexpect.py new file mode 100644 index 00000000000..e440dafce4e --- /dev/null +++ b/dbms/tests/queries/0_stateless/helpers/httpexpect.py @@ -0,0 +1,73 @@ +# Copyright (c) 2019 Vitaliy Zakaznikov +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +import os +import sys +import httplib + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, CURDIR) + +import uexpect + +from threading import Thread, Event +from Queue import Queue, Empty + +class IO(uexpect.IO): + def __init__(self, connection, response, queue, reader): + self.connection = connection + self.response = response + super(IO, self).__init__(None, None, queue, reader) + + def write(self, data): + raise NotImplementedError + + def close(self, force=True): + self.reader['kill_event'].set() + self.connection.close() + if self._logger: + self._logger.write('\n') + self._logger.flush() + + +def reader(response, queue, kill_event): + while True: + try: + if kill_event.is_set(): + break + data = response.read(1) + queue.put(data) + except Exception, e: + if kill_event.is_set(): + break + raise + +def spawn(connection, request): + connection = httplib.HTTPConnection(**connection) + connection.request(**request) + response = connection.getresponse() + + queue = Queue() + reader_kill_event = Event() + thread = Thread(target=reader, args=(response, queue, reader_kill_event)) + thread.daemon = True + thread.start() + + return IO(connection, response, queue, reader={'thread':thread, 'kill_event':reader_kill_event}) + +if __name__ == '__main__': + with http({'host':'localhost','port':8123},{'method':'GET', 'url':'?query=SELECT%201'}) as client: + client.logger(sys.stdout) + client.timeout(2) + print client.response.status, client.response.reason + client.expect('1\n') diff --git a/dbms/tests/queries/0_stateless/helpers/uexpect.py b/dbms/tests/queries/0_stateless/helpers/uexpect.py index 2a42b8a5de0..2f323cf6ca6 100644 --- a/dbms/tests/queries/0_stateless/helpers/uexpect.py +++ b/dbms/tests/queries/0_stateless/helpers/uexpect.py @@ -35,10 +35,13 @@ class ExpectTimeoutError(Exception): self.buffer = buffer def __str__(self): - return ('Timeout %.3fs ' % float(self.timeout) + - 'for %s ' % repr(self.pattern.pattern) + - 'buffer %s ' % repr(self.buffer[:]) + - 'or \'%s\'' % ','.join(['%x' % ord(c) for c in self.buffer[:]])) + s = 'Timeout %.3fs ' % float(self.timeout) + if self.pattern: + s += 'for %s ' % repr(self.pattern.pattern) + if self.buffer: + s += 'buffer %s ' % repr(self.buffer[:]) + s += 'or \'%s\'' % ','.join(['%x' % ord(c) for c in self.buffer[:]]) + return s class IO(object): class EOF(object): @@ -125,13 +128,22 @@ class IO(object): if timeout is None: timeout = self._timeout timeleft = timeout - while timeleft >= 0: + while True: start_time = time.time() + if self.buffer is not None: + self.match = pattern.search(self.buffer, 0) + if self.match is not None: + self.after = self.buffer[self.match.start():self.match.end()] + self.before = self.buffer[:self.match.start()] + self.buffer = self.buffer[self.match.end():] + break + if timeleft < 0: + break try: data = self.read(timeout=timeleft, raise_exception=True) except TimeoutError: if self._logger: - self._logger.write(self.buffer + '\n') + self._logger.write((self.buffer or '') + '\n') self._logger.flush() exception = ExpectTimeoutError(pattern, timeout, self.buffer) self.buffer = None @@ -139,12 +151,6 @@ class IO(object): timeleft -= (time.time() - start_time) if data: self.buffer = (self.buffer + data) if self.buffer else data - self.match = pattern.search(self.buffer, 0) - if self.match is not None: - self.after = self.buffer[self.match.start():self.match.end()] - self.before = self.buffer[:self.match.start()] - self.buffer = self.buffer[self.match.end():] - break if self._logger: self._logger.write((self.before or '') + (self.after or '')) self._logger.flush() From 5983273694c45c76da6619b0b732d52e02db8bd7 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Wed, 12 Jun 2019 09:11:44 -0400 Subject: [PATCH 072/509] Fixing bugs reported by sanitizers * data race condition on storage is_dropped access * invalid reference to storage after it is deleted * invalid reference to mutex after storage is deleted --- .../DataStreams/LiveViewBlockInputStream.h | 33 +++++++++---------- .../LiveViewEventsBlockInputStream.h | 28 ++++++++-------- dbms/src/Storages/IStorage.h | 2 +- dbms/src/Storages/StorageLiveView.cpp | 4 +-- 4 files changed, 32 insertions(+), 35 deletions(-) diff --git a/dbms/src/DataStreams/LiveViewBlockInputStream.h b/dbms/src/DataStreams/LiveViewBlockInputStream.h index 9de503fd6f7..e9bb599a1d6 100644 --- a/dbms/src/DataStreams/LiveViewBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewBlockInputStream.h @@ -36,17 +36,18 @@ public: { /// Start storage no users thread /// if we are the last active user - if (!storage.is_dropped && blocks_ptr.use_count() < 3) - storage.startNoUsersThread(temporary_live_view_timeout); + if (!storage->is_dropped && blocks_ptr.use_count() < 3) + storage->startNoUsersThread(temporary_live_view_timeout); } /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update /// and LIMIT 0 just returns data without waiting for any updates - LiveViewBlockInputStream(StorageLiveView & storage_, std::shared_ptr blocks_ptr_, + LiveViewBlockInputStream(std::shared_ptr storage_, + std::shared_ptr blocks_ptr_, std::shared_ptr blocks_metadata_ptr_, - std::shared_ptr active_ptr_, Poco::Condition & condition_, Poco::FastMutex & mutex_, + std::shared_ptr active_ptr_, int64_t length_, const UInt64 & heartbeat_interval_, const UInt64 & temporary_live_view_timeout_) - : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_interval(heartbeat_interval_ * 1000000), temporary_live_view_timeout(temporary_live_view_timeout_), + : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), length(length_ + 1), heartbeat_interval(heartbeat_interval_ * 1000000), temporary_live_view_timeout(temporary_live_view_timeout_), blocks_hash("") { /// grab active pointer @@ -57,14 +58,14 @@ public: void cancel(bool kill) override { - if (isCancelled() || storage.is_dropped) + if (isCancelled() || storage->is_dropped) return; IBlockInputStream::cancel(kill); - Poco::FastMutex::ScopedLock lock(mutex); - condition.broadcast(); + Poco::FastMutex::ScopedLock lock(storage->mutex); + storage->condition.broadcast(); } - Block getHeader() const override { return storage.getHeader(); } + Block getHeader() const override { return storage->getHeader(); } void refresh() { @@ -117,7 +118,7 @@ protected: /// If blocks were never assigned get blocks if (!blocks) { - Poco::FastMutex::ScopedLock lock(mutex); + Poco::FastMutex::ScopedLock lock(storage->mutex); if (!active) return { Block(), false }; blocks = (*blocks_ptr); @@ -126,7 +127,7 @@ protected: end = blocks->end(); } - if (isCancelled() || storage.is_dropped) + if (isCancelled() || storage->is_dropped) { return { Block(), true }; } @@ -134,7 +135,7 @@ protected: if (it == end) { { - Poco::FastMutex::ScopedLock lock(mutex); + Poco::FastMutex::ScopedLock lock(storage->mutex); if (!active) return { Block(), false }; /// If we are done iterating over our blocks @@ -161,9 +162,9 @@ protected: while (true) { UInt64 timestamp_usec = static_cast(timestamp.epochMicroseconds()); - bool signaled = condition.tryWait(mutex, std::max(static_cast(0), heartbeat_interval - (timestamp_usec - last_event_timestamp)) / 1000); + bool signaled = storage->condition.tryWait(storage->mutex, std::max(static_cast(0), heartbeat_interval - (timestamp_usec - last_event_timestamp)) / 1000); - if (isCancelled() || storage.is_dropped) + if (isCancelled() || storage->is_dropped) { return { Block(), true }; } @@ -199,7 +200,7 @@ protected: } private: - StorageLiveView & storage; + std::shared_ptr storage; std::shared_ptr blocks_ptr; std::shared_ptr blocks_metadata_ptr; std::weak_ptr active_ptr; @@ -209,8 +210,6 @@ private: Blocks::iterator it; Blocks::iterator end; Blocks::iterator begin; - Poco::Condition & condition; - Poco::FastMutex & mutex; /// Length specifies number of updates to send, default -1 (no limit) int64_t length; bool end_of_blocks{0}; diff --git a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h index 215d5a17380..c7911074176 100644 --- a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h @@ -40,14 +40,14 @@ public: { /// Start storage no users thread /// if we are the last active user - if (!storage.is_dropped && blocks_ptr.use_count() < 3) - storage.startNoUsersThread(temporary_live_view_timeout); + if (!storage->is_dropped && blocks_ptr.use_count() < 3) + storage->startNoUsersThread(temporary_live_view_timeout); } /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update /// and LIMIT 0 just returns data without waiting for any updates - LiveViewEventsBlockInputStream(StorageLiveView & storage_, std::shared_ptr blocks_ptr_, std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, Poco::Condition & condition_, Poco::FastMutex & mutex_, + LiveViewEventsBlockInputStream(std::shared_ptr storage_, std::shared_ptr blocks_ptr_, std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, int64_t length_, const UInt64 & heartbeat_interval_, const UInt64 & temporary_live_view_timeout_) - : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), condition(condition_), mutex(mutex_), length(length_ + 1), heartbeat_interval(heartbeat_interval_ * 1000000), temporary_live_view_timeout(temporary_live_view_timeout_) + : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), length(length_ + 1), heartbeat_interval(heartbeat_interval_ * 1000000), temporary_live_view_timeout(temporary_live_view_timeout_) { /// grab active pointer active = active_ptr.lock(); @@ -57,11 +57,11 @@ public: void cancel(bool kill) override { - if (isCancelled() || storage.is_dropped) + if (isCancelled() || storage->is_dropped) return; IBlockInputStream::cancel(kill); - Poco::FastMutex::ScopedLock lock(mutex); - condition.broadcast(); + Poco::FastMutex::ScopedLock lock(storage->mutex); + storage->condition.broadcast(); } Block getHeader() const override @@ -136,7 +136,7 @@ protected: /// If blocks were never assigned get blocks if (!blocks) { - Poco::FastMutex::ScopedLock lock(mutex); + Poco::FastMutex::ScopedLock lock(storage->mutex); if (!active) return { Block(), false }; blocks = (*blocks_ptr); @@ -146,7 +146,7 @@ protected: end = blocks->end(); } - if (isCancelled() || storage.is_dropped) + if (isCancelled() || storage->is_dropped) { return { Block(), true }; } @@ -154,7 +154,7 @@ protected: if (it == end) { { - Poco::FastMutex::ScopedLock lock(mutex); + Poco::FastMutex::ScopedLock lock(storage->mutex); if (!active) return { Block(), false }; /// If we are done iterating over our blocks @@ -182,9 +182,9 @@ protected: while (true) { UInt64 timestamp_usec = static_cast(timestamp.epochMicroseconds()); - bool signaled = condition.tryWait(mutex, std::max(static_cast(0), heartbeat_interval - (timestamp_usec - last_event_timestamp)) / 1000); + bool signaled = storage->condition.tryWait(storage->mutex, std::max(static_cast(0), heartbeat_interval - (timestamp_usec - last_event_timestamp)) / 1000); - if (isCancelled() || storage.is_dropped) + if (isCancelled() || storage->is_dropped) { return { Block(), true }; } @@ -220,7 +220,7 @@ protected: } private: - StorageLiveView & storage; + std::shared_ptr storage; std::shared_ptr blocks_ptr; std::shared_ptr blocks_metadata_ptr; std::weak_ptr active_ptr; @@ -230,8 +230,6 @@ private: Blocks::iterator it; Blocks::iterator end; Blocks::iterator begin; - Poco::Condition & condition; - Poco::FastMutex & mutex; /// Length specifies number of updates to send, default -1 (no limit) int64_t length; bool end_of_blocks{0}; diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 4a63c1ef34a..9fcd8e5ddcf 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -298,7 +298,7 @@ public: return {}; } - bool is_dropped{false}; + std::atomic is_dropped{false}; /// Does table support index for IN sections virtual bool supportsIndexForIn() const { return false; } diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index 28e6717611f..9a129141a1b 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -401,7 +401,7 @@ BlockInputStreams StorageLiveView::watch( if (query.is_watch_events) { - auto reader = std::make_shared(*this, blocks_ptr, blocks_metadata_ptr, active_ptr, condition, mutex, length, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), + auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, length, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), context.getSettingsRef().temporary_live_view_timeout.totalSeconds()); if (no_users_thread.joinable()) @@ -426,7 +426,7 @@ BlockInputStreams StorageLiveView::watch( } else { - auto reader = std::make_shared(*this, blocks_ptr, blocks_metadata_ptr, active_ptr, condition, mutex, length, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), + auto reader = std::make_shared(std::static_pointer_cast( shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, length, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), context.getSettingsRef().temporary_live_view_timeout.totalSeconds()); if (no_users_thread.joinable()) From 576393272bc5d67983ca55bdd7f169598a7db654 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Wed, 12 Jun 2019 20:45:41 -0400 Subject: [PATCH 073/509] * Fixing styling check error --- dbms/src/Storages/StorageLiveView.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index 9a129141a1b..77e95ac2e48 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -426,7 +426,7 @@ BlockInputStreams StorageLiveView::watch( } else { - auto reader = std::make_shared(std::static_pointer_cast( shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, length, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), + auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, length, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), context.getSettingsRef().temporary_live_view_timeout.totalSeconds()); if (no_users_thread.joinable()) From 825c21f29a38e33debe9c931e6435cef84dba2c3 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 13 Jun 2019 06:36:59 -0400 Subject: [PATCH 074/509] * Increasing timeout from 2 to 20 sec. * Style fix --- .../queries/0_stateless/00962_temporary_live_view_watch_live.py | 2 +- dbms/tests/queries/0_stateless/helpers/client.py | 2 +- dbms/tests/queries/0_stateless/helpers/httpclient.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py index 48311d890c5..1967284c38b 100755 --- a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py @@ -18,7 +18,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.mt') + client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/helpers/client.py b/dbms/tests/queries/0_stateless/helpers/client.py index b509a321cee..59ea3d898ea 100644 --- a/dbms/tests/queries/0_stateless/helpers/client.py +++ b/dbms/tests/queries/0_stateless/helpers/client.py @@ -17,5 +17,5 @@ def client(command=None, name='', log=None): client = uexpect.spawn(command) client.eol('\r') client.logger(log, prefix=name) - client.timeout(2) + client.timeout(20) return client diff --git a/dbms/tests/queries/0_stateless/helpers/httpclient.py b/dbms/tests/queries/0_stateless/helpers/httpclient.py index 111841ba708..a42fad2cbc3 100644 --- a/dbms/tests/queries/0_stateless/helpers/httpclient.py +++ b/dbms/tests/queries/0_stateless/helpers/httpclient.py @@ -10,5 +10,5 @@ import httpexpect def client(request, name='', log=None): client = httpexpect.spawn({'host':'localhost','port':8123}, request) client.logger(log, prefix=name) - client.timeout(2) + client.timeout(20) return client From b82bb4a954391398fab7e60b86e536a5d215c350 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 13 Jun 2019 12:37:29 -0400 Subject: [PATCH 075/509] * Updates to sync with yandex/master --- contrib/googletest | 1 + .../boost/archive/archive_exception.hpp | 100 - .../boost/archive/basic_archive.hpp | 304 --- .../boost/archive/basic_binary_iarchive.hpp | 204 -- .../boost/archive/basic_binary_iprimitive.hpp | 198 -- .../boost/archive/basic_binary_oarchive.hpp | 185 -- .../boost/archive/basic_binary_oprimitive.hpp | 188 -- .../archive/basic_streambuf_locale_saver.hpp | 108 -- .../boost/archive/basic_text_iarchive.hpp | 96 - .../boost/archive/basic_text_iprimitive.hpp | 142 -- .../boost/archive/basic_text_oarchive.hpp | 119 -- .../boost/archive/basic_text_oprimitive.hpp | 209 -- .../boost/archive/basic_xml_archive.hpp | 67 - .../boost/archive/basic_xml_iarchive.hpp | 119 -- .../boost/archive/basic_xml_oarchive.hpp | 138 -- .../boost/archive/binary_iarchive.hpp | 64 - .../boost/archive/binary_iarchive_impl.hpp | 105 - .../boost/archive/binary_oarchive.hpp | 64 - .../boost/archive/binary_oarchive_impl.hpp | 106 - .../boost/archive/binary_wiarchive.hpp | 56 - .../boost/archive/binary_woarchive.hpp | 59 - .../boost/archive/codecvt_null.hpp | 109 -- .../boost/archive/detail/abi_prefix.hpp | 16 - .../boost/archive/detail/abi_suffix.hpp | 15 - .../archive/detail/archive_serializer_map.hpp | 54 - .../archive/detail/auto_link_archive.hpp | 48 - .../archive/detail/auto_link_warchive.hpp | 47 - .../boost/archive/detail/basic_iarchive.hpp | 105 - .../archive/detail/basic_iserializer.hpp | 91 - .../boost/archive/detail/basic_oarchive.hpp | 94 - .../archive/detail/basic_oserializer.hpp | 89 - .../detail/basic_pointer_iserializer.hpp | 70 - .../detail/basic_pointer_oserializer.hpp | 68 - .../boost/archive/detail/basic_serializer.hpp | 77 - .../archive/detail/basic_serializer_map.hpp | 69 - .../boost/archive/detail/check.hpp | 169 -- .../boost/archive/detail/common_iarchive.hpp | 88 - .../boost/archive/detail/common_oarchive.hpp | 88 - .../boost/archive/detail/decl.hpp | 57 - .../archive/detail/helper_collection.hpp | 99 - .../archive/detail/interface_iarchive.hpp | 85 - .../archive/detail/interface_oarchive.hpp | 87 - .../boost/archive/detail/iserializer.hpp | 631 ------ .../boost/archive/detail/oserializer.hpp | 540 ------ .../detail/polymorphic_iarchive_route.hpp | 218 --- .../detail/polymorphic_oarchive_route.hpp | 209 -- .../boost/archive/detail/register_archive.hpp | 91 - .../archive/detail/utf8_codecvt_facet.hpp | 39 - .../boost_1_65_0/boost/archive/dinkumware.hpp | 224 --- .../archive/impl/archive_serializer_map.ipp | 75 - .../archive/impl/basic_binary_iarchive.ipp | 134 -- .../archive/impl/basic_binary_iprimitive.ipp | 171 -- .../archive/impl/basic_binary_oarchive.ipp | 42 - .../archive/impl/basic_binary_oprimitive.ipp | 126 -- .../archive/impl/basic_text_iarchive.ipp | 76 - .../archive/impl/basic_text_iprimitive.ipp | 137 -- .../archive/impl/basic_text_oarchive.ipp | 62 - .../archive/impl/basic_text_oprimitive.ipp | 115 -- .../boost/archive/impl/basic_xml_grammar.hpp | 173 -- .../boost/archive/impl/basic_xml_iarchive.ipp | 115 -- .../boost/archive/impl/basic_xml_oarchive.ipp | 272 --- .../boost/archive/impl/text_iarchive_impl.ipp | 128 -- .../boost/archive/impl/text_oarchive_impl.ipp | 122 -- .../archive/impl/text_wiarchive_impl.ipp | 118 -- .../archive/impl/text_woarchive_impl.ipp | 85 - .../boost/archive/impl/xml_iarchive_impl.ipp | 199 -- .../boost/archive/impl/xml_oarchive_impl.ipp | 142 -- .../boost/archive/impl/xml_wiarchive_impl.ipp | 189 -- .../boost/archive/impl/xml_woarchive_impl.ipp | 171 -- .../archive/iterators/base64_exception.hpp | 68 - .../archive/iterators/base64_from_binary.hpp | 109 -- .../archive/iterators/binary_from_base64.hpp | 118 -- .../boost/archive/iterators/dataflow.hpp | 102 - .../archive/iterators/dataflow_exception.hpp | 80 - .../boost/archive/iterators/escape.hpp | 115 -- .../archive/iterators/insert_linebreaks.hpp | 99 - .../archive/iterators/istream_iterator.hpp | 92 - .../boost/archive/iterators/mb_from_wchar.hpp | 139 -- .../archive/iterators/ostream_iterator.hpp | 83 - .../archive/iterators/remove_whitespace.hpp | 167 -- .../archive/iterators/transform_width.hpp | 177 -- .../boost/archive/iterators/unescape.hpp | 89 - .../boost/archive/iterators/wchar_from_mb.hpp | 194 -- .../boost/archive/iterators/xml_escape.hpp | 121 -- .../boost/archive/iterators/xml_unescape.hpp | 125 -- .../iterators/xml_unescape_exception.hpp | 49 - .../archive/polymorphic_binary_iarchive.hpp | 54 - .../archive/polymorphic_binary_oarchive.hpp | 43 - .../boost/archive/polymorphic_iarchive.hpp | 168 -- .../boost/archive/polymorphic_oarchive.hpp | 154 -- .../archive/polymorphic_text_iarchive.hpp | 54 - .../archive/polymorphic_text_oarchive.hpp | 39 - .../archive/polymorphic_text_wiarchive.hpp | 59 - .../archive/polymorphic_text_woarchive.hpp | 44 - .../archive/polymorphic_xml_iarchive.hpp | 54 - .../archive/polymorphic_xml_oarchive.hpp | 39 - .../archive/polymorphic_xml_wiarchive.hpp | 50 - .../archive/polymorphic_xml_woarchive.hpp | 44 - .../boost/archive/text_iarchive.hpp | 132 -- .../boost/archive/text_oarchive.hpp | 121 -- .../boost/archive/text_wiarchive.hpp | 137 -- .../boost/archive/text_woarchive.hpp | 155 -- .../boost_1_65_0/boost/archive/tmpdir.hpp | 50 - .../boost_1_65_0/boost/archive/wcslen.hpp | 58 - .../boost/archive/xml_archive_exception.hpp | 57 - .../boost/archive/xml_iarchive.hpp | 142 -- .../boost/archive/xml_oarchive.hpp | 137 -- .../boost/archive/xml_wiarchive.hpp | 149 -- .../boost/archive/xml_woarchive.hpp | 134 -- .../boost_1_65_0/boost/foreach_fwd.hpp | 51 - .../boost/multi_index/composite_key.hpp | 1513 --------------- .../multi_index/detail/access_specifier.hpp | 54 - .../boost/multi_index/detail/adl_swap.hpp | 44 - .../detail/archive_constructed.hpp | 83 - .../boost/multi_index/detail/auto_space.hpp | 91 - .../boost/multi_index/detail/base_type.hpp | 74 - .../detail/bidir_node_iterator.hpp | 114 -- .../boost/multi_index/detail/bucket_array.hpp | 243 --- .../multi_index/detail/cons_stdtuple.hpp | 93 - .../boost/multi_index/detail/converter.hpp | 52 - .../boost/multi_index/detail/copy_map.hpp | 142 -- .../detail/do_not_copy_elements_tag.hpp | 34 - .../detail/duplicates_iterator.hpp | 120 -- .../boost/multi_index/detail/has_tag.hpp | 42 - .../multi_index/detail/hash_index_args.hpp | 105 - .../detail/hash_index_iterator.hpp | 166 -- .../multi_index/detail/hash_index_node.hpp | 778 -------- .../multi_index/detail/header_holder.hpp | 50 - .../detail/ignore_wstrict_aliasing.hpp | 18 - .../boost/multi_index/detail/index_base.hpp | 293 --- .../boost/multi_index/detail/index_loader.hpp | 139 -- .../multi_index/detail/index_matcher.hpp | 249 --- .../multi_index/detail/index_node_base.hpp | 135 -- .../boost/multi_index/detail/index_saver.hpp | 135 -- .../multi_index/detail/invariant_assert.hpp | 21 - .../multi_index/detail/is_index_list.hpp | 40 - .../multi_index/detail/is_transparent.hpp | 135 -- .../boost/multi_index/detail/iter_adaptor.hpp | 321 --- .../multi_index/detail/modify_key_adaptor.hpp | 49 - .../multi_index/detail/no_duplicate_tags.hpp | 97 - .../boost/multi_index/detail/node_type.hpp | 66 - .../multi_index/detail/ord_index_args.hpp | 83 - .../multi_index/detail/ord_index_impl.hpp | 1567 --------------- .../multi_index/detail/ord_index_impl_fwd.hpp | 128 -- .../multi_index/detail/ord_index_node.hpp | 658 ------- .../multi_index/detail/ord_index_ops.hpp | 266 --- .../boost/multi_index/detail/promotes_arg.hpp | 83 - .../boost/multi_index/detail/raw_ptr.hpp | 52 - .../detail/restore_wstrict_aliasing.hpp | 11 - .../multi_index/detail/rnd_index_loader.hpp | 173 -- .../multi_index/detail/rnd_index_node.hpp | 273 --- .../multi_index/detail/rnd_index_ops.hpp | 203 -- .../detail/rnd_index_ptr_array.hpp | 144 -- .../multi_index/detail/rnd_node_iterator.hpp | 140 -- .../multi_index/detail/rnk_index_ops.hpp | 300 --- .../boost/multi_index/detail/safe_mode.hpp | 588 ------ .../boost/multi_index/detail/scope_guard.hpp | 453 ----- .../multi_index/detail/seq_index_node.hpp | 217 --- .../multi_index/detail/seq_index_ops.hpp | 203 -- .../detail/serialization_version.hpp | 73 - .../boost/multi_index/detail/uintptr_type.hpp | 76 - .../boost/multi_index/detail/unbounded.hpp | 66 - .../multi_index/detail/value_compare.hpp | 56 - .../multi_index/detail/vartempl_support.hpp | 247 --- .../boost/multi_index/global_fun.hpp | 185 -- .../boost/multi_index/hashed_index.hpp | 1725 ----------------- .../boost/multi_index/hashed_index_fwd.hpp | 74 - .../boost/multi_index/identity.hpp | 145 -- .../boost/multi_index/identity_fwd.hpp | 26 - .../boost/multi_index/indexed_by.hpp | 68 - .../boost/multi_index/key_extractors.hpp | 22 - .../boost/multi_index/mem_fun.hpp | 205 -- .../boost_1_65_0/boost/multi_index/member.hpp | 262 --- .../boost/multi_index/ordered_index.hpp | 114 -- .../boost/multi_index/ordered_index_fwd.hpp | 35 - .../boost/multi_index/random_access_index.hpp | 1167 ----------- .../multi_index/random_access_index_fwd.hpp | 91 - .../boost/multi_index/ranked_index.hpp | 382 ---- .../boost/multi_index/ranked_index_fwd.hpp | 35 - .../boost/multi_index/safe_mode_errors.hpp | 48 - .../boost/multi_index/sequenced_index.hpp | 1062 ---------- .../boost/multi_index/sequenced_index_fwd.hpp | 91 - .../boost_1_65_0/boost/multi_index/tag.hpp | 88 - .../boost/multi_index_container.hpp | 1362 ------------- .../boost/multi_index_container_fwd.hpp | 121 -- .../boost/serialization/access.hpp | 145 -- .../archive_input_unordered_map.hpp | 85 - .../archive_input_unordered_set.hpp | 72 - .../boost/serialization/array.hpp | 48 - .../serialization/array_optimization.hpp | 37 - .../boost/serialization/array_wrapper.hpp | 121 -- .../boost/serialization/assume_abstract.hpp | 60 - .../boost/serialization/base_object.hpp | 100 - .../boost/serialization/binary_object.hpp | 79 - .../boost/serialization/bitset.hpp | 75 - .../boost/serialization/boost_array.hpp | 33 - .../serialization/boost_unordered_map.hpp | 154 -- .../serialization/boost_unordered_set.hpp | 150 -- .../serialization/collection_size_type.hpp | 62 - .../boost/serialization/collection_traits.hpp | 79 - .../serialization/collections_load_imp.hpp | 106 - .../serialization/collections_save_imp.hpp | 82 - .../boost/serialization/complex.hpp | 81 - .../boost/serialization/config.hpp | 74 - .../boost/serialization/deque.hpp | 80 - .../detail/is_default_constructible.hpp | 54 - .../serialization/detail/shared_count_132.hpp | 551 ------ .../serialization/detail/shared_ptr_132.hpp | 443 ----- .../detail/shared_ptr_nmt_132.hpp | 182 -- .../detail/stack_constructor.hpp | 66 - .../boost/serialization/ephemeral.hpp | 72 - .../boost/serialization/export.hpp | 225 --- .../serialization/extended_type_info.hpp | 116 -- .../extended_type_info_no_rtti.hpp | 182 -- .../extended_type_info_typeid.hpp | 167 -- .../boost/serialization/factory.hpp | 102 - .../boost/serialization/force_include.hpp | 55 - .../boost/serialization/forward_list.hpp | 124 -- .../hash_collections_load_imp.hpp | 77 - .../hash_collections_save_imp.hpp | 97 - .../boost/serialization/hash_map.hpp | 232 --- .../boost/serialization/hash_set.hpp | 222 --- .../serialization/is_bitwise_serializable.hpp | 46 - .../boost/serialization/item_version_type.hpp | 68 - .../boost/serialization/level.hpp | 116 -- .../boost/serialization/level_enum.hpp | 55 - .../boost_1_65_0/boost/serialization/list.hpp | 85 - .../boost_1_65_0/boost/serialization/map.hpp | 139 -- .../boost_1_65_0/boost/serialization/nvp.hpp | 123 -- .../boost/serialization/optional.hpp | 107 - .../boost/serialization/priority_queue.hpp | 76 - .../boost/serialization/queue.hpp | 76 - .../boost/serialization/scoped_ptr.hpp | 58 - .../boost/serialization/serialization.hpp | 154 -- .../boost_1_65_0/boost/serialization/set.hpp | 137 -- .../boost/serialization/shared_ptr.hpp | 281 --- .../boost/serialization/shared_ptr_132.hpp | 222 --- .../boost/serialization/shared_ptr_helper.hpp | 209 -- .../boost/serialization/singleton.hpp | 166 -- .../boost/serialization/slist.hpp | 145 -- .../boost/serialization/smart_cast.hpp | 275 --- .../boost/serialization/split_free.hpp | 93 - .../boost/serialization/split_member.hpp | 86 - .../boost/serialization/stack.hpp | 76 - .../boost/serialization/state_saver.hpp | 96 - .../boost/serialization/static_warning.hpp | 103 - .../boost/serialization/string.hpp | 30 - .../boost/serialization/strong_typedef.hpp | 50 - .../boost/serialization/throw_exception.hpp | 44 - .../boost/serialization/tracking.hpp | 118 -- .../boost/serialization/tracking_enum.hpp | 41 - .../boost/serialization/traits.hpp | 65 - .../type_info_implementation.hpp | 73 - .../boost/serialization/unique_ptr.hpp | 68 - .../unordered_collections_load_imp.hpp | 73 - .../unordered_collections_save_imp.hpp | 86 - .../boost/serialization/unordered_map.hpp | 160 -- .../boost/serialization/unordered_set.hpp | 162 -- .../boost/serialization/utility.hpp | 56 - .../boost/serialization/valarray.hpp | 86 - .../boost/serialization/variant.hpp | 158 -- .../boost/serialization/vector.hpp | 233 --- .../boost/serialization/vector_135.hpp | 26 - .../boost/serialization/version.hpp | 107 - .../boost/serialization/void_cast.hpp | 298 --- .../boost/serialization/void_cast_fwd.hpp | 37 - .../boost/serialization/weak_ptr.hpp | 99 - .../boost/serialization/wrapper.hpp | 60 - contrib/poco | 1 + contrib/zlib-ng | 1 + dbms/src/Storages/MergeTree/MergeTreeData.cpp | 3 - 271 files changed, 3 insertions(+), 41591 deletions(-) create mode 160000 contrib/googletest delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/archive_exception.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_archive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iprimitive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oprimitive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_streambuf_locale_saver.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_text_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_text_iprimitive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_text_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_text_oprimitive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_xml_archive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_xml_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/basic_xml_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive_impl.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive_impl.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_wiarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/binary_woarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/codecvt_null.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/abi_prefix.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/abi_suffix.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/archive_serializer_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_archive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_warchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iserializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oserializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_iserializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_oserializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/check.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/common_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/common_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/decl.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/helper_collection.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/interface_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/interface_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/iserializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/oserializer.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_iarchive_route.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_oarchive_route.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/register_archive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/detail/utf8_codecvt_facet.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/dinkumware.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/archive_serializer_map.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iarchive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iprimitive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oarchive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oprimitive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iarchive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iprimitive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oarchive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oprimitive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_grammar.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_iarchive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_oarchive.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/text_iarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/text_oarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/text_wiarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/text_woarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/xml_iarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/xml_oarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/xml_wiarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/impl/xml_woarchive_impl.ipp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_exception.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_from_binary.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/binary_from_base64.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow_exception.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/escape.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/insert_linebreaks.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/istream_iterator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/mb_from_wchar.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/ostream_iterator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/remove_whitespace.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/transform_width.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/unescape.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/wchar_from_mb.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_escape.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape_exception.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_wiarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_woarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_wiarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_woarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/text_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/text_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/text_wiarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/text_woarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/tmpdir.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/wcslen.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_archive_exception.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_iarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_oarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_wiarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/archive/xml_woarchive.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/foreach_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/composite_key.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/access_specifier.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/adl_swap.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/archive_constructed.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/auto_space.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/base_type.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/bidir_node_iterator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/bucket_array.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/cons_stdtuple.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/converter.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/copy_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/do_not_copy_elements_tag.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/duplicates_iterator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/has_tag.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_args.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_iterator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_node.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/header_holder.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ignore_wstrict_aliasing.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_base.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_loader.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_matcher.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_node_base.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_saver.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/invariant_assert.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_index_list.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_transparent.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/iter_adaptor.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/modify_key_adaptor.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/no_duplicate_tags.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/node_type.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_args.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_node.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_ops.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/promotes_arg.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/raw_ptr.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/restore_wstrict_aliasing.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_loader.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_node.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ops.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ptr_array.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_node_iterator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnk_index_ops.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/safe_mode.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/scope_guard.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_node.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_ops.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/serialization_version.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/uintptr_type.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/unbounded.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/value_compare.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/detail/vartempl_support.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/global_fun.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/identity.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/identity_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/indexed_by.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/key_extractors.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/mem_fun.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/member.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/safe_mode_errors.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index/tag.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index_container.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/multi_index_container_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/access.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_set.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/array.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/array_optimization.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/array_wrapper.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/assume_abstract.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/base_object.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/binary_object.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/bitset.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/boost_array.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_set.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/collection_size_type.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/collection_traits.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/collections_load_imp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/collections_save_imp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/complex.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/config.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/deque.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/is_default_constructible.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_count_132.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_132.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_nmt_132.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/detail/stack_constructor.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/ephemeral.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/export.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_no_rtti.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_typeid.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/factory.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/force_include.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/forward_list.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_load_imp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_save_imp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/hash_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/hash_set.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/is_bitwise_serializable.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/item_version_type.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/level.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/level_enum.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/list.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/nvp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/optional.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/priority_queue.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/queue.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/scoped_ptr.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/serialization.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/set.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_132.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_helper.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/singleton.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/slist.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/smart_cast.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/split_free.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/split_member.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/stack.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/state_saver.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/static_warning.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/string.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/strong_typedef.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/throw_exception.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/tracking.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/tracking_enum.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/traits.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/type_info_implementation.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unique_ptr.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_load_imp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_save_imp.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unordered_map.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/unordered_set.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/utility.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/valarray.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/variant.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/vector.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/vector_135.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/version.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/void_cast.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/void_cast_fwd.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/weak_ptr.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/serialization/wrapper.hpp create mode 160000 contrib/poco create mode 160000 contrib/zlib-ng diff --git a/contrib/googletest b/contrib/googletest new file mode 160000 index 00000000000..d175c8bf823 --- /dev/null +++ b/contrib/googletest @@ -0,0 +1 @@ +Subproject commit d175c8bf823e709d570772b038757fadf63bc632 diff --git a/contrib/libboost/boost_1_65_0/boost/archive/archive_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/archive_exception.hpp deleted file mode 100644 index fabcdb5fa71..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/archive_exception.hpp +++ /dev/null @@ -1,100 +0,0 @@ -#ifndef BOOST_ARCHIVE_ARCHIVE_EXCEPTION_HPP -#define BOOST_ARCHIVE_ARCHIVE_EXCEPTION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// archive/archive_exception.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#include -#include - -// note: the only reason this is in here is that windows header -// includes #define exception_code _exception_code (arrrgghhhh!). -// the most expedient way to address this is be sure that this -// header is always included whenever this header file is included. -#if defined(BOOST_WINDOWS) -#include -#endif - -#include // must be the last header - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// exceptions thrown by archives -// -class BOOST_SYMBOL_VISIBLE archive_exception : - public virtual std::exception -{ -private: - char m_buffer[128]; -protected: - BOOST_ARCHIVE_DECL unsigned int - append(unsigned int l, const char * a); - BOOST_ARCHIVE_DECL - archive_exception() BOOST_NOEXCEPT; -public: - typedef enum { - no_exception, // initialized without code - other_exception, // any excepton not listed below - unregistered_class, // attempt to serialize a pointer of - // an unregistered class - invalid_signature, // first line of archive does not contain - // expected string - unsupported_version,// archive created with library version - // subsequent to this one - pointer_conflict, // an attempt has been made to directly - // serialize an object which has - // already been serialized through a pointer. - // Were this permitted, the archive load would result - // in the creation of an extra copy of the obect. - incompatible_native_format, // attempt to read native binary format - // on incompatible platform - array_size_too_short,// array being loaded doesn't fit in array allocated - input_stream_error, // error on input stream - invalid_class_name, // class name greater than the maximum permitted. - // most likely a corrupted archive or an attempt - // to insert virus via buffer overrun method. - unregistered_cast, // base - derived relationship not registered with - // void_cast_register - unsupported_class_version, // type saved with a version # greater than the - // one used by the program. This indicates that the program - // needs to be rebuilt. - multiple_code_instantiation, // code for implementing serialization for some - // type has been instantiated in more than one module. - output_stream_error // error on input stream - } exception_code; - exception_code code; - - BOOST_ARCHIVE_DECL archive_exception( - exception_code c, - const char * e1 = NULL, - const char * e2 = NULL - ) BOOST_NOEXCEPT; - BOOST_ARCHIVE_DECL archive_exception(archive_exception const &) BOOST_NOEXCEPT ; - virtual BOOST_ARCHIVE_DECL ~archive_exception() BOOST_NOEXCEPT_OR_NOTHROW ; - virtual BOOST_ARCHIVE_DECL const char * what() const BOOST_NOEXCEPT_OR_NOTHROW ; -}; - -}// namespace archive -}// namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif //BOOST_ARCHIVE_ARCHIVE_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_archive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_archive.hpp deleted file mode 100644 index ce7ac99a6dd..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_archive.hpp +++ /dev/null @@ -1,304 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_ARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_ARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_archive.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include // count -#include -#include -#include // size_t -#include -#include - -#include -#include // must be the last header - -namespace boost { -namespace archive { - -#if defined(_MSC_VER) -#pragma warning( push ) -#pragma warning( disable : 4244 4267 ) -#endif - -/* NOTE : Warning : Warning : Warning : Warning : Warning - * Don't ever changes this. If you do, they previously created - * binary archives won't be readable !!! - */ -class library_version_type { -private: - typedef uint_least16_t base_type; - base_type t; -public: - library_version_type(): t(0) {}; - explicit library_version_type(const unsigned int & t_) : t(t_){ - BOOST_ASSERT(t_ <= boost::integer_traits::const_max); - } - library_version_type(const library_version_type & t_) : - t(t_.t) - {} - library_version_type & operator=(const library_version_type & rhs){ - t = rhs.t; - return *this; - } - // used for text output - operator base_type () const { - return t; - } - // used for text input - operator base_type & (){ - return t; - } - bool operator==(const library_version_type & rhs) const { - return t == rhs.t; - } - bool operator<(const library_version_type & rhs) const { - return t < rhs.t; - } -}; - -BOOST_ARCHIVE_DECL library_version_type -BOOST_ARCHIVE_VERSION(); - -class version_type { -private: - typedef uint_least32_t base_type; - base_type t; -public: - // should be private - but MPI fails if it's not!!! - version_type(): t(0) {}; - explicit version_type(const unsigned int & t_) : t(t_){ - BOOST_ASSERT(t_ <= boost::integer_traits::const_max); - } - version_type(const version_type & t_) : - t(t_.t) - {} - version_type & operator=(const version_type & rhs){ - t = rhs.t; - return *this; - } - // used for text output - operator base_type () const { - return t; - } - // used for text intput - operator base_type & (){ - return t; - } - bool operator==(const version_type & rhs) const { - return t == rhs.t; - } - bool operator<(const version_type & rhs) const { - return t < rhs.t; - } -}; - -class class_id_type { -private: - typedef int_least16_t base_type; - base_type t; -public: - // should be private - but then can't use BOOST_STRONG_TYPE below - class_id_type() : t(0) {}; - explicit class_id_type(const int t_) : t(t_){ - BOOST_ASSERT(t_ <= boost::integer_traits::const_max); - } - explicit class_id_type(const std::size_t t_) : t(t_){ - // BOOST_ASSERT(t_ <= boost::integer_traits::const_max); - } - class_id_type(const class_id_type & t_) : - t(t_.t) - {} - class_id_type & operator=(const class_id_type & rhs){ - t = rhs.t; - return *this; - } - - // used for text output - operator int () const { - return t; - } - // used for text input - operator int_least16_t &() { - return t; - } - bool operator==(const class_id_type & rhs) const { - return t == rhs.t; - } - bool operator<(const class_id_type & rhs) const { - return t < rhs.t; - } -}; - -#define NULL_POINTER_TAG boost::archive::class_id_type(-1) - -class object_id_type { -private: - typedef uint_least32_t base_type; - base_type t; -public: - object_id_type(): t(0) {}; - // note: presumes that size_t >= unsigned int. - explicit object_id_type(const std::size_t & t_) : t(t_){ - BOOST_ASSERT(t_ <= boost::integer_traits::const_max); - } - object_id_type(const object_id_type & t_) : - t(t_.t) - {} - object_id_type & operator=(const object_id_type & rhs){ - t = rhs.t; - return *this; - } - // used for text output - operator uint_least32_t () const { - return t; - } - // used for text input - operator uint_least32_t & () { - return t; - } - bool operator==(const object_id_type & rhs) const { - return t == rhs.t; - } - bool operator<(const object_id_type & rhs) const { - return t < rhs.t; - } -}; - -#if defined(_MSC_VER) -#pragma warning( pop ) -#endif - -struct tracking_type { - bool t; - explicit tracking_type(const bool t_ = false) - : t(t_) - {}; - tracking_type(const tracking_type & t_) - : t(t_.t) - {} - operator bool () const { - return t; - }; - operator bool & () { - return t; - }; - tracking_type & operator=(const bool t_){ - t = t_; - return *this; - } - bool operator==(const tracking_type & rhs) const { - return t == rhs.t; - } - bool operator==(const bool & rhs) const { - return t == rhs; - } - tracking_type & operator=(const tracking_type & rhs){ - t = rhs.t; - return *this; - } -}; - -struct class_name_type : - private boost::noncopyable -{ - char *t; - operator const char * & () const { - return const_cast(t); - } - operator char * () { - return t; - } - std::size_t size() const { - return std::strlen(t); - } - explicit class_name_type(const char *key_) - : t(const_cast(key_)){} - explicit class_name_type(char *key_) - : t(key_){} - class_name_type & operator=(const class_name_type & rhs){ - t = rhs.t; - return *this; - } -}; - -enum archive_flags { - no_header = 1, // suppress archive header info - no_codecvt = 2, // suppress alteration of codecvt facet - no_xml_tag_checking = 4, // suppress checking of xml tags - no_tracking = 8, // suppress ALL tracking - flags_last = 8 -}; - -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_SIGNATURE(); - -/* NOTE : Warning : Warning : Warning : Warning : Warning - * If any of these are changed to different sized types, - * binary_iarchive won't be able to read older archives - * unless you rev the library version and include conditional - * code based on the library version. There is nothing - * inherently wrong in doing this - but you have to be super - * careful because it's easy to get wrong and start breaking - * old archives !!! - */ - -#define BOOST_ARCHIVE_STRONG_TYPEDEF(T, D) \ - class D : public T { \ - public: \ - explicit D(const T tt) : T(tt){} \ - }; \ -/**/ - -BOOST_ARCHIVE_STRONG_TYPEDEF(class_id_type, class_id_reference_type) -BOOST_ARCHIVE_STRONG_TYPEDEF(class_id_type, class_id_optional_type) -BOOST_ARCHIVE_STRONG_TYPEDEF(object_id_type, object_reference_type) - -}// namespace archive -}// namespace boost - -#include // pops abi_suffix.hpp pragmas - -#include - -// set implementation level to primitive for all types -// used internally by the serialization library - -BOOST_CLASS_IMPLEMENTATION(boost::archive::library_version_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::version_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::class_id_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::class_id_reference_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::class_id_optional_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::class_name_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::object_id_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::object_reference_type, primitive_type) -BOOST_CLASS_IMPLEMENTATION(boost::archive::tracking_type, primitive_type) - -#include - -// set types used internally by the serialization library -// to be bitwise serializable - -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::library_version_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::version_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::class_id_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::class_id_reference_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::class_id_optional_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::class_name_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::object_id_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::object_reference_type) -BOOST_IS_BITWISE_SERIALIZABLE(boost::archive::tracking_type) - -#endif //BOOST_ARCHIVE_BASIC_ARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iarchive.hpp deleted file mode 100644 index c0cc655c997..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iarchive.hpp +++ /dev/null @@ -1,204 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_BINARY_IARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_BINARY_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_iarchive.hpp -// -// archives stored as native binary - this should be the fastest way -// to archive the state of a group of obects. It makes no attempt to -// convert to any canonical form. - -// IN GENERAL, ARCHIVES CREATED WITH THIS CLASS WILL NOT BE READABLE -// ON PLATFORM APART FROM THE ONE THEY ARE CREATED ON - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -#include // must be the last header - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -///////////////////////////////////////////////////////////////////////// -// class basic_binary_iarchive - read serialized objects from a input binary stream -template -class BOOST_SYMBOL_VISIBLE basic_binary_iarchive : - public detail::common_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_iarchive; - #else - friend class detail::interface_iarchive; - #endif -#endif - // intermediate level to support override of operators - // fot templates in the absence of partial function - // template ordering. If we get here pass to base class - // note extra nonsense to sneak it pass the borland compiers - typedef detail::common_iarchive detail_common_iarchive; - template - void load_override(T & t){ - this->detail_common_iarchive::load_override(t); - } - - // include these to trap a change in binary format which - // isn't specifically handled - // upto 32K classes - BOOST_STATIC_ASSERT(sizeof(class_id_type) == sizeof(int_least16_t)); - BOOST_STATIC_ASSERT(sizeof(class_id_reference_type) == sizeof(int_least16_t)); - // upto 2G objects - BOOST_STATIC_ASSERT(sizeof(object_id_type) == sizeof(uint_least32_t)); - BOOST_STATIC_ASSERT(sizeof(object_reference_type) == sizeof(uint_least32_t)); - - // binary files don't include the optional information - void load_override(class_id_optional_type & /* t */){} - - void load_override(tracking_type & t, int /*version*/){ - library_version_type lvt = this->get_library_version(); - if(boost::archive::library_version_type(6) < lvt){ - int_least8_t x=0; - * this->This() >> x; - t = boost::archive::tracking_type(x); - } - else{ - bool x=0; - * this->This() >> x; - t = boost::archive::tracking_type(x); - } - } - void load_override(class_id_type & t){ - library_version_type lvt = this->get_library_version(); - if(boost::archive::library_version_type(7) < lvt){ - this->detail_common_iarchive::load_override(t); - } - else - if(boost::archive::library_version_type(6) < lvt){ - int_least16_t x=0; - * this->This() >> x; - t = boost::archive::class_id_type(x); - } - else{ - int x=0; - * this->This() >> x; - t = boost::archive::class_id_type(x); - } - } - void load_override(class_id_reference_type & t){ - load_override(static_cast(t)); - } - - void load_override(version_type & t){ - library_version_type lvt = this->get_library_version(); - if(boost::archive::library_version_type(7) < lvt){ - this->detail_common_iarchive::load_override(t); - } - else - if(boost::archive::library_version_type(6) < lvt){ - uint_least8_t x=0; - * this->This() >> x; - t = boost::archive::version_type(x); - } - else - if(boost::archive::library_version_type(5) < lvt){ - uint_least16_t x=0; - * this->This() >> x; - t = boost::archive::version_type(x); - } - else - if(boost::archive::library_version_type(2) < lvt){ - // upto 255 versions - unsigned char x=0; - * this->This() >> x; - t = version_type(x); - } - else{ - unsigned int x=0; - * this->This() >> x; - t = boost::archive::version_type(x); - } - } - - void load_override(boost::serialization::item_version_type & t){ - library_version_type lvt = this->get_library_version(); -// if(boost::archive::library_version_type(7) < lvt){ - if(boost::archive::library_version_type(6) < lvt){ - this->detail_common_iarchive::load_override(t); - } - else - if(boost::archive::library_version_type(6) < lvt){ - uint_least16_t x=0; - * this->This() >> x; - t = boost::serialization::item_version_type(x); - } - else{ - unsigned int x=0; - * this->This() >> x; - t = boost::serialization::item_version_type(x); - } - } - - void load_override(serialization::collection_size_type & t){ - if(boost::archive::library_version_type(5) < this->get_library_version()){ - this->detail_common_iarchive::load_override(t); - } - else{ - unsigned int x=0; - * this->This() >> x; - t = serialization::collection_size_type(x); - } - } - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_override(class_name_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(); - - basic_binary_iarchive(unsigned int flags) : - detail::common_iarchive(flags) - {} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_BINARY_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iprimitive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iprimitive.hpp deleted file mode 100644 index 665d3e81e1f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_iprimitive.hpp +++ /dev/null @@ -1,198 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_IPRIMITIVE_HPP -#define BOOST_ARCHIVE_BINARY_IPRIMITIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -#if defined(_MSC_VER) -#pragma warning( disable : 4800 ) -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_iprimitive.hpp -// -// archives stored as native binary - this should be the fastest way -// to archive the state of a group of obects. It makes no attempt to -// convert to any canonical form. - -// IN GENERAL, ARCHIVES CREATED WITH THIS CLASS WILL NOT BE READABLE -// ON PLATFORM APART FROM THE ONE THEY ARE CREATED ON - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include // std::memcpy -#include // std::size_t -#include // basic_streambuf -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include - -//#include -#include -#include - -#include -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace archive { - -///////////////////////////////////////////////////////////////////////////// -// class binary_iarchive - read serialized objects from a input binary stream -template -class BOOST_SYMBOL_VISIBLE basic_binary_iprimitive { -#ifndef BOOST_NO_MEMBER_TEMPLATE_FRIENDS - friend class load_access; -protected: -#else -public: -#endif - std::basic_streambuf & m_sb; - // return a pointer to the most derived class - Archive * This(){ - return static_cast(this); - } - - #ifndef BOOST_NO_STD_LOCALE - // note order! - if you change this, libstd++ will fail! - // a) create new locale with new codecvt facet - // b) save current locale - // c) change locale to new one - // d) use stream buffer - // e) change locale back to original - // f) destroy new codecvt facet - boost::archive::codecvt_null codecvt_null_facet; - basic_streambuf_locale_saver locale_saver; - std::locale archive_locale; - #endif - - // main template for serilization of primitive types - template - void load(T & t){ - load_binary(& t, sizeof(T)); - } - - ///////////////////////////////////////////////////////// - // fundamental types that need special treatment - - // trap usage of invalid uninitialized boolean - void load(bool & t){ - load_binary(& t, sizeof(t)); - int i = t; - BOOST_ASSERT(0 == i || 1 == i); - (void)i; // warning suppression for release builds. - } - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load(std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load(std::wstring &ws); - #endif - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load(char * t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load(wchar_t * t); - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - basic_binary_iprimitive( - std::basic_streambuf & sb, - bool no_codecvt - ); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - ~basic_binary_iprimitive(); -public: - // we provide an optimized load for all fundamental types - // typedef serialization::is_bitwise_serializable - // use_array_optimization; - struct use_array_optimization { - template - #if defined(BOOST_NO_DEPENDENT_NESTED_DERIVATIONS) - struct apply { - typedef typename boost::serialization::is_bitwise_serializable< T >::type type; - }; - #else - struct apply : public boost::serialization::is_bitwise_serializable< T > {}; - #endif - }; - - // the optimized load_array dispatches to load_binary - template - void load_array(serialization::array_wrapper& a, unsigned int) - { - load_binary(a.address(),a.count()*sizeof(ValueType)); - } - - void - load_binary(void *address, std::size_t count); -}; - -template -inline void -basic_binary_iprimitive::load_binary( - void *address, - std::size_t count -){ - // note: an optimizer should eliminate the following for char files - BOOST_ASSERT( - static_cast(count / sizeof(Elem)) - <= boost::integer_traits::const_max - ); - std::streamsize s = static_cast(count / sizeof(Elem)); - std::streamsize scount = m_sb.sgetn( - static_cast(address), - s - ); - if(scount != s) - boost::serialization::throw_exception( - archive_exception(archive_exception::input_stream_error) - ); - // note: an optimizer should eliminate the following for char files - BOOST_ASSERT(count % sizeof(Elem) <= boost::integer_traits::const_max); - s = static_cast(count % sizeof(Elem)); - if(0 < s){ -// if(is.fail()) -// boost::serialization::throw_exception( -// archive_exception(archive_exception::stream_error) -// ); - Elem t; - scount = m_sb.sgetn(& t, 1); - if(scount != 1) - boost::serialization::throw_exception( - archive_exception(archive_exception::input_stream_error) - ); - std::memcpy(static_cast(address) + (count - s), &t, static_cast(s)); - } -} - -} // namespace archive -} // namespace boost - -#include // pop pragmas - -#endif // BOOST_ARCHIVE_BINARY_IPRIMITIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oarchive.hpp deleted file mode 100644 index f05f2f86d55..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oarchive.hpp +++ /dev/null @@ -1,185 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_BINARY_OARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_BINARY_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// archives stored as native binary - this should be the fastest way -// to archive the state of a group of obects. It makes no attempt to -// convert to any canonical form. - -// IN GENERAL, ARCHIVES CREATED WITH THIS CLASS WILL NOT BE READABLE -// ON PLATFORM APART FROM THE ONE THEY ARE CREATE ON - -#include -#include -#include - -#include -#include - -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -////////////////////////////////////////////////////////////////////// -// class basic_binary_oarchive - write serialized objects to a binary output stream -// note: this archive has no pretensions to portability. Archive format -// may vary across machine architectures and compilers. About the only -// guarentee is that an archive created with this code will be readable -// by a program built with the same tools for the same machne. This class -// does have the virtue of buiding the smalles archive in the minimum amount -// of time. So under some circumstances it may be he right choice. -template -class BOOST_SYMBOL_VISIBLE basic_binary_oarchive : - public detail::common_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_oarchive; - #else - friend class detail::interface_oarchive; - #endif -#endif - // any datatype not specifed below will be handled by base class - typedef detail::common_oarchive detail_common_oarchive; - template - void save_override(const T & t){ - this->detail_common_oarchive::save_override(t); - } - - // include these to trap a change in binary format which - // isn't specifically handled - BOOST_STATIC_ASSERT(sizeof(tracking_type) == sizeof(bool)); - // upto 32K classes - BOOST_STATIC_ASSERT(sizeof(class_id_type) == sizeof(int_least16_t)); - BOOST_STATIC_ASSERT(sizeof(class_id_reference_type) == sizeof(int_least16_t)); - // upto 2G objects - BOOST_STATIC_ASSERT(sizeof(object_id_type) == sizeof(uint_least32_t)); - BOOST_STATIC_ASSERT(sizeof(object_reference_type) == sizeof(uint_least32_t)); - - // binary files don't include the optional information - void save_override(const class_id_optional_type & /* t */){} - - // enable this if we decide to support generation of previous versions - #if 0 - void save_override(const boost::archive::version_type & t){ - library_version_type lvt = this->get_library_version(); - if(boost::archive::library_version_type(7) < lvt){ - this->detail_common_oarchive::save_override(t); - } - else - if(boost::archive::library_version_type(6) < lvt){ - const boost::uint_least16_t x = t; - * this->This() << x; - } - else{ - const unsigned int x = t; - * this->This() << x; - } - } - void save_override(const boost::serialization::item_version_type & t){ - library_version_type lvt = this->get_library_version(); - if(boost::archive::library_version_type(7) < lvt){ - this->detail_common_oarchive::save_override(t); - } - else - if(boost::archive::library_version_type(6) < lvt){ - const boost::uint_least16_t x = t; - * this->This() << x; - } - else{ - const unsigned int x = t; - * this->This() << x; - } - } - - void save_override(class_id_type & t){ - library_version_type lvt = this->get_library_version(); - if(boost::archive::library_version_type(7) < lvt){ - this->detail_common_oarchive::save_override(t); - } - else - if(boost::archive::library_version_type(6) < lvt){ - const boost::int_least16_t x = t; - * this->This() << x; - } - else{ - const int x = t; - * this->This() << x; - } - } - void save_override(class_id_reference_type & t){ - save_override(static_cast(t)); - } - - #endif - - // explicitly convert to char * to avoid compile ambiguities - void save_override(const class_name_type & t){ - const std::string s(t); - * this->This() << s; - } - - #if 0 - void save_override(const serialization::collection_size_type & t){ - if (get_library_version() < boost::archive::library_version_type(6)){ - unsigned int x=0; - * this->This() >> x; - t = serialization::collection_size_type(x); - } - else{ - * this->This() >> t; - } - } - #endif - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(); - - basic_binary_oarchive(unsigned int flags) : - detail::common_oarchive(flags) - {} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_BINARY_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oprimitive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oprimitive.hpp deleted file mode 100644 index 6dc770c60e8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_binary_oprimitive.hpp +++ /dev/null @@ -1,188 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_BINARY_OPRIMITIVE_HPP -#define BOOST_ARCHIVE_BASIC_BINARY_OPRIMITIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_oprimitive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// archives stored as native binary - this should be the fastest way -// to archive the state of a group of obects. It makes no attempt to -// convert to any canonical form. - -// IN GENERAL, ARCHIVES CREATED WITH THIS CLASS WILL NOT BE READABLE -// ON PLATFORM APART FROM THE ONE THEY ARE CREATE ON - -#include -#include -#include -#include // basic_streambuf -#include -#include // size_t - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include - -//#include -#include -#include - -#include -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace archive { - -///////////////////////////////////////////////////////////////////////// -// class basic_binary_oprimitive - binary output of prmitives - -template -class BOOST_SYMBOL_VISIBLE basic_binary_oprimitive { -#ifndef BOOST_NO_MEMBER_TEMPLATE_FRIENDS - friend class save_access; -protected: -#else -public: -#endif - std::basic_streambuf & m_sb; - // return a pointer to the most derived class - Archive * This(){ - return static_cast(this); - } - #ifndef BOOST_NO_STD_LOCALE - // note order! - if you change this, libstd++ will fail! - // a) create new locale with new codecvt facet - // b) save current locale - // c) change locale to new one - // d) use stream buffer - // e) change locale back to original - // f) destroy new codecvt facet - boost::archive::codecvt_null codecvt_null_facet; - basic_streambuf_locale_saver locale_saver; - std::locale archive_locale; - #endif - // default saving of primitives. - template - void save(const T & t) - { - save_binary(& t, sizeof(T)); - } - - ///////////////////////////////////////////////////////// - // fundamental types that need special treatment - - // trap usage of invalid uninitialized boolean which would - // otherwise crash on load. - void save(const bool t){ - BOOST_ASSERT(0 == static_cast(t) || 1 == static_cast(t)); - save_binary(& t, sizeof(t)); - } - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save(const std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save(const std::wstring &ws); - #endif - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save(const char * t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save(const wchar_t * t); - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(); - - BOOST_ARCHIVE_OR_WARCHIVE_DECL - basic_binary_oprimitive( - std::basic_streambuf & sb, - bool no_codecvt - ); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - ~basic_binary_oprimitive(); -public: - - // we provide an optimized save for all fundamental types - // typedef serialization::is_bitwise_serializable - // use_array_optimization; - // workaround without using mpl lambdas - struct use_array_optimization { - template - #if defined(BOOST_NO_DEPENDENT_NESTED_DERIVATIONS) - struct apply { - typedef typename boost::serialization::is_bitwise_serializable< T >::type type; - }; - #else - struct apply : public boost::serialization::is_bitwise_serializable< T > {}; - #endif - }; - - // the optimized save_array dispatches to save_binary - template - void save_array(boost::serialization::array_wrapper const& a, unsigned int) - { - save_binary(a.address(),a.count()*sizeof(ValueType)); - } - - void save_binary(const void *address, std::size_t count); -}; - -template -inline void -basic_binary_oprimitive::save_binary( - const void *address, - std::size_t count -){ - // BOOST_ASSERT(count <= std::size_t(boost::integer_traits::const_max)); - // note: if the following assertions fail - // a likely cause is that the output stream is set to "text" - // mode where by cr characters recieve special treatment. - // be sure that the output stream is opened with ios::binary - //if(os.fail()) - // boost::serialization::throw_exception( - // archive_exception(archive_exception::output_stream_error) - // ); - // figure number of elements to output - round up - count = ( count + sizeof(Elem) - 1) / sizeof(Elem); - std::streamsize scount = m_sb.sputn( - static_cast(address), - static_cast(count) - ); - if(count != static_cast(scount)) - boost::serialization::throw_exception( - archive_exception(archive_exception::output_stream_error) - ); - //os.write( - // static_cast(address), - // count - //); - //BOOST_ASSERT(os.good()); -} - -} //namespace boost -} //namespace archive - -#include // pop pragmas - -#endif // BOOST_ARCHIVE_BASIC_BINARY_OPRIMITIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_streambuf_locale_saver.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_streambuf_locale_saver.hpp deleted file mode 100644 index 5cd4b36f081..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_streambuf_locale_saver.hpp +++ /dev/null @@ -1,108 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_STREAMBUF_LOCALE_SAVER_HPP -#define BOOST_ARCHIVE_BASIC_STREAMBUF_LOCALE_SAVER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_streambuf_locale_saver.hpp - -// (C) Copyright 2005 Robert Ramey - http://www.rrsd.com - -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// note derived from boost/io/ios_state.hpp -// Copyright 2002, 2005 Daryle Walker. Use, modification, and distribution -// are subject to the Boost Software License, Version 1.0. (See accompanying -// file LICENSE_1_0.txt or a copy at .) - -// See for the library's home page. - -#ifndef BOOST_NO_STD_LOCALE - -#include // for std::locale -#include -#include // for std::basic_streambuf - -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost{ -namespace archive{ - -template < typename Ch, class Tr > -class basic_streambuf_locale_saver : - private boost::noncopyable -{ -public: - explicit basic_streambuf_locale_saver(std::basic_streambuf &s) : - m_streambuf(s), - m_locale(s.getloc()) - {} - ~basic_streambuf_locale_saver(){ - m_streambuf.pubsync(); - m_streambuf.pubimbue(m_locale); - } -private: - std::basic_streambuf & m_streambuf; - std::locale const m_locale; -}; - -template < typename Ch, class Tr > -class basic_istream_locale_saver : - private boost::noncopyable -{ -public: - explicit basic_istream_locale_saver(std::basic_istream &s) : - m_istream(s), - m_locale(s.getloc()) - {} - ~basic_istream_locale_saver(){ - // libstdc++ crashes without this - m_istream.sync(); - m_istream.imbue(m_locale); - } -private: - std::basic_istream & m_istream; - std::locale const m_locale; -}; - -template < typename Ch, class Tr > -class basic_ostream_locale_saver : - private boost::noncopyable -{ -public: - explicit basic_ostream_locale_saver(std::basic_ostream &s) : - m_ostream(s), - m_locale(s.getloc()) - {} - ~basic_ostream_locale_saver(){ - m_ostream.flush(); - m_ostream.imbue(m_locale); - } -private: - std::basic_ostream & m_ostream; - std::locale const m_locale; -}; - - -} // archive -} // boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_NO_STD_LOCALE -#endif // BOOST_ARCHIVE_BASIC_STREAMBUF_LOCALE_SAVER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iarchive.hpp deleted file mode 100644 index 48a646cc1f7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iarchive.hpp +++ /dev/null @@ -1,96 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_TEXT_IARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_TEXT_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// archives stored as text - note these ar templated on the basic -// stream templates to accommodate wide (and other?) kind of characters -// -// note the fact that on libraries without wide characters, ostream is -// is not a specialization of basic_ostream which in fact is not defined -// in such cases. So we can't use basic_istream but rather -// use two template parameters - -#include -#include - -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -///////////////////////////////////////////////////////////////////////// -// class basic_text_iarchive - read serialized objects from a input text stream -template -class BOOST_SYMBOL_VISIBLE basic_text_iarchive : - public detail::common_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_iarchive; - #else - friend class detail::interface_iarchive; - #endif -#endif - // intermediate level to support override of operators - // fot templates in the absence of partial function - // template ordering - typedef detail::common_iarchive detail_common_iarchive; - template - void load_override(T & t){ - this->detail_common_iarchive::load_override(t); - } - // text file don't include the optional information - void load_override(class_id_optional_type & /*t*/){} - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_override(class_name_type & t); - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(void); - - basic_text_iarchive(unsigned int flags) : - detail::common_iarchive(flags) - {} - ~basic_text_iarchive(){} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_TEXT_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iprimitive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iprimitive.hpp deleted file mode 100644 index bf936b55546..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_iprimitive.hpp +++ /dev/null @@ -1,142 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_TEXT_IPRIMITIVE_HPP -#define BOOST_ARCHIVE_BASIC_TEXT_IPRIMITIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_iprimitive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// archives stored as text - note these are templated on the basic -// stream templates to accommodate wide (and other?) kind of characters -// -// Note the fact that on libraries without wide characters, ostream is -// not a specialization of basic_ostream which in fact is not defined -// in such cases. So we can't use basic_ostream but rather -// use two template parameters - -#include -#include // size_t - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; - #if ! defined(BOOST_DINKUMWARE_STDLIB) && ! defined(__SGI_STL_PORT) - using ::locale; - #endif -} // namespace std -#endif - -#include -#include - -#include -#if BOOST_WORKAROUND(BOOST_DINKUMWARE_STDLIB, == 1) -#include -#endif -#include -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace archive { - -///////////////////////////////////////////////////////////////////////// -// class basic_text_iarchive - load serialized objects from a input text stream -#if defined(_MSC_VER) -#pragma warning( push ) -#pragma warning( disable : 4244 4267 ) -#endif - -template -class BOOST_SYMBOL_VISIBLE basic_text_iprimitive { -protected: - IStream &is; - io::ios_flags_saver flags_saver; - io::ios_precision_saver precision_saver; - - #ifndef BOOST_NO_STD_LOCALE - // note order! - if you change this, libstd++ will fail! - // a) create new locale with new codecvt facet - // b) save current locale - // c) change locale to new one - // d) use stream buffer - // e) change locale back to original - // f) destroy new codecvt facet - boost::archive::codecvt_null codecvt_null_facet; - std::locale archive_locale; - basic_istream_locale_saver< - typename IStream::char_type, - typename IStream::traits_type - > locale_saver; - #endif - - template - void load(T & t) - { - if(is >> t) - return; - boost::serialization::throw_exception( - archive_exception(archive_exception::input_stream_error) - ); - } - - void load(char & t) - { - short int i; - load(i); - t = i; - } - void load(signed char & t) - { - short int i; - load(i); - t = i; - } - void load(unsigned char & t) - { - unsigned short int i; - load(i); - t = i; - } - - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - void load(wchar_t & t) - { - BOOST_STATIC_ASSERT(sizeof(wchar_t) <= sizeof(int)); - int i; - load(i); - t = i; - } - #endif - BOOST_ARCHIVE_OR_WARCHIVE_DECL - basic_text_iprimitive(IStream &is, bool no_codecvt); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - ~basic_text_iprimitive(); -public: - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_binary(void *address, std::size_t count); -}; - -#if defined(_MSC_VER) -#pragma warning( pop ) -#endif - -} // namespace archive -} // namespace boost - -#include // pop pragmas - -#endif // BOOST_ARCHIVE_BASIC_TEXT_IPRIMITIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oarchive.hpp deleted file mode 100644 index 6f7f8fb167d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oarchive.hpp +++ /dev/null @@ -1,119 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_TEXT_OARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_TEXT_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// archives stored as text - note these ar templated on the basic -// stream templates to accommodate wide (and other?) kind of characters -// -// note the fact that on libraries without wide characters, ostream is -// is not a specialization of basic_ostream which in fact is not defined -// in such cases. So we can't use basic_ostream but rather -// use two template parameters - -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -///////////////////////////////////////////////////////////////////////// -// class basic_text_oarchive -template -class BOOST_SYMBOL_VISIBLE basic_text_oarchive : - public detail::common_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_oarchive; - #else - friend class detail::interface_oarchive; - #endif -#endif - - enum { - none, - eol, - space - } delimiter; - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - newtoken(); - - void newline(){ - delimiter = eol; - } - - // default processing - kick back to base class. Note the - // extra stuff to get it passed borland compilers - typedef detail::common_oarchive detail_common_oarchive; - template - void save_override(T & t){ - this->detail_common_oarchive::save_override(t); - } - - // start new objects on a new line - void save_override(const object_id_type & t){ - this->This()->newline(); - this->detail_common_oarchive::save_override(t); - } - - // text file don't include the optional information - void save_override(const class_id_optional_type & /* t */){} - - void save_override(const class_name_type & t){ - const std::string s(t); - * this->This() << s; - } - - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(); - - basic_text_oarchive(unsigned int flags) : - detail::common_oarchive(flags), - delimiter(none) - {} - ~basic_text_oarchive(){} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_TEXT_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oprimitive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oprimitive.hpp deleted file mode 100644 index 45f09358ece..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_text_oprimitive.hpp +++ /dev/null @@ -1,209 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_TEXT_OPRIMITIVE_HPP -#define BOOST_ARCHIVE_BASIC_TEXT_OPRIMITIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_oprimitive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// archives stored as text - note these ar templated on the basic -// stream templates to accommodate wide (and other?) kind of characters -// -// note the fact that on libraries without wide characters, ostream is -// is not a specialization of basic_ostream which in fact is not defined -// in such cases. So we can't use basic_ostream but rather -// use two template parameters - -#include -#include -#include // size_t - -#include -#include -#include - -#include -#if BOOST_WORKAROUND(BOOST_DINKUMWARE_STDLIB, == 1) -#include -#endif - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; - #if ! defined(BOOST_DINKUMWARE_STDLIB) && ! defined(__SGI_STL_PORT) - using ::locale; - #endif -} // namespace std -#endif - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace archive { - -///////////////////////////////////////////////////////////////////////// -// class basic_text_oprimitive - output of prmitives to stream -template -class BOOST_SYMBOL_VISIBLE basic_text_oprimitive -{ -protected: - OStream &os; - io::ios_flags_saver flags_saver; - io::ios_precision_saver precision_saver; - - #ifndef BOOST_NO_STD_LOCALE - // note order! - if you change this, libstd++ will fail! - // a) create new locale with new codecvt facet - // b) save current locale - // c) change locale to new one - // d) use stream buffer - // e) change locale back to original - // f) destroy new codecvt facet - boost::archive::codecvt_null codecvt_null_facet; - std::locale archive_locale; - basic_ostream_locale_saver< - typename OStream::char_type, - typename OStream::traits_type - > locale_saver; - #endif - - ///////////////////////////////////////////////////////// - // fundamental types that need special treatment - void save(const bool t){ - // trap usage of invalid uninitialized boolean which would - // otherwise crash on load. - BOOST_ASSERT(0 == static_cast(t) || 1 == static_cast(t)); - if(os.fail()) - boost::serialization::throw_exception( - archive_exception(archive_exception::output_stream_error) - ); - os << t; - } - void save(const signed char t) - { - save(static_cast(t)); - } - void save(const unsigned char t) - { - save(static_cast(t)); - } - void save(const char t) - { - save(static_cast(t)); - } - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - void save(const wchar_t t) - { - BOOST_STATIC_ASSERT(sizeof(wchar_t) <= sizeof(int)); - save(static_cast(t)); - } - #endif - - ///////////////////////////////////////////////////////// - // saving of any types not listed above - - template - void save_impl(const T &t, boost::mpl::bool_ &){ - if(os.fail()) - boost::serialization::throw_exception( - archive_exception(archive_exception::output_stream_error) - ); - os << t; - } - - ///////////////////////////////////////////////////////// - // floating point types need even more special treatment - // the following determines whether the type T is some sort - // of floating point type. Note that we then assume that - // the stream << operator is defined on that type - if not - // we'll get a compile time error. This is meant to automatically - // support synthesized types which support floating point - // operations. Also it should handle compiler dependent types - // such long double. Due to John Maddock. - - template - struct is_float { - typedef typename mpl::bool_< - boost::is_floating_point::value - || (std::numeric_limits::is_specialized - && !std::numeric_limits::is_integer - && !std::numeric_limits::is_exact - && std::numeric_limits::max_exponent) - >::type type; - }; - - template - void save_impl(const T &t, boost::mpl::bool_ &){ - // must be a user mistake - can't serialize un-initialized data - if(os.fail()) - boost::serialization::throw_exception( - archive_exception(archive_exception::output_stream_error) - ); - // The formulae for the number of decimla digits required is given in - // http://www2.open-std.org/JTC1/SC22/WG21/docs/papers/2005/n1822.pdf - // which is derived from Kahan's paper: - // www.eecs.berkeley.edu/~wkahan/ieee754status/ieee754.ps - // const unsigned int digits = (std::numeric_limits::digits * 3010) / 10000; - // note: I've commented out the above because I didn't get good results. e.g. - // in one case I got a difference of 19 units. - #ifndef BOOST_NO_CXX11_NUMERIC_LIMITS - const unsigned int digits = std::numeric_limits::max_digits10; - #else - const unsigned int digits = std::numeric_limits::digits10 + 2; - #endif - os << std::setprecision(digits) << std::scientific << t; - } - - template - void save(const T & t){ - typename is_float::type tf; - save_impl(t, tf); - } - - BOOST_ARCHIVE_OR_WARCHIVE_DECL - basic_text_oprimitive(OStream & os, bool no_codecvt); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - ~basic_text_oprimitive(); -public: - // unformatted append of one character - void put(typename OStream::char_type c){ - if(os.fail()) - boost::serialization::throw_exception( - archive_exception(archive_exception::output_stream_error) - ); - os.put(c); - } - // unformatted append of null terminated string - void put(const char * s){ - while('\0' != *s) - os.put(*s++); - } - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_binary(const void *address, std::size_t count); -}; - -} //namespace boost -} //namespace archive - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_TEXT_OPRIMITIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_archive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_archive.hpp deleted file mode 100644 index bef368b973b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_archive.hpp +++ /dev/null @@ -1,67 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_XML_TEXT_ARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_XML_TEXT_ARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_xml_archive.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include // must be the last header - -namespace boost { -namespace archive { - -// constant strings used in xml i/o - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_OBJECT_ID(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_OBJECT_REFERENCE(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_CLASS_ID(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_CLASS_ID_REFERENCE(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_CLASS_NAME(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_TRACKING(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_VERSION(); - -extern -BOOST_ARCHIVE_DECL const char * -BOOST_ARCHIVE_XML_SIGNATURE(); - -}// namespace archive -}// namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_XML_TEXT_ARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_iarchive.hpp deleted file mode 100644 index e9f7482f744..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_iarchive.hpp +++ /dev/null @@ -1,119 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_XML_IARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_XML_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_xml_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -///////////////////////////////////////////////////////////////////////// -// class basic_xml_iarchive - read serialized objects from a input text stream -template -class BOOST_SYMBOL_VISIBLE basic_xml_iarchive : - public detail::common_iarchive -{ - unsigned int depth; -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_iarchive; -#endif - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_start(const char *name); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_end(const char *name); - - // Anything not an attribute and not a name-value pair is an - // should be trapped here. - template - void load_override(T & t) - { - // If your program fails to compile here, its most likely due to - // not specifying an nvp wrapper around the variable to - // be serialized. - BOOST_MPL_ASSERT((serialization::is_wrapper< T >)); - this->detail_common_iarchive::load_override(t); - } - - // Anything not an attribute - see below - should be a name value - // pair and be processed here - typedef detail::common_iarchive detail_common_iarchive; - template - void load_override( - const boost::serialization::nvp< T > & t - ){ - this->This()->load_start(t.name()); - this->detail_common_iarchive::load_override(t.value()); - this->This()->load_end(t.name()); - } - - // specific overrides for attributes - handle as - // primitives. These are not name-value pairs - // so they have to be intercepted here and passed on to load. - // although the class_id is included in the xml text file in order - // to make the file self describing, it isn't used when loading - // an xml archive. So we can skip it here. Note: we MUST override - // it otherwise it will be loaded as a normal primitive w/o tag and - // leaving the archive in an undetermined state - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_override(class_id_type & t); - void load_override(class_id_optional_type & /* t */){} - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_override(object_id_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_override(version_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - load_override(tracking_type & t); - // class_name_type can't be handled here as it depends upon the - // char type used by the stream. So require the derived implementation - // handle this. - // void load_override(class_name_type & t); - - BOOST_ARCHIVE_OR_WARCHIVE_DECL - basic_xml_iarchive(unsigned int flags); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - ~basic_xml_iarchive(); -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_XML_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_oarchive.hpp deleted file mode 100644 index 107fca4ec65..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/basic_xml_oarchive.hpp +++ /dev/null @@ -1,138 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_XML_OARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_XML_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_xml_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -////////////////////////////////////////////////////////////////////// -// class basic_xml_oarchive - write serialized objects to a xml output stream -template -class BOOST_SYMBOL_VISIBLE basic_xml_oarchive : - public detail::common_oarchive -{ - // special stuff for xml output - unsigned int depth; - bool pending_preamble; -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_oarchive; -#endif - bool indent_next; - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - indent(); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - init(); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - windup(); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - write_attribute( - const char *attribute_name, - int t, - const char *conjunction = "=\"" - ); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - write_attribute( - const char *attribute_name, - const char *key - ); - // helpers used below - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_start(const char *name); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_end(const char *name); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - end_preamble(); - - // Anything not an attribute and not a name-value pair is an - // error and should be trapped here. - template - void save_override(T & t) - { - // If your program fails to compile here, its most likely due to - // not specifying an nvp wrapper around the variable to - // be serialized. - BOOST_MPL_ASSERT((serialization::is_wrapper< T >)); - this->detail_common_oarchive::save_override(t); - } - - // special treatment for name-value pairs. - typedef detail::common_oarchive detail_common_oarchive; - template - void save_override( - const ::boost::serialization::nvp< T > & t - ){ - this->This()->save_start(t.name()); - this->detail_common_oarchive::save_override(t.const_value()); - this->This()->save_end(t.name()); - } - - // specific overrides for attributes - not name value pairs so we - // want to trap them before the above "fall through" - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const class_id_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const class_id_optional_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const class_id_reference_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const object_id_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const object_reference_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const version_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const class_name_type & t); - BOOST_ARCHIVE_OR_WARCHIVE_DECL void - save_override(const tracking_type & t); - - BOOST_ARCHIVE_OR_WARCHIVE_DECL - basic_xml_oarchive(unsigned int flags); - BOOST_ARCHIVE_OR_WARCHIVE_DECL - ~basic_xml_oarchive(); -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_XML_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive.hpp deleted file mode 100644 index 785ce7610b1..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive.hpp +++ /dev/null @@ -1,64 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_IARCHIVE_HPP -#define BOOST_ARCHIVE_BINARY_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from binary_iarchive_impl instead. This will -// preserve correct static polymorphism. -class BOOST_SYMBOL_VISIBLE binary_iarchive : - public binary_iarchive_impl< - boost::archive::binary_iarchive, - std::istream::char_type, - std::istream::traits_type - >{ -public: - binary_iarchive(std::istream & is, unsigned int flags = 0) : - binary_iarchive_impl< - binary_iarchive, std::istream::char_type, std::istream::traits_type - >(is, flags) - {} - binary_iarchive(std::streambuf & bsb, unsigned int flags = 0) : - binary_iarchive_impl< - binary_iarchive, std::istream::char_type, std::istream::traits_type - >(bsb, flags) - {} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::binary_iarchive) -BOOST_SERIALIZATION_USE_ARRAY_OPTIMIZATION(boost::archive::binary_iarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_BINARY_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive_impl.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive_impl.hpp deleted file mode 100644 index b4747c98ece..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/binary_iarchive_impl.hpp +++ /dev/null @@ -1,105 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_IARCHIVE_IMPL_HPP -#define BOOST_ARCHIVE_BINARY_IARCHIVE_IMPL_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_iarchive_impl.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE binary_iarchive_impl : - public basic_binary_iprimitive, - public basic_binary_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_iarchive; - friend basic_binary_iarchive; - friend load_access; - #else - friend class detail::interface_iarchive; - friend class basic_binary_iarchive; - friend class load_access; - #endif -#endif - template - void load_override(T & t){ - this->basic_binary_iarchive::load_override(t); - } - void init(unsigned int flags){ - if(0 != (flags & no_header)){ - return; - } - #if ! defined(__MWERKS__) - this->basic_binary_iarchive::init(); - this->basic_binary_iprimitive::init(); - #else - basic_binary_iarchive::init(); - basic_binary_iprimitive::init(); - #endif - } - binary_iarchive_impl( - std::basic_streambuf & bsb, - unsigned int flags - ) : - basic_binary_iprimitive( - bsb, - 0 != (flags & no_codecvt) - ), - basic_binary_iarchive(flags) - { - init(flags); - } - binary_iarchive_impl( - std::basic_istream & is, - unsigned int flags - ) : - basic_binary_iprimitive( - * is.rdbuf(), - 0 != (flags & no_codecvt) - ), - basic_binary_iarchive(flags) - { - init(flags); - } -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_BINARY_IARCHIVE_IMPL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive.hpp deleted file mode 100644 index e8313fd7c95..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive.hpp +++ /dev/null @@ -1,64 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_OARCHIVE_HPP -#define BOOST_ARCHIVE_BINARY_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from binary_oarchive_impl instead. This will -// preserve correct static polymorphism. -class BOOST_SYMBOL_VISIBLE binary_oarchive : - public binary_oarchive_impl< - binary_oarchive, std::ostream::char_type, std::ostream::traits_type - > -{ -public: - binary_oarchive(std::ostream & os, unsigned int flags = 0) : - binary_oarchive_impl< - binary_oarchive, std::ostream::char_type, std::ostream::traits_type - >(os, flags) - {} - binary_oarchive(std::streambuf & bsb, unsigned int flags = 0) : - binary_oarchive_impl< - binary_oarchive, std::ostream::char_type, std::ostream::traits_type - >(bsb, flags) - {} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::binary_oarchive) -BOOST_SERIALIZATION_USE_ARRAY_OPTIMIZATION(boost::archive::binary_oarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_BINARY_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive_impl.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive_impl.hpp deleted file mode 100644 index 6b4d018a564..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/binary_oarchive_impl.hpp +++ /dev/null @@ -1,106 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_OARCHIVE_IMPL_HPP -#define BOOST_ARCHIVE_BINARY_OARCHIVE_IMPL_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_oarchive_impl.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE binary_oarchive_impl : - public basic_binary_oprimitive, - public basic_binary_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_oarchive; - friend basic_binary_oarchive; - friend save_access; - #else - friend class detail::interface_oarchive; - friend class basic_binary_oarchive; - friend class save_access; - #endif -#endif - template - void save_override(T & t){ - this->basic_binary_oarchive::save_override(t); - } - void init(unsigned int flags) { - if(0 != (flags & no_header)){ - return; - } - #if ! defined(__MWERKS__) - this->basic_binary_oarchive::init(); - this->basic_binary_oprimitive::init(); - #else - basic_binary_oarchive::init(); - basic_binary_oprimitive::init(); - #endif - } - binary_oarchive_impl( - std::basic_streambuf & bsb, - unsigned int flags - ) : - basic_binary_oprimitive( - bsb, - 0 != (flags & no_codecvt) - ), - basic_binary_oarchive(flags) - { - init(flags); - } - binary_oarchive_impl( - std::basic_ostream & os, - unsigned int flags - ) : - basic_binary_oprimitive( - * os.rdbuf(), - 0 != (flags & no_codecvt) - ), - basic_binary_oarchive(flags) - { - init(flags); - } -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_BINARY_OARCHIVE_IMPL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_wiarchive.hpp deleted file mode 100644 index 775d8f82726..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/binary_wiarchive.hpp +++ /dev/null @@ -1,56 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_WIARCHIVE_HPP -#define BOOST_ARCHIVE_BINARY_WIARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_wiarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include // wistream -#include -#include - -namespace boost { -namespace archive { - -class binary_wiarchive : - public binary_iarchive_impl< - binary_wiarchive, std::wistream::char_type, std::wistream::traits_type - > -{ -public: - binary_wiarchive(std::wistream & is, unsigned int flags = 0) : - binary_iarchive_impl< - binary_wiarchive, std::wistream::char_type, std::wistream::traits_type - >(is, flags) - {} - binary_wiarchive(std::wstreambuf & bsb, unsigned int flags = 0) : - binary_iarchive_impl< - binary_wiarchive, std::wistream::char_type, std::wistream::traits_type - >(bsb, flags) - {} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::binary_wiarchive) - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_BINARY_WIARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/binary_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/binary_woarchive.hpp deleted file mode 100644 index a8817d6f8b4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/binary_woarchive.hpp +++ /dev/null @@ -1,59 +0,0 @@ -#ifndef BOOST_ARCHIVE_BINARY_WOARCHIVE_HPP -#define BOOST_ARCHIVE_BINARY_WOARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_woarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include -#include -#include - -namespace boost { -namespace archive { - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from binary_oarchive_impl instead. This will -// preserve correct static polymorphism. -class binary_woarchive : - public binary_oarchive_impl< - binary_woarchive, std::wostream::char_type, std::wostream::traits_type - > -{ -public: - binary_woarchive(std::wostream & os, unsigned int flags = 0) : - binary_oarchive_impl< - binary_woarchive, std::wostream::char_type, std::wostream::traits_type - >(os, flags) - {} - binary_woarchive(std::wstreambuf & bsb, unsigned int flags = 0) : - binary_oarchive_impl< - binary_woarchive, std::wostream::char_type, std::wostream::traits_type - >(bsb, flags) - {} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::binary_woarchive) - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_BINARY_WOARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/codecvt_null.hpp b/contrib/libboost/boost_1_65_0/boost/archive/codecvt_null.hpp deleted file mode 100644 index 7bce2b9b329..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/codecvt_null.hpp +++ /dev/null @@ -1,109 +0,0 @@ -#ifndef BOOST_ARCHIVE_CODECVT_NULL_HPP -#define BOOST_ARCHIVE_CODECVT_NULL_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// codecvt_null.hpp: - -// (C) Copyright 2004 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL, size_t -#ifndef BOOST_NO_CWCHAR -#include // for mbstate_t -#endif -#include -#include -#include -#include // must be the last header - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std { -// For STLport on WinCE, BOOST_NO_STDC_NAMESPACE can get defined if STLport is putting symbols in its own namespace. -// In the case of codecvt, however, this does not mean that codecvt is in the global namespace (it will be in STLport's namespace) -# if !defined(__SGI_STL_PORT) && !defined(_STLPORT_VERSION) - using ::codecvt; -# endif - using ::mbstate_t; - using ::size_t; -} // namespace -#endif - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -template -class codecvt_null; - -template<> -class codecvt_null : public std::codecvt -{ - virtual bool do_always_noconv() const throw() { - return true; - } -public: - explicit codecvt_null(std::size_t no_locale_manage = 0) : - std::codecvt(no_locale_manage) - {} - virtual ~codecvt_null(){}; -}; - -template<> -class BOOST_SYMBOL_VISIBLE codecvt_null : public std::codecvt -{ - virtual BOOST_WARCHIVE_DECL BOOST_DLLEXPORT std::codecvt_base::result - do_out( - std::mbstate_t & state, - const wchar_t * first1, - const wchar_t * last1, - const wchar_t * & next1, - char * first2, - char * last2, - char * & next2 - ) const BOOST_USED; - virtual BOOST_WARCHIVE_DECL BOOST_DLLEXPORT std::codecvt_base::result - do_in( - std::mbstate_t & state, - const char * first1, - const char * last1, - const char * & next1, - wchar_t * first2, - wchar_t * last2, - wchar_t * & next2 - ) const BOOST_USED; - virtual int do_encoding( ) const throw( ){ - return sizeof(wchar_t) / sizeof(char); - } - virtual int do_max_length( ) const throw( ){ - return do_encoding(); - } -public: - BOOST_DLLEXPORT explicit codecvt_null(std::size_t no_locale_manage = 0) : - std::codecvt(no_locale_manage) - {} - virtual ~codecvt_null(){}; -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif -#include // pop pragmas - -#endif //BOOST_ARCHIVE_CODECVT_NULL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_prefix.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_prefix.hpp deleted file mode 100644 index debf79e9f0b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_prefix.hpp +++ /dev/null @@ -1,16 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// abi_prefix.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // must be the last header -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4251 4231 4660 4275) -#endif - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_suffix.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_suffix.hpp deleted file mode 100644 index 4e054d66214..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/abi_suffix.hpp +++ /dev/null @@ -1,15 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// abi_suffix.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif -#include // pops abi_suffix.hpp pragmas - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/archive_serializer_map.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/archive_serializer_map.hpp deleted file mode 100644 index 5432bfc73e7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/archive_serializer_map.hpp +++ /dev/null @@ -1,54 +0,0 @@ -#ifndef BOOST_ARCHIVE_SERIALIZER_MAP_HPP -#define BOOST_ARCHIVE_SERIALIZER_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// archive_serializer_map.hpp: extenstion of type_info required for -// serialization. - -// (C) Copyright 2009 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// note: this is nothing more than the thinest of wrappers around -// basic_serializer_map so we can have a one map / archive type. - -#include -#include -#include // must be the last header - -namespace boost { - -namespace serialization { - class extended_type_info; -} // namespace serialization - -namespace archive { -namespace detail { - -class basic_serializer; - -template -class BOOST_SYMBOL_VISIBLE archive_serializer_map { -public: - static BOOST_ARCHIVE_OR_WARCHIVE_DECL bool insert(const basic_serializer * bs); - static BOOST_ARCHIVE_OR_WARCHIVE_DECL void erase(const basic_serializer * bs); - static BOOST_ARCHIVE_OR_WARCHIVE_DECL const basic_serializer * find( - const boost::serialization::extended_type_info & type_ - ); -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#include // must be the last header - -#endif //BOOST_ARCHIVE_SERIALIZER_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_archive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_archive.hpp deleted file mode 100644 index 79b0e490d65..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_archive.hpp +++ /dev/null @@ -1,48 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_AUTO_LINK_ARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_AUTO_LINK_ARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// auto_link_archive.hpp -// -// (c) Copyright Robert Ramey 2004 -// Use, modification, and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See library home page at http://www.boost.org/libs/serialization - -//----------------------------------------------------------------------------// - -// This header implements separate compilation features as described in -// http://www.boost.org/more/separate_compilation.html - -// enable automatic library variant selection ------------------------------// - -#include - -#if !defined(BOOST_ALL_NO_LIB) && !defined(BOOST_SERIALIZATION_NO_LIB) \ -&& !defined(BOOST_ARCHIVE_SOURCE) && !defined(BOOST_WARCHIVE_SOURCE) \ -&& !defined(BOOST_SERIALIZATION_SOURCE) - - // Set the name of our library, this will get undef'ed by auto_link.hpp - // once it's done with it: - // - #define BOOST_LIB_NAME boost_serialization - // - // If we're importing code from a dll, then tell auto_link.hpp about it: - // - #if defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) - # define BOOST_DYN_LINK - #endif - // - // And include the header that does the work: - // - #include -#endif // auto-linking disabled - -#endif // BOOST_ARCHIVE_DETAIL_AUTO_LINK_ARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_warchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_warchive.hpp deleted file mode 100644 index 683d191c20d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/auto_link_warchive.hpp +++ /dev/null @@ -1,47 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_AUTO_LINK_WARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_AUTO_LINK_WARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// auto_link_warchive.hpp -// -// (c) Copyright Robert Ramey 2004 -// Use, modification, and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See library home page at http://www.boost.org/libs/serialization - -//----------------------------------------------------------------------------// - -// This header implements separate compilation features as described in -// http://www.boost.org/more/separate_compilation.html - -// enable automatic library variant selection ------------------------------// - -#include - -#if !defined(BOOST_WARCHIVE_SOURCE) \ -&& !defined(BOOST_ALL_NO_LIB) && !defined(BOOST_SERIALIZATION_NO_LIB) - -// Set the name of our library, this will get undef'ed by auto_link.hpp -// once it's done with it: -// -#define BOOST_LIB_NAME boost_wserialization -// -// If we're importing code from a dll, then tell auto_link.hpp about it: -// -#if defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) -# define BOOST_DYN_LINK -#endif -// -// And include the header that does the work: -// -#include -#endif // auto-linking disabled - -#endif // ARCHIVE_DETAIL_AUTO_LINK_ARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iarchive.hpp deleted file mode 100644 index 1f5a8bf63bf..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iarchive.hpp +++ /dev/null @@ -1,105 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_BASIC_IARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_BASIC_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_iarchive.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// can't use this - much as I'd like to as borland doesn't support it - -#include -#include -#include - -#include -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization - -namespace archive { -namespace detail { - -class basic_iarchive_impl; -class basic_iserializer; -class basic_pointer_iserializer; - -////////////////////////////////////////////////////////////////////// -// class basic_iarchive - read serialized objects from a input stream -class BOOST_SYMBOL_VISIBLE basic_iarchive : - private boost::noncopyable, - public boost::archive::detail::helper_collection -{ - friend class basic_iarchive_impl; - // hide implementation of this class to minimize header conclusion - boost::scoped_ptr pimpl; - - virtual void vload(version_type &t) = 0; - virtual void vload(object_id_type &t) = 0; - virtual void vload(class_id_type &t) = 0; - virtual void vload(class_id_optional_type &t) = 0; - virtual void vload(class_name_type &t) = 0; - virtual void vload(tracking_type &t) = 0; -protected: - BOOST_ARCHIVE_DECL basic_iarchive(unsigned int flags); - boost::archive::detail::helper_collection & - get_helper_collection(){ - return *this; - } -public: - // some msvc versions require that the following function be public - // otherwise it should really protected. - virtual BOOST_ARCHIVE_DECL ~basic_iarchive(); - // note: NOT part of the public API. - BOOST_ARCHIVE_DECL void next_object_pointer(void *t); - BOOST_ARCHIVE_DECL void register_basic_serializer( - const basic_iserializer & bis - ); - BOOST_ARCHIVE_DECL void load_object( - void *t, - const basic_iserializer & bis - ); - BOOST_ARCHIVE_DECL const basic_pointer_iserializer * - load_pointer( - void * & t, - const basic_pointer_iserializer * bpis_ptr, - const basic_pointer_iserializer * (*finder)( - const boost::serialization::extended_type_info & eti - ) - ); - // real public API starts here - BOOST_ARCHIVE_DECL void - set_library_version(library_version_type archive_library_version); - BOOST_ARCHIVE_DECL library_version_type - get_library_version() const; - BOOST_ARCHIVE_DECL unsigned int - get_flags() const; - BOOST_ARCHIVE_DECL void - reset_object_address(const void * new_address, const void * old_address); - BOOST_ARCHIVE_DECL void - delete_created_pointers(); -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif //BOOST_ARCHIVE_DETAIL_BASIC_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iserializer.hpp deleted file mode 100644 index 0d66674c349..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_iserializer.hpp +++ /dev/null @@ -1,91 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_BASIC_ISERIALIZER_HPP -#define BOOST_ARCHIVE_DETAIL_BASIC_ISERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_iserializer.hpp: extenstion of type_info required for serialization. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include - -#include -#include -#include -#include -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization - -// forward declarations -namespace archive { -namespace detail { - -class basic_iarchive; -class basic_pointer_iserializer; - -class BOOST_SYMBOL_VISIBLE basic_iserializer : - public basic_serializer -{ -private: - basic_pointer_iserializer *m_bpis; -protected: - explicit BOOST_ARCHIVE_DECL basic_iserializer( - const boost::serialization::extended_type_info & type - ); - virtual BOOST_ARCHIVE_DECL ~basic_iserializer(); -public: - bool serialized_as_pointer() const { - return m_bpis != NULL; - } - void set_bpis(basic_pointer_iserializer *bpis){ - m_bpis = bpis; - } - const basic_pointer_iserializer * get_bpis_ptr() const { - return m_bpis; - } - virtual void load_object_data( - basic_iarchive & ar, - void *x, - const unsigned int file_version - ) const = 0; - // returns true if class_info should be saved - virtual bool class_info() const = 0 ; - // returns true if objects should be tracked - virtual bool tracking(const unsigned int) const = 0 ; - // returns class version - virtual version_type version() const = 0 ; - // returns true if this class is polymorphic - virtual bool is_polymorphic() const = 0; - virtual void destroy(/*const*/ void *address) const = 0 ; -}; - -} // namespae detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_DETAIL_BASIC_ISERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oarchive.hpp deleted file mode 100644 index c379108d584..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oarchive.hpp +++ /dev/null @@ -1,94 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_OARCHIVE_HPP -#define BOOST_ARCHIVE_BASIC_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_oarchive.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include -#include -#include - -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization - -namespace archive { -namespace detail { - -class basic_oarchive_impl; -class basic_oserializer; -class basic_pointer_oserializer; - -////////////////////////////////////////////////////////////////////// -// class basic_oarchive - write serialized objects to an output stream -class BOOST_SYMBOL_VISIBLE basic_oarchive : - private boost::noncopyable, - public boost::archive::detail::helper_collection -{ - friend class basic_oarchive_impl; - // hide implementation of this class to minimize header conclusion - boost::scoped_ptr pimpl; - - // overload these to bracket object attributes. Used to implement - // xml archives - virtual void vsave(const version_type t) = 0; - virtual void vsave(const object_id_type t) = 0; - virtual void vsave(const object_reference_type t) = 0; - virtual void vsave(const class_id_type t) = 0; - virtual void vsave(const class_id_optional_type t) = 0; - virtual void vsave(const class_id_reference_type t) = 0; - virtual void vsave(const class_name_type & t) = 0; - virtual void vsave(const tracking_type t) = 0; -protected: - BOOST_ARCHIVE_DECL basic_oarchive(unsigned int flags = 0); - BOOST_ARCHIVE_DECL boost::archive::detail::helper_collection & - get_helper_collection(); - virtual BOOST_ARCHIVE_DECL ~basic_oarchive(); -public: - // note: NOT part of the public interface - BOOST_ARCHIVE_DECL void register_basic_serializer( - const basic_oserializer & bos - ); - BOOST_ARCHIVE_DECL void save_object( - const void *x, - const basic_oserializer & bos - ); - BOOST_ARCHIVE_DECL void save_pointer( - const void * t, - const basic_pointer_oserializer * bpos_ptr - ); - void save_null_pointer(){ - vsave(NULL_POINTER_TAG); - } - // real public interface starts here - BOOST_ARCHIVE_DECL void end_preamble(); // default implementation does nothing - BOOST_ARCHIVE_DECL library_version_type get_library_version() const; - BOOST_ARCHIVE_DECL unsigned int get_flags() const; -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif //BOOST_ARCHIVE_BASIC_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oserializer.hpp deleted file mode 100644 index 94247e90056..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_oserializer.hpp +++ /dev/null @@ -1,89 +0,0 @@ -#ifndef BOOST_SERIALIZATION_BASIC_OSERIALIZER_HPP -#define BOOST_SERIALIZATION_BASIC_OSERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_oserializer.hpp: extenstion of type_info required for serialization. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include -#include - -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization - -// forward declarations -namespace archive { -namespace detail { - -class basic_oarchive; -class basic_pointer_oserializer; - -class BOOST_SYMBOL_VISIBLE basic_oserializer : - public basic_serializer -{ -private: - basic_pointer_oserializer *m_bpos; -protected: - explicit BOOST_ARCHIVE_DECL basic_oserializer( - const boost::serialization::extended_type_info & type_ - ); - virtual BOOST_ARCHIVE_DECL ~basic_oserializer(); -public: - bool serialized_as_pointer() const { - return m_bpos != NULL; - } - void set_bpos(basic_pointer_oserializer *bpos){ - m_bpos = bpos; - } - const basic_pointer_oserializer * get_bpos() const { - return m_bpos; - } - virtual void save_object_data( - basic_oarchive & ar, const void * x - ) const = 0; - // returns true if class_info should be saved - virtual bool class_info() const = 0; - // returns true if objects should be tracked - virtual bool tracking(const unsigned int flags) const = 0; - // returns class version - virtual version_type version() const = 0; - // returns true if this class is polymorphic - virtual bool is_polymorphic() const = 0; -}; - -} // namespace detail -} // namespace serialization -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_SERIALIZATION_BASIC_OSERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_iserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_iserializer.hpp deleted file mode 100644 index 1fc4b14d6e9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_iserializer.hpp +++ /dev/null @@ -1,70 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_POINTER_ISERIALIZER_HPP -#define BOOST_ARCHIVE_BASIC_POINTER_ISERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_pointer_oserializer.hpp: extenstion of type_info required for -// serialization. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization - -// forward declarations -namespace archive { -namespace detail { - -class basic_iarchive; -class basic_iserializer; - -class BOOST_SYMBOL_VISIBLE basic_pointer_iserializer - : public basic_serializer { -protected: - explicit BOOST_ARCHIVE_DECL basic_pointer_iserializer( - const boost::serialization::extended_type_info & type_ - ); - virtual BOOST_ARCHIVE_DECL ~basic_pointer_iserializer(); -public: - virtual void * heap_allocation() const = 0; - virtual const basic_iserializer & get_basic_serializer() const = 0; - virtual void load_object_ptr( - basic_iarchive & ar, - void * x, - const unsigned int file_version - ) const = 0; -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_POINTER_ISERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_oserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_oserializer.hpp deleted file mode 100644 index 1a5d9549eab..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_pointer_oserializer.hpp +++ /dev/null @@ -1,68 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_POINTER_OSERIALIZER_HPP -#define BOOST_ARCHIVE_BASIC_POINTER_OSERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_pointer_oserializer.hpp: extenstion of type_info required for -// serialization. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization - -namespace archive { -namespace detail { - -class basic_oarchive; -class basic_oserializer; - -class BOOST_SYMBOL_VISIBLE basic_pointer_oserializer : - public basic_serializer -{ -protected: - explicit BOOST_ARCHIVE_DECL basic_pointer_oserializer( - const boost::serialization::extended_type_info & type_ - ); -public: - virtual BOOST_ARCHIVE_DECL ~basic_pointer_oserializer(); - virtual const basic_oserializer & get_basic_serializer() const = 0; - virtual void save_object_ptr( - basic_oarchive & ar, - const void * x - ) const = 0; -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_BASIC_POINTER_OSERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer.hpp deleted file mode 100644 index f9c4203f862..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer.hpp +++ /dev/null @@ -1,77 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_SERIALIZER_HPP -#define BOOST_ARCHIVE_BASIC_SERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_serializer.hpp: extenstion of type_info required for serialization. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { -namespace detail { - -class basic_serializer : - private boost::noncopyable -{ - const boost::serialization::extended_type_info * m_eti; -protected: - explicit basic_serializer( - const boost::serialization::extended_type_info & eti - ) : - m_eti(& eti) - {} -public: - inline bool - operator<(const basic_serializer & rhs) const { - // can't compare address since there can be multiple eti records - // for the same type in different execution modules (that is, DLLS) - // leave this here as a reminder not to do this! - // return & lhs.get_eti() < & rhs.get_eti(); - return get_eti() < rhs.get_eti(); - } - const char * get_debug_info() const { - return m_eti->get_debug_info(); - } - const boost::serialization::extended_type_info & get_eti() const { - return * m_eti; - } -}; - -class basic_serializer_arg : public basic_serializer { -public: - basic_serializer_arg(const serialization::extended_type_info & eti) : - basic_serializer(eti) - {} -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_BASIC_SERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer_map.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer_map.hpp deleted file mode 100644 index 79341803367..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/basic_serializer_map.hpp +++ /dev/null @@ -1,69 +0,0 @@ -#ifndef BOOST_SERIALIZER_MAP_HPP -#define BOOST_SERIALIZER_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_serializer_map.hpp: extenstion of type_info required for serialization. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include -#include - -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} - -namespace archive { -namespace detail { - -class basic_serializer; - -class BOOST_SYMBOL_VISIBLE -basic_serializer_map : public - boost::noncopyable -{ - struct type_info_pointer_compare - { - bool operator()( - const basic_serializer * lhs, const basic_serializer * rhs - ) const ; - }; - typedef std::set< - const basic_serializer *, - type_info_pointer_compare - > map_type; - map_type m_map; -public: - BOOST_ARCHIVE_DECL bool insert(const basic_serializer * bs); - BOOST_ARCHIVE_DECL void erase(const basic_serializer * bs); - BOOST_ARCHIVE_DECL const basic_serializer * find( - const boost::serialization::extended_type_info & type_ - ) const; -private: - // cw 8.3 requires this - basic_serializer_map& operator=(basic_serializer_map const&); -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#include // must be the last header - -#endif // BOOST_SERIALIZER_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/check.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/check.hpp deleted file mode 100644 index 10034e7d101..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/check.hpp +++ /dev/null @@ -1,169 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_CHECK_HPP -#define BOOST_ARCHIVE_DETAIL_CHECK_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#pragma inline_depth(511) -#pragma inline_recursion(on) -#endif - -#if defined(__MWERKS__) -#pragma inline_depth(511) -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// check.hpp: interface for serialization system. - -// (C) Copyright 2009 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -namespace boost { -namespace archive { -namespace detail { - -// checks for objects - -template -inline void check_object_level(){ - typedef - typename mpl::greater_equal< - serialization::implementation_level< T >, - mpl::int_ - >::type typex; - - // trap attempts to serialize objects marked - // not_serializable - BOOST_STATIC_ASSERT(typex::value); -} - -template -inline void check_object_versioning(){ - typedef - typename mpl::or_< - typename mpl::greater< - serialization::implementation_level< T >, - mpl::int_ - >, - typename mpl::equal_to< - serialization::version< T >, - mpl::int_<0> - > - > typex; - // trap attempts to serialize with objects that don't - // save class information in the archive with versioning. - BOOST_STATIC_ASSERT(typex::value); -} - -template -inline void check_object_tracking(){ - // presume it has already been determined that - // T is not a const - BOOST_STATIC_ASSERT(! boost::is_const< T >::value); - typedef typename mpl::equal_to< - serialization::tracking_level< T >, - mpl::int_ - >::type typex; - // saving an non-const object of a type not marked "track_never) - - // may be an indicator of an error usage of the - // serialization library and should be double checked. - // See documentation on object tracking. Also, see the - // "rationale" section of the documenation - // for motivation for this checking. - - BOOST_STATIC_WARNING(typex::value); -} - -// checks for pointers - -template -inline void check_pointer_level(){ - // we should only invoke this once we KNOW that T - // has been used as a pointer!! - typedef - typename mpl::or_< - typename mpl::greater< - serialization::implementation_level< T >, - mpl::int_ - >, - typename mpl::not_< - typename mpl::equal_to< - serialization::tracking_level< T >, - mpl::int_ - > - > - > typex; - // Address the following when serializing to a pointer: - - // a) This type doesn't save class information in the - // archive. That is, the serialization trait implementation - // level <= object_serializable. - // b) Tracking for this type is set to "track selectively" - - // in this case, indication that an object is tracked is - // not stored in the archive itself - see level == object_serializable - // but rather the existence of the operation ar >> T * is used to - // infer that an object of this type should be tracked. So, if - // you save via a pointer but don't load via a pointer the operation - // will fail on load without given any valid reason for the failure. - - // So if your program traps here, consider changing the - // tracking or implementation level traits - or not - // serializing via a pointer. - BOOST_STATIC_WARNING(typex::value); -} - -template -void inline check_pointer_tracking(){ - typedef typename mpl::greater< - serialization::tracking_level< T >, - mpl::int_ - >::type typex; - // serializing an object of a type marked "track_never" through a pointer - // could result in creating more objects than were saved! - BOOST_STATIC_WARNING(typex::value); -} - -template -inline void check_const_loading(){ - typedef - typename mpl::or_< - typename boost::serialization::is_wrapper< T >, - typename mpl::not_< - typename boost::is_const< T > - > - >::type typex; - // cannot load data into a "const" object unless it's a - // wrapper around some other non-const object. - BOOST_STATIC_ASSERT(typex::value); -} - -} // detail -} // archive -} // boost - -#endif // BOOST_ARCHIVE_DETAIL_CHECK_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/common_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/common_iarchive.hpp deleted file mode 100644 index 82304f1e5ac..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/common_iarchive.hpp +++ /dev/null @@ -1,88 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_COMMON_IARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_COMMON_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// common_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { -namespace detail { - -class extended_type_info; - -// note: referred to as Curiously Recurring Template Patter (CRTP) -template -class BOOST_SYMBOL_VISIBLE common_iarchive : - public basic_iarchive, - public interface_iarchive -{ - friend class interface_iarchive; -private: - virtual void vload(version_type & t){ - * this->This() >> t; - } - virtual void vload(object_id_type & t){ - * this->This() >> t; - } - virtual void vload(class_id_type & t){ - * this->This() >> t; - } - virtual void vload(class_id_optional_type & t){ - * this->This() >> t; - } - virtual void vload(tracking_type & t){ - * this->This() >> t; - } - virtual void vload(class_name_type &s){ - * this->This() >> s; - } -protected: - // default processing - invoke serialization library - template - void load_override(T & t){ - archive::load(* this->This(), t); - } - // default implementations of functions which emit start/end tags for - // archive types that require them. - void load_start(const char * /*name*/){} - void load_end(const char * /*name*/){} - // default archive initialization - common_iarchive(unsigned int flags = 0) : - basic_iarchive(flags), - interface_iarchive() - {} -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_DETAIL_COMMON_IARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/common_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/common_oarchive.hpp deleted file mode 100644 index ee42bbe5976..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/common_oarchive.hpp +++ /dev/null @@ -1,88 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_COMMON_OARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_COMMON_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// common_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { -namespace detail { - -// note: referred to as Curiously Recurring Template Patter (CRTP) -template - -class BOOST_SYMBOL_VISIBLE common_oarchive : - public basic_oarchive, - public interface_oarchive -{ - friend class interface_oarchive; -private: - virtual void vsave(const version_type t){ - * this->This() << t; - } - virtual void vsave(const object_id_type t){ - * this->This() << t; - } - virtual void vsave(const object_reference_type t){ - * this->This() << t; - } - virtual void vsave(const class_id_type t){ - * this->This() << t; - } - virtual void vsave(const class_id_reference_type t){ - * this->This() << t; - } - virtual void vsave(const class_id_optional_type t){ - * this->This() << t; - } - virtual void vsave(const class_name_type & t){ - * this->This() << t; - } - virtual void vsave(const tracking_type t){ - * this->This() << t; - } -protected: - // default processing - invoke serialization library - template - void save_override(T & t){ - archive::save(* this->This(), t); - } - void save_start(const char * /*name*/){} - void save_end(const char * /*name*/){} - common_oarchive(unsigned int flags = 0) : - basic_oarchive(flags), - interface_oarchive() - {} -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_DETAIL_COMMON_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/decl.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/decl.hpp deleted file mode 100644 index 4f731cded37..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/decl.hpp +++ /dev/null @@ -1,57 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_DECL_HPP -#define BOOST_ARCHIVE_DETAIL_DECL_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2///////// 3/////////4/////////5/////////6/////////7/////////8 -// decl.hpp -// -// (c) Copyright Robert Ramey 2004 -// Use, modification, and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See library home page at http://www.boost.org/libs/serialization - -//----------------------------------------------------------------------------// - -// This header implements separate compilation features as described in -// http://www.boost.org/more/separate_compilation.html - -#include - -#if (defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK)) - #if defined(BOOST_ARCHIVE_SOURCE) - #define BOOST_ARCHIVE_DECL BOOST_SYMBOL_EXPORT - #else - #define BOOST_ARCHIVE_DECL BOOST_SYMBOL_IMPORT - #endif - - #if defined(BOOST_WARCHIVE_SOURCE) - #define BOOST_WARCHIVE_DECL BOOST_SYMBOL_EXPORT - #else - #define BOOST_WARCHIVE_DECL BOOST_SYMBOL_IMPORT - #endif - - #if defined(BOOST_WARCHIVE_SOURCE) || defined(BOOST_ARCHIVE_SOURCE) - #define BOOST_ARCHIVE_OR_WARCHIVE_DECL BOOST_SYMBOL_EXPORT - #else - #define BOOST_ARCHIVE_OR_WARCHIVE_DECL BOOST_SYMBOL_IMPORT - #endif - -#endif - -#if ! defined(BOOST_ARCHIVE_DECL) - #define BOOST_ARCHIVE_DECL -#endif -#if ! defined(BOOST_WARCHIVE_DECL) - #define BOOST_WARCHIVE_DECL -#endif -#if ! defined(BOOST_ARCHIVE_OR_WARCHIVE_DECL) - #define BOOST_ARCHIVE_OR_WARCHIVE_DECL -#endif - -#endif // BOOST_ARCHIVE_DETAIL_DECL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/helper_collection.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/helper_collection.hpp deleted file mode 100644 index edb4125e308..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/helper_collection.hpp +++ /dev/null @@ -1,99 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_HELPER_COLLECTION_HPP -#define BOOST_ARCHIVE_DETAIL_HELPER_COLLECTION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// helper_collection.hpp: archive support for run-time helpers - -// (C) Copyright 2002-2008 Robert Ramey and Joaquin M Lopez Munoz -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include -#include -#include -#include - -#include - -#include -#include - -namespace boost { - -namespace archive { -namespace detail { - -class helper_collection -{ - helper_collection(const helper_collection&); // non-copyable - helper_collection& operator = (const helper_collection&); // non-copyable - - // note: we dont' actually "share" the function object pointer - // we only use shared_ptr to make sure that it get's deleted - - typedef std::pair< - const void *, - boost::shared_ptr - > helper_value_type; - template - boost::shared_ptr make_helper_ptr(){ - // use boost::shared_ptr rather than std::shared_ptr to maintain - // c++03 compatibility - return boost::make_shared(); - } - - typedef std::vector collection; - collection m_collection; - - struct predicate { - BOOST_DELETED_FUNCTION(predicate & operator=(const predicate & rhs)) - public: - const void * const m_ti; - bool operator()(helper_value_type const &rhs) const { - return m_ti == rhs.first; - } - predicate(const void * ti) : - m_ti(ti) - {} - }; -protected: - helper_collection(){} - ~helper_collection(){} -public: - template - Helper& find_helper(void * const id = 0) { - collection::const_iterator it = - std::find_if( - m_collection.begin(), - m_collection.end(), - predicate(id) - ); - - void * rval = 0; - if(it == m_collection.end()){ - m_collection.push_back( - std::make_pair(id, make_helper_ptr()) - ); - rval = m_collection.back().second.get(); - } - else{ - rval = it->second.get(); - } - return *static_cast(rval); - } -}; - -} // namespace detail -} // namespace serialization -} // namespace boost - -#endif // BOOST_ARCHIVE_DETAIL_HELPER_COLLECTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_iarchive.hpp deleted file mode 100644 index 4a99e28b59f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_iarchive.hpp +++ /dev/null @@ -1,85 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_INTERFACE_IARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_INTERFACE_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// interface_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include // NULL -#include -#include -#include -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace archive { -namespace detail { - -class basic_pointer_iserializer; - -template -class interface_iarchive -{ -protected: - interface_iarchive(){}; -public: - ///////////////////////////////////////////////////////// - // archive public interface - typedef mpl::bool_ is_loading; - typedef mpl::bool_ is_saving; - - // return a pointer to the most derived class - Archive * This(){ - return static_cast(this); - } - - template - const basic_pointer_iserializer * - register_type(T * = NULL){ - const basic_pointer_iserializer & bpis = - boost::serialization::singleton< - pointer_iserializer - >::get_const_instance(); - this->This()->register_basic_serializer(bpis.get_basic_serializer()); - return & bpis; - } - template - Helper & - get_helper(void * const id = 0){ - helper_collection & hc = this->This()->get_helper_collection(); - return hc.template find_helper(id); - } - - template - Archive & operator>>(T & t){ - this->This()->load_override(t); - return * this->This(); - } - - // the & operator - template - Archive & operator&(T & t){ - return *(this->This()) >> t; - } -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_DETAIL_INTERFACE_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_oarchive.hpp deleted file mode 100644 index 359463ed9d8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/interface_oarchive.hpp +++ /dev/null @@ -1,87 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_INTERFACE_OARCHIVE_HPP -#define BOOST_ARCHIVE_DETAIL_INTERFACE_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// interface_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include // NULL -#include -#include - -#include -#include -#include // must be the last header - -#include - -namespace boost { -namespace archive { -namespace detail { - -class basic_pointer_oserializer; - -template -class interface_oarchive -{ -protected: - interface_oarchive(){}; -public: - ///////////////////////////////////////////////////////// - // archive public interface - typedef mpl::bool_ is_loading; - typedef mpl::bool_ is_saving; - - // return a pointer to the most derived class - Archive * This(){ - return static_cast(this); - } - - template - const basic_pointer_oserializer * - register_type(const T * = NULL){ - const basic_pointer_oserializer & bpos = - boost::serialization::singleton< - pointer_oserializer - >::get_const_instance(); - this->This()->register_basic_serializer(bpos.get_basic_serializer()); - return & bpos; - } - - template - Helper & - get_helper(void * const id = 0){ - helper_collection & hc = this->This()->get_helper_collection(); - return hc.template find_helper(id); - } - - template - Archive & operator<<(const T & t){ - this->This()->save_override(t); - return * this->This(); - } - - // the & operator - template - Archive & operator&(const T & t){ - return * this ->This() << t; - } -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_DETAIL_INTERFACE_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/iserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/iserializer.hpp deleted file mode 100644 index 193e98a82e4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/iserializer.hpp +++ /dev/null @@ -1,631 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_ISERIALIZER_HPP -#define BOOST_ARCHIVE_DETAIL_ISERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#pragma inline_depth(511) -#pragma inline_recursion(on) -#endif - -#if defined(__MWERKS__) -#pragma inline_depth(511) -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// iserializer.hpp: interface for serialization system. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // for placement new -#include // size_t, NULL - -#include -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include - -#include -#include -#include -#include -#include - -#ifndef BOOST_SERIALIZATION_DEFAULT_TYPE_INFO - #include -#endif -#include -#include -#include - -#include -#include -#include -#include -#include -#include - -#include - -#ifndef BOOST_MSVC - #define DONT_USE_HAS_NEW_OPERATOR ( \ - BOOST_WORKAROUND(__IBMCPP__, < 1210) \ - || defined(__SUNPRO_CC) && (__SUNPRO_CC < 0x590) \ - ) -#else - #define DONT_USE_HAS_NEW_OPERATOR 0 -#endif - -#if ! DONT_USE_HAS_NEW_OPERATOR -#include -#endif - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -// the following is need only for dynamic cast of polymorphic pointers -#include -#include -#include -#include -#include -#include - -namespace boost { - -namespace serialization { - class extended_type_info; -} // namespace serialization - -namespace archive { - -// an accessor to permit friend access to archives. Needed because -// some compilers don't handle friend templates completely -class load_access { -public: - template - static void load_primitive(Archive &ar, T &t){ - ar.load(t); - } -}; - -namespace detail { - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -template -class iserializer : public basic_iserializer -{ -private: - virtual void destroy(/*const*/ void *address) const { - boost::serialization::access::destroy(static_cast(address)); - } -protected: - // protected constructor since it's always created by singleton - explicit iserializer() : - basic_iserializer( - boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< T >::type - >::get_const_instance() - ) - {} -public: - virtual BOOST_DLLEXPORT void load_object_data( - basic_iarchive & ar, - void *x, - const unsigned int file_version - ) const BOOST_USED; - virtual bool class_info() const { - return boost::serialization::implementation_level< T >::value - >= boost::serialization::object_class_info; - } - virtual bool tracking(const unsigned int /* flags */) const { - return boost::serialization::tracking_level< T >::value - == boost::serialization::track_always - || ( boost::serialization::tracking_level< T >::value - == boost::serialization::track_selectively - && serialized_as_pointer()); - } - virtual version_type version() const { - return version_type(::boost::serialization::version< T >::value); - } - virtual bool is_polymorphic() const { - return boost::is_polymorphic< T >::value; - } - virtual ~iserializer(){}; -}; - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -template -BOOST_DLLEXPORT void iserializer::load_object_data( - basic_iarchive & ar, - void *x, - const unsigned int file_version -) const { - // note: we now comment this out. Before we permited archive - // version # to be very large. Now we don't. To permit - // readers of these old archives, we have to suppress this - // code. Perhaps in the future we might re-enable it but - // permit its suppression with a runtime switch. - #if 0 - // trap case where the program cannot handle the current version - if(file_version > static_cast(version())) - boost::serialization::throw_exception( - archive::archive_exception( - boost::archive::archive_exception::unsupported_class_version, - get_debug_info() - ) - ); - #endif - // make sure call is routed through the higest interface that might - // be specialized by the user. - boost::serialization::serialize_adl( - boost::serialization::smart_cast_reference(ar), - * static_cast(x), - file_version - ); -} - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -// the purpose of this code is to allocate memory for an object -// without requiring the constructor to be called. Presumably -// the allocated object will be subsequently initialized with -// "placement new". -// note: we have the boost type trait has_new_operator but we -// have no corresponding has_delete_operator. So we presume -// that the former being true would imply that the a delete -// operator is also defined for the class T. - -template -struct heap_allocation { - // boost::has_new_operator< T > doesn't work on these compilers - #if DONT_USE_HAS_NEW_OPERATOR - // This doesn't handle operator new overload for class T - static T * invoke_new(){ - return static_cast(operator new(sizeof(T))); - } - static void invoke_delete(T *t){ - (operator delete(t)); - } - #else - // note: we presume that a true value for has_new_operator - // implies the existence of a class specific delete operator as well - // as a class specific new operator. - struct has_new_operator { - static T * invoke_new() { - return static_cast((T::operator new)(sizeof(T))); - } - static void invoke_delete(T * t) { - // if compilation fails here, the likely cause that the class - // T has a class specific new operator but no class specific - // delete operator which matches the following signature. - // note that this solution addresses the issue that two - // possible signatures. But it doesn't address the possibility - // that the class might have class specific new with NO - // class specific delete at all. Patches (compatible with - // C++03) welcome! - delete t; - } - }; - struct doesnt_have_new_operator { - static T* invoke_new() { - return static_cast(operator new(sizeof(T))); - } - static void invoke_delete(T * t) { - // Note: I'm reliance upon automatic conversion from T * to void * here - delete t; - } - }; - static T * invoke_new() { - typedef typename - mpl::eval_if< - boost::has_new_operator< T >, - mpl::identity, - mpl::identity - >::type typex; - return typex::invoke_new(); - } - static void invoke_delete(T *t) { - typedef typename - mpl::eval_if< - boost::has_new_operator< T >, - mpl::identity, - mpl::identity - >::type typex; - typex::invoke_delete(t); - } - #endif - explicit heap_allocation(){ - m_p = invoke_new(); - } - ~heap_allocation(){ - if (0 != m_p) - invoke_delete(m_p); - } - T* get() const { - return m_p; - } - - T* release() { - T* p = m_p; - m_p = 0; - return p; - } -private: - T* m_p; -}; - -template -class pointer_iserializer : - public basic_pointer_iserializer -{ -private: - virtual void * heap_allocation() const { - detail::heap_allocation h; - T * t = h.get(); - h.release(); - return t; - } - virtual const basic_iserializer & get_basic_serializer() const { - return boost::serialization::singleton< - iserializer - >::get_const_instance(); - } - BOOST_DLLEXPORT virtual void load_object_ptr( - basic_iarchive & ar, - void * x, - const unsigned int file_version - ) const BOOST_USED; -protected: - // this should alway be a singleton so make the constructor protected - pointer_iserializer(); - ~pointer_iserializer(); -}; - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -// note: BOOST_DLLEXPORT is so that code for polymorphic class -// serialized only through base class won't get optimized out -template -BOOST_DLLEXPORT void pointer_iserializer::load_object_ptr( - basic_iarchive & ar, - void * t, - const unsigned int file_version -) const -{ - Archive & ar_impl = - boost::serialization::smart_cast_reference(ar); - - // note that the above will throw std::bad_alloc if the allocation - // fails so we don't have to address this contingency here. - - // catch exception during load_construct_data so that we don't - // automatically delete the t which is most likely not fully - // constructed - BOOST_TRY { - // this addresses an obscure situation that occurs when - // load_constructor de-serializes something through a pointer. - ar.next_object_pointer(t); - boost::serialization::load_construct_data_adl( - ar_impl, - static_cast(t), - file_version - ); - } - BOOST_CATCH(...){ - // if we get here the load_construct failed. The heap_allocation - // will be automatically deleted so we don't have to do anything - // special here. - BOOST_RETHROW; - } - BOOST_CATCH_END - - ar_impl >> boost::serialization::make_nvp(NULL, * static_cast(t)); -} - -template -pointer_iserializer::pointer_iserializer() : - basic_pointer_iserializer( - boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< T >::type - >::get_const_instance() - ) -{ - boost::serialization::singleton< - iserializer - >::get_mutable_instance().set_bpis(this); - archive_serializer_map::insert(this); -} - -template -pointer_iserializer::~pointer_iserializer(){ - archive_serializer_map::erase(this); -} - -template -struct load_non_pointer_type { - // note this bounces the call right back to the archive - // with no runtime overhead - struct load_primitive { - template - static void invoke(Archive & ar, T & t){ - load_access::load_primitive(ar, t); - } - }; - // note this bounces the call right back to the archive - // with no runtime overhead - struct load_only { - template - static void invoke(Archive & ar, const T & t){ - // short cut to user's serializer - // make sure call is routed through the higest interface that might - // be specialized by the user. - boost::serialization::serialize_adl( - ar, - const_cast(t), - boost::serialization::version< T >::value - ); - } - }; - - // note this save class information including version - // and serialization level to the archive - struct load_standard { - template - static void invoke(Archive &ar, const T & t){ - void * x = & const_cast(t); - ar.load_object( - x, - boost::serialization::singleton< - iserializer - >::get_const_instance() - ); - } - }; - - struct load_conditional { - template - static void invoke(Archive &ar, T &t){ - //if(0 == (ar.get_flags() & no_tracking)) - load_standard::invoke(ar, t); - //else - // load_only::invoke(ar, t); - } - }; - - template - static void invoke(Archive & ar, T &t){ - typedef typename mpl::eval_if< - // if its primitive - mpl::equal_to< - boost::serialization::implementation_level< T >, - mpl::int_ - >, - mpl::identity, - // else - typename mpl::eval_if< - // class info / version - mpl::greater_equal< - boost::serialization::implementation_level< T >, - mpl::int_ - >, - // do standard load - mpl::identity, - // else - typename mpl::eval_if< - // no tracking - mpl::equal_to< - boost::serialization::tracking_level< T >, - mpl::int_ - >, - // do a fast load - mpl::identity, - // else - // do a fast load only tracking is turned off - mpl::identity - > > >::type typex; - check_object_versioning< T >(); - check_object_level< T >(); - typex::invoke(ar, t); - } -}; - -template -struct load_pointer_type { - struct abstract - { - template - static const basic_pointer_iserializer * register_type(Archive & /* ar */){ - // it has? to be polymorphic - BOOST_STATIC_ASSERT(boost::is_polymorphic< T >::value); - return static_cast(NULL); - } - }; - - struct non_abstract - { - template - static const basic_pointer_iserializer * register_type(Archive & ar){ - return ar.register_type(static_cast(NULL)); - } - }; - - template - static const basic_pointer_iserializer * register_type(Archive &ar, const T & /*t*/){ - // there should never be any need to load an abstract polymorphic - // class pointer. Inhibiting code generation for this - // permits abstract base classes to be used - note: exception - // virtual serialize functions used for plug-ins - typedef typename - mpl::eval_if< - boost::serialization::is_abstract, - boost::mpl::identity, - boost::mpl::identity - >::type typex; - return typex::template register_type< T >(ar); - } - - template - static T * pointer_tweak( - const boost::serialization::extended_type_info & eti, - void const * const t, - const T & - ) { - // tweak the pointer back to the base class - void * upcast = const_cast( - boost::serialization::void_upcast( - eti, - boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< T >::type - >::get_const_instance(), - t - ) - ); - if(NULL == upcast) - boost::serialization::throw_exception( - archive_exception(archive_exception::unregistered_class) - ); - return static_cast(upcast); - } - - template - static void check_load(T & /* t */){ - check_pointer_level< T >(); - check_pointer_tracking< T >(); - } - - static const basic_pointer_iserializer * - find(const boost::serialization::extended_type_info & type){ - return static_cast( - archive_serializer_map::find(type) - ); - } - - template - static void invoke(Archive & ar, Tptr & t){ - check_load(*t); - const basic_pointer_iserializer * bpis_ptr = register_type(ar, *t); - const basic_pointer_iserializer * newbpis_ptr = ar.load_pointer( - // note major hack here !!! - // I tried every way to convert Tptr &t (where Tptr might - // include const) to void * &. This is the only way - // I could make it work. RR - (void * & )t, - bpis_ptr, - find - ); - // if the pointer isn't that of the base class - if(newbpis_ptr != bpis_ptr){ - t = pointer_tweak(newbpis_ptr->get_eti(), t, *t); - } - } -}; - -template -struct load_enum_type { - template - static void invoke(Archive &ar, T &t){ - // convert integers to correct enum to load - int i; - ar >> boost::serialization::make_nvp(NULL, i); - t = static_cast< T >(i); - } -}; - -template -struct load_array_type { - template - static void invoke(Archive &ar, T &t){ - typedef typename remove_extent< T >::type value_type; - - // convert integers to correct enum to load - // determine number of elements in the array. Consider the - // fact that some machines will align elements on boundries - // other than characters. - std::size_t current_count = sizeof(t) / ( - static_cast(static_cast(&t[1])) - - static_cast(static_cast(&t[0])) - ); - boost::serialization::collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(static_cast(count) > current_count) - boost::serialization::throw_exception( - archive::archive_exception( - boost::archive::archive_exception::array_size_too_short - ) - ); - // explict template arguments to pass intel C++ compiler - ar >> serialization::make_array< - value_type, - boost::serialization::collection_size_type - >( - static_cast(&t[0]), - count - ); - } -}; - -} // detail - -template -inline void load(Archive & ar, T &t){ - // if this assertion trips. It means we're trying to load a - // const object with a compiler that doesn't have correct - // function template ordering. On other compilers, this is - // handled below. - detail::check_const_loading< T >(); - typedef - typename mpl::eval_if, - mpl::identity > - ,//else - typename mpl::eval_if, - mpl::identity > - ,//else - typename mpl::eval_if, - mpl::identity > - ,//else - mpl::identity > - > - > - >::type typex; - typex::invoke(ar, t); -} - -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_DETAIL_ISERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/oserializer.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/oserializer.hpp deleted file mode 100644 index c120ec55073..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/oserializer.hpp +++ /dev/null @@ -1,540 +0,0 @@ -#ifndef BOOST_ARCHIVE_OSERIALIZER_HPP -#define BOOST_ARCHIVE_OSERIALIZER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#pragma inline_depth(511) -#pragma inline_recursion(on) -#endif - -#if defined(__MWERKS__) -#pragma inline_depth(511) -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// oserializer.hpp: interface for serialization system. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL - -#include -#include -#include - -#include -#include -#include -#include -#include - -#ifndef BOOST_SERIALIZATION_DEFAULT_TYPE_INFO - #include -#endif -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -#include -#include -#include -#include -#include -#include - -namespace boost { - -namespace serialization { - class extended_type_info; -} // namespace serialization - -namespace archive { - -// an accessor to permit friend access to archives. Needed because -// some compilers don't handle friend templates completely -class save_access { -public: - template - static void end_preamble(Archive & ar){ - ar.end_preamble(); - } - template - static void save_primitive(Archive & ar, const T & t){ - ar.end_preamble(); - ar.save(t); - } -}; - -namespace detail { - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -template -class oserializer : public basic_oserializer -{ -private: - // private constructor to inhibit any existence other than the - // static one -public: - explicit BOOST_DLLEXPORT oserializer() : - basic_oserializer( - boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< T >::type - >::get_const_instance() - ) - {} - virtual BOOST_DLLEXPORT void save_object_data( - basic_oarchive & ar, - const void *x - ) const BOOST_USED; - virtual bool class_info() const { - return boost::serialization::implementation_level< T >::value - >= boost::serialization::object_class_info; - } - virtual bool tracking(const unsigned int /* flags */) const { - return boost::serialization::tracking_level< T >::value == boost::serialization::track_always - || (boost::serialization::tracking_level< T >::value == boost::serialization::track_selectively - && serialized_as_pointer()); - } - virtual version_type version() const { - return version_type(::boost::serialization::version< T >::value); - } - virtual bool is_polymorphic() const { - return boost::is_polymorphic< T >::value; - } - virtual ~oserializer(){} -}; - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -template -BOOST_DLLEXPORT void oserializer::save_object_data( - basic_oarchive & ar, - const void *x -) const { - // make sure call is routed through the highest interface that might - // be specialized by the user. - BOOST_STATIC_ASSERT(boost::is_const< T >::value == false); - boost::serialization::serialize_adl( - boost::serialization::smart_cast_reference(ar), - * static_cast(const_cast(x)), - version() - ); -} - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -template -class pointer_oserializer : - public basic_pointer_oserializer -{ -private: - const basic_oserializer & - get_basic_serializer() const { - return boost::serialization::singleton< - oserializer - >::get_const_instance(); - } - virtual BOOST_DLLEXPORT void save_object_ptr( - basic_oarchive & ar, - const void * x - ) const BOOST_USED; -public: - pointer_oserializer(); - ~pointer_oserializer(); -}; - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -template -BOOST_DLLEXPORT void pointer_oserializer::save_object_ptr( - basic_oarchive & ar, - const void * x -) const { - BOOST_ASSERT(NULL != x); - // make sure call is routed through the highest interface that might - // be specialized by the user. - T * t = static_cast(const_cast(x)); - const unsigned int file_version = boost::serialization::version< T >::value; - Archive & ar_impl - = boost::serialization::smart_cast_reference(ar); - boost::serialization::save_construct_data_adl( - ar_impl, - t, - file_version - ); - ar_impl << boost::serialization::make_nvp(NULL, * t); -} - -template -pointer_oserializer::pointer_oserializer() : - basic_pointer_oserializer( - boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< T >::type - >::get_const_instance() - ) -{ - // make sure appropriate member function is instantiated - boost::serialization::singleton< - oserializer - >::get_mutable_instance().set_bpos(this); - archive_serializer_map::insert(this); -} - -template -pointer_oserializer::~pointer_oserializer(){ - archive_serializer_map::erase(this); -} - -template -struct save_non_pointer_type { - // note this bounces the call right back to the archive - // with no runtime overhead - struct save_primitive { - template - static void invoke(Archive & ar, const T & t){ - save_access::save_primitive(ar, t); - } - }; - // same as above but passes through serialization - struct save_only { - template - static void invoke(Archive & ar, const T & t){ - // make sure call is routed through the highest interface that might - // be specialized by the user. - boost::serialization::serialize_adl( - ar, - const_cast(t), - ::boost::serialization::version< T >::value - ); - } - }; - // adds class information to the archive. This includes - // serialization level and class version - struct save_standard { - template - static void invoke(Archive &ar, const T & t){ - ar.save_object( - & t, - boost::serialization::singleton< - oserializer - >::get_const_instance() - ); - } - }; - - // adds class information to the archive. This includes - // serialization level and class version - struct save_conditional { - template - static void invoke(Archive &ar, const T &t){ - //if(0 == (ar.get_flags() & no_tracking)) - save_standard::invoke(ar, t); - //else - // save_only::invoke(ar, t); - } - }; - - - template - static void invoke(Archive & ar, const T & t){ - typedef - typename mpl::eval_if< - // if its primitive - mpl::equal_to< - boost::serialization::implementation_level< T >, - mpl::int_ - >, - mpl::identity, - // else - typename mpl::eval_if< - // class info / version - mpl::greater_equal< - boost::serialization::implementation_level< T >, - mpl::int_ - >, - // do standard save - mpl::identity, - // else - typename mpl::eval_if< - // no tracking - mpl::equal_to< - boost::serialization::tracking_level< T >, - mpl::int_ - >, - // do a fast save - mpl::identity, - // else - // do a fast save only tracking is turned off - mpl::identity - > > >::type typex; - check_object_versioning< T >(); - typex::invoke(ar, t); - } - template - static void invoke(Archive & ar, T & t){ - check_object_level< T >(); - check_object_tracking< T >(); - invoke(ar, const_cast(t)); - } -}; - -template -struct save_pointer_type { - struct abstract - { - template - static const basic_pointer_oserializer * register_type(Archive & /* ar */){ - // it has? to be polymorphic - BOOST_STATIC_ASSERT(boost::is_polymorphic< T >::value); - return NULL; - } - }; - - struct non_abstract - { - template - static const basic_pointer_oserializer * register_type(Archive & ar){ - return ar.register_type(static_cast(NULL)); - } - }; - - template - static const basic_pointer_oserializer * register_type(Archive &ar, T & /*t*/){ - // there should never be any need to save an abstract polymorphic - // class pointer. Inhibiting code generation for this - // permits abstract base classes to be used - note: exception - // virtual serialize functions used for plug-ins - typedef - typename mpl::eval_if< - boost::serialization::is_abstract< T >, - mpl::identity, - mpl::identity - >::type typex; - return typex::template register_type< T >(ar); - } - - struct non_polymorphic - { - template - static void save( - Archive &ar, - T & t - ){ - const basic_pointer_oserializer & bpos = - boost::serialization::singleton< - pointer_oserializer - >::get_const_instance(); - // save the requested pointer type - ar.save_pointer(& t, & bpos); - } - }; - - struct polymorphic - { - template - static void save( - Archive &ar, - T & t - ){ - typename - boost::serialization::type_info_implementation< T >::type const - & i = boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< T >::type - >::get_const_instance(); - - boost::serialization::extended_type_info const * const this_type = & i; - - // retrieve the true type of the object pointed to - // if this assertion fails its an error in this library - BOOST_ASSERT(NULL != this_type); - - const boost::serialization::extended_type_info * true_type = - i.get_derived_extended_type_info(t); - - // note:if this exception is thrown, be sure that derived pointer - // is either registered or exported. - if(NULL == true_type){ - boost::serialization::throw_exception( - archive_exception( - archive_exception::unregistered_class, - "derived class not registered or exported" - ) - ); - } - - // if its not a pointer to a more derived type - const void *vp = static_cast(&t); - if(*this_type == *true_type){ - const basic_pointer_oserializer * bpos = register_type(ar, t); - ar.save_pointer(vp, bpos); - return; - } - // convert pointer to more derived type. if this is thrown - // it means that the base/derived relationship hasn't be registered - vp = serialization::void_downcast( - *true_type, - *this_type, - static_cast(&t) - ); - if(NULL == vp){ - boost::serialization::throw_exception( - archive_exception( - archive_exception::unregistered_cast, - true_type->get_debug_info(), - this_type->get_debug_info() - ) - ); - } - - // since true_type is valid, and this only gets made if the - // pointer oserializer object has been created, this should never - // fail - const basic_pointer_oserializer * bpos - = static_cast( - boost::serialization::singleton< - archive_serializer_map - >::get_const_instance().find(*true_type) - ); - BOOST_ASSERT(NULL != bpos); - if(NULL == bpos) - boost::serialization::throw_exception( - archive_exception( - archive_exception::unregistered_class, - "derived class not registered or exported" - ) - ); - ar.save_pointer(vp, bpos); - } - }; - - template - static void save( - Archive & ar, - const T & t - ){ - check_pointer_level< T >(); - check_pointer_tracking< T >(); - typedef typename mpl::eval_if< - is_polymorphic< T >, - mpl::identity, - mpl::identity - >::type type; - type::save(ar, const_cast(t)); - } - - template - static void invoke(Archive &ar, const TPtr t){ - register_type(ar, * t); - if(NULL == t){ - basic_oarchive & boa - = boost::serialization::smart_cast_reference(ar); - boa.save_null_pointer(); - save_access::end_preamble(ar); - return; - } - save(ar, * t); - } -}; - -template -struct save_enum_type -{ - template - static void invoke(Archive &ar, const T &t){ - // convert enum to integers on save - const int i = static_cast(t); - ar << boost::serialization::make_nvp(NULL, i); - } -}; - -template -struct save_array_type -{ - template - static void invoke(Archive &ar, const T &t){ - typedef typename boost::remove_extent< T >::type value_type; - - save_access::end_preamble(ar); - // consider alignment - std::size_t c = sizeof(t) / ( - static_cast(static_cast(&t[1])) - - static_cast(static_cast(&t[0])) - ); - boost::serialization::collection_size_type count(c); - ar << BOOST_SERIALIZATION_NVP(count); - // explict template arguments to pass intel C++ compiler - ar << serialization::make_array< - const value_type, - boost::serialization::collection_size_type - >( - static_cast(&t[0]), - count - ); - } -}; - -} // detail - -template -inline void save(Archive & ar, /*const*/ T &t){ - typedef - typename mpl::eval_if, - mpl::identity >, - //else - typename mpl::eval_if, - mpl::identity >, - //else - typename mpl::eval_if, - mpl::identity >, - //else - mpl::identity > - > - > - >::type typex; - typex::invoke(ar, t); -} - -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_OSERIALIZER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_iarchive_route.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_iarchive_route.hpp deleted file mode 100644 index 105685ebbd8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_iarchive_route.hpp +++ /dev/null @@ -1,218 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_POLYMORPHIC_IARCHIVE_ROUTE_HPP -#define BOOST_ARCHIVE_DETAIL_POLYMORPHIC_IARCHIVE_ROUTE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_iarchive_route.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization -namespace archive { -namespace detail{ - -class basic_iserializer; -class basic_pointer_iserializer; - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -template -class polymorphic_iarchive_route : - public polymorphic_iarchive, - // note: gcc dynamic cross cast fails if the the derivation below is - // not public. I think this is a mistake. - public /*protected*/ ArchiveImplementation -{ -private: - // these are used by the serialization library. - virtual void load_object( - void *t, - const basic_iserializer & bis - ){ - ArchiveImplementation::load_object(t, bis); - } - virtual const basic_pointer_iserializer * load_pointer( - void * & t, - const basic_pointer_iserializer * bpis_ptr, - const basic_pointer_iserializer * (*finder)( - const boost::serialization::extended_type_info & type - ) - ){ - return ArchiveImplementation::load_pointer(t, bpis_ptr, finder); - } - virtual void set_library_version(library_version_type archive_library_version){ - ArchiveImplementation::set_library_version(archive_library_version); - } - virtual library_version_type get_library_version() const{ - return ArchiveImplementation::get_library_version(); - } - virtual unsigned int get_flags() const { - return ArchiveImplementation::get_flags(); - } - virtual void delete_created_pointers(){ - ArchiveImplementation::delete_created_pointers(); - } - virtual void reset_object_address( - const void * new_address, - const void * old_address - ){ - ArchiveImplementation::reset_object_address(new_address, old_address); - } - virtual void load_binary(void * t, std::size_t size){ - ArchiveImplementation::load_binary(t, size); - } - // primitive types the only ones permitted by polymorphic archives - virtual void load(bool & t){ - ArchiveImplementation::load(t); - } - virtual void load(char & t){ - ArchiveImplementation::load(t); - } - virtual void load(signed char & t){ - ArchiveImplementation::load(t); - } - virtual void load(unsigned char & t){ - ArchiveImplementation::load(t); - } - #ifndef BOOST_NO_CWCHAR - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - virtual void load(wchar_t & t){ - ArchiveImplementation::load(t); - } - #endif - #endif - virtual void load(short & t){ - ArchiveImplementation::load(t); - } - virtual void load(unsigned short & t){ - ArchiveImplementation::load(t); - } - virtual void load(int & t){ - ArchiveImplementation::load(t); - } - virtual void load(unsigned int & t){ - ArchiveImplementation::load(t); - } - virtual void load(long & t){ - ArchiveImplementation::load(t); - } - virtual void load(unsigned long & t){ - ArchiveImplementation::load(t); - } - #if defined(BOOST_HAS_LONG_LONG) - virtual void load(boost::long_long_type & t){ - ArchiveImplementation::load(t); - } - virtual void load(boost::ulong_long_type & t){ - ArchiveImplementation::load(t); - } - #elif defined(BOOST_HAS_MS_INT64) - virtual void load(__int64 & t){ - ArchiveImplementation::load(t); - } - virtual void load(unsigned __int64 & t){ - ArchiveImplementation::load(t); - } - #endif - virtual void load(float & t){ - ArchiveImplementation::load(t); - } - virtual void load(double & t){ - ArchiveImplementation::load(t); - } - virtual void load(std::string & t){ - ArchiveImplementation::load(t); - } - #ifndef BOOST_NO_STD_WSTRING - virtual void load(std::wstring & t){ - ArchiveImplementation::load(t); - } - #endif - // used for xml and other tagged formats default does nothing - virtual void load_start(const char * name){ - ArchiveImplementation::load_start(name); - } - virtual void load_end(const char * name){ - ArchiveImplementation::load_end(name); - } - virtual void register_basic_serializer(const basic_iserializer & bis){ - ArchiveImplementation::register_basic_serializer(bis); - } - virtual helper_collection & - get_helper_collection(){ - return ArchiveImplementation::get_helper_collection(); - } -public: - // this can't be inheriteded because they appear in mulitple - // parents - typedef mpl::bool_ is_loading; - typedef mpl::bool_ is_saving; - // the >> operator - template - polymorphic_iarchive & operator>>(T & t){ - return polymorphic_iarchive::operator>>(t); - } - // the & operator - template - polymorphic_iarchive & operator&(T & t){ - return polymorphic_iarchive::operator&(t); - } - // register type function - template - const basic_pointer_iserializer * - register_type(T * t = NULL){ - return ArchiveImplementation::register_type(t); - } - // all current archives take a stream as constructor argument - template - polymorphic_iarchive_route( - std::basic_istream<_Elem, _Tr> & is, - unsigned int flags = 0 - ) : - ArchiveImplementation(is, flags) - {} - virtual ~polymorphic_iarchive_route(){}; -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_DETAIL_POLYMORPHIC_IARCHIVE_DISPATCH_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_oarchive_route.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_oarchive_route.hpp deleted file mode 100644 index b23fd6bf39d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/polymorphic_oarchive_route.hpp +++ /dev/null @@ -1,209 +0,0 @@ -#ifndef BOOST_ARCHIVE_DETAIL_POLYMORPHIC_OARCHIVE_ROUTE_HPP -#define BOOST_ARCHIVE_DETAIL_POLYMORPHIC_OARCHIVE_ROUTE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_oarchive_route.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include // size_t - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization -namespace archive { -namespace detail{ - -class basic_oserializer; -class basic_pointer_oserializer; - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -template -class polymorphic_oarchive_route : - public polymorphic_oarchive, - // note: gcc dynamic cross cast fails if the the derivation below is - // not public. I think this is a mistake. - public /*protected*/ ArchiveImplementation -{ -private: - // these are used by the serialization library. - virtual void save_object( - const void *x, - const detail::basic_oserializer & bos - ){ - ArchiveImplementation::save_object(x, bos); - } - virtual void save_pointer( - const void * t, - const detail::basic_pointer_oserializer * bpos_ptr - ){ - ArchiveImplementation::save_pointer(t, bpos_ptr); - } - virtual void save_null_pointer(){ - ArchiveImplementation::save_null_pointer(); - } - // primitive types the only ones permitted by polymorphic archives - virtual void save(const bool t){ - ArchiveImplementation::save(t); - } - virtual void save(const char t){ - ArchiveImplementation::save(t); - } - virtual void save(const signed char t){ - ArchiveImplementation::save(t); - } - virtual void save(const unsigned char t){ - ArchiveImplementation::save(t); - } - #ifndef BOOST_NO_CWCHAR - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - virtual void save(const wchar_t t){ - ArchiveImplementation::save(t); - } - #endif - #endif - virtual void save(const short t){ - ArchiveImplementation::save(t); - } - virtual void save(const unsigned short t){ - ArchiveImplementation::save(t); - } - virtual void save(const int t){ - ArchiveImplementation::save(t); - } - virtual void save(const unsigned int t){ - ArchiveImplementation::save(t); - } - virtual void save(const long t){ - ArchiveImplementation::save(t); - } - virtual void save(const unsigned long t){ - ArchiveImplementation::save(t); - } - #if defined(BOOST_HAS_LONG_LONG) - virtual void save(const boost::long_long_type t){ - ArchiveImplementation::save(t); - } - virtual void save(const boost::ulong_long_type t){ - ArchiveImplementation::save(t); - } - #elif defined(BOOST_HAS_MS_INT64) - virtual void save(const boost::int64_t t){ - ArchiveImplementation::save(t); - } - virtual void save(const boost::uint64_t t){ - ArchiveImplementation::save(t); - } - #endif - virtual void save(const float t){ - ArchiveImplementation::save(t); - } - virtual void save(const double t){ - ArchiveImplementation::save(t); - } - virtual void save(const std::string & t){ - ArchiveImplementation::save(t); - } - #ifndef BOOST_NO_STD_WSTRING - virtual void save(const std::wstring & t){ - ArchiveImplementation::save(t); - } - #endif - virtual library_version_type get_library_version() const{ - return ArchiveImplementation::get_library_version(); - } - virtual unsigned int get_flags() const { - return ArchiveImplementation::get_flags(); - } - virtual void save_binary(const void * t, std::size_t size){ - ArchiveImplementation::save_binary(t, size); - } - // used for xml and other tagged formats default does nothing - virtual void save_start(const char * name){ - ArchiveImplementation::save_start(name); - } - virtual void save_end(const char * name){ - ArchiveImplementation::save_end(name); - } - virtual void end_preamble(){ - ArchiveImplementation::end_preamble(); - } - virtual void register_basic_serializer(const detail::basic_oserializer & bos){ - ArchiveImplementation::register_basic_serializer(bos); - } - virtual helper_collection & - get_helper_collection(){ - return ArchiveImplementation::get_helper_collection(); - } -public: - // this can't be inheriteded because they appear in mulitple - // parents - typedef mpl::bool_ is_loading; - typedef mpl::bool_ is_saving; - // the << operator - template - polymorphic_oarchive & operator<<(T & t){ - return polymorphic_oarchive::operator<<(t); - } - // the & operator - template - polymorphic_oarchive & operator&(T & t){ - return polymorphic_oarchive::operator&(t); - } - // register type function - template - const basic_pointer_oserializer * - register_type(T * t = NULL){ - return ArchiveImplementation::register_type(t); - } - // all current archives take a stream as constructor argument - template - polymorphic_oarchive_route( - std::basic_ostream<_Elem, _Tr> & os, - unsigned int flags = 0 - ) : - ArchiveImplementation(os, flags) - {} - virtual ~polymorphic_oarchive_route(){}; -}; - -} // namespace detail -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_DETAIL_POLYMORPHIC_OARCHIVE_DISPATCH_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/register_archive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/register_archive.hpp deleted file mode 100644 index 5ffecc702ce..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/register_archive.hpp +++ /dev/null @@ -1,91 +0,0 @@ -// Copyright David Abrahams 2006. Distributed under the Boost -// Software License, Version 1.0. (See accompanying -// file LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) -#ifndef BOOST_ARCHIVE_DETAIL_REGISTER_ARCHIVE_DWA2006521_HPP -# define BOOST_ARCHIVE_DETAIL_REGISTER_ARCHIVE_DWA2006521_HPP - -namespace boost { namespace archive { namespace detail { - -// No instantiate_ptr_serialization overloads generated by -// BOOST_SERIALIZATION_REGISTER_ARCHIVE that lexically follow the call -// will be seen *unless* they are in an associated namespace of one of -// the arguments, so we pass one of these along to make sure this -// namespace is considered. See temp.dep.candidate (14.6.4.2) in the -// standard. -struct adl_tag {}; - -template -struct ptr_serialization_support; - -// We could've just used ptr_serialization_support, above, but using -// it with only a forward declaration causes vc6/7 to complain about a -// missing instantiate member, even if it has one. This is just a -// friendly layer of indirection. -template -struct _ptr_serialization_support - : ptr_serialization_support -{ - typedef int type; -}; - -#if defined(__SUNPRO_CC) && (__SUNPRO_CC < 0x5130) - -template -struct counter : counter {}; -template<> -struct counter<0> {}; - -template -void instantiate_ptr_serialization(Serializable* s, int, adl_tag) { - instantiate_ptr_serialization(s, counter<20>()); -} - -template -struct get_counter { - static const int value = sizeof(adjust_counter(counter<20>())); - typedef counter type; - typedef counter prior; - typedef char (&next)[value+1]; -}; - -char adjust_counter(counter<0>); -template -void instantiate_ptr_serialization(Serializable*, counter<0>) {} - -#define BOOST_SERIALIZATION_REGISTER_ARCHIVE(Archive) \ -namespace boost { namespace archive { namespace detail { \ - get_counter::next adjust_counter(get_counter::type);\ - template \ - void instantiate_ptr_serialization(Serializable* s, \ - get_counter::type) { \ - ptr_serialization_support x; \ - instantiate_ptr_serialization(s, get_counter::prior()); \ - }\ -}}} - - -#else - -// This function gets called, but its only purpose is to participate -// in overload resolution with the functions declared by -// BOOST_SERIALIZATION_REGISTER_ARCHIVE, below. -template -void instantiate_ptr_serialization(Serializable*, int, adl_tag ) {} - -// The function declaration generated by this macro never actually -// gets called, but its return type gets instantiated, and that's -// enough to cause registration of serialization functions between -// Archive and any exported Serializable type. See also: -// boost/serialization/export.hpp -# define BOOST_SERIALIZATION_REGISTER_ARCHIVE(Archive) \ -namespace boost { namespace archive { namespace detail { \ - \ -template \ -typename _ptr_serialization_support::type \ -instantiate_ptr_serialization( Serializable*, Archive*, adl_tag ); \ - \ -}}} -#endif -}}} // namespace boost::archive::detail - -#endif // BOOST_ARCHIVE_DETAIL_INSTANTIATE_SERIALIZE_DWA2006521_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/detail/utf8_codecvt_facet.hpp b/contrib/libboost/boost_1_65_0/boost/archive/detail/utf8_codecvt_facet.hpp deleted file mode 100644 index a40104abea6..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/detail/utf8_codecvt_facet.hpp +++ /dev/null @@ -1,39 +0,0 @@ -// Copyright (c) 2001 Ronald Garcia, Indiana University (garcia@osl.iu.edu) -// Andrew Lumsdaine, Indiana University (lums@osl.iu.edu). -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#ifndef BOOST_ARCHIVE_DETAIL_UTF8_CODECVT_FACET_HPP -#define BOOST_ARCHIVE_DETAIL_UTF8_CODECVT_FACET_HPP - -#include - -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#endif - -// std::codecvt_utf8 doesn't seem to work for any versions of msvc - -#if defined(_MSC_VER) || defined(BOOST_NO_CXX11_HDR_CODECVT) - // use boost's utf8 codecvt facet - #include - #define BOOST_UTF8_BEGIN_NAMESPACE \ - namespace boost { namespace archive { namespace detail { - #define BOOST_UTF8_DECL BOOST_ARCHIVE_DECL - #define BOOST_UTF8_END_NAMESPACE }}} - - #include - - #undef BOOST_UTF8_END_NAMESPACE - #undef BOOST_UTF8_DECL - #undef BOOST_UTF8_BEGIN_NAMESPACE -#else - // use the standard vendor supplied facet - #include - namespace boost { namespace archive { namespace detail { - typedef std::codecvt_utf8 utf8_codecvt_facet; - } } } -#endif - -#endif // BOOST_ARCHIVE_DETAIL_UTF8_CODECVT_FACET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/dinkumware.hpp b/contrib/libboost/boost_1_65_0/boost/archive/dinkumware.hpp deleted file mode 100644 index 90ba6271cdd..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/dinkumware.hpp +++ /dev/null @@ -1,224 +0,0 @@ -#ifndef BOOST_ARCHIVE_DINKUMWARE_HPP -#define BOOST_ARCHIVE_DINKUMWARE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// dinkumware.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// this file adds a couple of things that are missing from the dinkumware -// implementation of the standard library. - -#include -#include - -#include -#include - -namespace std { - -// define i/o operators for 64 bit integers -template -basic_ostream & -operator<<(basic_ostream & os, boost::uint64_t t){ - // octal rendering of 64 bit number would be 22 octets + eos - CharType d[23]; - unsigned int radix; - - if(os.flags() & (int)std::ios_base::hex) - radix = 16; - else - if(os.flags() & (int)std::ios_base::oct) - radix = 8; - else - //if(s.flags() & (int)std::ios_base::dec) - radix = 10; - unsigned int i = 0; - do{ - unsigned int j = t % radix; - d[i++] = j + ((j < 10) ? '0' : ('a' - 10)); - t /= radix; - } - while(t > 0); - d[i--] = '\0'; - - // reverse digits - unsigned int j = 0; - while(j < i){ - CharType k = d[i]; - d[i] = d[j]; - d[j] = k; - --i;++j; - } - os << d; - return os; - -} - -template -basic_ostream & -operator<<(basic_ostream &os, boost::int64_t t){ - if(0 <= t){ - os << static_cast(t); - } - else{ - os.put('-'); - os << -t; - } - return os; -} - -template -basic_istream & -operator>>(basic_istream &is, boost::int64_t & t){ - CharType d; - do{ - d = is.get(); - } - while(::isspace(d)); - bool negative = (d == '-'); - if(negative) - d = is.get(); - unsigned int radix; - if(is.flags() & (int)std::ios_base::hex) - radix = 16; - else - if(is.flags() & (int)std::ios_base::oct) - radix = 8; - else - //if(s.flags() & (int)std::ios_base::dec) - radix = 10; - t = 0; - do{ - if('0' <= d && d <= '9') - t = t * radix + (d - '0'); - else - if('a' <= d && d <= 'f') - t = t * radix + (d - 'a' + 10); - else - break; - d = is.get(); - } - while(!is.fail()); - // restore the delimiter - is.putback(d); - is.clear(); - if(negative) - t = -t; - return is; -} - -template -basic_istream & -operator>>(basic_istream &is, boost::uint64_t & t){ - boost::int64_t it; - is >> it; - t = it; - return is; -} - -//#endif - -template<> -class back_insert_iterator > : public - iterator -{ -public: - typedef basic_string container_type; - typedef container_type::reference reference; - - explicit back_insert_iterator(container_type & s) - : container(& s) - {} // construct with container - - back_insert_iterator & operator=( - container_type::const_reference Val_ - ){ // push value into container - //container->push_back(Val_); - *container += Val_; - return (*this); - } - - back_insert_iterator & operator*(){ - return (*this); - } - - back_insert_iterator & operator++(){ - // pretend to preincrement - return (*this); - } - - back_insert_iterator operator++(int){ - // pretend to postincrement - return (*this); - } - -protected: - container_type *container; // pointer to container -}; - -template -inline back_insert_iterator > back_inserter( - basic_string & s -){ - return (std::back_insert_iterator >(s)); -} - -template<> -class back_insert_iterator > : public - iterator -{ -public: - typedef basic_string container_type; - typedef container_type::reference reference; - - explicit back_insert_iterator(container_type & s) - : container(& s) - {} // construct with container - - back_insert_iterator & operator=( - container_type::const_reference Val_ - ){ // push value into container - //container->push_back(Val_); - *container += Val_; - return (*this); - } - - back_insert_iterator & operator*(){ - return (*this); - } - - back_insert_iterator & operator++(){ - // pretend to preincrement - return (*this); - } - - back_insert_iterator operator++(int){ - // pretend to postincrement - return (*this); - } - -protected: - container_type *container; // pointer to container -}; - -template -inline back_insert_iterator > back_inserter( - basic_string & s -){ - return (std::back_insert_iterator >(s)); -} - -} // namespace std - -#endif //BOOST_ARCHIVE_DINKUMWARE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/archive_serializer_map.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/archive_serializer_map.ipp deleted file mode 100644 index 7f163ec4076..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/archive_serializer_map.ipp +++ /dev/null @@ -1,75 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// archive_serializer_map.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -////////////////////////////////////////////////////////////////////// -// implementation of basic_text_iprimitive overrides for the combination -// of template parameters used to implement a text_iprimitive - -#include -#include -#include -#include - -namespace boost { -namespace archive { -namespace detail { - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace extra_detail { // anon - template - class map : public basic_serializer_map - {}; -} - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL bool -archive_serializer_map::insert(const basic_serializer * bs){ - return boost::serialization::singleton< - extra_detail::map - >::get_mutable_instance().insert(bs); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -archive_serializer_map::erase(const basic_serializer * bs){ - BOOST_ASSERT(! boost::serialization::singleton< - extra_detail::map - >::is_destroyed() - ); - if(boost::serialization::singleton< - extra_detail::map - >::is_destroyed()) - return; - boost::serialization::singleton< - extra_detail::map - >::get_mutable_instance().erase(bs); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL const basic_serializer * -archive_serializer_map::find( - const boost::serialization::extended_type_info & eti -) { - return boost::serialization::singleton< - extra_detail::map - >::get_const_instance().find(eti); -} - -} // namespace detail -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iarchive.ipp deleted file mode 100644 index d5619ab6cf3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iarchive.ipp +++ /dev/null @@ -1,134 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_iarchive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; - using ::strlen; - using ::size_t; -} -#endif - -#include -#include - -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implementation of binary_binary_archive -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iarchive::load_override(class_name_type & t){ - std::string cn; - cn.reserve(BOOST_SERIALIZATION_MAX_KEY_SIZE); - load_override(cn); - if(cn.size() > (BOOST_SERIALIZATION_MAX_KEY_SIZE - 1)) - boost::serialization::throw_exception( - archive_exception(archive_exception::invalid_class_name) - ); - std::memcpy(t, cn.data(), cn.size()); - // borland tweak - t.t[cn.size()] = '\0'; -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iarchive::init(void){ - // read signature in an archive version independent manner - std::string file_signature; - - #if 0 // commented out since it interfers with derivation - BOOST_TRY { - std::size_t l; - this->This()->load(l); - if(l == std::strlen(BOOST_ARCHIVE_SIGNATURE())) { - // borland de-allocator fixup - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != file_signature.data()) - #endif - file_signature.resize(l); - // note breaking a rule here - could be a problem on some platform - if(0 < l) - this->This()->load_binary(&(*file_signature.begin()), l); - } - } - BOOST_CATCH(archive_exception const &) { // catch stream_error archive exceptions - // will cause invalid_signature archive exception to be thrown below - file_signature = ""; - } - BOOST_CATCH_END - #else - // https://svn.boost.org/trac/boost/ticket/7301 - * this->This() >> file_signature; - #endif - - if(file_signature != BOOST_ARCHIVE_SIGNATURE()) - boost::serialization::throw_exception( - archive_exception(archive_exception::invalid_signature) - ); - - // make sure the version of the reading archive library can - // support the format of the archive being read - library_version_type input_library_version; - //* this->This() >> input_library_version; - { - int v = 0; - v = this->This()->m_sb.sbumpc(); - #if defined(BOOST_LITTLE_ENDIAN) - if(v < 6){ - ; - } - else - if(v < 7){ - // version 6 - next byte should be zero - this->This()->m_sb.sbumpc(); - } - else - if(v < 8){ - int x1; - // version 7 = might be followed by zero or some other byte - x1 = this->This()->m_sb.sgetc(); - // it's =a zero, push it back - if(0 == x1) - this->This()->m_sb.sbumpc(); - } - else{ - // version 8+ followed by a zero - this->This()->m_sb.sbumpc(); - } - #elif defined(BOOST_BIG_ENDIAN) - if(v == 0) - v = this->This()->m_sb.sbumpc(); - #endif - input_library_version = static_cast(v); - } - - #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) - this->set_library_version(input_library_version); - #else - detail::basic_iarchive::set_library_version(input_library_version); - #endif - - if(BOOST_ARCHIVE_VERSION() < input_library_version) - boost::serialization::throw_exception( - archive_exception(archive_exception::unsupported_version) - ); -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iprimitive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iprimitive.ipp deleted file mode 100644 index bbe933ccf63..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_iprimitive.ipp +++ /dev/null @@ -1,171 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_iprimitive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // size_t, NULL -#include // memcpy - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; - using ::memcpy; -} // namespace std -#endif - -#include -#include -#include -#include - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// implementation of basic_binary_iprimitive - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iprimitive::init() -{ - // Detect attempts to pass native binary archives across - // incompatible platforms. This is not fool proof but its - // better than nothing. - unsigned char size; - this->This()->load(size); - if(sizeof(int) != size) - boost::serialization::throw_exception( - archive_exception( - archive_exception::incompatible_native_format, - "size of int" - ) - ); - this->This()->load(size); - if(sizeof(long) != size) - boost::serialization::throw_exception( - archive_exception( - archive_exception::incompatible_native_format, - "size of long" - ) - ); - this->This()->load(size); - if(sizeof(float) != size) - boost::serialization::throw_exception( - archive_exception( - archive_exception::incompatible_native_format, - "size of float" - ) - ); - this->This()->load(size); - if(sizeof(double) != size) - boost::serialization::throw_exception( - archive_exception( - archive_exception::incompatible_native_format, - "size of double" - ) - ); - - // for checking endian - int i; - this->This()->load(i); - if(1 != i) - boost::serialization::throw_exception( - archive_exception( - archive_exception::incompatible_native_format, - "endian setting" - ) - ); -} - -#ifndef BOOST_NO_CWCHAR -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iprimitive::load(wchar_t * ws) -{ - std::size_t l; // number of wchar_t !!! - this->This()->load(l); - load_binary(ws, l * sizeof(wchar_t) / sizeof(char)); - ws[l] = L'\0'; -} -#endif - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iprimitive::load(std::string & s) -{ - std::size_t l; - this->This()->load(l); - // borland de-allocator fixup - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != s.data()) - #endif - s.resize(l); - // note breaking a rule here - could be a problem on some platform - if(0 < l) - load_binary(&(*s.begin()), l); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iprimitive::load(char * s) -{ - std::size_t l; - this->This()->load(l); - load_binary(s, l); - s[l] = '\0'; -} - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_iprimitive::load(std::wstring & ws) -{ - std::size_t l; - this->This()->load(l); - // borland de-allocator fixup - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != ws.data()) - #endif - ws.resize(l); - // note breaking a rule here - is could be a problem on some platform - load_binary(const_cast(ws.data()), l * sizeof(wchar_t) / sizeof(char)); -} -#endif - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_binary_iprimitive::basic_binary_iprimitive( - std::basic_streambuf & sb, - bool no_codecvt -) : -#ifndef BOOST_NO_STD_LOCALE - m_sb(sb), - codecvt_null_facet(1), - locale_saver(m_sb), - archive_locale(sb.getloc(), & codecvt_null_facet) -{ - if(! no_codecvt){ - m_sb.pubsync(); - m_sb.pubimbue(archive_locale); - } -} -#else - m_sb(sb) -{} -#endif - -// scoped_ptr requires that g be a complete type at time of -// destruction so define destructor here rather than in the header -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_binary_iprimitive::~basic_binary_iprimitive(){} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oarchive.ipp deleted file mode 100644 index d5a019d32bc..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oarchive.ipp +++ /dev/null @@ -1,42 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_oarchive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; -} -#endif - -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implementation of binary_binary_oarchive - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_oarchive::init(){ - // write signature in an archive version independent manner - const std::string file_signature(BOOST_ARCHIVE_SIGNATURE()); - * this->This() << file_signature; - // write library version - const library_version_type v(BOOST_ARCHIVE_VERSION()); - * this->This() << v; -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oprimitive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oprimitive.ipp deleted file mode 100644 index 7b042173a48..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_binary_oprimitive.ipp +++ /dev/null @@ -1,126 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_binary_oprimitive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL -#include - -#include - -#if defined(BOOST_NO_STDC_NAMESPACE) && ! defined(__LIBCOMO__) -namespace std{ - using ::strlen; -} // namespace std -#endif - -#ifndef BOOST_NO_CWCHAR -#include -#ifdef BOOST_NO_STDC_NAMESPACE -namespace std{ using ::wcslen; } -#endif -#endif - -#include -#include - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// implementation of basic_binary_oprimitive - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_oprimitive::init() -{ - // record native sizes of fundamental types - // this is to permit detection of attempts to pass - // native binary archives accross incompatible machines. - // This is not foolproof but its better than nothing. - this->This()->save(static_cast(sizeof(int))); - this->This()->save(static_cast(sizeof(long))); - this->This()->save(static_cast(sizeof(float))); - this->This()->save(static_cast(sizeof(double))); - // for checking endianness - this->This()->save(int(1)); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_oprimitive::save(const char * s) -{ - std::size_t l = std::strlen(s); - this->This()->save(l); - save_binary(s, l); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_oprimitive::save(const std::string &s) -{ - std::size_t l = static_cast(s.size()); - this->This()->save(l); - save_binary(s.data(), l); -} - -#ifndef BOOST_NO_CWCHAR -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_oprimitive::save(const wchar_t * ws) -{ - std::size_t l = std::wcslen(ws); - this->This()->save(l); - save_binary(ws, l * sizeof(wchar_t) / sizeof(char)); -} -#endif - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_binary_oprimitive::save(const std::wstring &ws) -{ - std::size_t l = ws.size(); - this->This()->save(l); - save_binary(ws.data(), l * sizeof(wchar_t) / sizeof(char)); -} -#endif -#endif // BOOST_NO_CWCHAR - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_binary_oprimitive::basic_binary_oprimitive( - std::basic_streambuf & sb, - bool no_codecvt -) : -#ifndef BOOST_NO_STD_LOCALE - m_sb(sb), - codecvt_null_facet(1), - locale_saver(m_sb), - archive_locale(sb.getloc(), & codecvt_null_facet) -{ - if(! no_codecvt){ - m_sb.pubsync(); - m_sb.pubimbue(archive_locale); - } -} -#else - m_sb(sb) -{} -#endif - -// scoped_ptr requires that g be a complete type at time of -// destruction so define destructor here rather than in the header -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_binary_oprimitive::~basic_binary_oprimitive(){} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iarchive.ipp deleted file mode 100644 index 9ec8c6588c8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iarchive.ipp +++ /dev/null @@ -1,76 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_iarchive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; -} -#endif - -#include -#include -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implementation of text_text_archive - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_text_iarchive::load_override(class_name_type & t){ - std::string cn; - cn.reserve(BOOST_SERIALIZATION_MAX_KEY_SIZE); - load_override(cn); - if(cn.size() > (BOOST_SERIALIZATION_MAX_KEY_SIZE - 1)) - boost::serialization::throw_exception( - archive_exception(archive_exception::invalid_class_name) - ); - std::memcpy(t, cn.data(), cn.size()); - // borland tweak - t.t[cn.size()] = '\0'; -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_text_iarchive::init(void){ - // read signature in an archive version independent manner - std::string file_signature; - * this->This() >> file_signature; - if(file_signature != BOOST_ARCHIVE_SIGNATURE()) - boost::serialization::throw_exception( - archive_exception(archive_exception::invalid_signature) - ); - - // make sure the version of the reading archive library can - // support the format of the archive being read - library_version_type input_library_version; - * this->This() >> input_library_version; - - #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) - this->set_library_version(input_library_version); - #else - detail::basic_iarchive::set_library_version(input_library_version); - #endif - - // extra little .t is to get around borland quirk - if(BOOST_ARCHIVE_VERSION() < input_library_version) - boost::serialization::throw_exception( - archive_exception(archive_exception::unsupported_version) - ); -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iprimitive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iprimitive.ipp deleted file mode 100644 index 4e44728068d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_iprimitive.ipp +++ /dev/null @@ -1,137 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_iprimitive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // size_t, NULL -#include // NULL - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include - -#include - -#include -#include -#include -#include - -namespace boost { -namespace archive { - -namespace detail { - template - static inline bool is_whitespace(CharType c); - - template<> - inline bool is_whitespace(char t){ - return 0 != std::isspace(t); - } - - #ifndef BOOST_NO_CWCHAR - template<> - inline bool is_whitespace(wchar_t t){ - return 0 != std::iswspace(t); - } - #endif -} // detail - -// translate base64 text into binary and copy into buffer -// until buffer is full. -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_text_iprimitive::load_binary( - void *address, - std::size_t count -){ - typedef typename IStream::char_type CharType; - - if(0 == count) - return; - - BOOST_ASSERT( - static_cast((std::numeric_limits::max)()) - > (count + sizeof(CharType) - 1)/sizeof(CharType) - ); - - if(is.fail()) - boost::serialization::throw_exception( - archive_exception(archive_exception::input_stream_error) - ); - // convert from base64 to binary - typedef typename - iterators::transform_width< - iterators::binary_from_base64< - iterators::remove_whitespace< - iterators::istream_iterator - > - ,typename IStream::int_type - > - ,8 - ,6 - ,CharType - > - binary; - - binary i = binary(iterators::istream_iterator(is)); - - char * caddr = static_cast(address); - - // take care that we don't increment anymore than necessary - while(count-- > 0){ - *caddr++ = static_cast(*i++); - } - - // skip over any excess input - for(;;){ - typename IStream::int_type r; - r = is.get(); - if(is.eof()) - break; - if(detail::is_whitespace(static_cast(r))) - break; - } -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_text_iprimitive::basic_text_iprimitive( - IStream &is_, - bool no_codecvt -) : - is(is_), - flags_saver(is_), - precision_saver(is_), -#ifndef BOOST_NO_STD_LOCALE - codecvt_null_facet(1), - archive_locale(is.getloc(), & codecvt_null_facet), - locale_saver(is) -{ - if(! no_codecvt){ - is_.sync(); - is_.imbue(archive_locale); - } - is_ >> std::noboolalpha; -} -#else -{} -#endif - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_text_iprimitive::~basic_text_iprimitive(){ -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oarchive.ipp deleted file mode 100644 index 44bc1401fd6..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oarchive.ipp +++ /dev/null @@ -1,62 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_oarchive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; -} -#endif - -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implementation of basic_text_oarchive - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_text_oarchive::newtoken() -{ - switch(delimiter){ - default: - BOOST_ASSERT(false); - break; - case eol: - this->This()->put('\n'); - delimiter = space; - break; - case space: - this->This()->put(' '); - break; - case none: - delimiter = space; - break; - } -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_text_oarchive::init(){ - // write signature in an archive version independent manner - const std::string file_signature(BOOST_ARCHIVE_SIGNATURE()); - * this->This() << file_signature; - // write library version - const library_version_type v(BOOST_ARCHIVE_VERSION()); - * this->This() << v; -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oprimitive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oprimitive.ipp deleted file mode 100644 index 6030fd44c57..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_text_oprimitive.ipp +++ /dev/null @@ -1,115 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_text_oprimitive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include // std::copy -#include // std::uncaught_exception -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include - -#include -#include -#include -#include - -namespace boost { -namespace archive { - -// translate to base64 and copy in to buffer. -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_text_oprimitive::save_binary( - const void *address, - std::size_t count -){ - typedef typename OStream::char_type CharType; - - if(0 == count) - return; - - if(os.fail()) - boost::serialization::throw_exception( - archive_exception(archive_exception::output_stream_error) - ); - - os.put('\n'); - - typedef - boost::archive::iterators::insert_linebreaks< - boost::archive::iterators::base64_from_binary< - boost::archive::iterators::transform_width< - const char *, - 6, - 8 - > - > - ,76 - ,const char // cwpro8 needs this - > - base64_text; - - boost::archive::iterators::ostream_iterator oi(os); - std::copy( - base64_text(static_cast(address)), - base64_text( - static_cast(address) + count - ), - oi - ); - - std::size_t tail = count % 3; - if(tail > 0){ - *oi++ = '='; - if(tail < 2) - *oi = '='; - } -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_text_oprimitive::basic_text_oprimitive( - OStream & os_, - bool no_codecvt -) : - os(os_), - flags_saver(os_), - precision_saver(os_), -#ifndef BOOST_NO_STD_LOCALE - codecvt_null_facet(1), - archive_locale(os.getloc(), & codecvt_null_facet), - locale_saver(os) -{ - if(! no_codecvt){ - os_.flush(); - os_.imbue(archive_locale); - } - os_ << std::noboolalpha; -} -#else -{} -#endif - - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_text_oprimitive::~basic_text_oprimitive(){ - if(std::uncaught_exception()) - return; - os << std::endl; -} - -} //namespace boost -} //namespace archive diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_grammar.hpp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_grammar.hpp deleted file mode 100644 index 6d4e4683f6a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_grammar.hpp +++ /dev/null @@ -1,173 +0,0 @@ -#ifndef BOOST_ARCHIVE_BASIC_XML_GRAMMAR_HPP -#define BOOST_ARCHIVE_BASIC_XML_GRAMMAR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_xml_grammar.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// this module is derived from simplexml.cpp - an example shipped as part of -// the spirit parser. This example contains the following notice: -/*============================================================================= - simplexml.cpp - - Spirit V1.3 - URL: http://spirit.sourceforge.net/ - - Copyright (c) 2001, Daniel C. Nuffer - - This software is provided 'as-is', without any express or implied - warranty. In no event will the copyright holder be held liable for - any damages arising from the use of this software. - - Permission is granted to anyone to use this software for any purpose, - including commercial applications, and to alter it and redistribute - it freely, subject to the following restrictions: - - 1. The origin of this software must not be misrepresented; you must - not claim that you wrote the original software. If you use this - software in a product, an acknowledgment in the product documentation - would be appreciated but is not required. - - 2. Altered source versions must be plainly marked as such, and must - not be misrepresented as being the original software. - - 3. This notice may not be removed or altered from any source - distribution. -=============================================================================*/ -#include - -#include -#include - -#include -#include - -#include -#include -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// XML grammar parsing - -template -class basic_xml_grammar { -public: - // The following is not necessary according to DR45, but at least - // one compiler (Compaq C++ 6.5 in strict_ansi mode) chokes otherwise. - struct return_values; - friend struct return_values; - -private: - typedef typename std::basic_istream IStream; - typedef typename std::basic_string StringType; - typedef typename boost::spirit::classic::chset chset_t; - typedef typename boost::spirit::classic::chlit chlit_t; - typedef typename boost::spirit::classic::scanner< - typename std::basic_string::iterator - > scanner_t; - typedef typename boost::spirit::classic::rule rule_t; - // Start grammar definition - rule_t - Reference, - Eq, - STag, - ETag, - LetterOrUnderscoreOrColon, - AttValue, - CharRef1, - CharRef2, - CharRef, - AmpRef, - LTRef, - GTRef, - AposRef, - QuoteRef, - CharData, - CharDataChars, - content, - AmpName, - LTName, - GTName, - ClassNameChar, - ClassName, - Name, - XMLDecl, - XMLDeclChars, - DocTypeDecl, - DocTypeDeclChars, - ClassIDAttribute, - ObjectIDAttribute, - ClassNameAttribute, - TrackingAttribute, - VersionAttribute, - UnusedAttribute, - Attribute, - SignatureAttribute, - SerializationWrapper, - NameHead, - NameTail, - AttributeList, - S; - - // XML Character classes - chset_t - BaseChar, - Ideographic, - Char, - Letter, - Digit, - CombiningChar, - Extender, - Sch, - NameChar; - - void init_chset(); - - bool my_parse( - IStream & is, - const rule_t &rule_, - const CharType delimiter = L'>' - ) const ; -public: - struct return_values { - StringType object_name; - StringType contents; - //class_id_type class_id; - int_least16_t class_id; - //object_id_type object_id; - uint_least32_t object_id; - //version_type version; - unsigned int version; - tracking_type tracking_level; - StringType class_name; - return_values() : - version(0), - tracking_level(false) - {} - } rv; - bool parse_start_tag(IStream & is) /*const*/; - bool parse_end_tag(IStream & is) const; - bool parse_string(IStream & is, StringType & s) /*const*/; - void init(IStream & is); - bool windup(IStream & is); - basic_xml_grammar(); -}; - -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_BASIC_XML_GRAMMAR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_iarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_iarchive.ipp deleted file mode 100644 index 625458b9eb5..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_iarchive.ipp +++ /dev/null @@ -1,115 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_xml_iarchive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL -#include - -#include -#include -#include -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implementation of xml_text_archive - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_iarchive::load_start(const char *name){ - // if there's no name - if(NULL == name) - return; - bool result = this->This()->gimpl->parse_start_tag(this->This()->get_is()); - if(true != result){ - boost::serialization::throw_exception( - archive_exception(archive_exception::input_stream_error) - ); - } - // don't check start tag at highest level - ++depth; - return; -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_iarchive::load_end(const char *name){ - // if there's no name - if(NULL == name) - return; - bool result = this->This()->gimpl->parse_end_tag(this->This()->get_is()); - if(true != result){ - boost::serialization::throw_exception( - archive_exception(archive_exception::input_stream_error) - ); - } - - // don't check start tag at highest level - if(0 == --depth) - return; - - if(0 == (this->get_flags() & no_xml_tag_checking)){ - // double check that the tag matches what is expected - useful for debug - if(0 != name[this->This()->gimpl->rv.object_name.size()] - || ! std::equal( - this->This()->gimpl->rv.object_name.begin(), - this->This()->gimpl->rv.object_name.end(), - name - ) - ){ - boost::serialization::throw_exception( - xml_archive_exception( - xml_archive_exception::xml_archive_tag_mismatch, - name - ) - ); - } - } -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_iarchive::load_override(object_id_type & t){ - t = object_id_type(this->This()->gimpl->rv.object_id); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_iarchive::load_override(version_type & t){ - t = version_type(this->This()->gimpl->rv.version); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_iarchive::load_override(class_id_type & t){ - t = class_id_type(this->This()->gimpl->rv.class_id); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_iarchive::load_override(tracking_type & t){ - t = this->This()->gimpl->rv.tracking_level; -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_xml_iarchive::basic_xml_iarchive(unsigned int flags) : - detail::common_iarchive(flags), - depth(0) -{} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_xml_iarchive::~basic_xml_iarchive(){ -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_oarchive.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_oarchive.ipp deleted file mode 100644 index 3184413f382..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/basic_xml_oarchive.ipp +++ /dev/null @@ -1,272 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// basic_xml_oarchive.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL -#include -#if defined(BOOST_NO_STDC_NAMESPACE) && ! defined(__LIBCOMO__) -namespace std{ - using ::strlen; -} // namespace std -#endif - -#include -#include -#include -#include - -namespace boost { -namespace archive { - -namespace detail { -template -struct XML_name { - void operator()(CharType t) const{ - const unsigned char lookup_table[] = { - 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, - 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, - 0,0,0,0,0,0,0,0,0,0,0,0,0,1,1,0, // -. - 1,1,1,1,1,1,1,1,1,1,0,0,0,0,0,0, // 0-9 - 0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1, // A- - 1,1,1,1,1,1,1,1,1,1,1,0,0,0,0,1, // -Z _ - 0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1, // a- - 1,1,1,1,1,1,1,1,1,1,1,0,0,0,0,0, // -z - 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, - 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 - }; - if((unsigned)t > 127) - return; - if(0 == lookup_table[(unsigned)t]) - boost::serialization::throw_exception( - xml_archive_exception( - xml_archive_exception::xml_archive_tag_name_error - ) - ); - } -}; - -} // namespace detail - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implemenations of functions common to both types of xml output - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::write_attribute( - const char *attribute_name, - int t, - const char *conjunction -){ - this->This()->put(' '); - this->This()->put(attribute_name); - this->This()->put(conjunction); - this->This()->save(t); - this->This()->put('"'); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::write_attribute( - const char *attribute_name, - const char *key -){ - this->This()->put(' '); - this->This()->put(attribute_name); - this->This()->put("=\""); - this->This()->save(key); - this->This()->put('"'); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::indent(){ - int i; - for(i = depth; i-- > 0;) - this->This()->put('\t'); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_start(const char *name) -{ - if(NULL == name) - return; - - // be sure name has no invalid characters - std::for_each(name, name + std::strlen(name), detail::XML_name()); - - end_preamble(); - if(depth > 0){ - this->This()->put('\n'); - indent(); - } - ++depth; - this->This()->put('<'); - this->This()->save(name); - pending_preamble = true; - indent_next = false; -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_end(const char *name) -{ - if(NULL == name) - return; - - // be sure name has no invalid characters - std::for_each(name, name + std::strlen(name), detail::XML_name()); - - end_preamble(); - --depth; - if(indent_next){ - this->This()->put('\n'); - indent(); - } - indent_next = true; - this->This()->put("This()->save(name); - this->This()->put('>'); - if(0 == depth) - this->This()->put('\n'); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::end_preamble(){ - if(pending_preamble){ - this->This()->put('>'); - pending_preamble = false; - } -} -#if 0 -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const object_id_type & t) -{ - int i = t.t; // extra .t is for borland - write_attribute(BOOST_ARCHIVE_XML_OBJECT_ID(), i, "=\"_"); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override( - const object_reference_type & t, - int -){ - int i = t.t; // extra .t is for borland - write_attribute(BOOST_ARCHIVE_XML_OBJECT_REFERENCE(), i, "=\"_"); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const version_type & t) -{ - int i = t.t; // extra .t is for borland - write_attribute(BOOST_ARCHIVE_XML_VERSION(), i); -} -#endif - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const object_id_type & t) -{ - // borland doesn't do conversion of STRONG_TYPEDEFs very well - const unsigned int i = t; - write_attribute(BOOST_ARCHIVE_XML_OBJECT_ID(), i, "=\"_"); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override( - const object_reference_type & t -){ - const unsigned int i = t; - write_attribute(BOOST_ARCHIVE_XML_OBJECT_REFERENCE(), i, "=\"_"); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const version_type & t) -{ - const unsigned int i = t; - write_attribute(BOOST_ARCHIVE_XML_VERSION(), i); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const class_id_type & t) -{ - write_attribute(BOOST_ARCHIVE_XML_CLASS_ID(), t); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override( - const class_id_reference_type & t -){ - write_attribute(BOOST_ARCHIVE_XML_CLASS_ID_REFERENCE(), t); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override( - const class_id_optional_type & t -){ - write_attribute(BOOST_ARCHIVE_XML_CLASS_ID(), t); -} -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const class_name_type & t) -{ - const char * key = t; - if(NULL == key) - return; - write_attribute(BOOST_ARCHIVE_XML_CLASS_NAME(), key); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::save_override(const tracking_type & t) -{ - write_attribute(BOOST_ARCHIVE_XML_TRACKING(), t.t); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::init(){ - // xml header - this->This()->put("\n"); - this->This()->put("\n"); - // xml document wrapper - outer root - this->This()->put("This()->put(">\n"); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL void -basic_xml_oarchive::windup(){ - // xml_trailer - this->This()->put("\n"); -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_xml_oarchive::basic_xml_oarchive(unsigned int flags) : - detail::common_oarchive(flags), - depth(0), - pending_preamble(false), - indent_next(false) -{ -} - -template -BOOST_ARCHIVE_OR_WARCHIVE_DECL -basic_xml_oarchive::~basic_xml_oarchive(){ -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_iarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_iarchive_impl.ipp deleted file mode 100644 index ae4e2750ce8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_iarchive_impl.ipp +++ /dev/null @@ -1,128 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_iarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -////////////////////////////////////////////////////////////////////// -// implementation of basic_text_iprimitive overrides for the combination -// of template parameters used to implement a text_iprimitive - -#include // size_t, NULL -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include // RogueWave - -#include - -namespace boost { -namespace archive { - -template -BOOST_ARCHIVE_DECL void -text_iarchive_impl::load(char *s) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - // Works on all tested platforms - is.read(s, size); - s[size] = '\0'; -} - -template -BOOST_ARCHIVE_DECL void -text_iarchive_impl::load(std::string &s) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - // borland de-allocator fixup - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != s.data()) - #endif - s.resize(size); - if(0 < size) - is.read(&(*s.begin()), size); -} - -#ifndef BOOST_NO_CWCHAR -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_ARCHIVE_DECL void -text_iarchive_impl::load(wchar_t *ws) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - is.read((char *)ws, size * sizeof(wchar_t)/sizeof(char)); - ws[size] = L'\0'; -} -#endif // BOOST_NO_INTRINSIC_WCHAR_T - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_ARCHIVE_DECL void -text_iarchive_impl::load(std::wstring &ws) -{ - std::size_t size; - * this->This() >> size; - // borland de-allocator fixup - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != ws.data()) - #endif - ws.resize(size); - // skip separating space - is.get(); - is.read((char *)ws.data(), size * sizeof(wchar_t)/sizeof(char)); -} - -#endif // BOOST_NO_STD_WSTRING -#endif // BOOST_NO_CWCHAR - -template -BOOST_ARCHIVE_DECL void -text_iarchive_impl::load_override(class_name_type & t){ - basic_text_iarchive::load_override(t); -} - -template -BOOST_ARCHIVE_DECL void -text_iarchive_impl::init(){ - basic_text_iarchive::init(); -} - -template -BOOST_ARCHIVE_DECL -text_iarchive_impl::text_iarchive_impl( - std::istream & is, - unsigned int flags -) : - basic_text_iprimitive( - is, - 0 != (flags & no_codecvt) - ), - basic_text_iarchive(flags) -{ - if(0 == (flags & no_header)) - #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) - this->init(); - #else - this->basic_text_iarchive::init(); - #endif -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_oarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_oarchive_impl.ipp deleted file mode 100644 index 37d8664a98c..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_oarchive_impl.ipp +++ /dev/null @@ -1,122 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_oarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include // size_t - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#ifndef BOOST_NO_CWCHAR -#include -#ifdef BOOST_NO_STDC_NAMESPACE -namespace std{ using ::wcslen; } -#endif -#endif - -#include - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// implementation of basic_text_oprimitive overrides for the combination -// of template parameters used to create a text_oprimitive - -template -BOOST_ARCHIVE_DECL void -text_oarchive_impl::save(const char * s) -{ - const std::size_t len = std::ostream::traits_type::length(s); - *this->This() << len; - this->This()->newtoken(); - os << s; -} - -template -BOOST_ARCHIVE_DECL void -text_oarchive_impl::save(const std::string &s) -{ - const std::size_t size = s.size(); - *this->This() << size; - this->This()->newtoken(); - os << s; -} - -#ifndef BOOST_NO_CWCHAR -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_ARCHIVE_DECL void -text_oarchive_impl::save(const wchar_t * ws) -{ - const std::size_t l = std::wcslen(ws); - * this->This() << l; - this->This()->newtoken(); - os.write((const char *)ws, l * sizeof(wchar_t)/sizeof(char)); -} -#endif - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_ARCHIVE_DECL void -text_oarchive_impl::save(const std::wstring &ws) -{ - const std::size_t l = ws.size(); - * this->This() << l; - this->This()->newtoken(); - os.write((const char *)(ws.data()), l * sizeof(wchar_t)/sizeof(char)); -} -#endif -#endif // BOOST_NO_CWCHAR - -template -BOOST_ARCHIVE_DECL -text_oarchive_impl::text_oarchive_impl( - std::ostream & os, - unsigned int flags -) : - basic_text_oprimitive( - os, - 0 != (flags & no_codecvt) - ), - basic_text_oarchive(flags) -{ - if(0 == (flags & no_header)) - #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) - this->init(); - #else - this->basic_text_oarchive::init(); - #endif -} - -template -BOOST_ARCHIVE_DECL void -text_oarchive_impl::save_binary(const void *address, std::size_t count){ - put('\n'); - this->end_preamble(); - #if ! defined(__MWERKS__) - this->basic_text_oprimitive::save_binary( - #else - this->basic_text_oprimitive::save_binary( - #endif - address, - count - ); - this->delimiter = this->eol; -} - -} // namespace archive -} // namespace boost - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_wiarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_wiarchive_impl.ipp deleted file mode 100644 index e85625ac326..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_wiarchive_impl.ipp +++ /dev/null @@ -1,118 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_text_wiarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // size_t, NULL - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include // fixup for RogueWave - -#ifndef BOOST_NO_STD_WSTREAMBUF -#include - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// implementation of wiprimtives functions -// -template -BOOST_WARCHIVE_DECL void -text_wiarchive_impl::load(char *s) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - while(size-- > 0){ - *s++ = is.narrow(is.get(), '\0'); - } - *s = '\0'; -} - -template -BOOST_WARCHIVE_DECL void -text_wiarchive_impl::load(std::string &s) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != s.data()) - #endif - s.resize(0); - s.reserve(size); - while(size-- > 0){ - char x = is.narrow(is.get(), '\0'); - s += x; - } -} - -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_WARCHIVE_DECL void -text_wiarchive_impl::load(wchar_t *s) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - // Works on all tested platforms - is.read(s, size); - s[size] = L'\0'; -} -#endif - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_WARCHIVE_DECL void -text_wiarchive_impl::load(std::wstring &ws) -{ - std::size_t size; - * this->This() >> size; - // skip separating space - is.get(); - // borland complains about resize - // borland de-allocator fixup - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != ws.data()) - #endif - ws.resize(size); - // note breaking a rule here - is this a problem on some platform - is.read(const_cast(ws.data()), size); -} -#endif - -template -BOOST_WARCHIVE_DECL -text_wiarchive_impl::text_wiarchive_impl( - std::wistream & is, - unsigned int flags -) : - basic_text_iprimitive( - is, - 0 != (flags & no_codecvt) - ), - basic_text_iarchive(flags) -{ - if(0 == (flags & no_header)) - basic_text_iarchive::init(); -} - -} // archive -} // boost - -#endif // BOOST_NO_STD_WSTREAMBUF diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_woarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/text_woarchive_impl.ipp deleted file mode 100644 index 2b6d427cd3a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/text_woarchive_impl.ipp +++ /dev/null @@ -1,85 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_woarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifndef BOOST_NO_STD_WSTREAMBUF - -#include -#include // size_t -#if defined(BOOST_NO_STDC_NAMESPACE) && ! defined(__LIBCOMO__) -namespace std{ - using ::strlen; - using ::size_t; -} // namespace std -#endif - -#include - -#include - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// implementation of woarchive functions -// -template -BOOST_WARCHIVE_DECL void -text_woarchive_impl::save(const char *s) -{ - // note: superfluous local variable fixes borland warning - const std::size_t size = std::strlen(s); - * this->This() << size; - this->This()->newtoken(); - while(*s != '\0') - os.put(os.widen(*s++)); -} - -template -BOOST_WARCHIVE_DECL void -text_woarchive_impl::save(const std::string &s) -{ - const std::size_t size = s.size(); - * this->This() << size; - this->This()->newtoken(); - const char * cptr = s.data(); - for(std::size_t i = size; i-- > 0;) - os.put(os.widen(*cptr++)); -} - -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_WARCHIVE_DECL void -text_woarchive_impl::save(const wchar_t *ws) -{ - const std::size_t size = std::wostream::traits_type::length(ws); - * this->This() << size; - this->This()->newtoken(); - os.write(ws, size); -} -#endif - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_WARCHIVE_DECL void -text_woarchive_impl::save(const std::wstring &ws) -{ - const std::size_t size = ws.length(); - * this->This() << size; - this->This()->newtoken(); - os.write(ws.data(), size); -} -#endif - -} // namespace archive -} // namespace boost - -#endif - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_iarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_iarchive_impl.ipp deleted file mode 100644 index efc32e01632..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_iarchive_impl.ipp +++ /dev/null @@ -1,199 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_iarchive_impl.cpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // memcpy -#include // NULL -#include - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; -} // namespace std -#endif - -#ifndef BOOST_NO_CWCHAR -#include // mbstate_t and mbrtowc -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::mbstate_t; - using ::mbrtowc; - } // namespace std -#endif -#endif // BOOST_NO_CWCHAR - -#include // RogueWave and Dinkumware -#if BOOST_WORKAROUND(BOOST_DINKUMWARE_STDLIB, == 1) -#include -#endif - -#include - -#include -#include -#include -#include - -#include "basic_xml_grammar.hpp" - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implemenations of functions specific to char archives - -// wide char stuff used by char archives - -#ifndef BOOST_NO_CWCHAR -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_ARCHIVE_DECL void -xml_iarchive_impl::load(std::wstring &ws){ - std::string s; - bool result = gimpl->parse_string(is, s); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); - - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != ws.data()) - #endif - ws.resize(0); - std::mbstate_t mbs = std::mbstate_t(); - const char * start = s.data(); - const char * end = start + s.size(); - while(start < end){ - wchar_t wc; - std::size_t count = std::mbrtowc(&wc, start, end - start, &mbs); - if(count == static_cast(-1)) - boost::serialization::throw_exception( - iterators::dataflow_exception( - iterators::dataflow_exception::invalid_conversion - ) - ); - if(count == static_cast(-2)) - continue; - start += count; - ws += wc; - } -} -#endif // BOOST_NO_STD_WSTRING - -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_ARCHIVE_DECL void -xml_iarchive_impl::load(wchar_t * ws){ - std::string s; - bool result = gimpl->parse_string(is, s); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception( - xml_archive_exception::xml_archive_parsing_error - ) - ); - - std::mbstate_t mbs = std::mbstate_t(); - const char * start = s.data(); - const char * end = start + s.size(); - while(start < end){ - wchar_t wc; - std::size_t length = std::mbrtowc(&wc, start, end - start, &mbs); - if(static_cast(-1) == length) - boost::serialization::throw_exception( - iterators::dataflow_exception( - iterators::dataflow_exception::invalid_conversion - ) - ); - if(static_cast(-2) == length) - continue; - - start += length; - *ws++ = wc; - } - *ws = L'\0'; -} -#endif // BOOST_NO_INTRINSIC_WCHAR_T - -#endif // BOOST_NO_CWCHAR - -template -BOOST_ARCHIVE_DECL void -xml_iarchive_impl::load(std::string &s){ - bool result = gimpl->parse_string(is, s); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); -} - -template -BOOST_ARCHIVE_DECL void -xml_iarchive_impl::load(char * s){ - std::string tstring; - bool result = gimpl->parse_string(is, tstring); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); - std::memcpy(s, tstring.data(), tstring.size()); - s[tstring.size()] = 0; -} - -template -BOOST_ARCHIVE_DECL void -xml_iarchive_impl::load_override(class_name_type & t){ - const std::string & s = gimpl->rv.class_name; - if(s.size() > BOOST_SERIALIZATION_MAX_KEY_SIZE - 1) - boost::serialization::throw_exception( - archive_exception(archive_exception::invalid_class_name) - ); - char * tptr = t; - std::memcpy(tptr, s.data(), s.size()); - tptr[s.size()] = '\0'; -} - -template -BOOST_ARCHIVE_DECL void -xml_iarchive_impl::init(){ - gimpl->init(is); - this->set_library_version( - library_version_type(gimpl->rv.version) - ); -} - -template -BOOST_ARCHIVE_DECL -xml_iarchive_impl::xml_iarchive_impl( - std::istream &is_, - unsigned int flags -) : - basic_text_iprimitive( - is_, - 0 != (flags & no_codecvt) - ), - basic_xml_iarchive(flags), - gimpl(new xml_grammar()) -{ - if(0 == (flags & no_header)) - init(); -} - -template -BOOST_ARCHIVE_DECL -xml_iarchive_impl::~xml_iarchive_impl(){ - if(std::uncaught_exception()) - return; - if(0 == (this->get_flags() & no_header)){ - gimpl->windup(is); - } -} -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_oarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_oarchive_impl.ipp deleted file mode 100644 index 5ebd454e722..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_oarchive_impl.ipp +++ /dev/null @@ -1,142 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_oarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include -#include -#include // std::copy -#include -#include - -#include // strlen -#include // msvc 6.0 needs this to suppress warnings -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::strlen; -} // namespace std -#endif - -#include -#include - -#ifndef BOOST_NO_CWCHAR -#include -#include -#endif - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implemenations of functions specific to char archives - -// wide char stuff used by char archives -#ifndef BOOST_NO_CWCHAR -// copy chars to output escaping to xml and translating wide chars to mb chars -template -void save_iterator(std::ostream &os, InputIterator begin, InputIterator end){ - typedef boost::archive::iterators::mb_from_wchar< - boost::archive::iterators::xml_escape - > translator; - std::copy( - translator(begin), - translator(end), - boost::archive::iterators::ostream_iterator(os) - ); -} - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_ARCHIVE_DECL void -xml_oarchive_impl::save(const std::wstring & ws){ -// at least one library doesn't typedef value_type for strings -// so rather than using string directly make a pointer iterator out of it -// save_iterator(os, ws.data(), ws.data() + std::wcslen(ws.data())); - save_iterator(os, ws.data(), ws.data() + ws.size()); -} -#endif - -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_ARCHIVE_DECL void -xml_oarchive_impl::save(const wchar_t * ws){ - save_iterator(os, ws, ws + std::wcslen(ws)); -} -#endif - -#endif // BOOST_NO_CWCHAR - -template -BOOST_ARCHIVE_DECL void -xml_oarchive_impl::save(const std::string & s){ -// at least one library doesn't typedef value_type for strings -// so rather than using string directly make a pointer iterator out of it - typedef boost::archive::iterators::xml_escape< - const char * - > xml_escape_translator; - std::copy( - xml_escape_translator(s.data()), - xml_escape_translator(s.data()+ s.size()), - boost::archive::iterators::ostream_iterator(os) - ); -} - -template -BOOST_ARCHIVE_DECL void -xml_oarchive_impl::save(const char * s){ - typedef boost::archive::iterators::xml_escape< - const char * - > xml_escape_translator; - std::copy( - xml_escape_translator(s), - xml_escape_translator(s + std::strlen(s)), - boost::archive::iterators::ostream_iterator(os) - ); -} - -template -BOOST_ARCHIVE_DECL -xml_oarchive_impl::xml_oarchive_impl( - std::ostream & os_, - unsigned int flags -) : - basic_text_oprimitive( - os_, - 0 != (flags & no_codecvt) - ), - basic_xml_oarchive(flags) -{ - if(0 == (flags & no_header)) - this->init(); -} - -template -BOOST_ARCHIVE_DECL void -xml_oarchive_impl::save_binary(const void *address, std::size_t count){ - this->end_preamble(); - #if ! defined(__MWERKS__) - this->basic_text_oprimitive::save_binary( - #else - this->basic_text_oprimitive::save_binary( - #endif - address, - count - ); - this->indent_next = true; -} - -template -BOOST_ARCHIVE_DECL -xml_oarchive_impl::~xml_oarchive_impl(){ - if(std::uncaught_exception()) - return; - if(0 == (this->get_flags() & no_header)) - this->windup(); -} - -} // namespace archive -} // namespace boost diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_wiarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_wiarchive_impl.ipp deleted file mode 100644 index ee66c1263e6..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_wiarchive_impl.ipp +++ /dev/null @@ -1,189 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_wiarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::memcpy; -} //std -#endif - -#include // msvc 6.0 needs this to suppress warnings -#ifndef BOOST_NO_STD_WSTREAMBUF - -#include -#include // std::copy -#include // uncaught exception -#include // Dinkumware and RogueWave -#if BOOST_WORKAROUND(BOOST_DINKUMWARE_STDLIB, == 1) -#include -#endif - -#include -#include -#include - -#include -#include - -#include -#include - -#include - -#include "basic_xml_grammar.hpp" - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implemenations of functions specific to wide char archives - -namespace { // anonymous - -void copy_to_ptr(char * s, const std::wstring & ws){ - std::copy( - iterators::mb_from_wchar( - ws.begin() - ), - iterators::mb_from_wchar( - ws.end() - ), - s - ); - s[ws.size()] = 0; -} - -} // anonymous - -template -BOOST_WARCHIVE_DECL void -xml_wiarchive_impl::load(std::string & s){ - std::wstring ws; - bool result = gimpl->parse_string(is, ws); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); - #if BOOST_WORKAROUND(_RWSTD_VER, BOOST_TESTED_AT(20101)) - if(NULL != s.data()) - #endif - s.resize(0); - s.reserve(ws.size()); - std::copy( - iterators::mb_from_wchar( - ws.begin() - ), - iterators::mb_from_wchar( - ws.end() - ), - std::back_inserter(s) - ); -} - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_WARCHIVE_DECL void -xml_wiarchive_impl::load(std::wstring & ws){ - bool result = gimpl->parse_string(is, ws); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); -} -#endif - -template -BOOST_WARCHIVE_DECL void -xml_wiarchive_impl::load(char * s){ - std::wstring ws; - bool result = gimpl->parse_string(is, ws); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); - copy_to_ptr(s, ws); -} - -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_WARCHIVE_DECL void -xml_wiarchive_impl::load(wchar_t * ws){ - std::wstring twstring; - bool result = gimpl->parse_string(is, twstring); - if(! result) - boost::serialization::throw_exception( - xml_archive_exception(xml_archive_exception::xml_archive_parsing_error) - ); - std::memcpy(ws, twstring.c_str(), twstring.size()); - ws[twstring.size()] = L'\0'; -} -#endif - -template -BOOST_WARCHIVE_DECL void -xml_wiarchive_impl::load_override(class_name_type & t){ - const std::wstring & ws = gimpl->rv.class_name; - if(ws.size() > BOOST_SERIALIZATION_MAX_KEY_SIZE - 1) - boost::serialization::throw_exception( - archive_exception(archive_exception::invalid_class_name) - ); - copy_to_ptr(t, ws); -} - -template -BOOST_WARCHIVE_DECL void -xml_wiarchive_impl::init(){ - gimpl->init(is); - this->set_library_version( - library_version_type(gimpl->rv.version) - ); -} - -template -BOOST_WARCHIVE_DECL -xml_wiarchive_impl::xml_wiarchive_impl( - std::wistream &is_, - unsigned int flags -) : - basic_text_iprimitive( - is_, - true // don't change the codecvt - use the one below - ), - basic_xml_iarchive(flags), - gimpl(new xml_wgrammar()) -{ - if(0 == (flags & no_codecvt)){ - std::locale l = std::locale( - is_.getloc(), - new boost::archive::detail::utf8_codecvt_facet - ); - // libstdc++ crashes without this - is_.sync(); - is_.imbue(l); - } - if(0 == (flags & no_header)) - init(); -} - -template -BOOST_WARCHIVE_DECL -xml_wiarchive_impl::~xml_wiarchive_impl(){ - if(std::uncaught_exception()) - return; - if(0 == (this->get_flags() & no_header)){ - gimpl->windup(is); - } -} - -} // namespace archive -} // namespace boost - -#endif // BOOST_NO_STD_WSTREAMBUF diff --git a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_woarchive_impl.ipp b/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_woarchive_impl.ipp deleted file mode 100644 index 01b1a052d51..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/impl/xml_woarchive_impl.ipp +++ /dev/null @@ -1,171 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_woarchive_impl.ipp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include -#ifndef BOOST_NO_STD_WSTREAMBUF - -#include -#include -#include // std::copy -#include -#include - -#include // strlen -#include // mbtowc -#ifndef BOOST_NO_CWCHAR -#include // wcslen -#endif - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::strlen; - #if ! defined(BOOST_NO_INTRINSIC_WCHAR_T) - using ::mbtowc; - using ::wcslen; - #endif -} // namespace std -#endif - -#include -#include - -#include - -#include -#include -#include -#include - -namespace boost { -namespace archive { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implemenations of functions specific to wide char archives - -// copy chars to output escaping to xml and widening characters as we go -template -void save_iterator(std::wostream &os, InputIterator begin, InputIterator end){ - typedef iterators::wchar_from_mb< - iterators::xml_escape - > xmbtows; - std::copy( - xmbtows(begin), - xmbtows(end), - boost::archive::iterators::ostream_iterator(os) - ); -} - -template -BOOST_WARCHIVE_DECL void -xml_woarchive_impl::save(const std::string & s){ - // note: we don't use s.begin() and s.end() because dinkumware - // doesn't have string::value_type defined. So use a wrapper - // around these values to implement the definitions. - const char * begin = s.data(); - const char * end = begin + s.size(); - save_iterator(os, begin, end); -} - -#ifndef BOOST_NO_STD_WSTRING -template -BOOST_WARCHIVE_DECL void -xml_woarchive_impl::save(const std::wstring & ws){ -#if 0 - typedef iterators::xml_escape xmbtows; - std::copy( - xmbtows(ws.begin()), - xmbtows(ws.end()), - boost::archive::iterators::ostream_iterator(os) - ); -#endif - typedef iterators::xml_escape xmbtows; - std::copy( - xmbtows(ws.data()), - xmbtows(ws.data() + ws.size()), - boost::archive::iterators::ostream_iterator(os) - ); -} -#endif //BOOST_NO_STD_WSTRING - -template -BOOST_WARCHIVE_DECL void -xml_woarchive_impl::save(const char * s){ - save_iterator(os, s, s + std::strlen(s)); -} - -#ifndef BOOST_NO_INTRINSIC_WCHAR_T -template -BOOST_WARCHIVE_DECL void -xml_woarchive_impl::save(const wchar_t * ws){ - os << ws; - typedef iterators::xml_escape xmbtows; - std::copy( - xmbtows(ws), - xmbtows(ws + std::wcslen(ws)), - boost::archive::iterators::ostream_iterator(os) - ); -} -#endif - -template -BOOST_WARCHIVE_DECL -xml_woarchive_impl::xml_woarchive_impl( - std::wostream & os_, - unsigned int flags -) : - basic_text_oprimitive( - os_, - true // don't change the codecvt - use the one below - ), - basic_xml_oarchive(flags) -{ - if(0 == (flags & no_codecvt)){ - std::locale l = std::locale( - os_.getloc(), - new boost::archive::detail::utf8_codecvt_facet - ); - os_.flush(); - os_.imbue(l); - } - if(0 == (flags & no_header)) - this->init(); -} - -template -BOOST_WARCHIVE_DECL -xml_woarchive_impl::~xml_woarchive_impl(){ - if(std::uncaught_exception()) - return; - if(0 == (this->get_flags() & no_header)){ - save(L"\n"); - } -} - -template -BOOST_WARCHIVE_DECL void -xml_woarchive_impl::save_binary( - const void *address, - std::size_t count -){ - this->end_preamble(); - #if ! defined(__MWERKS__) - this->basic_text_oprimitive::save_binary( - #else - this->basic_text_oprimitive::save_binary( - #endif - address, - count - ); - this->indent_next = true; -} - -} // namespace archive -} // namespace boost - -#endif //BOOST_NO_STD_WSTREAMBUF diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_exception.hpp deleted file mode 100644 index 8f9208b60ea..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_exception.hpp +++ /dev/null @@ -1,68 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_BASE64_EXCEPTION_HPP -#define BOOST_ARCHIVE_ITERATORS_BASE64_EXCEPTION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// base64_exception.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifndef BOOST_NO_EXCEPTIONS -#include - -#include - -namespace boost { -namespace archive { -namespace iterators { - -////////////////////////////////////////////////////////////////////// -// exceptions thrown by base64s -// -class base64_exception : public std::exception -{ -public: - typedef enum { - invalid_code, // attempt to encode a value > 6 bits - invalid_character, // decode a value not in base64 char set - other_exception - } exception_code; - exception_code code; - - base64_exception(exception_code c = other_exception) : code(c) - {} - - virtual const char *what( ) const throw( ) - { - const char *msg = "unknown exception code"; - switch(code){ - case invalid_code: - msg = "attempt to encode a value > 6 bits"; - break; - case invalid_character: - msg = "attempt to decode a value not in base64 char set"; - break; - default: - BOOST_ASSERT(false); - break; - } - return msg; - } -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif //BOOST_NO_EXCEPTIONS -#endif //BOOST_ARCHIVE_ITERATORS_ARCHIVE_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_from_binary.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_from_binary.hpp deleted file mode 100644 index ee849944397..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/base64_from_binary.hpp +++ /dev/null @@ -1,109 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_BASE64_FROM_BINARY_HPP -#define BOOST_ARCHIVE_ITERATORS_BASE64_FROM_BINARY_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// base64_from_binary.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include // size_t -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// convert binary integers to base64 characters - -namespace detail { - -template -struct from_6_bit { - typedef CharType result_type; - CharType operator()(CharType t) const{ - static const char * lookup_table = - "ABCDEFGHIJKLMNOPQRSTUVWXYZ" - "abcdefghijklmnopqrstuvwxyz" - "0123456789" - "+/"; - BOOST_ASSERT(t < 64); - return lookup_table[static_cast(t)]; - } -}; - -} // namespace detail - -// note: what we would like to do is -// template -// typedef transform_iterator< -// from_6_bit, -// transform_width -// > base64_from_binary; -// but C++ won't accept this. Rather than using a "type generator" and -// using a different syntax, make a derivation which should be equivalent. -// -// Another issue addressed here is that the transform_iterator doesn't have -// a templated constructor. This makes it incompatible with the dataflow -// ideal. This is also addressed here. - -//template -template< - class Base, - class CharType = typename boost::iterator_value::type -> -class base64_from_binary : - public transform_iterator< - detail::from_6_bit, - Base - > -{ - friend class boost::iterator_core_access; - typedef transform_iterator< - typename detail::from_6_bit, - Base - > super_t; - -public: - // make composible buy using templated constructor - template - base64_from_binary(T start) : - super_t( - Base(static_cast< T >(start)), - detail::from_6_bit() - ) - {} - // intel 7.1 doesn't like default copy constructor - base64_from_binary(const base64_from_binary & rhs) : - super_t( - Base(rhs.base_reference()), - detail::from_6_bit() - ) - {} -// base64_from_binary(){}; -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_BASE64_FROM_BINARY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/binary_from_base64.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/binary_from_base64.hpp deleted file mode 100644 index 89b8f889da3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/binary_from_base64.hpp +++ /dev/null @@ -1,118 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_BINARY_FROM_BASE64_HPP -#define BOOST_ARCHIVE_ITERATORS_BINARY_FROM_BASE64_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// binary_from_base64.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// convert base64 characters to binary data - -namespace detail { - -template -struct to_6_bit { - typedef CharType result_type; - CharType operator()(CharType t) const{ - static const signed char lookup_table[] = { - -1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1, - -1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1, - -1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,62,-1,-1,-1,63, - 52,53,54,55,56,57,58,59,60,61,-1,-1,-1, 0,-1,-1, // render '=' as 0 - -1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9,10,11,12,13,14, - 15,16,17,18,19,20,21,22,23,24,25,-1,-1,-1,-1,-1, - -1,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,-1,-1,-1,-1,-1 - }; - // metrowerks trips this assertion - how come? - #if ! defined(__MWERKS__) - BOOST_STATIC_ASSERT(128 == sizeof(lookup_table)); - #endif - signed char value = -1; - if((unsigned)t <= 127) - value = lookup_table[(unsigned)t]; - if(-1 == value) - boost::serialization::throw_exception( - dataflow_exception(dataflow_exception::invalid_base64_character) - ); - return value; - } -}; - -} // namespace detail - -// note: what we would like to do is -// template -// typedef transform_iterator< -// from_6_bit, -// transform_width -// > base64_from_binary; -// but C++ won't accept this. Rather than using a "type generator" and -// using a different syntax, make a derivation which should be equivalent. -// -// Another issue addressed here is that the transform_iterator doesn't have -// a templated constructor. This makes it incompatible with the dataflow -// ideal. This is also addressed here. - -template< - class Base, - class CharType = typename boost::iterator_value::type -> -class binary_from_base64 : public - transform_iterator< - detail::to_6_bit, - Base - > -{ - friend class boost::iterator_core_access; - typedef transform_iterator< - detail::to_6_bit, - Base - > super_t; -public: - // make composible buy using templated constructor - template - binary_from_base64(T start) : - super_t( - Base(static_cast< T >(start)), - detail::to_6_bit() - ) - {} - // intel 7.1 doesn't like default copy constructor - binary_from_base64(const binary_from_base64 & rhs) : - super_t( - Base(rhs.base_reference()), - detail::to_6_bit() - ) - {} -// binary_from_base64(){}; -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_BINARY_FROM_BASE64_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow.hpp deleted file mode 100644 index 07733d5fd62..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow.hpp +++ /dev/null @@ -1,102 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_DATAFLOW_HPP -#define BOOST_ARCHIVE_ITERATORS_DATAFLOW_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// dataflow.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -// poor man's tri-state -struct tri_state { - enum state_enum { - is_false = false, - is_true = true, - is_indeterminant - } m_state; - // convert to bool - operator bool (){ - BOOST_ASSERT(is_indeterminant != m_state); - return is_true == m_state ? true : false; - } - // assign from bool - tri_state & operator=(bool rhs) { - m_state = rhs ? is_true : is_false; - return *this; - } - tri_state(bool rhs) : - m_state(rhs ? is_true : is_false) - {} - tri_state(state_enum state) : - m_state(state) - {} - bool operator==(const tri_state & rhs) const { - return m_state == rhs.m_state; - } - bool operator!=(const tri_state & rhs) const { - return m_state != rhs.m_state; - } -}; - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implement functions common to dataflow iterators -template -class dataflow { - bool m_eoi; -protected: - // test for iterator equality - tri_state equal(const Derived & rhs) const { - if(m_eoi && rhs.m_eoi) - return true; - if(m_eoi || rhs.m_eoi) - return false; - return tri_state(tri_state::is_indeterminant); - } - void eoi(bool tf){ - m_eoi = tf; - } - bool eoi() const { - return m_eoi; - } -public: - dataflow(bool tf) : - m_eoi(tf) - {} - dataflow() : // used for iterator end - m_eoi(true) - {} -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_DATAFLOW_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow_exception.hpp deleted file mode 100644 index e3e18605b38..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/dataflow_exception.hpp +++ /dev/null @@ -1,80 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_DATAFLOW_EXCEPTION_HPP -#define BOOST_ARCHIVE_ITERATORS_DATAFLOW_EXCEPTION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// dataflow_exception.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifndef BOOST_NO_EXCEPTIONS -#include -#endif //BOOST_NO_EXCEPTIONS - -#include - -namespace boost { -namespace archive { -namespace iterators { - -////////////////////////////////////////////////////////////////////// -// exceptions thrown by dataflows -// -class dataflow_exception : public std::exception -{ -public: - typedef enum { - invalid_6_bitcode, - invalid_base64_character, - invalid_xml_escape_sequence, - comparison_not_permitted, - invalid_conversion, - other_exception - } exception_code; - exception_code code; - - dataflow_exception(exception_code c = other_exception) : code(c) - {} - - virtual const char *what( ) const throw( ) - { - const char *msg = "unknown exception code"; - switch(code){ - case invalid_6_bitcode: - msg = "attempt to encode a value > 6 bits"; - break; - case invalid_base64_character: - msg = "attempt to decode a value not in base64 char set"; - break; - case invalid_xml_escape_sequence: - msg = "invalid xml escape_sequence"; - break; - case comparison_not_permitted: - msg = "cannot invoke iterator comparison now"; - break; - case invalid_conversion: - msg = "invalid multbyte/wide char conversion"; - break; - default: - BOOST_ASSERT(false); - break; - } - return msg; - } -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif //BOOST_ARCHIVE_ITERATORS_DATAFLOW_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/escape.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/escape.hpp deleted file mode 100644 index 103b31e0fef..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/escape.hpp +++ /dev/null @@ -1,115 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_ESCAPE_HPP -#define BOOST_ARCHIVE_ITERATORS_ESCAPE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// escape.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // NULL - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// insert escapes into text - -template -class escape : - public boost::iterator_adaptor< - Derived, - Base, - typename boost::iterator_value::type, - single_pass_traversal_tag, - typename boost::iterator_value::type - > -{ - typedef typename boost::iterator_value::type base_value_type; - typedef typename boost::iterator_reference::type reference_type; - friend class boost::iterator_core_access; - - typedef typename boost::iterator_adaptor< - Derived, - Base, - base_value_type, - single_pass_traversal_tag, - base_value_type - > super_t; - - typedef escape this_t; - - void dereference_impl() { - m_current_value = static_cast(this)->fill(m_bnext, m_bend); - m_full = true; - } - - //Access the value referred to - reference_type dereference() const { - if(!m_full) - const_cast(this)->dereference_impl(); - return m_current_value; - } - - bool equal(const this_t & rhs) const { - if(m_full){ - if(! rhs.m_full) - const_cast(& rhs)->dereference_impl(); - } - else{ - if(rhs.m_full) - const_cast(this)->dereference_impl(); - } - if(m_bnext != rhs.m_bnext) - return false; - if(this->base_reference() != rhs.base_reference()) - return false; - return true; - } - - void increment(){ - if(++m_bnext < m_bend){ - m_current_value = *m_bnext; - return; - } - ++(this->base_reference()); - m_bnext = NULL; - m_bend = NULL; - m_full = false; - } - - // buffer to handle pending characters - const base_value_type *m_bnext; - const base_value_type *m_bend; - bool m_full; - base_value_type m_current_value; -public: - escape(Base base) : - super_t(base), - m_bnext(NULL), - m_bend(NULL), - m_full(false), - m_current_value(0) - { - } -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_ESCAPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/insert_linebreaks.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/insert_linebreaks.hpp deleted file mode 100644 index 2504b030db1..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/insert_linebreaks.hpp +++ /dev/null @@ -1,99 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_INSERT_LINEBREAKS_HPP -#define BOOST_ARCHIVE_ITERATORS_INSERT_LINEBREAKS_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// insert_linebreaks.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ using ::memcpy; } -#endif - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// insert line break every N characters -template< - class Base, - int N, - class CharType = typename boost::iterator_value::type -> -class insert_linebreaks : - public iterator_adaptor< - insert_linebreaks, - Base, - CharType, - single_pass_traversal_tag, - CharType - > -{ -private: - friend class boost::iterator_core_access; - typedef iterator_adaptor< - insert_linebreaks, - Base, - CharType, - single_pass_traversal_tag, - CharType - > super_t; - - bool equal(const insert_linebreaks & rhs) const { - return -// m_count == rhs.m_count -// && base_reference() == rhs.base_reference() - this->base_reference() == rhs.base_reference() - ; - } - - void increment() { - if(m_count == N){ - m_count = 0; - return; - } - ++m_count; - ++(this->base_reference()); - } - CharType dereference() const { - if(m_count == N) - return '\n'; - return * (this->base_reference()); - } - unsigned int m_count; -public: - // make composible buy using templated constructor - template - insert_linebreaks(T start) : - super_t(Base(static_cast< T >(start))), - m_count(0) - {} - // intel 7.1 doesn't like default copy constructor - insert_linebreaks(const insert_linebreaks & rhs) : - super_t(rhs.base_reference()), - m_count(rhs.m_count) - {} -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_INSERT_LINEBREAKS_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/istream_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/istream_iterator.hpp deleted file mode 100644 index a187f605e69..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/istream_iterator.hpp +++ /dev/null @@ -1,92 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_ISTREAM_ITERATOR_HPP -#define BOOST_ARCHIVE_ITERATORS_ISTREAM_ITERATOR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// istream_iterator.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// note: this is a custom version of the standard istream_iterator. -// This is necessary as the standard version doesn't work as expected -// for wchar_t based streams on systems for which wchar_t not a true -// type but rather a synonym for some integer type. - -#include // NULL -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -// given a type, make an input iterator based on a pointer to that type -template -class istream_iterator : - public boost::iterator_facade< - istream_iterator, - Elem, - std::input_iterator_tag, - Elem - > -{ - friend class boost::iterator_core_access; - typedef istream_iterator this_t ; - typedef typename boost::iterator_facade< - istream_iterator, - Elem, - std::input_iterator_tag, - Elem - > super_t; - typedef typename std::basic_istream istream_type; - - bool equal(const this_t & rhs) const { - // note: only works for comparison against end of stream - return m_istream == rhs.m_istream; - } - - //Access the value referred to - Elem dereference() const { - return static_cast(m_istream->peek()); - } - - void increment(){ - if(NULL != m_istream){ - m_istream->ignore(1); - } - } - - istream_type *m_istream; - Elem m_current_value; -public: - istream_iterator(istream_type & is) : - m_istream(& is) - { - //increment(); - } - - istream_iterator() : - m_istream(NULL), - m_current_value(NULL) - {} - - istream_iterator(const istream_iterator & rhs) : - m_istream(rhs.m_istream), - m_current_value(rhs.m_current_value) - {} -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_ISTREAM_ITERATOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/mb_from_wchar.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/mb_from_wchar.hpp deleted file mode 100644 index 05df71c258e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/mb_from_wchar.hpp +++ /dev/null @@ -1,139 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_MB_FROM_WCHAR_HPP -#define BOOST_ARCHIVE_ITERATORS_MB_FROM_WCHAR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// mb_from_wchar.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // size_t -#ifndef BOOST_NO_CWCHAR -#include // mbstate_t -#endif -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::mbstate_t; -} // namespace std -#endif - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// class used by text archives to translate wide strings and to char -// strings of the currently selected locale -template // the input iterator -class mb_from_wchar - : public boost::iterator_adaptor< - mb_from_wchar, - Base, - wchar_t, - single_pass_traversal_tag, - char - > -{ - friend class boost::iterator_core_access; - - typedef typename boost::iterator_adaptor< - mb_from_wchar, - Base, - wchar_t, - single_pass_traversal_tag, - char - > super_t; - - typedef mb_from_wchar this_t; - - char dereference_impl() { - if(! m_full){ - fill(); - m_full = true; - } - return m_buffer[m_bnext]; - } - - char dereference() const { - return (const_cast(this))->dereference_impl(); - } - // test for iterator equality - bool equal(const mb_from_wchar & rhs) const { - // once the value is filled, the base_reference has been incremented - // so don't permit comparison anymore. - return - 0 == m_bend - && 0 == m_bnext - && this->base_reference() == rhs.base_reference() - ; - } - - void fill(){ - wchar_t value = * this->base_reference(); - const wchar_t *wend; - char *bend; - std::codecvt_base::result r = m_codecvt_facet.out( - m_mbs, - & value, & value + 1, wend, - m_buffer, m_buffer + sizeof(m_buffer), bend - ); - BOOST_ASSERT(std::codecvt_base::ok == r); - m_bnext = 0; - m_bend = bend - m_buffer; - } - - void increment(){ - if(++m_bnext < m_bend) - return; - m_bend = - m_bnext = 0; - ++(this->base_reference()); - m_full = false; - } - - boost::archive::detail::utf8_codecvt_facet m_codecvt_facet; - std::mbstate_t m_mbs; - // buffer to handle pending characters - char m_buffer[9 /* MB_CUR_MAX */]; - std::size_t m_bend; - std::size_t m_bnext; - bool m_full; - -public: - // make composible buy using templated constructor - template - mb_from_wchar(T start) : - super_t(Base(static_cast< T >(start))), - m_mbs(std::mbstate_t()), - m_bend(0), - m_bnext(0), - m_full(false) - {} - // intel 7.1 doesn't like default copy constructor - mb_from_wchar(const mb_from_wchar & rhs) : - super_t(rhs.base_reference()), - m_bend(rhs.m_bend), - m_bnext(rhs.m_bnext), - m_full(rhs.m_full) - {} -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_MB_FROM_WCHAR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/ostream_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/ostream_iterator.hpp deleted file mode 100644 index 49a9b99034b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/ostream_iterator.hpp +++ /dev/null @@ -1,83 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_OSTREAM_ITERATOR_HPP -#define BOOST_ARCHIVE_ITERATORS_OSTREAM_ITERATOR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// ostream_iterator.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// note: this is a custom version of the standard ostream_iterator. -// This is necessary as the standard version doesn't work as expected -// for wchar_t based streams on systems for which wchar_t not a true -// type but rather a synonym for some integer type. - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -// given a type, make an input iterator based on a pointer to that type -template -class ostream_iterator : - public boost::iterator_facade< - ostream_iterator, - Elem, - std::output_iterator_tag, - ostream_iterator & - > -{ - friend class boost::iterator_core_access; - typedef ostream_iterator this_t ; - typedef Elem char_type; - typedef std::basic_ostream ostream_type; - - //emulate the behavior of std::ostream - ostream_iterator & dereference() const { - return const_cast(*this); - } - bool equal(const this_t & rhs) const { - return m_ostream == rhs.m_ostream; - } - void increment(){} -protected: - ostream_type *m_ostream; - void put_val(char_type e){ - if(NULL != m_ostream){ - m_ostream->put(e); - if(! m_ostream->good()) - m_ostream = NULL; - } - } -public: - this_t & operator=(char_type c){ - put_val(c); - return *this; - } - ostream_iterator(ostream_type & os) : - m_ostream (& os) - {} - ostream_iterator() : - m_ostream (NULL) - {} - ostream_iterator(const ostream_iterator & rhs) : - m_ostream (rhs.m_ostream) - {} -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_OSTREAM_ITERATOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/remove_whitespace.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/remove_whitespace.hpp deleted file mode 100644 index c3580ab258a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/remove_whitespace.hpp +++ /dev/null @@ -1,167 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_REMOVE_WHITESPACE_HPP -#define BOOST_ARCHIVE_ITERATORS_REMOVE_WHITESPACE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// remove_whitespace.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include -#include - -// here is the default standard implementation of the functor used -// by the filter iterator to remove spaces. Unfortunately usage -// of this implementation in combination with spirit trips a bug -// VC 6.5. The only way I can find to work around it is to -// implement a special non-standard version for this platform - -#ifndef BOOST_NO_CWCTYPE -#include // iswspace -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ using ::iswspace; } -#endif -#endif - -#include // isspace -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ using ::isspace; } -#endif - -#if defined(__STD_RWCOMPILER_H__) || defined(_RWSTD_VER) -// this is required for the RW STL on Linux and Tru64. -#undef isspace -#undef iswspace -#endif - -namespace { // anonymous - -template -struct remove_whitespace_predicate; - -template<> -struct remove_whitespace_predicate -{ - bool operator()(unsigned char t){ - return ! std::isspace(t); - } -}; - -#ifndef BOOST_NO_CWCHAR -template<> -struct remove_whitespace_predicate -{ - bool operator()(wchar_t t){ - return ! std::iswspace(t); - } -}; -#endif - -} // namespace anonymous - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// convert base64 file data (including whitespace and padding) to binary - -namespace boost { -namespace archive { -namespace iterators { - -// custom version of filter iterator which doesn't look ahead further than -// necessary - -template -class filter_iterator - : public boost::iterator_adaptor< - filter_iterator, - Base, - use_default, - single_pass_traversal_tag - > -{ - friend class boost::iterator_core_access; - typedef typename boost::iterator_adaptor< - filter_iterator, - Base, - use_default, - single_pass_traversal_tag - > super_t; - typedef filter_iterator this_t; - typedef typename super_t::reference reference_type; - - reference_type dereference_impl(){ - if(! m_full){ - while(! m_predicate(* this->base_reference())) - ++(this->base_reference()); - m_full = true; - } - return * this->base_reference(); - } - - reference_type dereference() const { - return const_cast(this)->dereference_impl(); - } - - Predicate m_predicate; - bool m_full; -public: - // note: this function is public only because comeau compiler complained - // I don't know if this is because the compiler is wrong or what - void increment(){ - m_full = false; - ++(this->base_reference()); - } - filter_iterator(Base start) : - super_t(start), - m_full(false) - {} - filter_iterator(){} -}; - -template -class remove_whitespace : - public filter_iterator< - remove_whitespace_predicate< - typename boost::iterator_value::type - //typename Base::value_type - >, - Base - > -{ - friend class boost::iterator_core_access; - typedef filter_iterator< - remove_whitespace_predicate< - typename boost::iterator_value::type - //typename Base::value_type - >, - Base - > super_t; -public: -// remove_whitespace(){} // why is this needed? - // make composible buy using templated constructor - template - remove_whitespace(T start) : - super_t(Base(static_cast< T >(start))) - {} - // intel 7.1 doesn't like default copy constructor - remove_whitespace(const remove_whitespace & rhs) : - super_t(rhs.base_reference()) - {} -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_REMOVE_WHITESPACE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/transform_width.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/transform_width.hpp deleted file mode 100644 index 09c050a9274..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/transform_width.hpp +++ /dev/null @@ -1,177 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_TRANSFORM_WIDTH_HPP -#define BOOST_ARCHIVE_ITERATORS_TRANSFORM_WIDTH_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// transform_width.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// iterator which takes elements of x bits and returns elements of y bits. -// used to change streams of 8 bit characters into streams of 6 bit characters. -// and vice-versa for implementing base64 encodeing/decoding. Be very careful -// when using and end iterator. end is only reliable detected when the input -// stream length is some common multiple of x and y. E.G. Base64 6 bit -// character and 8 bit bytes. Lowest common multiple is 24 => 4 6 bit characters -// or 3 8 bit characters - -#include -#include - -#include // std::min - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// class used by text archives to translate char strings to wchar_t -// strings of the currently selected locale -template< - class Base, - int BitsOut, - int BitsIn, - class CharType = typename boost::iterator_value::type // output character -> -class transform_width : - public boost::iterator_adaptor< - transform_width, - Base, - CharType, - single_pass_traversal_tag, - CharType - > -{ - friend class boost::iterator_core_access; - typedef typename boost::iterator_adaptor< - transform_width, - Base, - CharType, - single_pass_traversal_tag, - CharType - > super_t; - - typedef transform_width this_t; - typedef typename iterator_value::type base_value_type; - - void fill(); - - CharType dereference() const { - if(!m_buffer_out_full) - const_cast(this)->fill(); - return m_buffer_out; - } - - bool equal_impl(const this_t & rhs){ - if(BitsIn < BitsOut) // discard any left over bits - return this->base_reference() == rhs.base_reference(); - else{ - // BitsIn > BitsOut // zero fill - if(this->base_reference() == rhs.base_reference()){ - m_end_of_sequence = true; - return 0 == m_remaining_bits; - } - return false; - } - } - - // standard iterator interface - bool equal(const this_t & rhs) const { - return const_cast(this)->equal_impl(rhs); - } - - void increment(){ - m_buffer_out_full = false; - } - - bool m_buffer_out_full; - CharType m_buffer_out; - - // last read element from input - base_value_type m_buffer_in; - - // number of bits to left in the input buffer. - unsigned int m_remaining_bits; - - // flag to indicate we've reached end of data. - bool m_end_of_sequence; - -public: - // make composible buy using templated constructor - template - transform_width(T start) : - super_t(Base(static_cast< T >(start))), - m_buffer_out_full(false), - m_buffer_out(0), - // To disable GCC warning, but not truly necessary - //(m_buffer_in will be initialized later before being - //used because m_remaining_bits == 0) - m_buffer_in(0), - m_remaining_bits(0), - m_end_of_sequence(false) - {} - // intel 7.1 doesn't like default copy constructor - transform_width(const transform_width & rhs) : - super_t(rhs.base_reference()), - m_buffer_out_full(rhs.m_buffer_out_full), - m_buffer_out(rhs.m_buffer_out), - m_buffer_in(rhs.m_buffer_in), - m_remaining_bits(rhs.m_remaining_bits), - m_end_of_sequence(false) - {} -}; - -template< - class Base, - int BitsOut, - int BitsIn, - class CharType -> -void transform_width::fill() { - unsigned int missing_bits = BitsOut; - m_buffer_out = 0; - do{ - if(0 == m_remaining_bits){ - if(m_end_of_sequence){ - m_buffer_in = 0; - m_remaining_bits = missing_bits; - } - else{ - m_buffer_in = * this->base_reference()++; - m_remaining_bits = BitsIn; - } - } - - // append these bits to the next output - // up to the size of the output - unsigned int i = (std::min)(missing_bits, m_remaining_bits); - // shift interesting bits to least significant position - base_value_type j = m_buffer_in >> (m_remaining_bits - i); - // and mask off the un interesting higher bits - // note presumption of twos complement notation - j &= (1 << i) - 1; - // append then interesting bits to the output value - m_buffer_out <<= i; - m_buffer_out |= j; - - // and update counters - missing_bits -= i; - m_remaining_bits -= i; - }while(0 < missing_bits); - m_buffer_out_full = true; -} - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_TRANSFORM_WIDTH_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/unescape.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/unescape.hpp deleted file mode 100644 index abf62406088..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/unescape.hpp +++ /dev/null @@ -1,89 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_UNESCAPE_HPP -#define BOOST_ARCHIVE_ITERATORS_UNESCAPE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// unescape.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// class used by text archives to translate char strings to wchar_t -// strings of the currently selected locale -template -class unescape - : public boost::iterator_adaptor< - unescape, - Base, - typename pointee::type, - single_pass_traversal_tag, - typename pointee::type - > -{ - friend class boost::iterator_core_access; - typedef typename boost::iterator_adaptor< - unescape, - Base, - typename pointee::type, - single_pass_traversal_tag, - typename pointee::type - > super_t; - - typedef unescape this_t; -public: - typedef typename this_t::value_type value_type; - typedef typename this_t::reference reference; -private: - value_type dereference_impl() { - if(! m_full){ - m_current_value = static_cast(this)->drain(); - m_full = true; - } - return m_current_value; - } - - reference dereference() const { - return const_cast(this)->dereference_impl(); - } - - value_type m_current_value; - bool m_full; - - void increment(){ - ++(this->base_reference()); - dereference_impl(); - m_full = false; - }; - -public: - - unescape(Base base) : - super_t(base), - m_full(false) - {} - -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_UNESCAPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/wchar_from_mb.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/wchar_from_mb.hpp deleted file mode 100644 index 2af8f6401f2..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/wchar_from_mb.hpp +++ /dev/null @@ -1,194 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_WCHAR_FROM_MB_HPP -#define BOOST_ARCHIVE_ITERATORS_WCHAR_FROM_MB_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// wchar_from_mb.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include // size_t -#ifndef BOOST_NO_CWCHAR -#include // mbstate_t -#endif -#include // copy - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::mbstate_t; -} // namespace std -#endif -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// class used by text archives to translate char strings to wchar_t -// strings of the currently selected locale -template -class wchar_from_mb - : public boost::iterator_adaptor< - wchar_from_mb, - Base, - wchar_t, - single_pass_traversal_tag, - wchar_t - > -{ - friend class boost::iterator_core_access; - typedef typename boost::iterator_adaptor< - wchar_from_mb, - Base, - wchar_t, - single_pass_traversal_tag, - wchar_t - > super_t; - - typedef wchar_from_mb this_t; - - void drain(); - - wchar_t dereference() const { - if(m_output.m_next == m_output.m_next_available) - return static_cast(0); - return * m_output.m_next; - } - - void increment(){ - if(m_output.m_next == m_output.m_next_available) - return; - if(++m_output.m_next == m_output.m_next_available){ - if(m_input.m_done) - return; - drain(); - } - } - - bool equal(this_t const & rhs) const { - return dereference() == rhs.dereference(); - } - - boost::archive::detail::utf8_codecvt_facet m_codecvt_facet; - std::mbstate_t m_mbs; - - template - struct sliding_buffer { - boost::array m_buffer; - typename boost::array::const_iterator m_next_available; - typename boost::array::iterator m_next; - bool m_done; - // default ctor - sliding_buffer() : - m_next_available(m_buffer.begin()), - m_next(m_buffer.begin()), - m_done(false) - {} - // copy ctor - sliding_buffer(const sliding_buffer & rhs) : - m_next_available( - std::copy( - rhs.m_buffer.begin(), - rhs.m_next_available, - m_buffer.begin() - ) - ), - m_next( - m_buffer.begin() + (rhs.m_next - rhs.m_buffer.begin()) - ), - m_done(rhs.m_done) - {} - }; - - sliding_buffer::type> m_input; - sliding_buffer::type> m_output; - -public: - // make composible buy using templated constructor - template - wchar_from_mb(T start) : - super_t(Base(static_cast< T >(start))), - m_mbs(std::mbstate_t()) - { - BOOST_ASSERT(std::mbsinit(&m_mbs)); - drain(); - } - // default constructor used as an end iterator - wchar_from_mb(){} - - // copy ctor - wchar_from_mb(const wchar_from_mb & rhs) : - super_t(rhs.base_reference()), - m_mbs(rhs.m_mbs), - m_input(rhs.m_input), - m_output(rhs.m_output) - {} -}; - -template -void wchar_from_mb::drain(){ - BOOST_ASSERT(! m_input.m_done); - for(;;){ - typename boost::iterators::iterator_reference::type c = *(this->base_reference()); - // a null character in a multibyte stream is takes as end of string - if(0 == c){ - m_input.m_done = true; - break; - } - ++(this->base_reference()); - * const_cast::type *>( - (m_input.m_next_available++) - ) = c; - // if input buffer is full - we're done for now - if(m_input.m_buffer.end() == m_input.m_next_available) - break; - } - const typename boost::iterators::iterator_value::type * input_new_start; - typename iterator_value::type * next_available; - - std::codecvt_base::result r = m_codecvt_facet.in( - m_mbs, - m_input.m_buffer.begin(), - m_input.m_next_available, - input_new_start, - m_output.m_buffer.begin(), - m_output.m_buffer.end(), - next_available - ); - BOOST_ASSERT(std::codecvt_base::ok == r); - m_output.m_next_available = next_available; - m_output.m_next = m_output.m_buffer.begin(); - - // we're done with some of the input so shift left. - m_input.m_next_available = std::copy( - input_new_start, - m_input.m_next_available, - m_input.m_buffer.begin() - ); - m_input.m_next = m_input.m_buffer.begin(); -} - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_WCHAR_FROM_MB_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_escape.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_escape.hpp deleted file mode 100644 index c838a73b864..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_escape.hpp +++ /dev/null @@ -1,121 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_XML_ESCAPE_HPP -#define BOOST_ARCHIVE_ITERATORS_XML_ESCAPE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_escape.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// insert escapes into xml text - -template -class xml_escape - : public escape, Base> -{ - friend class boost::iterator_core_access; - - typedef escape, Base> super_t; - -public: - char fill(const char * & bstart, const char * & bend); - wchar_t fill(const wchar_t * & bstart, const wchar_t * & bend); - - template - xml_escape(T start) : - super_t(Base(static_cast< T >(start))) - {} - // intel 7.1 doesn't like default copy constructor - xml_escape(const xml_escape & rhs) : - super_t(rhs.base_reference()) - {} -}; - -template -char xml_escape::fill( - const char * & bstart, - const char * & bend -){ - char current_value = * this->base_reference(); - switch(current_value){ - case '<': - bstart = "<"; - bend = bstart + 4; - break; - case '>': - bstart = ">"; - bend = bstart + 4; - break; - case '&': - bstart = "&"; - bend = bstart + 5; - break; - case '"': - bstart = """; - bend = bstart + 6; - break; - case '\'': - bstart = "'"; - bend = bstart + 6; - break; - default: - return current_value; - } - return *bstart; -} - -template -wchar_t xml_escape::fill( - const wchar_t * & bstart, - const wchar_t * & bend -){ - wchar_t current_value = * this->base_reference(); - switch(current_value){ - case '<': - bstart = L"<"; - bend = bstart + 4; - break; - case '>': - bstart = L">"; - bend = bstart + 4; - break; - case '&': - bstart = L"&"; - bend = bstart + 5; - break; - case '"': - bstart = L"""; - bend = bstart + 6; - break; - case '\'': - bstart = L"'"; - bend = bstart + 6; - break; - default: - return current_value; - } - return *bstart; -} - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_XML_ESCAPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape.hpp deleted file mode 100644 index 69977404567..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape.hpp +++ /dev/null @@ -1,125 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_HPP -#define BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_unescape.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include - -namespace boost { -namespace archive { -namespace iterators { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// replace &??? xml escape sequences with the corresponding characters -template -class xml_unescape - : public unescape, Base> -{ - friend class boost::iterator_core_access; - typedef xml_unescape this_t; - typedef unescape super_t; - typedef typename boost::iterator_reference reference_type; - - reference_type dereference() const { - return unescape, Base>::dereference(); - } -public: - // workaround msvc 7.1 ICU crash - #if defined(BOOST_MSVC) - typedef int value_type; - #else - typedef typename this_t::value_type value_type; - #endif - - void drain_residue(const char *literal); - value_type drain(); - - template - xml_unescape(T start) : - super_t(Base(static_cast< T >(start))) - {} - // intel 7.1 doesn't like default copy constructor - xml_unescape(const xml_unescape & rhs) : - super_t(rhs.base_reference()) - {} -}; - -template -void xml_unescape::drain_residue(const char * literal){ - do{ - if(* literal != * ++(this->base_reference())) - boost::serialization::throw_exception( - dataflow_exception( - dataflow_exception::invalid_xml_escape_sequence - ) - ); - } - while('\0' != * ++literal); -} - -// note key constraint on this function is that can't "look ahead" any -// more than necessary into base iterator. Doing so would alter the base -// iterator refenence which would make subsequent iterator comparisons -// incorrect and thereby break the composiblity of iterators. -template -typename xml_unescape::value_type -//int -xml_unescape::drain(){ - value_type retval = * this->base_reference(); - if('&' != retval){ - return retval; - } - retval = * ++(this->base_reference()); - switch(retval){ - case 'l': // < - drain_residue("t;"); - retval = '<'; - break; - case 'g': // > - drain_residue("t;"); - retval = '>'; - break; - case 'a': - retval = * ++(this->base_reference()); - switch(retval){ - case 'p': // ' - drain_residue("os;"); - retval = '\''; - break; - case 'm': // & - drain_residue("p;"); - retval = '&'; - break; - } - break; - case 'q': - drain_residue("uot;"); - retval = '"'; - break; - } - return retval; -} - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif // BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape_exception.hpp deleted file mode 100644 index 71a64378c20..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/iterators/xml_unescape_exception.hpp +++ /dev/null @@ -1,49 +0,0 @@ -#ifndef BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_EXCEPTION_HPP -#define BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_EXCEPTION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_unescape_exception.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifndef BOOST_NO_EXCEPTIONS -#include - -#include - -namespace boost { -namespace archive { -namespace iterators { - -////////////////////////////////////////////////////////////////////// -// exceptions thrown by xml_unescapes -// -class xml_unescape_exception : public std::exception -{ -public: - xml_unescape_exception() - {} - - virtual const char *what( ) const throw( ) - { - return "xml contained un-recognized escape code"; - } -}; - -} // namespace iterators -} // namespace archive -} // namespace boost - -#endif //BOOST_NO_EXCEPTIONS -#endif //BOOST_ARCHIVE_ITERATORS_XML_UNESCAPE_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_iarchive.hpp deleted file mode 100644 index 4a898a8ad16..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_iarchive.hpp +++ /dev/null @@ -1,54 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_BINARY_IARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_BINARY_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_binary_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class polymorphic_binary_iarchive : - public detail::polymorphic_iarchive_route -{ -public: - polymorphic_binary_iarchive(std::istream & is, unsigned int flags = 0) : - detail::polymorphic_iarchive_route(is, flags) - {} - ~polymorphic_binary_iarchive(){} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_binary_iarchive -) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_BINARY_IARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_oarchive.hpp deleted file mode 100644 index 931b243feb8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_binary_oarchive.hpp +++ /dev/null @@ -1,43 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_BINARY_OARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_BINARY_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_binary_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -namespace boost { -namespace archive { - -typedef detail::polymorphic_oarchive_route< - binary_oarchive_impl< - binary_oarchive, - std::ostream::char_type, - std::ostream::traits_type - > - > polymorphic_binary_oarchive; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_binary_oarchive -) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_BINARY_OARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_iarchive.hpp deleted file mode 100644 index d3c59a9f0f4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_iarchive.hpp +++ /dev/null @@ -1,168 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_IARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // std::size_t -#include // ULONG_MAX -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include - -#include -#include -#include -#include - -#include -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization -namespace archive { -namespace detail { - class basic_iarchive; - class basic_iserializer; -} - -class polymorphic_iarchive; - -class BOOST_SYMBOL_VISIBLE polymorphic_iarchive_impl : - public detail::interface_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else - friend class detail::interface_iarchive; - friend class load_access; -#endif - // primitive types the only ones permitted by polymorphic archives - virtual void load(bool & t) = 0; - - virtual void load(char & t) = 0; - virtual void load(signed char & t) = 0; - virtual void load(unsigned char & t) = 0; - #ifndef BOOST_NO_CWCHAR - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - virtual void load(wchar_t & t) = 0; - #endif - #endif - virtual void load(short & t) = 0; - virtual void load(unsigned short & t) = 0; - virtual void load(int & t) = 0; - virtual void load(unsigned int & t) = 0; - virtual void load(long & t) = 0; - virtual void load(unsigned long & t) = 0; - - #if defined(BOOST_HAS_LONG_LONG) - virtual void load(boost::long_long_type & t) = 0; - virtual void load(boost::ulong_long_type & t) = 0; - #elif defined(BOOST_HAS_MS_INT64) - virtual void load(__int64 & t) = 0; - virtual void load(unsigned __int64 & t) = 0; - #endif - - virtual void load(float & t) = 0; - virtual void load(double & t) = 0; - - // string types are treated as primitives - virtual void load(std::string & t) = 0; - #ifndef BOOST_NO_STD_WSTRING - virtual void load(std::wstring & t) = 0; - #endif - - // used for xml and other tagged formats - virtual void load_start(const char * name) = 0; - virtual void load_end(const char * name) = 0; - virtual void register_basic_serializer(const detail::basic_iserializer & bis) = 0; - virtual detail::helper_collection & get_helper_collection() = 0; - - // msvc and borland won't automatically pass these to the base class so - // make it explicit here - template - void load_override(T & t) - { - archive::load(* this->This(), t); - } - // special treatment for name-value pairs. - template - void load_override( - const boost::serialization::nvp< T > & t - ){ - load_start(t.name()); - archive::load(* this->This(), t.value()); - load_end(t.name()); - } -protected: - virtual ~polymorphic_iarchive_impl(){}; -public: - // utility function implemented by all legal archives - virtual void set_library_version(library_version_type archive_library_version) = 0; - virtual library_version_type get_library_version() const = 0; - virtual unsigned int get_flags() const = 0; - virtual void delete_created_pointers() = 0; - virtual void reset_object_address( - const void * new_address, - const void * old_address - ) = 0; - - virtual void load_binary(void * t, std::size_t size) = 0; - - // these are used by the serialization library implementation. - virtual void load_object( - void *t, - const detail::basic_iserializer & bis - ) = 0; - virtual const detail::basic_pointer_iserializer * load_pointer( - void * & t, - const detail::basic_pointer_iserializer * bpis_ptr, - const detail::basic_pointer_iserializer * (*finder)( - const boost::serialization::extended_type_info & type - ) - ) = 0; -}; - -} // namespace archive -} // namespace boost - -#include // pops abi_suffix.hpp pragmas - -namespace boost { -namespace archive { - -class BOOST_SYMBOL_VISIBLE polymorphic_iarchive : - public polymorphic_iarchive_impl -{ -public: - virtual ~polymorphic_iarchive(){}; -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::polymorphic_iarchive) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_oarchive.hpp deleted file mode 100644 index edac4edb1e8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_oarchive.hpp +++ /dev/null @@ -1,154 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_OARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // size_t -#include // ULONG_MAX -#include - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include - -#include -#include // must be the last header - -namespace boost { -namespace serialization { - class extended_type_info; -} // namespace serialization -namespace archive { -namespace detail { - class basic_oarchive; - class basic_oserializer; -} - -class polymorphic_oarchive; - -class BOOST_SYMBOL_VISIBLE polymorphic_oarchive_impl : - public detail::interface_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else - friend class detail::interface_oarchive; - friend class save_access; -#endif - // primitive types the only ones permitted by polymorphic archives - virtual void save(const bool t) = 0; - - virtual void save(const char t) = 0; - virtual void save(const signed char t) = 0; - virtual void save(const unsigned char t) = 0; - #ifndef BOOST_NO_CWCHAR - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - virtual void save(const wchar_t t) = 0; - #endif - #endif - virtual void save(const short t) = 0; - virtual void save(const unsigned short t) = 0; - virtual void save(const int t) = 0; - virtual void save(const unsigned int t) = 0; - virtual void save(const long t) = 0; - virtual void save(const unsigned long t) = 0; - - #if defined(BOOST_HAS_LONG_LONG) - virtual void save(const boost::long_long_type t) = 0; - virtual void save(const boost::ulong_long_type t) = 0; - #elif defined(BOOST_HAS_MS_INT64) - virtual void save(const __int64 t) = 0; - virtual void save(const unsigned __int64 t) = 0; - #endif - - virtual void save(const float t) = 0; - virtual void save(const double t) = 0; - - // string types are treated as primitives - virtual void save(const std::string & t) = 0; - #ifndef BOOST_NO_STD_WSTRING - virtual void save(const std::wstring & t) = 0; - #endif - - virtual void save_null_pointer() = 0; - // used for xml and other tagged formats - virtual void save_start(const char * name) = 0; - virtual void save_end(const char * name) = 0; - virtual void register_basic_serializer(const detail::basic_oserializer & bos) = 0; - virtual detail::helper_collection & get_helper_collection() = 0; - - virtual void end_preamble() = 0; - - // msvc and borland won't automatically pass these to the base class so - // make it explicit here - template - void save_override(T & t) - { - archive::save(* this->This(), t); - } - // special treatment for name-value pairs. - template - void save_override( - const ::boost::serialization::nvp< T > & t - ){ - save_start(t.name()); - archive::save(* this->This(), t.const_value()); - save_end(t.name()); - } -protected: - virtual ~polymorphic_oarchive_impl(){}; -public: - // utility functions implemented by all legal archives - virtual unsigned int get_flags() const = 0; - virtual library_version_type get_library_version() const = 0; - virtual void save_binary(const void * t, std::size_t size) = 0; - - virtual void save_object( - const void *x, - const detail::basic_oserializer & bos - ) = 0; - virtual void save_pointer( - const void * t, - const detail::basic_pointer_oserializer * bpos_ptr - ) = 0; -}; - -// note: preserve naming symmetry -class BOOST_SYMBOL_VISIBLE polymorphic_oarchive : - public polymorphic_oarchive_impl -{ -public: - virtual ~polymorphic_oarchive(){}; -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::polymorphic_oarchive) - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_POLYMORPHIC_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_iarchive.hpp deleted file mode 100644 index 7bef2927865..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_iarchive.hpp +++ /dev/null @@ -1,54 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_TEXT_IARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_TEXT_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_text_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class polymorphic_text_iarchive : - public detail::polymorphic_iarchive_route -{ -public: - polymorphic_text_iarchive(std::istream & is, unsigned int flags = 0) : - detail::polymorphic_iarchive_route(is, flags) - {} - ~polymorphic_text_iarchive(){} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_text_iarchive -) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_TEXT_IARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_oarchive.hpp deleted file mode 100644 index 457aad9fd75..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_oarchive.hpp +++ /dev/null @@ -1,39 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_TEXT_OARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_TEXT_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_text_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -namespace boost { -namespace archive { - -typedef detail::polymorphic_oarchive_route< - text_oarchive_impl -> polymorphic_text_oarchive; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_text_oarchive -) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_TEXT_OARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_wiarchive.hpp deleted file mode 100644 index 8466f05d6a6..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_wiarchive.hpp +++ /dev/null @@ -1,59 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_TEXT_WIARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_TEXT_WIARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_text_wiarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class polymorphic_text_wiarchive : - public detail::polymorphic_iarchive_route -{ -public: - polymorphic_text_wiarchive(std::wistream & is, unsigned int flags = 0) : - detail::polymorphic_iarchive_route(is, flags) - {} - ~polymorphic_text_wiarchive(){} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_text_wiarchive -) - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_POLYMORPHIC_TEXT_WIARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_woarchive.hpp deleted file mode 100644 index 295625d1bcf..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_text_woarchive.hpp +++ /dev/null @@ -1,44 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_TEXT_WOARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_TEXT_WOARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_text_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include -#include - -namespace boost { -namespace archive { - -typedef detail::polymorphic_oarchive_route< - text_woarchive_impl -> polymorphic_text_woarchive; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_text_woarchive -) - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_POLYMORPHIC_TEXT_WOARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_iarchive.hpp deleted file mode 100644 index 4dc3f894b38..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_iarchive.hpp +++ /dev/null @@ -1,54 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_XML_IARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_XML_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_xml_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class polymorphic_xml_iarchive : - public detail::polymorphic_iarchive_route -{ -public: - polymorphic_xml_iarchive(std::istream & is, unsigned int flags = 0) : - detail::polymorphic_iarchive_route(is, flags) - {} - ~polymorphic_xml_iarchive(){} -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_xml_iarchive -) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_XML_IARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_oarchive.hpp deleted file mode 100644 index 514f9e530a8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_oarchive.hpp +++ /dev/null @@ -1,39 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_XML_OARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_XML_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_xml_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -namespace boost { -namespace archive { - -typedef detail::polymorphic_oarchive_route< - xml_oarchive_impl -> polymorphic_xml_oarchive; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_xml_oarchive -) - -#endif // BOOST_ARCHIVE_POLYMORPHIC_XML_OARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_wiarchive.hpp deleted file mode 100644 index d4ab731267f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_wiarchive.hpp +++ /dev/null @@ -1,50 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_XML_WIARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_XML_WIARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_xml_wiarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include -#include - -namespace boost { -namespace archive { - -class polymorphic_xml_wiarchive : - public detail::polymorphic_iarchive_route -{ -public: - polymorphic_xml_wiarchive(std::wistream & is, unsigned int flags = 0) : - detail::polymorphic_iarchive_route(is, flags) - {} - ~polymorphic_xml_wiarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_xml_wiarchive -) - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_POLYMORPHIC_XML_WIARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_woarchive.hpp deleted file mode 100644 index dd8963fbb14..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/polymorphic_xml_woarchive.hpp +++ /dev/null @@ -1,44 +0,0 @@ -#ifndef BOOST_ARCHIVE_POLYMORPHIC_XML_WOARCHIVE_HPP -#define BOOST_ARCHIVE_POLYMORPHIC_XML_WOARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// polymorphic_xml_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include -#include - -namespace boost { -namespace archive { - -typedef detail::polymorphic_oarchive_route< - xml_woarchive_impl -> polymorphic_xml_woarchive; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE( - boost::archive::polymorphic_xml_woarchive -) - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_POLYMORPHIC_XML_WOARCHIVE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/archive/text_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/text_iarchive.hpp deleted file mode 100644 index d9d60adf0b8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/text_iarchive.hpp +++ /dev/null @@ -1,132 +0,0 @@ -#ifndef BOOST_ARCHIVE_TEXT_IARCHIVE_HPP -#define BOOST_ARCHIVE_TEXT_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE text_iarchive_impl : - public basic_text_iprimitive, - public basic_text_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_iarchive; - friend class load_access; -#endif - template - void load(T & t){ - basic_text_iprimitive::load(t); - } - void load(version_type & t){ - unsigned int v; - load(v); - t = version_type(v); - } - void load(boost::serialization::item_version_type & t){ - unsigned int v; - load(v); - t = boost::serialization::item_version_type(v); - } - BOOST_ARCHIVE_DECL void - load(char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_ARCHIVE_DECL void - load(wchar_t * t); - #endif - BOOST_ARCHIVE_DECL void - load(std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_ARCHIVE_DECL void - load(std::wstring &ws); - #endif - template - void load_override(T & t){ - basic_text_iarchive::load_override(t); - } - BOOST_ARCHIVE_DECL void - load_override(class_name_type & t); - BOOST_ARCHIVE_DECL void - init(); - BOOST_ARCHIVE_DECL - text_iarchive_impl(std::istream & is, unsigned int flags); - // don't import inline definitions! leave this as a reminder. - //BOOST_ARCHIVE_DECL - ~text_iarchive_impl(){}; -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class BOOST_SYMBOL_VISIBLE text_iarchive : - public text_iarchive_impl{ -public: - text_iarchive(std::istream & is_, unsigned int flags = 0) : - // note: added _ to suppress useless gcc warning - text_iarchive_impl(is_, flags) - {} - ~text_iarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::text_iarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_TEXT_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/text_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/text_oarchive.hpp deleted file mode 100644 index 9ba0dafffb4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/text_oarchive.hpp +++ /dev/null @@ -1,121 +0,0 @@ -#ifndef BOOST_ARCHIVE_TEXT_OARCHIVE_HPP -#define BOOST_ARCHIVE_TEXT_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include // std::size_t - -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE text_oarchive_impl : - /* protected ? */ public basic_text_oprimitive, - public basic_text_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_oarchive; - friend class basic_text_oarchive; - friend class save_access; -#endif - template - void save(const T & t){ - this->newtoken(); - basic_text_oprimitive::save(t); - } - void save(const version_type & t){ - save(static_cast(t)); - } - void save(const boost::serialization::item_version_type & t){ - save(static_cast(t)); - } - BOOST_ARCHIVE_DECL void - save(const char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_ARCHIVE_DECL void - save(const wchar_t * t); - #endif - BOOST_ARCHIVE_DECL void - save(const std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_ARCHIVE_DECL void - save(const std::wstring &ws); - #endif - BOOST_ARCHIVE_DECL - text_oarchive_impl(std::ostream & os, unsigned int flags); - // don't import inline definitions! leave this as a reminder. - //BOOST_ARCHIVE_DECL - ~text_oarchive_impl(){}; -public: - BOOST_ARCHIVE_DECL void - save_binary(const void *address, std::size_t count); -}; - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from text_oarchive_impl instead. This will -// preserve correct static polymorphism. -class BOOST_SYMBOL_VISIBLE text_oarchive : - public text_oarchive_impl -{ -public: - text_oarchive(std::ostream & os_, unsigned int flags = 0) : - // note: added _ to suppress useless gcc warning - text_oarchive_impl(os_, flags) - {} - ~text_oarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::text_oarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_ARCHIVE_TEXT_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/text_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/text_wiarchive.hpp deleted file mode 100644 index 3adf068a51a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/text_wiarchive.hpp +++ /dev/null @@ -1,137 +0,0 @@ -#ifndef BOOST_ARCHIVE_TEXT_WIARCHIVE_HPP -#define BOOST_ARCHIVE_TEXT_WIARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_wiarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include - -#include -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE text_wiarchive_impl : - public basic_text_iprimitive, - public basic_text_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_iarchive; - friend load_access; - #else - friend class detail::interface_iarchive; - friend class load_access; - #endif -#endif - template - void load(T & t){ - basic_text_iprimitive::load(t); - } - void load(version_type & t){ - unsigned int v; - load(v); - t = version_type(v); - } - void load(boost::serialization::item_version_type & t){ - unsigned int v; - load(v); - t = boost::serialization::item_version_type(v); - } - BOOST_WARCHIVE_DECL void - load(char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_WARCHIVE_DECL void - load(wchar_t * t); - #endif - BOOST_WARCHIVE_DECL void - load(std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_WARCHIVE_DECL void - load(std::wstring &ws); - #endif - template - void load_override(T & t){ - basic_text_iarchive::load_override(t); - } - BOOST_WARCHIVE_DECL - text_wiarchive_impl(std::wistream & is, unsigned int flags); - ~text_wiarchive_impl(){}; -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class BOOST_SYMBOL_VISIBLE text_wiarchive : - public text_wiarchive_impl{ -public: - text_wiarchive(std::wistream & is, unsigned int flags = 0) : - text_wiarchive_impl(is, flags) - {} - ~text_wiarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::text_wiarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_TEXT_WIARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/text_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/text_woarchive.hpp deleted file mode 100644 index b6b4f8ed59a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/text_woarchive.hpp +++ /dev/null @@ -1,155 +0,0 @@ -#ifndef BOOST_ARCHIVE_TEXT_WOARCHIVE_HPP -#define BOOST_ARCHIVE_TEXT_WOARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// text_woarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include -#include // size_t - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE text_woarchive_impl : - public basic_text_oprimitive, - public basic_text_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - #if BOOST_WORKAROUND(BOOST_MSVC, < 1500) - // for some inexplicable reason insertion of "class" generates compile erro - // on msvc 7.1 - friend detail::interface_oarchive; - friend basic_text_oarchive; - friend save_access; - #else - friend class detail::interface_oarchive; - friend class basic_text_oarchive; - friend class save_access; - #endif -#endif - template - void save(const T & t){ - this->newtoken(); - basic_text_oprimitive::save(t); - } - void save(const version_type & t){ - save(static_cast(t)); - } - void save(const boost::serialization::item_version_type & t){ - save(static_cast(t)); - } - BOOST_WARCHIVE_DECL void - save(const char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_WARCHIVE_DECL void - save(const wchar_t * t); - #endif - BOOST_WARCHIVE_DECL void - save(const std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_WARCHIVE_DECL void - save(const std::wstring &ws); - #endif - text_woarchive_impl(std::wostream & os, unsigned int flags) : - basic_text_oprimitive( - os, - 0 != (flags & no_codecvt) - ), - basic_text_oarchive(flags) - { - if(0 == (flags & no_header)) - basic_text_oarchive::init(); - } -public: - void save_binary(const void *address, std::size_t count){ - put(static_cast('\n')); - this->end_preamble(); - #if ! defined(__MWERKS__) - this->basic_text_oprimitive::save_binary( - #else - this->basic_text_oprimitive::save_binary( - #endif - address, - count - ); - put(static_cast('\n')); - this->delimiter = this->none; - } - -}; - -// we use the following because we can't use -// typedef text_oarchive_impl > text_oarchive; - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from text_oarchive_impl instead. This will -// preserve correct static polymorphism. -class BOOST_SYMBOL_VISIBLE text_woarchive : - public text_woarchive_impl -{ -public: - text_woarchive(std::wostream & os, unsigned int flags = 0) : - text_woarchive_impl(os, flags) - {} - ~text_woarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::text_woarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_TEXT_WOARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/tmpdir.hpp b/contrib/libboost/boost_1_65_0/boost/archive/tmpdir.hpp deleted file mode 100644 index 400d23b9f68..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/tmpdir.hpp +++ /dev/null @@ -1,50 +0,0 @@ -#ifndef BOOST_ARCHIVE_TMPDIR_HPP -#define BOOST_ARCHIVE_TMPDIR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// tmpdir.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // getenv -#include // NULL -//#include - -#include -#ifdef BOOST_NO_STDC_NAMESPACE -namespace std { - using ::getenv; -} -#endif - -namespace boost { -namespace archive { - -inline const char * tmpdir(){ - const char *dirname; - dirname = std::getenv("TMP"); - if(NULL == dirname) - dirname = std::getenv("TMPDIR"); - if(NULL == dirname) - dirname = std::getenv("TEMP"); - if(NULL == dirname){ - //BOOST_ASSERT(false); // no temp directory found - dirname = "."; - } - return dirname; -} - -} // archive -} // boost - -#endif // BOOST_ARCHIVE_TMPDIR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/wcslen.hpp b/contrib/libboost/boost_1_65_0/boost/archive/wcslen.hpp deleted file mode 100644 index 0b60004f095..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/wcslen.hpp +++ /dev/null @@ -1,58 +0,0 @@ -#ifndef BOOST_ARCHIVE_WCSLEN_HPP -#define BOOST_ARCHIVE_WCSLEN_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// wcslen.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // size_t -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#ifndef BOOST_NO_CWCHAR - -// a couple of libraries which include wchar_t don't include -// wcslen - -#if defined(BOOST_DINKUMWARE_STDLIB) && BOOST_DINKUMWARE_STDLIB < 306 \ -|| defined(__LIBCOMO__) - -namespace std { -inline std::size_t wcslen(const wchar_t * ws) -{ - const wchar_t * eows = ws; - while(* eows != 0) - ++eows; - return eows - ws; -} -} // namespace std - -#else - -#ifndef BOOST_NO_CWCHAR -#include -#endif -#ifdef BOOST_NO_STDC_NAMESPACE -namespace std{ using ::wcslen; } -#endif - -#endif // wcslen - -#endif //BOOST_NO_CWCHAR - -#endif //BOOST_ARCHIVE_WCSLEN_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_archive_exception.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_archive_exception.hpp deleted file mode 100644 index 82c53ef5d3e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/xml_archive_exception.hpp +++ /dev/null @@ -1,57 +0,0 @@ -#ifndef BOOST_ARCHIVE_XML_ARCHIVE_EXCEPTION_HPP -#define BOOST_ARCHIVE_XML_ARCHIVE_EXCEPTION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_archive_exception.hpp: - -// (C) Copyright 2007 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include - -#include // must be the last header - -namespace boost { -namespace archive { - -////////////////////////////////////////////////////////////////////// -// exceptions thrown by xml archives -// -class BOOST_SYMBOL_VISIBLE xml_archive_exception : - public virtual boost::archive::archive_exception -{ -public: - typedef enum { - xml_archive_parsing_error, // see save_register - xml_archive_tag_mismatch, - xml_archive_tag_name_error - } exception_code; - BOOST_ARCHIVE_DECL xml_archive_exception( - exception_code c, - const char * e1 = NULL, - const char * e2 = NULL - ); - BOOST_ARCHIVE_DECL xml_archive_exception(xml_archive_exception const &) ; - virtual BOOST_ARCHIVE_DECL ~xml_archive_exception() BOOST_NOEXCEPT_OR_NOTHROW ; -}; - -}// namespace archive -}// namespace boost - -#include // pops abi_suffix.hpp pragmas - -#endif //BOOST_XML_ARCHIVE_ARCHIVE_EXCEPTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_iarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_iarchive.hpp deleted file mode 100644 index abd2f9fc4e3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/xml_iarchive.hpp +++ /dev/null @@ -1,142 +0,0 @@ -#ifndef BOOST_ARCHIVE_XML_IARCHIVE_HPP -#define BOOST_ARCHIVE_XML_IARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_iarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -template -class basic_xml_grammar; -typedef basic_xml_grammar xml_grammar; - -template -class BOOST_SYMBOL_VISIBLE xml_iarchive_impl : - public basic_text_iprimitive, - public basic_xml_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_iarchive; - friend class basic_xml_iarchive; - friend class load_access; -#endif - // use boost:scoped_ptr to implement automatic deletion; - boost::scoped_ptr gimpl; - - std::istream & get_is(){ - return is; - } - template - void load(T & t){ - basic_text_iprimitive::load(t); - } - void - load(version_type & t){ - unsigned int v; - load(v); - t = version_type(v); - } - void - load(boost::serialization::item_version_type & t){ - unsigned int v; - load(v); - t = boost::serialization::item_version_type(v); - } - BOOST_ARCHIVE_DECL void - load(char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_ARCHIVE_DECL void - load(wchar_t * t); - #endif - BOOST_ARCHIVE_DECL void - load(std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_ARCHIVE_DECL void - load(std::wstring &ws); - #endif - template - void load_override(T & t){ - basic_xml_iarchive::load_override(t); - } - BOOST_ARCHIVE_DECL void - load_override(class_name_type & t); - BOOST_ARCHIVE_DECL void - init(); - BOOST_ARCHIVE_DECL - xml_iarchive_impl(std::istream & is, unsigned int flags); - BOOST_ARCHIVE_DECL - ~xml_iarchive_impl(); -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class BOOST_SYMBOL_VISIBLE xml_iarchive : - public xml_iarchive_impl{ -public: - xml_iarchive(std::istream & is, unsigned int flags = 0) : - xml_iarchive_impl(is, flags) - {} - ~xml_iarchive(){}; -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::xml_iarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_XML_IARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_oarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_oarchive.hpp deleted file mode 100644 index eea12680372..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/xml_oarchive.hpp +++ /dev/null @@ -1,137 +0,0 @@ -#ifndef BOOST_ARCHIVE_XML_OARCHIVE_HPP -#define BOOST_ARCHIVE_XML_OARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_oarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include // size_t -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE xml_oarchive_impl : - public basic_text_oprimitive, - public basic_xml_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_oarchive; - friend class basic_xml_oarchive; - friend class save_access; -#endif - template - void save(const T & t){ - basic_text_oprimitive::save(t); - } - void - save(const version_type & t){ - save(static_cast(t)); - } - void - save(const boost::serialization::item_version_type & t){ - save(static_cast(t)); - } - BOOST_ARCHIVE_DECL void - save(const char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_ARCHIVE_DECL void - save(const wchar_t * t); - #endif - BOOST_ARCHIVE_DECL void - save(const std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_ARCHIVE_DECL void - save(const std::wstring &ws); - #endif - BOOST_ARCHIVE_DECL - xml_oarchive_impl(std::ostream & os, unsigned int flags); - BOOST_ARCHIVE_DECL - ~xml_oarchive_impl(); -public: - BOOST_ARCHIVE_DECL - void save_binary(const void *address, std::size_t count); -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -// we use the following because we can't use -// typedef xml_oarchive_impl > xml_oarchive; - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from xml_oarchive_impl instead. This will -// preserve correct static polymorphism. -class BOOST_SYMBOL_VISIBLE xml_oarchive : - public xml_oarchive_impl -{ -public: - xml_oarchive(std::ostream & os, unsigned int flags = 0) : - xml_oarchive_impl(os, flags) - {} - ~xml_oarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::xml_oarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_ARCHIVE_XML_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_wiarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_wiarchive.hpp deleted file mode 100644 index ac24289ac11..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/xml_wiarchive.hpp +++ /dev/null @@ -1,149 +0,0 @@ -#ifndef BOOST_ARCHIVE_XML_WIARCHIVE_HPP -#define BOOST_ARCHIVE_XML_WIARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_wiarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else - -#include - -#include -#include -#include -#include -#include -#include -// #include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_iarchive; -} // namespace detail - -template -class basic_xml_grammar; -typedef basic_xml_grammar xml_wgrammar; - -template -class BOOST_SYMBOL_VISIBLE xml_wiarchive_impl : - public basic_text_iprimitive, - public basic_xml_iarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_iarchive; - friend class basic_xml_iarchive; - friend class load_access; -#endif - boost::scoped_ptr gimpl; - std::wistream & get_is(){ - return is; - } - template - void - load(T & t){ - basic_text_iprimitive::load(t); - } - void - load(version_type & t){ - unsigned int v; - load(v); - t = version_type(v); - } - void - load(boost::serialization::item_version_type & t){ - unsigned int v; - load(v); - t = boost::serialization::item_version_type(v); - } - BOOST_WARCHIVE_DECL void - load(char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_WARCHIVE_DECL void - load(wchar_t * t); - #endif - BOOST_WARCHIVE_DECL void - load(std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_WARCHIVE_DECL void - load(std::wstring &ws); - #endif - template - void load_override(T & t){ - basic_xml_iarchive::load_override(t); - } - BOOST_WARCHIVE_DECL void - load_override(class_name_type & t); - BOOST_WARCHIVE_DECL void - init(); - BOOST_WARCHIVE_DECL - xml_wiarchive_impl(std::wistream & is, unsigned int flags) ; - BOOST_WARCHIVE_DECL - ~xml_wiarchive_impl(); -}; - -} // namespace archive -} // namespace boost - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -class BOOST_SYMBOL_VISIBLE xml_wiarchive : - public xml_wiarchive_impl{ -public: - xml_wiarchive(std::wistream & is, unsigned int flags = 0) : - xml_wiarchive_impl(is, flags) - {} - ~xml_wiarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::xml_wiarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_XML_WIARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/archive/xml_woarchive.hpp b/contrib/libboost/boost_1_65_0/boost/archive/xml_woarchive.hpp deleted file mode 100644 index cb7ce68cb6f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/archive/xml_woarchive.hpp +++ /dev/null @@ -1,134 +0,0 @@ -#ifndef BOOST_ARCHIVE_XML_WOARCHIVE_HPP -#define BOOST_ARCHIVE_XML_WOARCHIVE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// xml_woarchive.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_NO_STD_WSTREAMBUF -#error "wide char i/o not supported on this platform" -#else -#include // size_t -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include - -//#include -#include -#include -#include -#include -#include -//#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace archive { - -namespace detail { - template class interface_oarchive; -} // namespace detail - -template -class BOOST_SYMBOL_VISIBLE xml_woarchive_impl : - public basic_text_oprimitive, - public basic_xml_oarchive -{ -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else -protected: - friend class detail::interface_oarchive; - friend class basic_xml_oarchive; - friend class save_access; -#endif - //void end_preamble(){ - // basic_xml_oarchive::end_preamble(); - //} - template - void - save(const T & t){ - basic_text_oprimitive::save(t); - } - void - save(const version_type & t){ - save(static_cast(t)); - } - void - save(const boost::serialization::item_version_type & t){ - save(static_cast(t)); - } - BOOST_WARCHIVE_DECL void - save(const char * t); - #ifndef BOOST_NO_INTRINSIC_WCHAR_T - BOOST_WARCHIVE_DECL void - save(const wchar_t * t); - #endif - BOOST_WARCHIVE_DECL void - save(const std::string &s); - #ifndef BOOST_NO_STD_WSTRING - BOOST_WARCHIVE_DECL void - save(const std::wstring &ws); - #endif - BOOST_WARCHIVE_DECL - xml_woarchive_impl(std::wostream & os, unsigned int flags); - BOOST_WARCHIVE_DECL - ~xml_woarchive_impl(); -public: - BOOST_WARCHIVE_DECL void - save_binary(const void *address, std::size_t count); - -}; - -// we use the following because we can't use -// typedef xml_woarchive_impl > xml_woarchive; - -// do not derive from this class. If you want to extend this functionality -// via inhertance, derived from xml_woarchive_impl instead. This will -// preserve correct static polymorphism. -class BOOST_SYMBOL_VISIBLE xml_woarchive : - public xml_woarchive_impl -{ -public: - xml_woarchive(std::wostream & os, unsigned int flags = 0) : - xml_woarchive_impl(os, flags) - {} - ~xml_woarchive(){} -}; - -} // namespace archive -} // namespace boost - -// required by export -BOOST_SERIALIZATION_REGISTER_ARCHIVE(boost::archive::xml_woarchive) - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_NO_STD_WSTREAMBUF -#endif // BOOST_ARCHIVE_XML_OARCHIVE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/foreach_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/foreach_fwd.hpp deleted file mode 100644 index 4e0bb370c2f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/foreach_fwd.hpp +++ /dev/null @@ -1,51 +0,0 @@ -/////////////////////////////////////////////////////////////////////////////// -// foreach.hpp header file -// -// Copyright 2010 Eric Niebler. -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// See http://www.boost.org/libs/foreach for documentation -// -// Credits: -// Kazutoshi Satoda: for suggesting the need for a _fwd header for foreach's -// customization points. - -#ifndef BOOST_FOREACH_FWD_HPP -#define BOOST_FOREACH_FWD_HPP - -// This must be at global scope, hence the uglified name -enum boost_foreach_argument_dependent_lookup_hack -{ - boost_foreach_argument_dependent_lookup_hack_value -}; - -namespace boost -{ - -namespace foreach -{ - /////////////////////////////////////////////////////////////////////////////// - // boost::foreach::tag - // - typedef boost_foreach_argument_dependent_lookup_hack tag; - - /////////////////////////////////////////////////////////////////////////////// - // boost::foreach::is_lightweight_proxy - // Specialize this for user-defined collection types if they are inexpensive to copy. - // This tells BOOST_FOREACH it can avoid the rvalue/lvalue detection stuff. - template - struct is_lightweight_proxy; - - /////////////////////////////////////////////////////////////////////////////// - // boost::foreach::is_noncopyable - // Specialize this for user-defined collection types if they cannot be copied. - // This also tells BOOST_FOREACH to avoid the rvalue/lvalue detection stuff. - template - struct is_noncopyable; - -} // namespace foreach - -} // namespace boost - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/composite_key.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/composite_key.hpp deleted file mode 100644 index 787cdf83195..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/composite_key.hpp +++ /dev/null @@ -1,1513 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_COMPOSITE_KEY_HPP -#define BOOST_MULTI_INDEX_COMPOSITE_KEY_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_FUNCTION_TEMPLATE_ORDERING) -#include -#endif - -#if !defined(BOOST_NO_SFINAE) -#include -#endif - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) -#include -#endif - -/* A composite key stores n key extractors and "computes" the - * result on a given value as a packed reference to the value and - * the composite key itself. Actual invocations to the component - * key extractors are lazily performed when executing an operation - * on composite_key results (equality, comparison, hashing.) - * As the other key extractors in Boost.MultiIndex, composite_key - * is overloaded to work on chained pointers to T and reference_wrappers - * of T. - */ - -/* This user_definable macro limits the number of elements of a composite - * key; useful for shortening resulting symbol names (MSVC++ 6.0, for - * instance has problems coping with very long symbol names.) - * NB: This cannot exceed the maximum number of arguments of - * boost::tuple. In Boost 1.32, the limit is 10. - */ - -#if !defined(BOOST_MULTI_INDEX_LIMIT_COMPOSITE_KEY_SIZE) -#define BOOST_MULTI_INDEX_LIMIT_COMPOSITE_KEY_SIZE 10 -#endif - -/* maximum number of key extractors in a composite key */ - -#if BOOST_MULTI_INDEX_LIMIT_COMPOSITE_KEY_SIZE<10 /* max length of a tuple */ -#define BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE \ - BOOST_MULTI_INDEX_LIMIT_COMPOSITE_KEY_SIZE -#else -#define BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE 10 -#endif - -/* BOOST_PP_ENUM of BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE elements */ - -#define BOOST_MULTI_INDEX_CK_ENUM(macro,data) \ - BOOST_PP_ENUM(BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE,macro,data) - -/* BOOST_PP_ENUM_PARAMS of BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE elements */ - -#define BOOST_MULTI_INDEX_CK_ENUM_PARAMS(param) \ - BOOST_PP_ENUM_PARAMS(BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE,param) - -/* if n==0 -> text0 - * otherwise -> textn=tuples::null_type - */ - -#define BOOST_MULTI_INDEX_CK_TEMPLATE_PARM(z,n,text) \ - typename BOOST_PP_CAT(text,n) BOOST_PP_EXPR_IF(n,=tuples::null_type) - -/* const textn& kn=textn() */ - -#define BOOST_MULTI_INDEX_CK_CTOR_ARG(z,n,text) \ - const BOOST_PP_CAT(text,n)& BOOST_PP_CAT(k,n) = BOOST_PP_CAT(text,n)() - -/* typename list(0)::type */ - -#define BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N(z,n,list) \ - BOOST_DEDUCED_TYPENAME BOOST_PP_LIST_AT(list,0)< \ - BOOST_PP_LIST_AT(list,1),n \ - >::type - -namespace boost{ - -template class reference_wrapper; /* fwd decl. */ - -namespace multi_index{ - -namespace detail{ - -/* n-th key extractor of a composite key */ - -template -struct nth_key_from_value -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename mpl::eval_if_c< - N::value, - tuples::element, - mpl::identity - >::type type; -}; - -/* nth_composite_key_##name::type yields - * functor >, or tuples::null_type - * if N exceeds the length of the composite key. - */ - -#define BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(name,functor) \ -template \ -struct BOOST_PP_CAT(key_,name) \ -{ \ - typedef functor type; \ -}; \ - \ -template<> \ -struct BOOST_PP_CAT(key_,name) \ -{ \ - typedef tuples::null_type type; \ -}; \ - \ -template \ -struct BOOST_PP_CAT(nth_composite_key_,name) \ -{ \ - typedef typename nth_key_from_value::type key_from_value; \ - typedef typename BOOST_PP_CAT(key_,name)::type type; \ -}; - -/* nth_composite_key_equal_to - * nth_composite_key_less - * nth_composite_key_greater - * nth_composite_key_hash - */ - -BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(equal_to,std::equal_to) -BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(less,std::less) -BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(greater,std::greater) -BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR(hash,boost::hash) - -/* used for defining equality and comparison ops of composite_key_result */ - -#define BOOST_MULTI_INDEX_CK_IDENTITY_ENUM_MACRO(z,n,text) text - -struct generic_operator_equal -{ - template - bool operator()(const T& x,const Q& y)const{return x==y;} -}; - -typedef tuple< - BOOST_MULTI_INDEX_CK_ENUM( - BOOST_MULTI_INDEX_CK_IDENTITY_ENUM_MACRO, - detail::generic_operator_equal)> generic_operator_equal_tuple; - -struct generic_operator_less -{ - template - bool operator()(const T& x,const Q& y)const{return x generic_operator_less_tuple; - -/* Metaprogramming machinery for implementing equality, comparison and - * hashing operations of composite_key_result. - * - * equal_* checks for equality between composite_key_results and - * between those and tuples, accepting a tuple of basic equality functors. - * compare_* does lexicographical comparison. - * hash_* computes a combination of elementwise hash values. - */ - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename EqualCons -> -struct equal_ckey_ckey; /* fwd decl. */ - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename EqualCons -> -struct equal_ckey_ckey_terminal -{ - static bool compare( - const KeyCons1&,const Value1&, - const KeyCons2&,const Value2&, - const EqualCons&) - { - return true; - } -}; - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename EqualCons -> -struct equal_ckey_ckey_normal -{ - static bool compare( - const KeyCons1& c0,const Value1& v0, - const KeyCons2& c1,const Value2& v1, - const EqualCons& eq) - { - if(!eq.get_head()(c0.get_head()(v0),c1.get_head()(v1)))return false; - return equal_ckey_ckey< - BOOST_DEDUCED_TYPENAME KeyCons1::tail_type,Value1, - BOOST_DEDUCED_TYPENAME KeyCons2::tail_type,Value2, - BOOST_DEDUCED_TYPENAME EqualCons::tail_type - >::compare(c0.get_tail(),v0,c1.get_tail(),v1,eq.get_tail()); - } -}; - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename EqualCons -> -struct equal_ckey_ckey: - mpl::if_< - mpl::or_< - is_same, - is_same - >, - equal_ckey_ckey_terminal, - equal_ckey_ckey_normal - >::type -{ -}; - -template -< - typename KeyCons,typename Value, - typename ValCons,typename EqualCons -> -struct equal_ckey_cval; /* fwd decl. */ - -template -< - typename KeyCons,typename Value, - typename ValCons,typename EqualCons -> -struct equal_ckey_cval_terminal -{ - static bool compare( - const KeyCons&,const Value&,const ValCons&,const EqualCons&) - { - return true; - } - - static bool compare( - const ValCons&,const KeyCons&,const Value&,const EqualCons&) - { - return true; - } -}; - -template -< - typename KeyCons,typename Value, - typename ValCons,typename EqualCons -> -struct equal_ckey_cval_normal -{ - static bool compare( - const KeyCons& c,const Value& v,const ValCons& vc, - const EqualCons& eq) - { - if(!eq.get_head()(c.get_head()(v),vc.get_head()))return false; - return equal_ckey_cval< - BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, - BOOST_DEDUCED_TYPENAME ValCons::tail_type, - BOOST_DEDUCED_TYPENAME EqualCons::tail_type - >::compare(c.get_tail(),v,vc.get_tail(),eq.get_tail()); - } - - static bool compare( - const ValCons& vc,const KeyCons& c,const Value& v, - const EqualCons& eq) - { - if(!eq.get_head()(vc.get_head(),c.get_head()(v)))return false; - return equal_ckey_cval< - BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, - BOOST_DEDUCED_TYPENAME ValCons::tail_type, - BOOST_DEDUCED_TYPENAME EqualCons::tail_type - >::compare(vc.get_tail(),c.get_tail(),v,eq.get_tail()); - } -}; - -template -< - typename KeyCons,typename Value, - typename ValCons,typename EqualCons -> -struct equal_ckey_cval: - mpl::if_< - mpl::or_< - is_same, - is_same - >, - equal_ckey_cval_terminal, - equal_ckey_cval_normal - >::type -{ -}; - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename CompareCons -> -struct compare_ckey_ckey; /* fwd decl. */ - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename CompareCons -> -struct compare_ckey_ckey_terminal -{ - static bool compare( - const KeyCons1&,const Value1&, - const KeyCons2&,const Value2&, - const CompareCons&) - { - return false; - } -}; - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename CompareCons -> -struct compare_ckey_ckey_normal -{ - static bool compare( - const KeyCons1& c0,const Value1& v0, - const KeyCons2& c1,const Value2& v1, - const CompareCons& comp) - { - if(comp.get_head()(c0.get_head()(v0),c1.get_head()(v1)))return true; - if(comp.get_head()(c1.get_head()(v1),c0.get_head()(v0)))return false; - return compare_ckey_ckey< - BOOST_DEDUCED_TYPENAME KeyCons1::tail_type,Value1, - BOOST_DEDUCED_TYPENAME KeyCons2::tail_type,Value2, - BOOST_DEDUCED_TYPENAME CompareCons::tail_type - >::compare(c0.get_tail(),v0,c1.get_tail(),v1,comp.get_tail()); - } -}; - -template -< - typename KeyCons1,typename Value1, - typename KeyCons2, typename Value2, - typename CompareCons -> -struct compare_ckey_ckey: - mpl::if_< - mpl::or_< - is_same, - is_same - >, - compare_ckey_ckey_terminal, - compare_ckey_ckey_normal - >::type -{ -}; - -template -< - typename KeyCons,typename Value, - typename ValCons,typename CompareCons -> -struct compare_ckey_cval; /* fwd decl. */ - -template -< - typename KeyCons,typename Value, - typename ValCons,typename CompareCons -> -struct compare_ckey_cval_terminal -{ - static bool compare( - const KeyCons&,const Value&,const ValCons&,const CompareCons&) - { - return false; - } - - static bool compare( - const ValCons&,const KeyCons&,const Value&,const CompareCons&) - { - return false; - } -}; - -template -< - typename KeyCons,typename Value, - typename ValCons,typename CompareCons -> -struct compare_ckey_cval_normal -{ - static bool compare( - const KeyCons& c,const Value& v,const ValCons& vc, - const CompareCons& comp) - { - if(comp.get_head()(c.get_head()(v),vc.get_head()))return true; - if(comp.get_head()(vc.get_head(),c.get_head()(v)))return false; - return compare_ckey_cval< - BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, - BOOST_DEDUCED_TYPENAME ValCons::tail_type, - BOOST_DEDUCED_TYPENAME CompareCons::tail_type - >::compare(c.get_tail(),v,vc.get_tail(),comp.get_tail()); - } - - static bool compare( - const ValCons& vc,const KeyCons& c,const Value& v, - const CompareCons& comp) - { - if(comp.get_head()(vc.get_head(),c.get_head()(v)))return true; - if(comp.get_head()(c.get_head()(v),vc.get_head()))return false; - return compare_ckey_cval< - BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, - BOOST_DEDUCED_TYPENAME ValCons::tail_type, - BOOST_DEDUCED_TYPENAME CompareCons::tail_type - >::compare(vc.get_tail(),c.get_tail(),v,comp.get_tail()); - } -}; - -template -< - typename KeyCons,typename Value, - typename ValCons,typename CompareCons -> -struct compare_ckey_cval: - mpl::if_< - mpl::or_< - is_same, - is_same - >, - compare_ckey_cval_terminal, - compare_ckey_cval_normal - >::type -{ -}; - -template -struct hash_ckey; /* fwd decl. */ - -template -struct hash_ckey_terminal -{ - static std::size_t hash( - const KeyCons&,const Value&,const HashCons&,std::size_t carry) - { - return carry; - } -}; - -template -struct hash_ckey_normal -{ - static std::size_t hash( - const KeyCons& c,const Value& v,const HashCons& h,std::size_t carry=0) - { - /* same hashing formula as boost::hash_combine */ - - carry^=h.get_head()(c.get_head()(v))+0x9e3779b9+(carry<<6)+(carry>>2); - return hash_ckey< - BOOST_DEDUCED_TYPENAME KeyCons::tail_type,Value, - BOOST_DEDUCED_TYPENAME HashCons::tail_type - >::hash(c.get_tail(),v,h.get_tail(),carry); - } -}; - -template -struct hash_ckey: - mpl::if_< - is_same, - hash_ckey_terminal, - hash_ckey_normal - >::type -{ -}; - -template -struct hash_cval; /* fwd decl. */ - -template -struct hash_cval_terminal -{ - static std::size_t hash(const ValCons&,const HashCons&,std::size_t carry) - { - return carry; - } -}; - -template -struct hash_cval_normal -{ - static std::size_t hash( - const ValCons& vc,const HashCons& h,std::size_t carry=0) - { - carry^=h.get_head()(vc.get_head())+0x9e3779b9+(carry<<6)+(carry>>2); - return hash_cval< - BOOST_DEDUCED_TYPENAME ValCons::tail_type, - BOOST_DEDUCED_TYPENAME HashCons::tail_type - >::hash(vc.get_tail(),h.get_tail(),carry); - } -}; - -template -struct hash_cval: - mpl::if_< - is_same, - hash_cval_terminal, - hash_cval_normal - >::type -{ -}; - -} /* namespace multi_index::detail */ - -/* composite_key_result */ - -#if defined(BOOST_MSVC) -#pragma warning(push) -#pragma warning(disable:4512) -#endif - -template -struct composite_key_result -{ - typedef CompositeKey composite_key_type; - typedef typename composite_key_type::value_type value_type; - - composite_key_result( - const composite_key_type& composite_key_,const value_type& value_): - composite_key(composite_key_),value(value_) - {} - - const composite_key_type& composite_key; - const value_type& value; -}; - -#if defined(BOOST_MSVC) -#pragma warning(pop) -#endif - -/* composite_key */ - -template< - typename Value, - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_TEMPLATE_PARM,KeyFromValue) -> -struct composite_key: - private tuple -{ -private: - typedef tuple super; - -public: - typedef super key_extractor_tuple; - typedef Value value_type; - typedef composite_key_result result_type; - - composite_key( - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_CTOR_ARG,KeyFromValue)): - super(BOOST_MULTI_INDEX_CK_ENUM_PARAMS(k)) - {} - - composite_key(const key_extractor_tuple& x):super(x){} - - const key_extractor_tuple& key_extractors()const{return *this;} - key_extractor_tuple& key_extractors(){return *this;} - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,result_type>::type -#else - result_type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - result_type operator()(const value_type& x)const - { - return result_type(*this,x); - } - - result_type operator()(const reference_wrapper& x)const - { - return result_type(*this,x.get()); - } - - result_type operator()(const reference_wrapper& x)const - { - return result_type(*this,x.get()); - } -}; - -/* comparison operators */ - -/* == */ - -template -inline bool operator==( - const composite_key_result& x, - const composite_key_result& y) -{ - typedef typename CompositeKey1::key_extractor_tuple key_extractor_tuple1; - typedef typename CompositeKey1::value_type value_type1; - typedef typename CompositeKey2::key_extractor_tuple key_extractor_tuple2; - typedef typename CompositeKey2::value_type value_type2; - - BOOST_STATIC_ASSERT( - tuples::length::value== - tuples::length::value); - - return detail::equal_ckey_ckey< - key_extractor_tuple1,value_type1, - key_extractor_tuple2,value_type2, - detail::generic_operator_equal_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - y.composite_key.key_extractors(),y.value, - detail::generic_operator_equal_tuple()); -} - -template< - typename CompositeKey, - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value) -> -inline bool operator==( - const composite_key_result& x, - const tuple& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value== - tuples::length::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,detail::generic_operator_equal_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - y,detail::generic_operator_equal_tuple()); -} - -template -< - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), - typename CompositeKey -> -inline bool operator==( - const tuple& x, - const composite_key_result& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value== - tuples::length::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,detail::generic_operator_equal_tuple - >::compare( - x,y.composite_key.key_extractors(), - y.value,detail::generic_operator_equal_tuple()); -} - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) -template -inline bool operator==( - const composite_key_result& x, - const std::tuple& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - static_cast(tuples::length::value)== - std::tuple_size::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,detail::generic_operator_equal_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - detail::make_cons_stdtuple(y),detail::generic_operator_equal_tuple()); -} - -template -inline bool operator==( - const std::tuple& x, - const composite_key_result& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - static_cast(tuples::length::value)== - std::tuple_size::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,detail::generic_operator_equal_tuple - >::compare( - detail::make_cons_stdtuple(x),y.composite_key.key_extractors(), - y.value,detail::generic_operator_equal_tuple()); -} -#endif - -/* < */ - -template -inline bool operator<( - const composite_key_result& x, - const composite_key_result& y) -{ - typedef typename CompositeKey1::key_extractor_tuple key_extractor_tuple1; - typedef typename CompositeKey1::value_type value_type1; - typedef typename CompositeKey2::key_extractor_tuple key_extractor_tuple2; - typedef typename CompositeKey2::value_type value_type2; - - return detail::compare_ckey_ckey< - key_extractor_tuple1,value_type1, - key_extractor_tuple2,value_type2, - detail::generic_operator_less_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - y.composite_key.key_extractors(),y.value, - detail::generic_operator_less_tuple()); -} - -template -< - typename CompositeKey, - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value) -> -inline bool operator<( - const composite_key_result& x, - const tuple& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,detail::generic_operator_less_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - y,detail::generic_operator_less_tuple()); -} - -template -< - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), - typename CompositeKey -> -inline bool operator<( - const tuple& x, - const composite_key_result& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,detail::generic_operator_less_tuple - >::compare( - x,y.composite_key.key_extractors(), - y.value,detail::generic_operator_less_tuple()); -} - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) -template -inline bool operator<( - const composite_key_result& x, - const std::tuple& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,detail::generic_operator_less_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - detail::make_cons_stdtuple(y),detail::generic_operator_less_tuple()); -} - -template -inline bool operator<( - const std::tuple& x, - const composite_key_result& y) -{ - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,detail::generic_operator_less_tuple - >::compare( - detail::make_cons_stdtuple(x),y.composite_key.key_extractors(), - y.value,detail::generic_operator_less_tuple()); -} -#endif - -/* rest of comparison operators */ - -#define BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS(t1,t2,a1,a2) \ -template inline bool operator!=(const a1& x,const a2& y) \ -{ \ - return !(x==y); \ -} \ - \ -template inline bool operator>(const a1& x,const a2& y) \ -{ \ - return y inline bool operator>=(const a1& x,const a2& y) \ -{ \ - return !(x inline bool operator<=(const a1& x,const a2& y) \ -{ \ - return !(y, - composite_key_result -) - -BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS( - typename CompositeKey, - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), - composite_key_result, - tuple -) - -BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS( - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), - typename CompositeKey, - tuple, - composite_key_result -) - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) -BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS( - typename CompositeKey, - typename... Values, - composite_key_result, - std::tuple -) - -BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS( - typename CompositeKey, - typename... Values, - std::tuple, - composite_key_result -) -#endif - -/* composite_key_equal_to */ - -template -< - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_TEMPLATE_PARM,Pred) -> -struct composite_key_equal_to: - private tuple -{ -private: - typedef tuple super; - -public: - typedef super key_eq_tuple; - - composite_key_equal_to( - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_CTOR_ARG,Pred)): - super(BOOST_MULTI_INDEX_CK_ENUM_PARAMS(k)) - {} - - composite_key_equal_to(const key_eq_tuple& x):super(x){} - - const key_eq_tuple& key_eqs()const{return *this;} - key_eq_tuple& key_eqs(){return *this;} - - template - bool operator()( - const composite_key_result & x, - const composite_key_result & y)const - { - typedef typename CompositeKey1::key_extractor_tuple key_extractor_tuple1; - typedef typename CompositeKey1::value_type value_type1; - typedef typename CompositeKey2::key_extractor_tuple key_extractor_tuple2; - typedef typename CompositeKey2::value_type value_type2; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value&& - tuples::length::value== - tuples::length::value); - - return detail::equal_ckey_ckey< - key_extractor_tuple1,value_type1, - key_extractor_tuple2,value_type2, - key_eq_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - y.composite_key.key_extractors(),y.value, - key_eqs()); - } - - template - < - typename CompositeKey, - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value) - > - bool operator()( - const composite_key_result& x, - const tuple& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value&& - tuples::length::value== - tuples::length::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,key_eq_tuple - >::compare(x.composite_key.key_extractors(),x.value,y,key_eqs()); - } - - template - < - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), - typename CompositeKey - > - bool operator()( - const tuple& x, - const composite_key_result& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value&& - tuples::length::value== - tuples::length::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,key_eq_tuple - >::compare(x,y.composite_key.key_extractors(),y.value,key_eqs()); - } - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) - template - bool operator()( - const composite_key_result& x, - const std::tuple& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value&& - static_cast(tuples::length::value)== - std::tuple_size::value); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,key_eq_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - detail::make_cons_stdtuple(y),key_eqs()); - } - - template - bool operator()( - const std::tuple& x, - const composite_key_result& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - std::tuple_size::value<= - static_cast(tuples::length::value)&& - std::tuple_size::value== - static_cast(tuples::length::value)); - - return detail::equal_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,key_eq_tuple - >::compare( - detail::make_cons_stdtuple(x),y.composite_key.key_extractors(), - y.value,key_eqs()); - } -#endif -}; - -/* composite_key_compare */ - -template -< - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_TEMPLATE_PARM,Compare) -> -struct composite_key_compare: - private tuple -{ -private: - typedef tuple super; - -public: - typedef super key_comp_tuple; - - composite_key_compare( - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_CTOR_ARG,Compare)): - super(BOOST_MULTI_INDEX_CK_ENUM_PARAMS(k)) - {} - - composite_key_compare(const key_comp_tuple& x):super(x){} - - const key_comp_tuple& key_comps()const{return *this;} - key_comp_tuple& key_comps(){return *this;} - - template - bool operator()( - const composite_key_result & x, - const composite_key_result & y)const - { - typedef typename CompositeKey1::key_extractor_tuple key_extractor_tuple1; - typedef typename CompositeKey1::value_type value_type1; - typedef typename CompositeKey2::key_extractor_tuple key_extractor_tuple2; - typedef typename CompositeKey2::value_type value_type2; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value|| - tuples::length::value<= - tuples::length::value); - - return detail::compare_ckey_ckey< - key_extractor_tuple1,value_type1, - key_extractor_tuple2,value_type2, - key_comp_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - y.composite_key.key_extractors(),y.value, - key_comps()); - } - -#if !defined(BOOST_NO_FUNCTION_TEMPLATE_ORDERING) - template - bool operator()( - const composite_key_result& x, - const Value& y)const - { - return operator()(x,boost::make_tuple(boost::cref(y))); - } -#endif - - template - < - typename CompositeKey, - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value) - > - bool operator()( - const composite_key_result& x, - const tuple& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value|| - tuples::length::value<= - tuples::length::value); - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,key_comp_tuple - >::compare(x.composite_key.key_extractors(),x.value,y,key_comps()); - } - -#if !defined(BOOST_NO_FUNCTION_TEMPLATE_ORDERING) - template - bool operator()( - const Value& x, - const composite_key_result& y)const - { - return operator()(boost::make_tuple(boost::cref(x)),y); - } -#endif - - template - < - BOOST_MULTI_INDEX_CK_ENUM_PARAMS(typename Value), - typename CompositeKey - > - bool operator()( - const tuple& x, - const composite_key_result& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value|| - tuples::length::value<= - tuples::length::value); - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - key_tuple,key_comp_tuple - >::compare(x,y.composite_key.key_extractors(),y.value,key_comps()); - } - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) - template - bool operator()( - const composite_key_result& x, - const std::tuple& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value<= - tuples::length::value|| - std::tuple_size::value<= - static_cast(tuples::length::value)); - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,key_comp_tuple - >::compare( - x.composite_key.key_extractors(),x.value, - detail::make_cons_stdtuple(y),key_comps()); - } - - template - bool operator()( - const std::tuple& x, - const composite_key_result& y)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - std::tuple_size::value<= - static_cast(tuples::length::value)|| - tuples::length::value<= - tuples::length::value); - - return detail::compare_ckey_cval< - key_extractor_tuple,value_type, - cons_key_tuple,key_comp_tuple - >::compare( - detail::make_cons_stdtuple(x),y.composite_key.key_extractors(), - y.value,key_comps()); - } -#endif -}; - -/* composite_key_hash */ - -template -< - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_TEMPLATE_PARM,Hash) -> -struct composite_key_hash: - private tuple -{ -private: - typedef tuple super; - -public: - typedef super key_hasher_tuple; - - composite_key_hash( - BOOST_MULTI_INDEX_CK_ENUM(BOOST_MULTI_INDEX_CK_CTOR_ARG,Hash)): - super(BOOST_MULTI_INDEX_CK_ENUM_PARAMS(k)) - {} - - composite_key_hash(const key_hasher_tuple& x):super(x){} - - const key_hasher_tuple& key_hash_functions()const{return *this;} - key_hasher_tuple& key_hash_functions(){return *this;} - - template - std::size_t operator()(const composite_key_result & x)const - { - typedef typename CompositeKey::key_extractor_tuple key_extractor_tuple; - typedef typename CompositeKey::value_type value_type; - - BOOST_STATIC_ASSERT( - tuples::length::value== - tuples::length::value); - - return detail::hash_ckey< - key_extractor_tuple,value_type, - key_hasher_tuple - >::hash(x.composite_key.key_extractors(),x.value,key_hash_functions()); - } - - template - std::size_t operator()( - const tuple& x)const - { - typedef tuple key_tuple; - - BOOST_STATIC_ASSERT( - tuples::length::value== - tuples::length::value); - - return detail::hash_cval< - key_tuple,key_hasher_tuple - >::hash(x,key_hash_functions()); - } - -#if !defined(BOOST_NO_CXX11_HDR_TUPLE)&&\ - !defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) - template - std::size_t operator()(const std::tuple& x)const - { - typedef std::tuple key_tuple; - typedef typename detail::cons_stdtuple_ctor< - key_tuple>::result_type cons_key_tuple; - - BOOST_STATIC_ASSERT( - std::tuple_size::value== - static_cast(tuples::length::value)); - - return detail::hash_cval< - cons_key_tuple,key_hasher_tuple - >::hash(detail::make_cons_stdtuple(x),key_hash_functions()); - } -#endif -}; - -/* Instantiations of the former functors with "natural" basic components: - * composite_key_result_equal_to uses std::equal_to of the values. - * composite_key_result_less uses std::less. - * composite_key_result_greater uses std::greater. - * composite_key_result_hash uses boost::hash. - */ - -#define BOOST_MULTI_INDEX_CK_RESULT_EQUAL_TO_SUPER \ -composite_key_equal_to< \ - BOOST_MULTI_INDEX_CK_ENUM( \ - BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N, \ - /* the argument is a PP list */ \ - (detail::nth_composite_key_equal_to, \ - (BOOST_DEDUCED_TYPENAME CompositeKeyResult::composite_key_type, \ - BOOST_PP_NIL))) \ - > - -template -struct composite_key_result_equal_to: -BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS -BOOST_MULTI_INDEX_CK_RESULT_EQUAL_TO_SUPER -{ -private: - typedef BOOST_MULTI_INDEX_CK_RESULT_EQUAL_TO_SUPER super; - -public: - typedef CompositeKeyResult first_argument_type; - typedef first_argument_type second_argument_type; - typedef bool result_type; - - using super::operator(); -}; - -#define BOOST_MULTI_INDEX_CK_RESULT_LESS_SUPER \ -composite_key_compare< \ - BOOST_MULTI_INDEX_CK_ENUM( \ - BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N, \ - /* the argument is a PP list */ \ - (detail::nth_composite_key_less, \ - (BOOST_DEDUCED_TYPENAME CompositeKeyResult::composite_key_type, \ - BOOST_PP_NIL))) \ - > - -template -struct composite_key_result_less: -BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS -BOOST_MULTI_INDEX_CK_RESULT_LESS_SUPER -{ -private: - typedef BOOST_MULTI_INDEX_CK_RESULT_LESS_SUPER super; - -public: - typedef CompositeKeyResult first_argument_type; - typedef first_argument_type second_argument_type; - typedef bool result_type; - - using super::operator(); -}; - -#define BOOST_MULTI_INDEX_CK_RESULT_GREATER_SUPER \ -composite_key_compare< \ - BOOST_MULTI_INDEX_CK_ENUM( \ - BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N, \ - /* the argument is a PP list */ \ - (detail::nth_composite_key_greater, \ - (BOOST_DEDUCED_TYPENAME CompositeKeyResult::composite_key_type, \ - BOOST_PP_NIL))) \ - > - -template -struct composite_key_result_greater: -BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS -BOOST_MULTI_INDEX_CK_RESULT_GREATER_SUPER -{ -private: - typedef BOOST_MULTI_INDEX_CK_RESULT_GREATER_SUPER super; - -public: - typedef CompositeKeyResult first_argument_type; - typedef first_argument_type second_argument_type; - typedef bool result_type; - - using super::operator(); -}; - -#define BOOST_MULTI_INDEX_CK_RESULT_HASH_SUPER \ -composite_key_hash< \ - BOOST_MULTI_INDEX_CK_ENUM( \ - BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N, \ - /* the argument is a PP list */ \ - (detail::nth_composite_key_hash, \ - (BOOST_DEDUCED_TYPENAME CompositeKeyResult::composite_key_type, \ - BOOST_PP_NIL))) \ - > - -template -struct composite_key_result_hash: -BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS -BOOST_MULTI_INDEX_CK_RESULT_HASH_SUPER -{ -private: - typedef BOOST_MULTI_INDEX_CK_RESULT_HASH_SUPER super; - -public: - typedef CompositeKeyResult argument_type; - typedef std::size_t result_type; - - using super::operator(); -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -/* Specializations of std::equal_to, std::less, std::greater and boost::hash - * for composite_key_results enabling interoperation with tuples of values. - */ - -namespace std{ - -template -struct equal_to >: - boost::multi_index::composite_key_result_equal_to< - boost::multi_index::composite_key_result - > -{ -}; - -template -struct less >: - boost::multi_index::composite_key_result_less< - boost::multi_index::composite_key_result - > -{ -}; - -template -struct greater >: - boost::multi_index::composite_key_result_greater< - boost::multi_index::composite_key_result - > -{ -}; - -} /* namespace std */ - -namespace boost{ - -template -struct hash >: - boost::multi_index::composite_key_result_hash< - boost::multi_index::composite_key_result - > -{ -}; - -} /* namespace boost */ - -#undef BOOST_MULTI_INDEX_CK_RESULT_HASH_SUPER -#undef BOOST_MULTI_INDEX_CK_RESULT_GREATER_SUPER -#undef BOOST_MULTI_INDEX_CK_RESULT_LESS_SUPER -#undef BOOST_MULTI_INDEX_CK_RESULT_EQUAL_TO_SUPER -#undef BOOST_MULTI_INDEX_CK_COMPLETE_COMP_OPS -#undef BOOST_MULTI_INDEX_CK_IDENTITY_ENUM_MACRO -#undef BOOST_MULTI_INDEX_CK_NTH_COMPOSITE_KEY_FUNCTOR -#undef BOOST_MULTI_INDEX_CK_APPLY_METAFUNCTION_N -#undef BOOST_MULTI_INDEX_CK_CTOR_ARG -#undef BOOST_MULTI_INDEX_CK_TEMPLATE_PARM -#undef BOOST_MULTI_INDEX_CK_ENUM_PARAMS -#undef BOOST_MULTI_INDEX_CK_ENUM -#undef BOOST_MULTI_INDEX_COMPOSITE_KEY_SIZE - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/access_specifier.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/access_specifier.hpp deleted file mode 100644 index f3346e836d4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/access_specifier.hpp +++ /dev/null @@ -1,54 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ACCESS_SPECIFIER_HPP -#define BOOST_MULTI_INDEX_DETAIL_ACCESS_SPECIFIER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include -#include - -/* In those compilers that do not accept the member template friend syntax, - * some protected and private sections might need to be specified as - * public. - */ - -#if defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) -#define BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS public -#define BOOST_MULTI_INDEX_PRIVATE_IF_MEMBER_TEMPLATE_FRIENDS public -#else -#define BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS protected -#define BOOST_MULTI_INDEX_PRIVATE_IF_MEMBER_TEMPLATE_FRIENDS private -#endif - -/* GCC does not correctly support in-class using declarations for template - * functions. See http://gcc.gnu.org/bugzilla/show_bug.cgi?id=9810 - * MSVC 7.1/8.0 seem to have a similar problem, though the conditions in - * which the error happens are not that simple. I have yet to isolate this - * into a snippet suitable for bug reporting. - * Sun Studio also has this problem, which might be related, from the - * information gathered at Sun forums, with a known issue notified at the - * internal bug report 6421933. The bug is present up to Studio Express 2, - * the latest preview version of the future Sun Studio 12. As of this writing - * (October 2006) it is not known whether a fix will finally make it into the - * official Sun Studio 12. - */ - -#if BOOST_WORKAROUND(__GNUC__,==3)&&(__GNUC_MINOR__<4)||\ - BOOST_WORKAROUND(BOOST_MSVC,==1310)||\ - BOOST_WORKAROUND(BOOST_MSVC,==1400)||\ - BOOST_WORKAROUND(__SUNPRO_CC,BOOST_TESTED_AT(0x590)) -#define BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS public -#else -#define BOOST_MULTI_INDEX_PRIVATE_IF_USING_DECL_FOR_TEMPL_FUNCTIONS private -#endif - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/adl_swap.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/adl_swap.hpp deleted file mode 100644 index 02b06442290..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/adl_swap.hpp +++ /dev/null @@ -1,44 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ADL_SWAP_HPP -#define BOOST_MULTI_INDEX_DETAIL_ADL_SWAP_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -void adl_swap(T& x,T& y) -{ - -#if !defined(BOOST_FUNCTION_SCOPE_USING_DECLARATION_BREAKS_ADL) - using std::swap; - swap(x,y); -#else - std::swap(x,y); -#endif - -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/archive_constructed.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/archive_constructed.hpp deleted file mode 100644 index 0a7a26e0d4e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/archive_constructed.hpp +++ /dev/null @@ -1,83 +0,0 @@ -/* Copyright 2003-2016 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ARCHIVE_CONSTRUCTED_HPP -#define BOOST_MULTI_INDEX_DETAIL_ARCHIVE_CONSTRUCTED_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* constructs a stack-based object from a serialization archive */ - -template -struct archive_constructed:private noncopyable -{ - template - archive_constructed(Archive& ar,const unsigned int version) - { - serialization::load_construct_data_adl(ar,&get(),version); - BOOST_TRY{ - ar>>get(); - } - BOOST_CATCH(...){ - (&get())->~T(); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - template - archive_constructed(const char* name,Archive& ar,const unsigned int version) - { - serialization::load_construct_data_adl(ar,&get(),version); - BOOST_TRY{ - ar>>serialization::make_nvp(name,get()); - } - BOOST_CATCH(...){ - (&get())->~T(); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - ~archive_constructed() - { - (&get())->~T(); - } - -#include - - T& get(){return *reinterpret_cast(&space);} - -#include - -private: - typename aligned_storage::value>::type space; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/auto_space.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/auto_space.hpp deleted file mode 100644 index 9d78c3a363f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/auto_space.hpp +++ /dev/null @@ -1,91 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_AUTO_SPACE_HPP -#define BOOST_MULTI_INDEX_DETAIL_AUTO_SPACE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* auto_space provides uninitialized space suitably to store - * a given number of elements of a given type. - */ - -/* NB: it is not clear whether using an allocator to handle - * zero-sized arrays of elements is conformant or not. GCC 3.3.1 - * and prior fail here, other stdlibs handle the issue gracefully. - * To be on the safe side, the case n==0 is given special treatment. - * References: - * GCC Bugzilla, "standard allocator crashes when deallocating segment - * "of zero length", http://gcc.gnu.org/bugzilla/show_bug.cgi?id=14176 - * C++ Standard Library Defect Report List (Revision 28), issue 199 - * "What does allocate(0) return?", - * http://www.open-std.org/jtc1/sc22/wg21/docs/lwg-defects.html#199 - */ - -template > -struct auto_space:private noncopyable -{ - typedef typename boost::detail::allocator::rebind_to< - Allocator,T - >::type::pointer pointer; - - explicit auto_space(const Allocator& al=Allocator(),std::size_t n=1): - al_(al),n_(n),data_(n_?al_.allocate(n_):pointer(0)) - {} - - ~auto_space() - { - if(n_)al_.deallocate(data_,n_); - } - - Allocator get_allocator()const{return al_;} - - pointer data()const{return data_;} - - void swap(auto_space& x) - { - if(al_!=x.al_)adl_swap(al_,x.al_); - std::swap(n_,x.n_); - std::swap(data_,x.data_); - } - -private: - typename boost::detail::allocator::rebind_to< - Allocator,T>::type al_; - std::size_t n_; - pointer data_; -}; - -template -void swap(auto_space& x,auto_space& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/base_type.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/base_type.hpp deleted file mode 100644 index 8c9b62b716a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/base_type.hpp +++ /dev/null @@ -1,74 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_BASE_TYPE_HPP -#define BOOST_MULTI_INDEX_DETAIL_BASE_TYPE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* MPL machinery to construct a linear hierarchy of indices out of - * a index list. - */ - -struct index_applier -{ - template - struct apply - { - typedef typename IndexSpecifierMeta::type index_specifier; - typedef typename index_specifier:: - BOOST_NESTED_TEMPLATE index_class::type type; - }; -}; - -template -struct nth_layer -{ - BOOST_STATIC_CONSTANT(int,length=mpl::size::value); - - typedef typename mpl::eval_if_c< - N==length, - mpl::identity >, - mpl::apply2< - index_applier, - mpl::at_c, - nth_layer - > - >::type type; -}; - -template -struct multi_index_base_type:nth_layer<0,Value,IndexSpecifierList,Allocator> -{ - BOOST_STATIC_ASSERT(detail::is_index_list::value); -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bidir_node_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bidir_node_iterator.hpp deleted file mode 100644 index 9be5ec84b43..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bidir_node_iterator.hpp +++ /dev/null @@ -1,114 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_BIDIR_NODE_ITERATOR_HPP -#define BOOST_MULTI_INDEX_DETAIL_BIDIR_NODE_ITERATOR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Iterator class for node-based indices with bidirectional - * iterators (ordered and sequenced indices.) - */ - -template -class bidir_node_iterator: - public bidirectional_iterator_helper< - bidir_node_iterator, - typename Node::value_type, - std::ptrdiff_t, - const typename Node::value_type*, - const typename Node::value_type&> -{ -public: - /* coverity[uninit_ctor]: suppress warning */ - bidir_node_iterator(){} - explicit bidir_node_iterator(Node* node_):node(node_){} - - const typename Node::value_type& operator*()const - { - return node->value(); - } - - bidir_node_iterator& operator++() - { - Node::increment(node); - return *this; - } - - bidir_node_iterator& operator--() - { - Node::decrement(node); - return *this; - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* Serialization. As for why the following is public, - * see explanation in safe_mode_iterator notes in safe_mode.hpp. - */ - - BOOST_SERIALIZATION_SPLIT_MEMBER() - - typedef typename Node::base_type node_base_type; - - template - void save(Archive& ar,const unsigned int)const - { - node_base_type* bnode=node; - ar< - void load(Archive& ar,const unsigned int) - { - node_base_type* bnode; - ar>>serialization::make_nvp("pointer",bnode); - node=static_cast(bnode); - } -#endif - - /* get_node is not to be used by the user */ - - typedef Node node_type; - - Node* get_node()const{return node;} - -private: - Node* node; -}; - -template -bool operator==( - const bidir_node_iterator& x, - const bidir_node_iterator& y) -{ - return x.get_node()==y.get_node(); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bucket_array.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bucket_array.hpp deleted file mode 100644 index d9fa434d9a9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/bucket_array.hpp +++ /dev/null @@ -1,243 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_BUCKET_ARRAY_HPP -#define BOOST_MULTI_INDEX_DETAIL_BUCKET_ARRAY_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* bucket structure for use by hashed indices */ - -#define BOOST_MULTI_INDEX_BA_SIZES_32BIT \ -(53ul)(97ul)(193ul)(389ul)(769ul) \ -(1543ul)(3079ul)(6151ul)(12289ul)(24593ul) \ -(49157ul)(98317ul)(196613ul)(393241ul)(786433ul) \ -(1572869ul)(3145739ul)(6291469ul)(12582917ul)(25165843ul) \ -(50331653ul)(100663319ul)(201326611ul)(402653189ul)(805306457ul) \ -(1610612741ul)(3221225473ul) - -#if ((((ULONG_MAX>>16)>>16)>>16)>>15)==0 /* unsigned long less than 64 bits */ -#define BOOST_MULTI_INDEX_BA_SIZES \ -BOOST_MULTI_INDEX_BA_SIZES_32BIT \ -(4294967291ul) -#else - /* obtained with aid from - * http://javaboutique.internet.com/prime_numb/ - * http://www.rsok.com/~jrm/next_ten_primes.html - * and verified with - * http://www.alpertron.com.ar/ECM.HTM - */ - -#define BOOST_MULTI_INDEX_BA_SIZES \ -BOOST_MULTI_INDEX_BA_SIZES_32BIT \ -(6442450939ul)(12884901893ul)(25769803751ul)(51539607551ul) \ -(103079215111ul)(206158430209ul)(412316860441ul)(824633720831ul) \ -(1649267441651ul)(3298534883309ul)(6597069766657ul)(13194139533299ul) \ -(26388279066623ul)(52776558133303ul)(105553116266489ul)(211106232532969ul) \ -(422212465066001ul)(844424930131963ul)(1688849860263953ul) \ -(3377699720527861ul)(6755399441055731ul)(13510798882111483ul) \ -(27021597764222939ul)(54043195528445957ul)(108086391056891903ul) \ -(216172782113783843ul)(432345564227567621ul)(864691128455135207ul) \ -(1729382256910270481ul)(3458764513820540933ul)(6917529027641081903ul) \ -(13835058055282163729ul)(18446744073709551557ul) -#endif - -template /* templatized to have in-header static var defs */ -class bucket_array_base:private noncopyable -{ -protected: - static const std::size_t sizes[ - BOOST_PP_SEQ_SIZE(BOOST_MULTI_INDEX_BA_SIZES)]; - - static std::size_t size_index(std::size_t n) - { - const std::size_t *bound=std::lower_bound(sizes,sizes+sizes_length,n); - if(bound==sizes+sizes_length)--bound; - return bound-sizes; - } - -#define BOOST_MULTI_INDEX_BA_POSITION_CASE(z,n,_) \ - case n:return hash%BOOST_PP_SEQ_ELEM(n,BOOST_MULTI_INDEX_BA_SIZES); - - static std::size_t position(std::size_t hash,std::size_t size_index_) - { - /* Accelerate hash%sizes[size_index_] by replacing with a switch on - * hash%Ci expressions, each Ci a compile-time constant, which the - * compiler can implement without using integer division. - */ - - switch(size_index_){ - default: /* never used */ - BOOST_PP_REPEAT( - BOOST_PP_SEQ_SIZE(BOOST_MULTI_INDEX_BA_SIZES), - BOOST_MULTI_INDEX_BA_POSITION_CASE,~) - } - } - -private: - static const std::size_t sizes_length; -}; - -template -const std::size_t bucket_array_base<_>::sizes[]={ - BOOST_PP_SEQ_ENUM(BOOST_MULTI_INDEX_BA_SIZES) -}; - -template -const std::size_t bucket_array_base<_>::sizes_length= - sizeof(bucket_array_base<_>::sizes)/ - sizeof(bucket_array_base<_>::sizes[0]); - -#undef BOOST_MULTI_INDEX_BA_POSITION_CASE -#undef BOOST_MULTI_INDEX_BA_SIZES -#undef BOOST_MULTI_INDEX_BA_SIZES_32BIT - -template -class bucket_array:bucket_array_base<> -{ - typedef bucket_array_base<> super; - typedef hashed_index_base_node_impl< - typename boost::detail::allocator::rebind_to< - Allocator, - char - >::type - > base_node_impl_type; - -public: - typedef typename base_node_impl_type::base_pointer base_pointer; - typedef typename base_node_impl_type::pointer pointer; - - bucket_array(const Allocator& al,pointer end_,std::size_t size_): - size_index_(super::size_index(size_)), - spc(al,super::sizes[size_index_]+1) - { - clear(end_); - } - - std::size_t size()const - { - return super::sizes[size_index_]; - } - - std::size_t position(std::size_t hash)const - { - return super::position(hash,size_index_); - } - - base_pointer begin()const{return buckets();} - base_pointer end()const{return buckets()+size();} - base_pointer at(std::size_t n)const{return buckets()+n;} - - void clear(pointer end_) - { - for(base_pointer x=begin(),y=end();x!=y;++x)x->prior()=pointer(0); - end()->prior()=end_->prior()=end_; - end_->next()=end(); - } - - void swap(bucket_array& x) - { - std::swap(size_index_,x.size_index_); - spc.swap(x.spc); - } - -private: - std::size_t size_index_; - auto_space spc; - - base_pointer buckets()const - { - return spc.data(); - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - friend class boost::serialization::access; - - /* bucket_arrays do not emit any kind of serialization info. They are - * fed to Boost.Serialization as hashed index iterators need to track - * them during serialization. - */ - - template - void serialize(Archive&,const unsigned int) - { - } -#endif -}; - -template -void swap(bucket_array& x,bucket_array& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -/* bucket_arrays never get constructed directly by Boost.Serialization, - * as archives are always fed pointers to previously existent - * arrays. So, if this is called it means we are dealing with a - * somehow invalid archive. - */ - -#if defined(BOOST_NO_ARGUMENT_DEPENDENT_LOOKUP) -namespace serialization{ -#else -namespace multi_index{ -namespace detail{ -#endif - -template -inline void load_construct_data( - Archive&,boost::multi_index::detail::bucket_array*, - const unsigned int) -{ - throw_exception( - archive::archive_exception(archive::archive_exception::other_exception)); -} - -#if defined(BOOST_NO_ARGUMENT_DEPENDENT_LOOKUP) -} /* namespace serialization */ -#else -} /* namespace multi_index::detail */ -} /* namespace multi_index */ -#endif - -#endif - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/cons_stdtuple.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/cons_stdtuple.hpp deleted file mode 100644 index 855c5e06aa9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/cons_stdtuple.hpp +++ /dev/null @@ -1,93 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_CONS_STDTUPLE_HPP -#define BOOST_MULTI_INDEX_DETAIL_CONS_STDTUPLE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* std::tuple wrapper providing the cons-based interface of boost::tuple for - * composite_key interoperability. - */ - -template -struct cons_stdtuple; - -struct cons_stdtuple_ctor_terminal -{ - typedef boost::tuples::null_type result_type; - - template - static result_type create(const StdTuple&) - { - return boost::tuples::null_type(); - } -}; - -template -struct cons_stdtuple_ctor_normal -{ - typedef cons_stdtuple result_type; - - static result_type create(const StdTuple& t) - { - return result_type(t); - } -}; - -template -struct cons_stdtuple_ctor: - boost::mpl::if_c< - N::value, - cons_stdtuple_ctor_normal, - cons_stdtuple_ctor_terminal - >::type -{}; - -template -struct cons_stdtuple -{ - typedef typename std::tuple_element::type head_type; - typedef cons_stdtuple_ctor tail_ctor; - typedef typename tail_ctor::result_type tail_type; - - cons_stdtuple(const StdTuple& t_):t(t_){} - - const head_type& get_head()const{return std::get(t);} - tail_type get_tail()const{return tail_ctor::create(t);} - - const StdTuple& t; -}; - -template -typename cons_stdtuple_ctor::result_type -make_cons_stdtuple(const StdTuple& t) -{ - return cons_stdtuple_ctor::create(t); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/converter.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/converter.hpp deleted file mode 100644 index 3e04a3e8295..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/converter.hpp +++ /dev/null @@ -1,52 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_CONVERTER_HPP -#define BOOST_MULTI_INDEX_DETAIL_CONVERTER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* converter offers means to access indices of a given multi_index_container - * and for convertibilty between index iterators, so providing a - * localized access point for get() and project() functions. - */ - -template -struct converter -{ - static const Index& index(const MultiIndexContainer& x){return x;} - static Index& index(MultiIndexContainer& x){return x;} - - static typename Index::const_iterator const_iterator( - const MultiIndexContainer& x,typename MultiIndexContainer::node_type* node) - { - return x.Index::make_iterator(node); - } - - static typename Index::iterator iterator( - MultiIndexContainer& x,typename MultiIndexContainer::node_type* node) - { - return x.Index::make_iterator(node); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/copy_map.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/copy_map.hpp deleted file mode 100644 index 9a34b259cf3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/copy_map.hpp +++ /dev/null @@ -1,142 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_COPY_MAP_HPP -#define BOOST_MULTI_INDEX_DETAIL_COPY_MAP_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* copy_map is used as an auxiliary structure during copy_() operations. - * When a container with n nodes is replicated, node_map holds the pairings - * between original and copied nodes, and provides a fast way to find a - * copied node from an original one. - * The semantics of the class are not simple, and no attempt has been made - * to enforce it: multi_index_container handles it right. On the other hand, - * the const interface, which is the one provided to index implementations, - * only allows for: - * - Enumeration of pairs of (original,copied) nodes (excluding the headers), - * - fast retrieval of copied nodes (including the headers.) - */ - -template -struct copy_map_entry -{ - copy_map_entry(Node* f,Node* s):first(f),second(s){} - - Node* first; - Node* second; - - bool operator<(const copy_map_entry& x)const - { - return std::less()(first,x.first); - } -}; - -template -class copy_map:private noncopyable -{ -public: - typedef const copy_map_entry* const_iterator; - - copy_map( - const Allocator& al,std::size_t size,Node* header_org,Node* header_cpy): - al_(al),size_(size),spc(al_,size_),n(0), - header_org_(header_org),header_cpy_(header_cpy),released(false) - {} - - ~copy_map() - { - if(!released){ - for(std::size_t i=0;isecond->value()); - deallocate((spc.data()+i)->second); - } - } - } - - const_iterator begin()const{return raw_ptr(spc.data());} - const_iterator end()const{return raw_ptr(spc.data()+n);} - - void clone(Node* node) - { - (spc.data()+n)->first=node; - (spc.data()+n)->second=raw_ptr(al_.allocate(1)); - BOOST_TRY{ - boost::detail::allocator::construct( - &(spc.data()+n)->second->value(),node->value()); - } - BOOST_CATCH(...){ - deallocate((spc.data()+n)->second); - BOOST_RETHROW; - } - BOOST_CATCH_END - ++n; - - if(n==size_){ - std::sort( - raw_ptr*>(spc.data()), - raw_ptr*>(spc.data())+size_); - } - } - - Node* find(Node* node)const - { - if(node==header_org_)return header_cpy_; - return std::lower_bound( - begin(),end(),copy_map_entry(node,0))->second; - } - - void release() - { - released=true; - } - -private: - typedef typename boost::detail::allocator::rebind_to< - Allocator,Node - >::type allocator_type; - typedef typename allocator_type::pointer allocator_pointer; - - allocator_type al_; - std::size_t size_; - auto_space,Allocator> spc; - std::size_t n; - Node* header_org_; - Node* header_cpy_; - bool released; - - void deallocate(Node* node) - { - al_.deallocate(static_cast(node),1); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/do_not_copy_elements_tag.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/do_not_copy_elements_tag.hpp deleted file mode 100644 index f0fa7304253..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/do_not_copy_elements_tag.hpp +++ /dev/null @@ -1,34 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_DO_NOT_COPY_ELEMENTS_TAG_HPP -#define BOOST_MULTI_INDEX_DETAIL_DO_NOT_COPY_ELEMENTS_TAG_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Used to mark a special ctor variant that copies the internal objects of - * a container but not its elements. - */ - -struct do_not_copy_elements_tag{}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/duplicates_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/duplicates_iterator.hpp deleted file mode 100644 index cbebf264045..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/duplicates_iterator.hpp +++ /dev/null @@ -1,120 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_DUPLICATES_ITERATOR_HPP -#define BOOST_MULTI_INDEX_DETAIL_DUPLICATES_ITERATOR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* duplicates_operator is given a range of ordered elements and - * passes only over those which are duplicated. - */ - -template -class duplicates_iterator -{ -public: - typedef typename Node::value_type value_type; - typedef std::ptrdiff_t difference_type; - typedef const typename Node::value_type* pointer; - typedef const typename Node::value_type& reference; - typedef std::forward_iterator_tag iterator_category; - - duplicates_iterator(Node* node_,Node* end_,Predicate pred_): - node(node_),begin_chunk(0),end(end_),pred(pred_) - { - advance(); - } - - duplicates_iterator(Node* end_,Predicate pred_): - node(end_),begin_chunk(end_),end(end_),pred(pred_) - { - } - - reference operator*()const - { - return node->value(); - } - - pointer operator->()const - { - return &node->value(); - } - - duplicates_iterator& operator++() - { - Node::increment(node); - sync(); - return *this; - } - - duplicates_iterator operator++(int) - { - duplicates_iterator tmp(*this); - ++(*this); - return tmp; - } - - Node* get_node()const{return node;} - -private: - void sync() - { - if(node!=end&&pred(begin_chunk->value(),node->value()))advance(); - } - - void advance() - { - for(Node* node2=node;node!=end;node=node2){ - Node::increment(node2); - if(node2!=end&&!pred(node->value(),node2->value()))break; - } - begin_chunk=node; - } - - Node* node; - Node* begin_chunk; - Node* end; - Predicate pred; -}; - -template -bool operator==( - const duplicates_iterator& x, - const duplicates_iterator& y) -{ - return x.get_node()==y.get_node(); -} - -template -bool operator!=( - const duplicates_iterator& x, - const duplicates_iterator& y) -{ - return !(x==y); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/has_tag.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/has_tag.hpp deleted file mode 100644 index 217b61143af..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/has_tag.hpp +++ /dev/null @@ -1,42 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_HAS_TAG_HPP -#define BOOST_MULTI_INDEX_DETAIL_HAS_TAG_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* determines whether an index type has a given tag in its tag list */ - -template -struct has_tag -{ - template - struct apply:mpl::contains - { - }; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_args.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_args.hpp deleted file mode 100644 index 81902f5a4a5..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_args.hpp +++ /dev/null @@ -1,105 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_ARGS_HPP -#define BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_ARGS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Hashed index specifiers can be instantiated in two forms: - * - * (hashed_unique|hashed_non_unique)< - * KeyFromValue, - * Hash=boost::hash, - * Pred=std::equal_to > - * (hashed_unique|hashed_non_unique)< - * TagList, - * KeyFromValue, - * Hash=boost::hash, - * Pred=std::equal_to > - * - * hashed_index_args implements the machinery to accept this - * argument-dependent polymorphism. - */ - -template -struct index_args_default_hash -{ - typedef ::boost::hash type; -}; - -template -struct index_args_default_pred -{ - typedef std::equal_to type; -}; - -template -struct hashed_index_args -{ - typedef is_tag full_form; - - typedef typename mpl::if_< - full_form, - Arg1, - tag< > >::type tag_list_type; - typedef typename mpl::if_< - full_form, - Arg2, - Arg1>::type key_from_value_type; - typedef typename mpl::if_< - full_form, - Arg3, - Arg2>::type supplied_hash_type; - typedef typename mpl::eval_if< - mpl::is_na, - index_args_default_hash, - mpl::identity - >::type hash_type; - typedef typename mpl::if_< - full_form, - Arg4, - Arg3>::type supplied_pred_type; - typedef typename mpl::eval_if< - mpl::is_na, - index_args_default_pred, - mpl::identity - >::type pred_type; - - BOOST_STATIC_ASSERT(is_tag::value); - BOOST_STATIC_ASSERT(!mpl::is_na::value); - BOOST_STATIC_ASSERT(!mpl::is_na::value); - BOOST_STATIC_ASSERT(!mpl::is_na::value); -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_iterator.hpp deleted file mode 100644 index 8d063002a1d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_iterator.hpp +++ /dev/null @@ -1,166 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_ITERATOR_HPP -#define BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_ITERATOR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Iterator class for hashed indices. - */ - -struct hashed_index_global_iterator_tag{}; -struct hashed_index_local_iterator_tag{}; - -template -class hashed_index_iterator: - public forward_iterator_helper< - hashed_index_iterator, - typename Node::value_type, - std::ptrdiff_t, - const typename Node::value_type*, - const typename Node::value_type&> -{ -public: - /* coverity[uninit_ctor]: suppress warning */ - hashed_index_iterator(){} - hashed_index_iterator(Node* node_):node(node_){} - - const typename Node::value_type& operator*()const - { - return node->value(); - } - - hashed_index_iterator& operator++() - { - this->increment(Category()); - return *this; - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* Serialization. As for why the following is public, - * see explanation in safe_mode_iterator notes in safe_mode.hpp. - */ - - BOOST_SERIALIZATION_SPLIT_MEMBER() - - typedef typename Node::base_type node_base_type; - - template - void save(Archive& ar,const unsigned int)const - { - node_base_type* bnode=node; - ar< - void load(Archive& ar,const unsigned int version) - { - load(ar,version,Category()); - } - - template - void load( - Archive& ar,const unsigned int version,hashed_index_global_iterator_tag) - { - node_base_type* bnode; - ar>>serialization::make_nvp("pointer",bnode); - node=static_cast(bnode); - if(version<1){ - BucketArray* throw_away; /* consume unused ptr */ - ar>>serialization::make_nvp("pointer",throw_away); - } - } - - template - void load( - Archive& ar,const unsigned int version,hashed_index_local_iterator_tag) - { - node_base_type* bnode; - ar>>serialization::make_nvp("pointer",bnode); - node=static_cast(bnode); - if(version<1){ - BucketArray* buckets; - ar>>serialization::make_nvp("pointer",buckets); - if(buckets&&node&&node->impl()==buckets->end()->prior()){ - /* end local_iterators used to point to end node, now they are null */ - node=0; - } - } - } -#endif - - /* get_node is not to be used by the user */ - - typedef Node node_type; - - Node* get_node()const{return node;} - -private: - - void increment(hashed_index_global_iterator_tag) - { - Node::increment(node); - } - - void increment(hashed_index_local_iterator_tag) - { - Node::increment_local(node); - } - - Node* node; -}; - -template -bool operator==( - const hashed_index_iterator& x, - const hashed_index_iterator& y) -{ - return x.get_node()==y.get_node(); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -/* class version = 1 : hashed_index_iterator does no longer serialize a bucket - * array pointer. - */ - -namespace serialization { -template -struct version< - boost::multi_index::detail::hashed_index_iterator -> -{ - BOOST_STATIC_CONSTANT(int,value=1); -}; -} /* namespace serialization */ -#endif - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_node.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_node.hpp deleted file mode 100644 index 7788e810ac9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/hash_index_node.hpp +++ /dev/null @@ -1,778 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_NODE_HPP -#define BOOST_MULTI_INDEX_DETAIL_HASH_INDEX_NODE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Certain C++ requirements on unordered associative containers (see LWG issue - * #579) imply a data structure where nodes are linked in a single list, which - * in its turn forces implementors to add additional overhed per node to - * associate each with its corresponding bucket. Others resort to storing hash - * values, we use an alternative structure providing unconditional O(1) - * manipulation, even in situations of unfair hash distribution, plus some - * lookup speedups. For unique indices we maintain a doubly linked list of - * nodes except that if N is the first node of a bucket its associated - * bucket node is embedded between N and the preceding node in the following - * manner: - * - * +---+ +---+ +---+ +---+ - * <--+ |<--+ | <--+ |<--+ | - * ... | B0| | B1| ... | B1| | B2| ... - * | |-+ | +--> | |-+ | +--> - * +-+-+ | +---+ +-+-+ | +---+ - * | ^ | ^ - * | | | | - * | +-+ | +-+ - * | | | | - * v | v | - * --+---+---+---+-- --+---+---+---+-- - * ... | | B1| | ... | | B2| | ... - * --+---+---+---+-- --+---+---+---+-- - * - * - * The fist and last nodes of buckets can be checked with - * - * first node of a bucket: Npn != N - * last node of a bucket: Nnp != N - * - * (n and p short for ->next(), ->prior(), bucket nodes have prior pointers - * only). Pure insert and erase (without lookup) can be unconditionally done - * in O(1). - * For non-unique indices we add the following additional complexity: when - * there is a group of 3 or more equivalent elements, they are linked as - * follows: - * - * +-----------------------+ - * | v - * +---+ | +---+ +---+ +---+ - * | | +-+ | | |<--+ | - * | F | | S | ... | P | | L | - * | +-->| | | +-+ | | - * +---+ +---+ +---+ | +---+ - * ^ | - * +-----------------------+ - * - * F, S, P and L are the first, second, penultimate and last node in the - * group, respectively (S and P can coincide if the group has size 3.) This - * arrangement is used to skip equivalent elements in O(1) when doing lookup, - * while preserving O(1) insert/erase. The following invariants identify - * special positions (some of the operations have to be carefully implemented - * as Xnn is not valid if Xn points to a bucket): - * - * first node of a bucket: Npnp == N - * last node of a bucket: Nnpp == N - * first node of a group: Nnp != N && Nnppn == N - * second node of a group: Npn != N && Nppnn == N - * n-1 node of a group: Nnp != N && Nnnpp == N - * last node of a group: Npn != N && Npnnp == N - * - * The memory overhead is one pointer per bucket plus two pointers per node, - * probably unbeatable. The resulting structure is bidirectonally traversable, - * though currently we are just providing forward iteration. - */ - -template -struct hashed_index_node_impl; - -/* half-header (only prior() pointer) to use for the bucket array */ - -template -struct hashed_index_base_node_impl -{ - typedef typename - boost::detail::allocator::rebind_to< - Allocator,hashed_index_base_node_impl - >::type::pointer base_pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator,hashed_index_base_node_impl - >::type::const_pointer const_base_pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator, - hashed_index_node_impl - >::type::pointer pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator, - hashed_index_node_impl - >::type::const_pointer const_pointer; - - pointer& prior(){return prior_;} - pointer prior()const{return prior_;} - -private: - pointer prior_; -}; - -/* full header (prior() and next()) for the nodes */ - -template -struct hashed_index_node_impl:hashed_index_base_node_impl -{ -private: - typedef hashed_index_base_node_impl super; - -public: - typedef typename super::base_pointer base_pointer; - typedef typename super::const_base_pointer const_base_pointer; - typedef typename super::pointer pointer; - typedef typename super::const_pointer const_pointer; - - base_pointer& next(){return next_;} - base_pointer next()const{return next_;} - - static pointer pointer_from(base_pointer x) - { - return static_cast( - static_cast( - raw_ptr(x))); - } - - static base_pointer base_pointer_from(pointer x) - { - return static_cast( - raw_ptr(x)); - } - -private: - base_pointer next_; -}; - -/* Boost.MultiIndex requires machinery to reverse unlink operations. A simple - * way to make a pointer-manipulation function undoable is to templatize - * its internal pointer assignments with a functor that, besides doing the - * assignment, keeps track of the original pointer values and can later undo - * the operations in reverse order. - */ - -struct default_assigner -{ - template void operator()(T& x,const T& val){x=val;} -}; - -template -struct unlink_undo_assigner -{ - typedef typename Node::base_pointer base_pointer; - typedef typename Node::pointer pointer; - - unlink_undo_assigner():pointer_track_count(0),base_pointer_track_count(0){} - - void operator()(pointer& x,pointer val) - { - pointer_tracks[pointer_track_count].x=&x; - pointer_tracks[pointer_track_count++].val=x; - x=val; - } - - void operator()(base_pointer& x,base_pointer val) - { - base_pointer_tracks[base_pointer_track_count].x=&x; - base_pointer_tracks[base_pointer_track_count++].val=x; - x=val; - } - - void operator()() /* undo op */ - { - /* in the absence of aliasing, restitution order is immaterial */ - - while(pointer_track_count--){ - *(pointer_tracks[pointer_track_count].x)= - pointer_tracks[pointer_track_count].val; - } - while(base_pointer_track_count--){ - *(base_pointer_tracks[base_pointer_track_count].x)= - base_pointer_tracks[base_pointer_track_count].val; - } - } - - struct pointer_track {pointer* x; pointer val;}; - struct base_pointer_track{base_pointer* x; base_pointer val;}; - - /* We know the maximum number of pointer and base pointer assignments that - * the two unlink versions do, so we can statically reserve the needed - * storage. - */ - - pointer_track pointer_tracks[3]; - int pointer_track_count; - base_pointer_track base_pointer_tracks[2]; - int base_pointer_track_count; -}; - -/* algorithmic stuff for unique and non-unique variants */ - -struct hashed_unique_tag{}; -struct hashed_non_unique_tag{}; - -template -struct hashed_index_node_alg; - -template -struct hashed_index_node_alg -{ - typedef typename Node::base_pointer base_pointer; - typedef typename Node::const_base_pointer const_base_pointer; - typedef typename Node::pointer pointer; - typedef typename Node::const_pointer const_pointer; - - static bool is_first_of_bucket(pointer x) - { - return x->prior()->next()!=base_pointer_from(x); - } - - static pointer after(pointer x) - { - return is_last_of_bucket(x)?x->next()->prior():pointer_from(x->next()); - } - - static pointer after_local(pointer x) - { - return is_last_of_bucket(x)?pointer(0):pointer_from(x->next()); - } - - static pointer next_to_inspect(pointer x) - { - return is_last_of_bucket(x)?pointer(0):pointer_from(x->next()); - } - - static void link(pointer x,base_pointer buc,pointer end) - { - if(buc->prior()==pointer(0)){ /* empty bucket */ - x->prior()=end->prior(); - x->next()=end->prior()->next(); - x->prior()->next()=buc; - buc->prior()=x; - end->prior()=x; - } - else{ - x->prior()=buc->prior()->prior(); - x->next()=base_pointer_from(buc->prior()); - buc->prior()=x; - x->next()->prior()=x; - } - } - - static void unlink(pointer x) - { - default_assigner assign; - unlink(x,assign); - } - - typedef unlink_undo_assigner unlink_undo; - - template - static void unlink(pointer x,Assigner& assign) - { - if(is_first_of_bucket(x)){ - if(is_last_of_bucket(x)){ - assign(x->prior()->next()->prior(),pointer(0)); - assign(x->prior()->next(),x->next()); - assign(x->next()->prior()->prior(),x->prior()); - } - else{ - assign(x->prior()->next()->prior(),pointer_from(x->next())); - assign(x->next()->prior(),x->prior()); - } - } - else if(is_last_of_bucket(x)){ - assign(x->prior()->next(),x->next()); - assign(x->next()->prior()->prior(),x->prior()); - } - else{ - assign(x->prior()->next(),x->next()); - assign(x->next()->prior(),x->prior()); - } - } - - /* used only at rehashing */ - - static void append(pointer x,pointer end) - { - x->prior()=end->prior(); - x->next()=end->prior()->next(); - x->prior()->next()=base_pointer_from(x); - end->prior()=x; - } - - static bool unlink_last(pointer end) - { - /* returns true iff bucket is emptied */ - - pointer x=end->prior(); - if(x->prior()->next()==base_pointer_from(x)){ - x->prior()->next()=x->next(); - end->prior()=x->prior(); - return false; - } - else{ - x->prior()->next()->prior()=pointer(0); - x->prior()->next()=x->next(); - end->prior()=x->prior(); - return true; - } - } - -private: - static pointer pointer_from(base_pointer x) - { - return Node::pointer_from(x); - } - - static base_pointer base_pointer_from(pointer x) - { - return Node::base_pointer_from(x); - } - - static bool is_last_of_bucket(pointer x) - { - return x->next()->prior()!=x; - } -}; - -template -struct hashed_index_node_alg -{ - typedef typename Node::base_pointer base_pointer; - typedef typename Node::const_base_pointer const_base_pointer; - typedef typename Node::pointer pointer; - typedef typename Node::const_pointer const_pointer; - - static bool is_first_of_bucket(pointer x) - { - return x->prior()->next()->prior()==x; - } - - static bool is_first_of_group(pointer x) - { - return - x->next()->prior()!=x&& - x->next()->prior()->prior()->next()==base_pointer_from(x); - } - - static pointer after(pointer x) - { - if(x->next()->prior()==x)return pointer_from(x->next()); - if(x->next()->prior()->prior()==x)return x->next()->prior(); - if(x->next()->prior()->prior()->next()==base_pointer_from(x)) - return pointer_from(x->next()); - return pointer_from(x->next())->next()->prior(); - } - - static pointer after_local(pointer x) - { - if(x->next()->prior()==x)return pointer_from(x->next()); - if(x->next()->prior()->prior()==x)return pointer(0); - if(x->next()->prior()->prior()->next()==base_pointer_from(x)) - return pointer_from(x->next()); - return pointer_from(x->next())->next()->prior(); - } - - static pointer next_to_inspect(pointer x) - { - if(x->next()->prior()==x)return pointer_from(x->next()); - if(x->next()->prior()->prior()==x)return pointer(0); - if(x->next()->prior()->next()->prior()!=x->next()->prior()) - return pointer(0); - return pointer_from(x->next()->prior()->next()); - } - - static void link(pointer x,base_pointer buc,pointer end) - { - if(buc->prior()==pointer(0)){ /* empty bucket */ - x->prior()=end->prior(); - x->next()=end->prior()->next(); - x->prior()->next()=buc; - buc->prior()=x; - end->prior()=x; - } - else{ - x->prior()=buc->prior()->prior(); - x->next()=base_pointer_from(buc->prior()); - buc->prior()=x; - x->next()->prior()=x; - } - }; - - static void link(pointer x,pointer first,pointer last) - { - x->prior()=first->prior(); - x->next()=base_pointer_from(first); - if(is_first_of_bucket(first)){ - x->prior()->next()->prior()=x; - } - else{ - x->prior()->next()=base_pointer_from(x); - } - - if(first==last){ - last->prior()=x; - } - else if(first->next()==base_pointer_from(last)){ - first->prior()=last; - first->next()=base_pointer_from(x); - } - else{ - pointer second=pointer_from(first->next()), - lastbutone=last->prior(); - second->prior()=first; - first->prior()=last; - lastbutone->next()=base_pointer_from(x); - } - } - - static void unlink(pointer x) - { - default_assigner assign; - unlink(x,assign); - } - - typedef unlink_undo_assigner unlink_undo; - - template - static void unlink(pointer x,Assigner& assign) - { - if(x->prior()->next()==base_pointer_from(x)){ - if(x->next()->prior()==x){ - left_unlink(x,assign); - right_unlink(x,assign); - } - else if(x->next()->prior()->prior()==x){ /* last of bucket */ - left_unlink(x,assign); - right_unlink_last_of_bucket(x,assign); - } - else if(x->next()->prior()->prior()->next()== - base_pointer_from(x)){ /* first of group size */ - left_unlink(x,assign); - right_unlink_first_of_group(x,assign); - } - else{ /* n-1 of group */ - unlink_last_but_one_of_group(x,assign); - } - } - else if(x->prior()->next()->prior()==x){ /* first of bucket */ - if(x->next()->prior()==x){ - left_unlink_first_of_bucket(x,assign); - right_unlink(x,assign); - } - else if(x->next()->prior()->prior()==x){ /* last of bucket */ - assign(x->prior()->next()->prior(),pointer(0)); - assign(x->prior()->next(),x->next()); - assign(x->next()->prior()->prior(),x->prior()); - } - else{ /* first of group */ - left_unlink_first_of_bucket(x,assign); - right_unlink_first_of_group(x,assign); - } - } - else if(x->next()->prior()->prior()==x){ /* last of group and bucket */ - left_unlink_last_of_group(x,assign); - right_unlink_last_of_bucket(x,assign); - } - else if(pointer_from(x->prior()->prior()->next()) - ->next()==base_pointer_from(x)){ /* second of group */ - unlink_second_of_group(x,assign); - } - else{ /* last of group, ~(last of bucket) */ - left_unlink_last_of_group(x,assign); - right_unlink(x,assign); - } - } - - /* used only at rehashing */ - - static void link_range( - pointer first,pointer last,base_pointer buc,pointer cend) - { - if(buc->prior()==pointer(0)){ /* empty bucket */ - first->prior()=cend->prior(); - last->next()=cend->prior()->next(); - first->prior()->next()=buc; - buc->prior()=first; - cend->prior()=last; - } - else{ - first->prior()=buc->prior()->prior(); - last->next()=base_pointer_from(buc->prior()); - buc->prior()=first; - last->next()->prior()=last; - } - } - - static void append_range(pointer first,pointer last,pointer cend) - { - first->prior()=cend->prior(); - last->next()=cend->prior()->next(); - first->prior()->next()=base_pointer_from(first); - cend->prior()=last; - } - - static std::pair unlink_last_group(pointer end) - { - /* returns first of group true iff bucket is emptied */ - - pointer x=end->prior(); - if(x->prior()->next()==base_pointer_from(x)){ - x->prior()->next()=x->next(); - end->prior()=x->prior(); - return std::make_pair(x,false); - } - else if(x->prior()->next()->prior()==x){ - x->prior()->next()->prior()=pointer(0); - x->prior()->next()=x->next(); - end->prior()=x->prior(); - return std::make_pair(x,true); - } - else{ - pointer y=pointer_from(x->prior()->next()); - - if(y->prior()->next()==base_pointer_from(y)){ - y->prior()->next()=x->next(); - end->prior()=y->prior(); - return std::make_pair(y,false); - } - else{ - y->prior()->next()->prior()=pointer(0); - y->prior()->next()=x->next(); - end->prior()=y->prior(); - return std::make_pair(y,true); - } - } - } - - static void unlink_range(pointer first,pointer last) - { - if(is_first_of_bucket(first)){ - if(is_last_of_bucket(last)){ - first->prior()->next()->prior()=pointer(0); - first->prior()->next()=last->next(); - last->next()->prior()->prior()=first->prior(); - } - else{ - first->prior()->next()->prior()=pointer_from(last->next()); - last->next()->prior()=first->prior(); - } - } - else if(is_last_of_bucket(last)){ - first->prior()->next()=last->next(); - last->next()->prior()->prior()=first->prior(); - } - else{ - first->prior()->next()=last->next(); - last->next()->prior()=first->prior(); - } - } - -private: - static pointer pointer_from(base_pointer x) - { - return Node::pointer_from(x); - } - - static base_pointer base_pointer_from(pointer x) - { - return Node::base_pointer_from(x); - } - - static bool is_last_of_bucket(pointer x) - { - return x->next()->prior()->prior()==x; - } - - template - static void left_unlink(pointer x,Assigner& assign) - { - assign(x->prior()->next(),x->next()); - } - - template - static void right_unlink(pointer x,Assigner& assign) - { - assign(x->next()->prior(),x->prior()); - } - - template - static void left_unlink_first_of_bucket(pointer x,Assigner& assign) - { - assign(x->prior()->next()->prior(),pointer_from(x->next())); - } - - template - static void right_unlink_last_of_bucket(pointer x,Assigner& assign) - { - assign(x->next()->prior()->prior(),x->prior()); - } - - template - static void right_unlink_first_of_group(pointer x,Assigner& assign) - { - pointer second=pointer_from(x->next()), - last=second->prior(), - lastbutone=last->prior(); - if(second==lastbutone){ - assign(second->next(),base_pointer_from(last)); - assign(second->prior(),x->prior()); - } - else{ - assign(lastbutone->next(),base_pointer_from(second)); - assign(second->next()->prior(),last); - assign(second->prior(),x->prior()); - } - } - - template - static void left_unlink_last_of_group(pointer x,Assigner& assign) - { - pointer lastbutone=x->prior(), - first=pointer_from(lastbutone->next()), - second=pointer_from(first->next()); - if(lastbutone==second){ - assign(lastbutone->prior(),first); - assign(lastbutone->next(),x->next()); - } - else{ - assign(second->prior(),lastbutone); - assign(lastbutone->prior()->next(),base_pointer_from(first)); - assign(lastbutone->next(),x->next()); - } - } - - template - static void unlink_last_but_one_of_group(pointer x,Assigner& assign) - { - pointer first=pointer_from(x->next()), - second=pointer_from(first->next()), - last=second->prior(); - if(second==x){ - assign(last->prior(),first); - assign(first->next(),base_pointer_from(last)); - } - else{ - assign(last->prior(),x->prior()); - assign(x->prior()->next(),base_pointer_from(first)); - } - } - - template - static void unlink_second_of_group(pointer x,Assigner& assign) - { - pointer last=x->prior(), - lastbutone=last->prior(), - first=pointer_from(lastbutone->next()); - if(lastbutone==x){ - assign(first->next(),base_pointer_from(last)); - assign(last->prior(),first); - } - else{ - assign(first->next(),x->next()); - assign(x->next()->prior(),last); - } - } -}; - -template -struct hashed_index_node_trampoline: - hashed_index_node_impl< - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > -{ - typedef typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type impl_allocator_type; - typedef hashed_index_node_impl impl_type; -}; - -template -struct hashed_index_node: - Super,hashed_index_node_trampoline -{ -private: - typedef hashed_index_node_trampoline trampoline; - -public: - typedef typename trampoline::impl_type impl_type; - typedef hashed_index_node_alg< - impl_type,Category> node_alg; - typedef typename trampoline::base_pointer impl_base_pointer; - typedef typename trampoline::const_base_pointer const_impl_base_pointer; - typedef typename trampoline::pointer impl_pointer; - typedef typename trampoline::const_pointer const_impl_pointer; - - impl_pointer& prior(){return trampoline::prior();} - impl_pointer prior()const{return trampoline::prior();} - impl_base_pointer& next(){return trampoline::next();} - impl_base_pointer next()const{return trampoline::next();} - - impl_pointer impl() - { - return static_cast( - static_cast(static_cast(this))); - } - - const_impl_pointer impl()const - { - return static_cast( - static_cast(static_cast(this))); - } - - static hashed_index_node* from_impl(impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - static const hashed_index_node* from_impl(const_impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - /* interoperability with hashed_index_iterator */ - - static void increment(hashed_index_node*& x) - { - x=from_impl(node_alg::after(x->impl())); - } - - static void increment_local(hashed_index_node*& x) - { - x=from_impl(node_alg::after_local(x->impl())); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/header_holder.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/header_holder.hpp deleted file mode 100644 index ca8a9b2edb1..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/header_holder.hpp +++ /dev/null @@ -1,50 +0,0 @@ -/* Copyright 2003-2008 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_HEADER_HOLDER_HPP -#define BOOST_MULTI_INDEX_DETAIL_HEADER_HOLDER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* A utility class used to hold a pointer to the header node. - * The base from member idiom is used because index classes, which are - * superclasses of multi_index_container, need this header in construction - * time. The allocation is made by the allocator of the multi_index_container - * class --hence, this allocator needs also be stored resorting - * to the base from member trick. - */ - -template -struct header_holder:private noncopyable -{ - header_holder():member(final().allocate_node()){} - ~header_holder(){final().deallocate_node(&*member);} - - NodeTypePtr member; - -private: - Final& final(){return *static_cast(this);} -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ignore_wstrict_aliasing.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ignore_wstrict_aliasing.hpp deleted file mode 100644 index ae398456d1f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ignore_wstrict_aliasing.hpp +++ /dev/null @@ -1,18 +0,0 @@ -/* Copyright 2003-2016 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#include - -#if defined(BOOST_GCC)&&(BOOST_GCC>=4*10000+6*100) -#if !defined(BOOST_MULTI_INDEX_DETAIL_RESTORE_WSTRICT_ALIASING) -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wstrict-aliasing" -#else -#pragma GCC diagnostic pop -#endif -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_base.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_base.hpp deleted file mode 100644 index 99000ed4813..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_base.hpp +++ /dev/null @@ -1,293 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_BASE_HPP -#define BOOST_MULTI_INDEX_DETAIL_INDEX_BASE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* The role of this class is threefold: - * - tops the linear hierarchy of indices. - * - terminates some cascading backbone function calls (insert_, etc.), - * - grants access to the backbone functions of the final - * multi_index_container class (for access restriction reasons, these - * cannot be called directly from the index classes.) - */ - -struct lvalue_tag{}; -struct rvalue_tag{}; -struct emplaced_tag{}; - -template -class index_base -{ -protected: - typedef index_node_base node_type; - typedef typename multi_index_node_type< - Value,IndexSpecifierList,Allocator>::type final_node_type; - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> final_type; - typedef tuples::null_type ctor_args_list; - typedef typename - boost::detail::allocator::rebind_to< - Allocator, - typename Allocator::value_type - >::type final_allocator_type; - typedef mpl::vector0<> index_type_list; - typedef mpl::vector0<> iterator_type_list; - typedef mpl::vector0<> const_iterator_type_list; - typedef copy_map< - final_node_type, - final_allocator_type> copy_map_type; - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - typedef index_saver< - node_type, - final_allocator_type> index_saver_type; - typedef index_loader< - node_type, - final_node_type, - final_allocator_type> index_loader_type; -#endif - -private: - typedef Value value_type; - -protected: - explicit index_base(const ctor_args_list&,const Allocator&){} - - index_base( - const index_base&, - do_not_copy_elements_tag) - {} - - void copy_( - const index_base&,const copy_map_type&) - {} - - final_node_type* insert_(const value_type& v,final_node_type*& x,lvalue_tag) - { - x=final().allocate_node(); - BOOST_TRY{ - boost::detail::allocator::construct(&x->value(),v); - } - BOOST_CATCH(...){ - final().deallocate_node(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - return x; - } - - final_node_type* insert_(const value_type& v,final_node_type*& x,rvalue_tag) - { - x=final().allocate_node(); - BOOST_TRY{ - /* This shoud have used a modified, T&&-compatible version of - * boost::detail::allocator::construct, but - * is too old and venerable to - * mess with; besides, it is a general internal utility and the imperfect - * perfect forwarding emulation of Boost.Move might break other libs. - */ - - new (&x->value()) value_type(boost::move(const_cast(v))); - } - BOOST_CATCH(...){ - final().deallocate_node(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - return x; - } - - final_node_type* insert_(const value_type&,final_node_type*& x,emplaced_tag) - { - return x; - } - - final_node_type* insert_( - const value_type& v,node_type*,final_node_type*& x,lvalue_tag) - { - return insert_(v,x,lvalue_tag()); - } - - final_node_type* insert_( - const value_type& v,node_type*,final_node_type*& x,rvalue_tag) - { - return insert_(v,x,rvalue_tag()); - } - - final_node_type* insert_( - const value_type&,node_type*,final_node_type*& x,emplaced_tag) - { - return x; - } - - void erase_(node_type* x) - { - boost::detail::allocator::destroy(&x->value()); - } - - void delete_node_(node_type* x) - { - boost::detail::allocator::destroy(&x->value()); - } - - void clear_(){} - - void swap_(index_base&){} - - void swap_elements_(index_base&){} - - bool replace_(const value_type& v,node_type* x,lvalue_tag) - { - x->value()=v; - return true; - } - - bool replace_(const value_type& v,node_type* x,rvalue_tag) - { - x->value()=boost::move(const_cast(v)); - return true; - } - - bool modify_(node_type*){return true;} - - bool modify_rollback_(node_type*){return true;} - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* serialization */ - - template - void save_(Archive&,const unsigned int,const index_saver_type&)const{} - - template - void load_(Archive&,const unsigned int,const index_loader_type&){} -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - bool invariant_()const{return true;} -#endif - - /* access to backbone memfuns of Final class */ - - final_type& final(){return *static_cast(this);} - const final_type& final()const{return *static_cast(this);} - - final_node_type* final_header()const{return final().header();} - - bool final_empty_()const{return final().empty_();} - std::size_t final_size_()const{return final().size_();} - std::size_t final_max_size_()const{return final().max_size_();} - - std::pair final_insert_(const value_type& x) - {return final().insert_(x);} - std::pair final_insert_rv_(const value_type& x) - {return final().insert_rv_(x);} - template - std::pair final_insert_ref_(const T& t) - {return final().insert_ref_(t);} - template - std::pair final_insert_ref_(T& t) - {return final().insert_ref_(t);} - - template - std::pair final_emplace_( - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - return final().emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - } - - std::pair final_insert_( - const value_type& x,final_node_type* position) - {return final().insert_(x,position);} - std::pair final_insert_rv_( - const value_type& x,final_node_type* position) - {return final().insert_rv_(x,position);} - template - std::pair final_insert_ref_( - const T& t,final_node_type* position) - {return final().insert_ref_(t,position);} - template - std::pair final_insert_ref_( - T& t,final_node_type* position) - {return final().insert_ref_(t,position);} - - template - std::pair final_emplace_hint_( - final_node_type* position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - return final().emplace_hint_( - position,BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - } - - void final_erase_(final_node_type* x){final().erase_(x);} - - void final_delete_node_(final_node_type* x){final().delete_node_(x);} - void final_delete_all_nodes_(){final().delete_all_nodes_();} - void final_clear_(){final().clear_();} - - void final_swap_(final_type& x){final().swap_(x);} - - bool final_replace_( - const value_type& k,final_node_type* x) - {return final().replace_(k,x);} - bool final_replace_rv_( - const value_type& k,final_node_type* x) - {return final().replace_rv_(k,x);} - - template - bool final_modify_(Modifier& mod,final_node_type* x) - {return final().modify_(mod,x);} - - template - bool final_modify_(Modifier& mod,Rollback& back,final_node_type* x) - {return final().modify_(mod,back,x);} - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - void final_check_invariant_()const{final().check_invariant_();} -#endif -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_loader.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_loader.hpp deleted file mode 100644 index 71418a10e19..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_loader.hpp +++ /dev/null @@ -1,139 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_LOADER_HPP -#define BOOST_MULTI_INDEX_DETAIL_INDEX_LOADER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Counterpart of index_saver (check index_saver.hpp for serialization - * details.)* multi_index_container is in charge of supplying the info about - * the base sequence, and each index can subsequently load itself using the - * const interface of index_loader. - */ - -template -class index_loader:private noncopyable -{ -public: - index_loader(const Allocator& al,std::size_t size): - spc(al,size),size_(size),n(0),sorted(false) - { - } - - template - void add(Node* node,Archive& ar,const unsigned int) - { - ar>>serialization::make_nvp("position",*node); - entries()[n++]=node; - } - - template - void add_track(Node* node,Archive& ar,const unsigned int) - { - ar>>serialization::make_nvp("position",*node); - } - - /* A rearranger is passed two nodes, and is expected to - * reposition the second after the first. - * If the first node is 0, then the second should be moved - * to the beginning of the sequence. - */ - - template - void load(Rearranger r,Archive& ar,const unsigned int)const - { - FinalNode* prev=unchecked_load_node(ar); - if(!prev)return; - - if(!sorted){ - std::sort(entries(),entries()+size_); - sorted=true; - } - - check_node(prev); - - for(;;){ - for(;;){ - FinalNode* node=load_node(ar); - if(!node)break; - - if(node==prev)prev=0; - r(prev,node); - - prev=node; - } - prev=load_node(ar); - if(!prev)break; - } - } - -private: - Node** entries()const{return raw_ptr(spc.data());} - - /* We try to delay sorting as much as possible just in case it - * is not necessary, hence this version of load_node. - */ - - template - FinalNode* unchecked_load_node(Archive& ar)const - { - Node* node=0; - ar>>serialization::make_nvp("pointer",node); - return static_cast(node); - } - - template - FinalNode* load_node(Archive& ar)const - { - Node* node=0; - ar>>serialization::make_nvp("pointer",node); - check_node(node); - return static_cast(node); - } - - void check_node(Node* node)const - { - if(node!=0&&!std::binary_search(entries(),entries()+size_,node)){ - throw_exception( - archive::archive_exception( - archive::archive_exception::other_exception)); - } - } - - auto_space spc; - std::size_t size_; - std::size_t n; - mutable bool sorted; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_matcher.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_matcher.hpp deleted file mode 100644 index 34d1f9d5a8d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_matcher.hpp +++ /dev/null @@ -1,249 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_MATCHER_HPP -#define BOOST_MULTI_INDEX_DETAIL_INDEX_MATCHER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* index_matcher compares a sequence of elements against a - * base sequence, identifying those elements that belong to the - * longest subsequence which is ordered with respect to the base. - * For instance, if the base sequence is: - * - * 0 1 2 3 4 5 6 7 8 9 - * - * and the compared sequence (not necesarilly the same length): - * - * 1 4 2 3 0 7 8 9 - * - * the elements of the longest ordered subsequence are: - * - * 1 2 3 7 8 9 - * - * The algorithm for obtaining such a subsequence is called - * Patience Sorting, described in ch. 1 of: - * Aldous, D., Diaconis, P.: "Longest increasing subsequences: from - * patience sorting to the Baik-Deift-Johansson Theorem", Bulletin - * of the American Mathematical Society, vol. 36, no 4, pp. 413-432, - * July 1999. - * http://www.ams.org/bull/1999-36-04/S0273-0979-99-00796-X/ - * S0273-0979-99-00796-X.pdf - * - * This implementation is not fully generic since it assumes that - * the sequences given are pointed to by index iterators (having a - * get_node() memfun.) - */ - -namespace index_matcher{ - -/* The algorithm stores the nodes of the base sequence and a number - * of "piles" that are dynamically updated during the calculation - * stage. From a logical point of view, nodes form an independent - * sequence from piles. They are stored together so as to minimize - * allocated memory. - */ - -struct entry -{ - entry(void* node_,std::size_t pos_=0):node(node_),pos(pos_){} - - /* node stuff */ - - void* node; - std::size_t pos; - entry* previous; - bool ordered; - - struct less_by_node - { - bool operator()( - const entry& x,const entry& y)const - { - return std::less()(x.node,y.node); - } - }; - - /* pile stuff */ - - std::size_t pile_top; - entry* pile_top_entry; - - struct less_by_pile_top - { - bool operator()( - const entry& x,const entry& y)const - { - return x.pile_top -class algorithm_base:private noncopyable -{ -protected: - algorithm_base(const Allocator& al,std::size_t size): - spc(al,size),size_(size),n_(0),sorted(false) - { - } - - void add(void* node) - { - entries()[n_]=entry(node,n_); - ++n_; - } - - void begin_algorithm()const - { - if(!sorted){ - std::sort(entries(),entries()+size_,entry::less_by_node()); - sorted=true; - } - num_piles=0; - } - - void add_node_to_algorithm(void* node)const - { - entry* ent= - std::lower_bound( - entries(),entries()+size_, - entry(node),entry::less_by_node()); /* localize entry */ - ent->ordered=false; - std::size_t n=ent->pos; /* get its position */ - - entry dummy(0); - dummy.pile_top=n; - - entry* pile_ent= /* find the first available pile */ - std::lower_bound( /* to stack the entry */ - entries(),entries()+num_piles, - dummy,entry::less_by_pile_top()); - - pile_ent->pile_top=n; /* stack the entry */ - pile_ent->pile_top_entry=ent; - - /* if not the first pile, link entry to top of the preceding pile */ - if(pile_ent>&entries()[0]){ - ent->previous=(pile_ent-1)->pile_top_entry; - } - - if(pile_ent==&entries()[num_piles]){ /* new pile? */ - ++num_piles; - } - } - - void finish_algorithm()const - { - if(num_piles>0){ - /* Mark those elements which are in their correct position, i.e. those - * belonging to the longest increasing subsequence. These are those - * elements linked from the top of the last pile. - */ - - entry* ent=entries()[num_piles-1].pile_top_entry; - for(std::size_t n=num_piles;n--;){ - ent->ordered=true; - ent=ent->previous; - } - } - } - - bool is_ordered(void * node)const - { - return std::lower_bound( - entries(),entries()+size_, - entry(node),entry::less_by_node())->ordered; - } - -private: - entry* entries()const{return raw_ptr(spc.data());} - - auto_space spc; - std::size_t size_; - std::size_t n_; - mutable bool sorted; - mutable std::size_t num_piles; -}; - -/* The algorithm has three phases: - * - Initialization, during which the nodes of the base sequence are added. - * - Execution. - * - Results querying, through the is_ordered memfun. - */ - -template -class algorithm:private algorithm_base -{ - typedef algorithm_base super; - -public: - algorithm(const Allocator& al,std::size_t size):super(al,size){} - - void add(Node* node) - { - super::add(node); - } - - template - void execute(IndexIterator first,IndexIterator last)const - { - super::begin_algorithm(); - - for(IndexIterator it=first;it!=last;++it){ - add_node_to_algorithm(get_node(it)); - } - - super::finish_algorithm(); - } - - bool is_ordered(Node* node)const - { - return super::is_ordered(node); - } - -private: - void add_node_to_algorithm(Node* node)const - { - super::add_node_to_algorithm(node); - } - - template - static Node* get_node(IndexIterator it) - { - return static_cast(it.get_node()); - } -}; - -} /* namespace multi_index::detail::index_matcher */ - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_node_base.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_node_base.hpp deleted file mode 100644 index 1a1f0cae4be..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_node_base.hpp +++ /dev/null @@ -1,135 +0,0 @@ -/* Copyright 2003-2016 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_NODE_BASE_HPP -#define BOOST_MULTI_INDEX_DETAIL_INDEX_NODE_BASE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* index_node_base tops the node hierarchy of multi_index_container. It holds - * the value of the element contained. - */ - -template -struct pod_value_holder -{ - typename aligned_storage< - sizeof(Value), - alignment_of::value - >::type space; -}; - -template -struct index_node_base:private pod_value_holder -{ - typedef index_node_base base_type; /* used for serialization purposes */ - typedef Value value_type; - typedef Allocator allocator_type; - -#include - - value_type& value() - { - return *reinterpret_cast(&this->space); - } - - const value_type& value()const - { - return *reinterpret_cast(&this->space); - } - -#include - - static index_node_base* from_value(const value_type* p) - { - return static_cast( - reinterpret_cast*>( /* std 9.2.17 */ - const_cast(p))); - } - -private: -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - friend class boost::serialization::access; - - /* nodes do not emit any kind of serialization info. They are - * fed to Boost.Serialization so that pointers to nodes are - * tracked correctly. - */ - - template - void serialize(Archive&,const unsigned int) - { - } -#endif -}; - -template -Node* node_from_value(const Value* p) -{ - typedef typename Node::allocator_type allocator_type; - return static_cast( - index_node_base::from_value(p)); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -/* Index nodes never get constructed directly by Boost.Serialization, - * as archives are always fed pointers to previously existent - * nodes. So, if this is called it means we are dealing with a - * somehow invalid archive. - */ - -#if defined(BOOST_NO_ARGUMENT_DEPENDENT_LOOKUP) -namespace serialization{ -#else -namespace multi_index{ -namespace detail{ -#endif - -template -inline void load_construct_data( - Archive&,boost::multi_index::detail::index_node_base*, - const unsigned int) -{ - throw_exception( - archive::archive_exception(archive::archive_exception::other_exception)); -} - -#if defined(BOOST_NO_ARGUMENT_DEPENDENT_LOOKUP) -} /* namespace serialization */ -#else -} /* namespace multi_index::detail */ -} /* namespace multi_index */ -#endif - -#endif - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_saver.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_saver.hpp deleted file mode 100644 index ae09d4eba4f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/index_saver.hpp +++ /dev/null @@ -1,135 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_INDEX_SAVER_HPP -#define BOOST_MULTI_INDEX_DETAIL_INDEX_SAVER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* index_saver accepts a base sequence of previously saved elements - * and saves a possibly reordered subsequence in an efficient manner, - * serializing only the information needed to rearrange the subsequence - * based on the original order of the base. - * multi_index_container is in charge of supplying the info about the - * base sequence, and each index can subsequently save itself using the - * const interface of index_saver. - */ - -template -class index_saver:private noncopyable -{ -public: - index_saver(const Allocator& al,std::size_t size):alg(al,size){} - - template - void add(Node* node,Archive& ar,const unsigned int) - { - ar< - void add_track(Node* node,Archive& ar,const unsigned int) - { - ar< - void save( - IndexIterator first,IndexIterator last,Archive& ar, - const unsigned int)const - { - /* calculate ordered positions */ - - alg.execute(first,last); - - /* Given a consecutive subsequence of displaced elements - * x1,...,xn, the following information is serialized: - * - * p0,p1,...,pn,0 - * - * where pi is a pointer to xi and p0 is a pointer to the element - * preceding x1. Crealy, from this information is possible to - * restore the original order on loading time. If x1 is the first - * element in the sequence, the following is serialized instead: - * - * p1,p1,...,pn,0 - * - * For each subsequence of n elements, n+2 pointers are serialized. - * An optimization policy is applied: consider for instance the - * sequence - * - * a,B,c,D - * - * where B and D are displaced, but c is in its correct position. - * Applying the schema described above we would serialize 6 pointers: - * - * p(a),p(B),0 - * p(c),p(D),0 - * - * but this can be reduced to 5 pointers by treating c as a displaced - * element: - * - * p(a),p(B),p(c),p(D),0 - */ - - std::size_t last_saved=3; /* distance to last pointer saved */ - for(IndexIterator it=first,prev=first;it!=last;prev=it++,++last_saved){ - if(!alg.is_ordered(get_node(it))){ - if(last_saved>1)save_node(get_node(prev),ar); - save_node(get_node(it),ar); - last_saved=0; - } - else if(last_saved==2)save_node(null_node(),ar); - } - if(last_saved<=2)save_node(null_node(),ar); - - /* marks the end of the serialization info for [first,last) */ - - save_node(null_node(),ar); - } - -private: - template - static Node* get_node(IndexIterator it) - { - return it.get_node(); - } - - static Node* null_node(){return 0;} - - template - static void save_node(Node* node,Archive& ar) - { - ar< alg; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/invariant_assert.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/invariant_assert.hpp deleted file mode 100644 index c6c547c7c33..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/invariant_assert.hpp +++ /dev/null @@ -1,21 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_INVARIANT_ASSERT_HPP -#define BOOST_MULTI_INDEX_DETAIL_INVARIANT_ASSERT_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#if !defined(BOOST_MULTI_INDEX_INVARIANT_ASSERT) -#include -#define BOOST_MULTI_INDEX_INVARIANT_ASSERT BOOST_ASSERT -#endif - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_index_list.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_index_list.hpp deleted file mode 100644 index f6a24218b81..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_index_list.hpp +++ /dev/null @@ -1,40 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_IS_INDEX_LIST_HPP -#define BOOST_MULTI_INDEX_DETAIL_IS_INDEX_LIST_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -struct is_index_list -{ - BOOST_STATIC_CONSTANT(bool,mpl_sequence=mpl::is_sequence::value); - BOOST_STATIC_CONSTANT(bool,non_empty=!mpl::empty::value); - BOOST_STATIC_CONSTANT(bool,value=mpl_sequence&&non_empty); -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_transparent.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_transparent.hpp deleted file mode 100644 index 72036d257e2..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/is_transparent.hpp +++ /dev/null @@ -1,135 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_IS_TRANSPARENT_HPP -#define BOOST_MULTI_INDEX_DETAIL_IS_TRANSPARENT_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Metafunction that checks if f(arg,arg2) executes without argument type - * conversion. By default (i.e. when it cannot be determined) it evaluates to - * true. - */ - -template -struct is_transparent:mpl::true_{}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#if !defined(BOOST_NO_SFINAE)&&!defined(BOOST_NO_SFINAE_EXPR)&& \ - !defined(BOOST_NO_CXX11_DECLTYPE)&& \ - (defined(BOOST_NO_CXX11_FINAL)||defined(BOOST_IS_FINAL)) - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -struct not_is_transparent_result_type{}; - -template -struct is_transparent_class_helper:F -{ - using F::operator(); - template - not_is_transparent_result_type operator()(const T&,const Q&)const; -}; - -template -struct is_transparent_class:mpl::true_{}; - -template -struct is_transparent_class< - F,Arg1,Arg2, - typename enable_if< - is_same< - decltype( - declval >()( - declval(),declval()) - ), - not_is_transparent_result_type - > - >::type ->:mpl::false_{}; - -template -struct is_transparent< - F,Arg1,Arg2, - typename enable_if< - mpl::and_< - is_class, - mpl::not_ > /* is_transparent_class_helper derives from F */ - > - >::type ->:is_transparent_class{}; - -template -struct is_transparent_function:mpl::true_{}; - -template -struct is_transparent_function< - F,Arg1,Arg2, - typename enable_if< - mpl::or_< - mpl::not_::arg1_type,const Arg1&>, - is_same::arg1_type,Arg1> - > >, - mpl::not_::arg2_type,const Arg2&>, - is_same::arg2_type,Arg2> - > > - > - >::type ->:mpl::false_{}; - -template -struct is_transparent< - F,Arg1,Arg2, - typename enable_if< - is_function::type> - >::type ->:is_transparent_function::type,Arg1,Arg2>{}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/iter_adaptor.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/iter_adaptor.hpp deleted file mode 100644 index 7a032350b36..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/iter_adaptor.hpp +++ /dev/null @@ -1,321 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ITER_ADAPTOR_HPP -#define BOOST_MULTI_INDEX_DETAIL_ITER_ADAPTOR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Poor man's version of boost::iterator_adaptor. Used instead of the - * original as compile times for the latter are significantly higher. - * The interface is not replicated exactly, only to the extent necessary - * for internal consumption. - */ - -/* NB. The purpose of the (non-inclass) global operators ==, < and - defined - * above is to partially alleviate a problem of MSVC++ 6.0 by * which - * friend-injected operators on T are not visible if T is instantiated only - * in template code where T is a dependent type. - */ - -class iter_adaptor_access -{ -public: - template - static typename Class::reference dereference(const Class& x) - { - return x.dereference(); - } - - template - static bool equal(const Class& x,const Class& y) - { - return x.equal(y); - } - - template - static void increment(Class& x) - { - x.increment(); - } - - template - static void decrement(Class& x) - { - x.decrement(); - } - - template - static void advance(Class& x,typename Class::difference_type n) - { - x.advance(n); - } - - template - static typename Class::difference_type distance_to( - const Class& x,const Class& y) - { - return x.distance_to(y); - } -}; - -template -struct iter_adaptor_selector; - -template -class forward_iter_adaptor_base: - public forward_iterator_helper< - Derived, - typename Base::value_type, - typename Base::difference_type, - typename Base::pointer, - typename Base::reference> -{ -public: - typedef typename Base::reference reference; - - reference operator*()const - { - return iter_adaptor_access::dereference(final()); - } - - friend bool operator==(const Derived& x,const Derived& y) - { - return iter_adaptor_access::equal(x,y); - } - - Derived& operator++() - { - iter_adaptor_access::increment(final()); - return final(); - } - -private: - Derived& final(){return *static_cast(this);} - const Derived& final()const{return *static_cast(this);} -}; - -template -bool operator==( - const forward_iter_adaptor_base& x, - const forward_iter_adaptor_base& y) -{ - return iter_adaptor_access::equal( - static_cast(x),static_cast(y)); -} - -template<> -struct iter_adaptor_selector -{ - template - struct apply - { - typedef forward_iter_adaptor_base type; - }; -}; - -template -class bidirectional_iter_adaptor_base: - public bidirectional_iterator_helper< - Derived, - typename Base::value_type, - typename Base::difference_type, - typename Base::pointer, - typename Base::reference> -{ -public: - typedef typename Base::reference reference; - - reference operator*()const - { - return iter_adaptor_access::dereference(final()); - } - - friend bool operator==(const Derived& x,const Derived& y) - { - return iter_adaptor_access::equal(x,y); - } - - Derived& operator++() - { - iter_adaptor_access::increment(final()); - return final(); - } - - Derived& operator--() - { - iter_adaptor_access::decrement(final()); - return final(); - } - -private: - Derived& final(){return *static_cast(this);} - const Derived& final()const{return *static_cast(this);} -}; - -template -bool operator==( - const bidirectional_iter_adaptor_base& x, - const bidirectional_iter_adaptor_base& y) -{ - return iter_adaptor_access::equal( - static_cast(x),static_cast(y)); -} - -template<> -struct iter_adaptor_selector -{ - template - struct apply - { - typedef bidirectional_iter_adaptor_base type; - }; -}; - -template -class random_access_iter_adaptor_base: - public random_access_iterator_helper< - Derived, - typename Base::value_type, - typename Base::difference_type, - typename Base::pointer, - typename Base::reference> -{ -public: - typedef typename Base::reference reference; - typedef typename Base::difference_type difference_type; - - reference operator*()const - { - return iter_adaptor_access::dereference(final()); - } - - friend bool operator==(const Derived& x,const Derived& y) - { - return iter_adaptor_access::equal(x,y); - } - - friend bool operator<(const Derived& x,const Derived& y) - { - return iter_adaptor_access::distance_to(x,y)>0; - } - - Derived& operator++() - { - iter_adaptor_access::increment(final()); - return final(); - } - - Derived& operator--() - { - iter_adaptor_access::decrement(final()); - return final(); - } - - Derived& operator+=(difference_type n) - { - iter_adaptor_access::advance(final(),n); - return final(); - } - - Derived& operator-=(difference_type n) - { - iter_adaptor_access::advance(final(),-n); - return final(); - } - - friend difference_type operator-(const Derived& x,const Derived& y) - { - return iter_adaptor_access::distance_to(y,x); - } - -private: - Derived& final(){return *static_cast(this);} - const Derived& final()const{return *static_cast(this);} -}; - -template -bool operator==( - const random_access_iter_adaptor_base& x, - const random_access_iter_adaptor_base& y) -{ - return iter_adaptor_access::equal( - static_cast(x),static_cast(y)); -} - -template -bool operator<( - const random_access_iter_adaptor_base& x, - const random_access_iter_adaptor_base& y) -{ - return iter_adaptor_access::distance_to( - static_cast(x),static_cast(y))>0; -} - -template -typename random_access_iter_adaptor_base::difference_type -operator-( - const random_access_iter_adaptor_base& x, - const random_access_iter_adaptor_base& y) -{ - return iter_adaptor_access::distance_to( - static_cast(y),static_cast(x)); -} - -template<> -struct iter_adaptor_selector -{ - template - struct apply - { - typedef random_access_iter_adaptor_base type; - }; -}; - -template -struct iter_adaptor_base -{ - typedef iter_adaptor_selector< - typename Base::iterator_category> selector; - typedef typename mpl::apply2< - selector,Derived,Base>::type type; -}; - -template -class iter_adaptor:public iter_adaptor_base::type -{ -protected: - iter_adaptor(){} - explicit iter_adaptor(const Base& b_):b(b_){} - - const Base& base_reference()const{return b;} - Base& base_reference(){return b;} - -private: - Base b; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/modify_key_adaptor.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/modify_key_adaptor.hpp deleted file mode 100644 index 6df89b18386..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/modify_key_adaptor.hpp +++ /dev/null @@ -1,49 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_MODIFY_KEY_ADAPTOR_HPP -#define BOOST_MULTI_INDEX_DETAIL_MODIFY_KEY_ADAPTOR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Functional adaptor to resolve modify_key as a call to modify. - * Preferred over compose_f_gx and stuff cause it eliminates problems - * with references to references, dealing with function pointers, etc. - */ - -template -struct modify_key_adaptor -{ - - modify_key_adaptor(Fun f_,KeyFromValue kfv_):f(f_),kfv(kfv_){} - - void operator()(Value& x) - { - f(kfv(x)); - } - -private: - Fun f; - KeyFromValue kfv; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/no_duplicate_tags.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/no_duplicate_tags.hpp deleted file mode 100644 index ba216ed82cf..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/no_duplicate_tags.hpp +++ /dev/null @@ -1,97 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_NO_DUPLICATE_TAGS_HPP -#define BOOST_MULTI_INDEX_DETAIL_NO_DUPLICATE_TAGS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* no_duplicate_tags check at compile-time that a tag list - * has no duplicate tags. - * The algorithm deserves some explanation: tags - * are sequentially inserted into a mpl::set if they were - * not already present. Due to the magic of mpl::set - * (mpl::has_key is contant time), this operation takes linear - * time, and even MSVC++ 6.5 handles it gracefully (other obvious - * solutions are quadratic.) - */ - -struct duplicate_tag_mark{}; - -struct duplicate_tag_marker -{ - template - struct apply - { - typedef mpl::s_item< - typename mpl::if_,duplicate_tag_mark,Tag>::type, - MplSet - > type; - }; -}; - -template -struct no_duplicate_tags -{ - typedef typename mpl::fold< - TagList, - mpl::set0<>, - duplicate_tag_marker - >::type aux; - - BOOST_STATIC_CONSTANT( - bool,value=!(mpl::has_key::value)); -}; - -/* Variant for an index list: duplication is checked - * across all the indices. - */ - -struct duplicate_tag_list_marker -{ - template - struct apply:mpl::fold< - BOOST_DEDUCED_TYPENAME Index::tag_list, - MplSet, - duplicate_tag_marker> - { - }; -}; - -template -struct no_duplicate_tags_in_index_list -{ - typedef typename mpl::fold< - IndexList, - mpl::set0<>, - duplicate_tag_list_marker - >::type aux; - - BOOST_STATIC_CONSTANT( - bool,value=!(mpl::has_key::value)); -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/node_type.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/node_type.hpp deleted file mode 100644 index 7fe85cf968b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/node_type.hpp +++ /dev/null @@ -1,66 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_NODE_TYPE_HPP -#define BOOST_MULTI_INDEX_DETAIL_NODE_TYPE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* MPL machinery to construct the internal node type associated to an - * index list. - */ - -struct index_node_applier -{ - template - struct apply - { - typedef typename mpl::deref::type index_specifier; - typedef typename index_specifier:: - BOOST_NESTED_TEMPLATE node_class::type type; - }; -}; - -template -struct multi_index_node_type -{ - BOOST_STATIC_ASSERT(detail::is_index_list::value); - - typedef typename mpl::reverse_iter_fold< - IndexSpecifierList, - index_node_base, - mpl::bind2 - >::type type; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_args.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_args.hpp deleted file mode 100644 index 3e2641f2f4d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_args.hpp +++ /dev/null @@ -1,83 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_ARGS_HPP -#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_ARGS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Oredered index specifiers can be instantiated in two forms: - * - * (ordered_unique|ordered_non_unique)< - * KeyFromValue,Compare=std::less > - * (ordered_unique|ordered_non_unique)< - * TagList,KeyFromValue,Compare=std::less > - * - * index_args implements the machinery to accept this argument-dependent - * polymorphism. - */ - -template -struct index_args_default_compare -{ - typedef std::less type; -}; - -template -struct ordered_index_args -{ - typedef is_tag full_form; - - typedef typename mpl::if_< - full_form, - Arg1, - tag< > >::type tag_list_type; - typedef typename mpl::if_< - full_form, - Arg2, - Arg1>::type key_from_value_type; - typedef typename mpl::if_< - full_form, - Arg3, - Arg2>::type supplied_compare_type; - typedef typename mpl::eval_if< - mpl::is_na, - index_args_default_compare, - mpl::identity - >::type compare_type; - - BOOST_STATIC_ASSERT(is_tag::value); - BOOST_STATIC_ASSERT(!mpl::is_na::value); - BOOST_STATIC_ASSERT(!mpl::is_na::value); -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl.hpp deleted file mode 100644 index 040cb989630..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl.hpp +++ /dev/null @@ -1,1567 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - * - * The internal implementation of red-black trees is based on that of SGI STL - * stl_tree.h file: - * - * Copyright (c) 1996,1997 - * Silicon Graphics Computer Systems, Inc. - * - * Permission to use, copy, modify, distribute and sell this software - * and its documentation for any purpose is hereby granted without fee, - * provided that the above copyright notice appear in all copies and - * that both that copyright notice and this permission notice appear - * in supporting documentation. Silicon Graphics makes no - * representations about the suitability of this software for any - * purpose. It is provided "as is" without express or implied warranty. - * - * - * Copyright (c) 1994 - * Hewlett-Packard Company - * - * Permission to use, copy, modify, distribute and sell this software - * and its documentation for any purpose is hereby granted without fee, - * provided that the above copyright notice appear in all copies and - * that both that copyright notice and this permission notice appear - * in supporting documentation. Hewlett-Packard Company makes no - * representations about the suitability of this software for any - * purpose. It is provided "as is" without express or implied warranty. - * - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_IMPL_HPP -#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_IMPL_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) -#include -#endif - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#include -#include -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) -#define BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF(x) \ - detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ - detail::make_obj_guard(x,&ordered_index_impl::check_invariant_); \ - BOOST_JOIN(check_invariant_,__LINE__).touch(); -#define BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT \ - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF(*this) -#else -#define BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF(x) -#define BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* ordered_index adds a layer of ordered indexing to a given Super and accepts - * an augmenting policy for optional addition of order statistics. - */ - -/* Most of the implementation of unique and non-unique indices is - * shared. We tell from one another on instantiation time by using - * these tags. - */ - -struct ordered_unique_tag{}; -struct ordered_non_unique_tag{}; - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -class ordered_index; - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -class ordered_index_impl: - BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS SuperMeta::type - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - ,public safe_mode::safe_container< - ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy> > -#endif - -{ -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the - * lifetime of const references bound to temporaries --precisely what - * scopeguards are. - */ - -#pragma parse_mfunc_templ off -#endif - - typedef typename SuperMeta::type super; - -protected: - typedef ordered_index_node< - AugmentPolicy,typename super::node_type> node_type; - -protected: /* for the benefit of AugmentPolicy::augmented_interface */ - typedef typename node_type::impl_type node_impl_type; - typedef typename node_impl_type::pointer node_impl_pointer; - -public: - /* types */ - - typedef typename KeyFromValue::result_type key_type; - typedef typename node_type::value_type value_type; - typedef KeyFromValue key_from_value; - typedef Compare key_compare; - typedef value_comparison< - value_type,KeyFromValue,Compare> value_compare; - typedef tuple ctor_args; - typedef typename super::final_allocator_type allocator_type; - typedef typename allocator_type::reference reference; - typedef typename allocator_type::const_reference const_reference; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_iterator< - bidir_node_iterator, - ordered_index_impl> iterator; -#else - typedef bidir_node_iterator iterator; -#endif - - typedef iterator const_iterator; - - typedef std::size_t size_type; - typedef std::ptrdiff_t difference_type; - typedef typename allocator_type::pointer pointer; - typedef typename allocator_type::const_pointer const_pointer; - typedef typename - boost::reverse_iterator reverse_iterator; - typedef typename - boost::reverse_iterator const_reverse_iterator; - typedef TagList tag_list; - -protected: - typedef typename super::final_node_type final_node_type; - typedef tuples::cons< - ctor_args, - typename super::ctor_args_list> ctor_args_list; - typedef typename mpl::push_front< - typename super::index_type_list, - ordered_index< - KeyFromValue,Compare, - SuperMeta,TagList,Category,AugmentPolicy - > >::type index_type_list; - typedef typename mpl::push_front< - typename super::iterator_type_list, - iterator>::type iterator_type_list; - typedef typename mpl::push_front< - typename super::const_iterator_type_list, - const_iterator>::type const_iterator_type_list; - typedef typename super::copy_map_type copy_map_type; - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - typedef typename super::index_saver_type index_saver_type; - typedef typename super::index_loader_type index_loader_type; -#endif - -protected: -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_container< - ordered_index_impl> safe_super; -#endif - - typedef typename call_traits< - value_type>::param_type value_param_type; - typedef typename call_traits< - key_type>::param_type key_param_type; - - /* Needed to avoid commas in BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL - * expansion. - */ - - typedef std::pair emplace_return_type; - -public: - - /* construct/copy/destroy - * Default and copy ctors are in the protected section as indices are - * not supposed to be created on their own. No range ctor either. - * Assignment operators defined at ordered_index rather than here. - */ - - allocator_type get_allocator()const BOOST_NOEXCEPT - { - return this->final().get_allocator(); - } - - /* iterators */ - - iterator - begin()BOOST_NOEXCEPT{return make_iterator(leftmost());} - const_iterator - begin()const BOOST_NOEXCEPT{return make_iterator(leftmost());} - iterator - end()BOOST_NOEXCEPT{return make_iterator(header());} - const_iterator - end()const BOOST_NOEXCEPT{return make_iterator(header());} - reverse_iterator - rbegin()BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} - const_reverse_iterator - rbegin()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} - reverse_iterator - rend()BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} - const_reverse_iterator - rend()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} - const_iterator - cbegin()const BOOST_NOEXCEPT{return begin();} - const_iterator - cend()const BOOST_NOEXCEPT{return end();} - const_reverse_iterator - crbegin()const BOOST_NOEXCEPT{return rbegin();} - const_reverse_iterator - crend()const BOOST_NOEXCEPT{return rend();} - - iterator iterator_to(const value_type& x) - { - return make_iterator(node_from_value(&x)); - } - - const_iterator iterator_to(const value_type& x)const - { - return make_iterator(node_from_value(&x)); - } - - /* capacity */ - - bool empty()const BOOST_NOEXCEPT{return this->final_empty_();} - size_type size()const BOOST_NOEXCEPT{return this->final_size_();} - size_type max_size()const BOOST_NOEXCEPT{return this->final_max_size_();} - - /* modifiers */ - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( - emplace_return_type,emplace,emplace_impl) - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( - iterator,emplace_hint,emplace_hint_impl,iterator,position) - - std::pair insert(const value_type& x) - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_(x); - return std::pair(make_iterator(p.first),p.second); - } - - std::pair insert(BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_rv_(x); - return std::pair(make_iterator(p.first),p.second); - } - - iterator insert(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_( - x,static_cast(position.get_node())); - return make_iterator(p.first); - } - - iterator insert(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_rv_( - x,static_cast(position.get_node())); - return make_iterator(p.first); - } - - template - void insert(InputIterator first,InputIterator last) - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - node_type* hint=header(); /* end() */ - for(;first!=last;++first){ - hint=this->final_insert_ref_( - *first,static_cast(hint)).first; - node_type::increment(hint); - } - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - void insert(std::initializer_list list) - { - insert(list.begin(),list.end()); - } -#endif - - iterator erase(iterator position) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - this->final_erase_(static_cast(position++.get_node())); - return position; - } - - size_type erase(key_param_type x) - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pair p=equal_range(x); - size_type s=0; - while(p.first!=p.second){ - p.first=erase(p.first); - ++s; - } - return s; - } - - iterator erase(iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - while(first!=last){ - first=erase(first); - } - return first; - } - - bool replace(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - return this->final_replace_( - x,static_cast(position.get_node())); - } - - bool replace(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - return this->final_replace_rv_( - x,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod,Rollback back_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,back_,static_cast(position.get_node())); - } - - template - bool modify_key(iterator position,Modifier mod) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - return modify( - position,modify_key_adaptor(mod,key)); - } - - template - bool modify_key(iterator position,Modifier mod,Rollback back_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - return modify( - position, - modify_key_adaptor(mod,key), - modify_key_adaptor(back_,key)); - } - - void swap( - ordered_index< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x) - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF(x); - this->final_swap_(x.final()); - } - - void clear()BOOST_NOEXCEPT - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - this->final_clear_(); - } - - /* observers */ - - key_from_value key_extractor()const{return key;} - key_compare key_comp()const{return comp_;} - value_compare value_comp()const{return value_compare(key,comp_);} - - /* set operations */ - - /* Internally, these ops rely on const_iterator being the same - * type as iterator. - */ - - template - iterator find(const CompatibleKey& x)const - { - return make_iterator(ordered_index_find(root(),header(),key,x,comp_)); - } - - template - iterator find( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return make_iterator(ordered_index_find(root(),header(),key,x,comp)); - } - - template - size_type count(const CompatibleKey& x)const - { - return count(x,comp_); - } - - template - size_type count(const CompatibleKey& x,const CompatibleCompare& comp)const - { - std::pair p=equal_range(x,comp); - size_type n=std::distance(p.first,p.second); - return n; - } - - template - iterator lower_bound(const CompatibleKey& x)const - { - return make_iterator( - ordered_index_lower_bound(root(),header(),key,x,comp_)); - } - - template - iterator lower_bound( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return make_iterator( - ordered_index_lower_bound(root(),header(),key,x,comp)); - } - - template - iterator upper_bound(const CompatibleKey& x)const - { - return make_iterator( - ordered_index_upper_bound(root(),header(),key,x,comp_)); - } - - template - iterator upper_bound( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return make_iterator( - ordered_index_upper_bound(root(),header(),key,x,comp)); - } - - template - std::pair equal_range( - const CompatibleKey& x)const - { - std::pair p= - ordered_index_equal_range(root(),header(),key,x,comp_); - return std::pair( - make_iterator(p.first),make_iterator(p.second)); - } - - template - std::pair equal_range( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - std::pair p= - ordered_index_equal_range(root(),header(),key,x,comp); - return std::pair( - make_iterator(p.first),make_iterator(p.second)); - } - - /* range */ - - template - std::pair - range(LowerBounder lower,UpperBounder upper)const - { - typedef typename mpl::if_< - is_same, - BOOST_DEDUCED_TYPENAME mpl::if_< - is_same, - both_unbounded_tag, - lower_unbounded_tag - >::type, - BOOST_DEDUCED_TYPENAME mpl::if_< - is_same, - upper_unbounded_tag, - none_unbounded_tag - >::type - >::type dispatch; - - return range(lower,upper,dispatch()); - } - -BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: - ordered_index_impl(const ctor_args_list& args_list,const allocator_type& al): - super(args_list.get_tail(),al), - key(tuples::get<0>(args_list.get_head())), - comp_(tuples::get<1>(args_list.get_head())) - { - empty_initialize(); - } - - ordered_index_impl( - const ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x): - super(x), - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super(), -#endif - - key(x.key), - comp_(x.comp_) - { - /* Copy ctor just takes the key and compare objects from x. The rest is - * done in a subsequent call to copy_(). - */ - } - - ordered_index_impl( - const ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x, - do_not_copy_elements_tag): - super(x,do_not_copy_elements_tag()), - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super(), -#endif - - key(x.key), - comp_(x.comp_) - { - empty_initialize(); - } - - ~ordered_index_impl() - { - /* the container is guaranteed to be empty by now */ - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - iterator make_iterator(node_type* node){return iterator(node,this);} - const_iterator make_iterator(node_type* node)const - {return const_iterator(node,const_cast(this));} -#else - iterator make_iterator(node_type* node){return iterator(node);} - const_iterator make_iterator(node_type* node)const - {return const_iterator(node);} -#endif - - void copy_( - const ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x, - const copy_map_type& map) - { - if(!x.root()){ - empty_initialize(); - } - else{ - header()->color()=x.header()->color(); - AugmentPolicy::copy(x.header()->impl(),header()->impl()); - - node_type* root_cpy=map.find(static_cast(x.root())); - header()->parent()=root_cpy->impl(); - - node_type* leftmost_cpy=map.find( - static_cast(x.leftmost())); - header()->left()=leftmost_cpy->impl(); - - node_type* rightmost_cpy=map.find( - static_cast(x.rightmost())); - header()->right()=rightmost_cpy->impl(); - - typedef typename copy_map_type::const_iterator copy_map_iterator; - for(copy_map_iterator it=map.begin(),it_end=map.end();it!=it_end;++it){ - node_type* org=it->first; - node_type* cpy=it->second; - - cpy->color()=org->color(); - AugmentPolicy::copy(org->impl(),cpy->impl()); - - node_impl_pointer parent_org=org->parent(); - if(parent_org==node_impl_pointer(0))cpy->parent()=node_impl_pointer(0); - else{ - node_type* parent_cpy=map.find( - static_cast(node_type::from_impl(parent_org))); - cpy->parent()=parent_cpy->impl(); - if(parent_org->left()==org->impl()){ - parent_cpy->left()=cpy->impl(); - } - else if(parent_org->right()==org->impl()){ - /* header() does not satisfy this nor the previous check */ - parent_cpy->right()=cpy->impl(); - } - } - - if(org->left()==node_impl_pointer(0)) - cpy->left()=node_impl_pointer(0); - if(org->right()==node_impl_pointer(0)) - cpy->right()=node_impl_pointer(0); - } - } - - super::copy_(x,map); - } - - template - final_node_type* insert_( - value_param_type v,final_node_type*& x,Variant variant) - { - link_info inf; - if(!link_point(key(v),inf,Category())){ - return static_cast(node_type::from_impl(inf.pos)); - } - - final_node_type* res=super::insert_(v,x,variant); - if(res==x){ - node_impl_type::link( - static_cast(x)->impl(),inf.side,inf.pos,header()->impl()); - } - return res; - } - - template - final_node_type* insert_( - value_param_type v,node_type* position,final_node_type*& x,Variant variant) - { - link_info inf; - if(!hinted_link_point(key(v),position,inf,Category())){ - return static_cast(node_type::from_impl(inf.pos)); - } - - final_node_type* res=super::insert_(v,position,x,variant); - if(res==x){ - node_impl_type::link( - static_cast(x)->impl(),inf.side,inf.pos,header()->impl()); - } - return res; - } - - void erase_(node_type* x) - { - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - } - - void delete_all_nodes_() - { - delete_all_nodes(root()); - } - - void clear_() - { - super::clear_(); - empty_initialize(); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::detach_dereferenceable_iterators(); -#endif - } - - void swap_( - ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x) - { - std::swap(key,x.key); - std::swap(comp_,x.comp_); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_(x); - } - - void swap_elements_( - ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x) - { -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_elements_(x); - } - - template - bool replace_(value_param_type v,node_type* x,Variant variant) - { - if(in_place(v,x,Category())){ - return super::replace_(v,x,variant); - } - - node_type* next=x; - node_type::increment(next); - - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - - BOOST_TRY{ - link_info inf; - if(link_point(key(v),inf,Category())&&super::replace_(v,x,variant)){ - node_impl_type::link(x->impl(),inf.side,inf.pos,header()->impl()); - return true; - } - node_impl_type::restore(x->impl(),next->impl(),header()->impl()); - return false; - } - BOOST_CATCH(...){ - node_impl_type::restore(x->impl(),next->impl(),header()->impl()); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - bool modify_(node_type* x) - { - bool b; - BOOST_TRY{ - b=in_place(x->value(),x,Category()); - } - BOOST_CATCH(...){ - erase_(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - if(!b){ - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - BOOST_TRY{ - link_info inf; - if(!link_point(key(x->value()),inf,Category())){ - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - return false; - } - node_impl_type::link(x->impl(),inf.side,inf.pos,header()->impl()); - } - BOOST_CATCH(...){ - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - BOOST_TRY{ - if(!super::modify_(x)){ - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - return false; - } - else return true; - } - BOOST_CATCH(...){ - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - bool modify_rollback_(node_type* x) - { - if(in_place(x->value(),x,Category())){ - return super::modify_rollback_(x); - } - - node_type* next=x; - node_type::increment(next); - - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - - BOOST_TRY{ - link_info inf; - if(link_point(key(x->value()),inf,Category())&& - super::modify_rollback_(x)){ - node_impl_type::link(x->impl(),inf.side,inf.pos,header()->impl()); - return true; - } - node_impl_type::restore(x->impl(),next->impl(),header()->impl()); - return false; - } - BOOST_CATCH(...){ - node_impl_type::restore(x->impl(),next->impl(),header()->impl()); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* serialization */ - - template - void save_( - Archive& ar,const unsigned int version,const index_saver_type& sm)const - { - save_(ar,version,sm,Category()); - } - - template - void load_(Archive& ar,const unsigned int version,const index_loader_type& lm) - { - load_(ar,version,lm,Category()); - } -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - bool invariant_()const - { - if(size()==0||begin()==end()){ - if(size()!=0||begin()!=end()|| - header()->left()!=header()->impl()|| - header()->right()!=header()->impl())return false; - } - else{ - if((size_type)std::distance(begin(),end())!=size())return false; - - std::size_t len=node_impl_type::black_count( - leftmost()->impl(),root()->impl()); - for(const_iterator it=begin(),it_end=end();it!=it_end;++it){ - node_type* x=it.get_node(); - node_type* left_x=node_type::from_impl(x->left()); - node_type* right_x=node_type::from_impl(x->right()); - - if(x->color()==red){ - if((left_x&&left_x->color()==red)|| - (right_x&&right_x->color()==red))return false; - } - if(left_x&&comp_(key(x->value()),key(left_x->value())))return false; - if(right_x&&comp_(key(right_x->value()),key(x->value())))return false; - if(!left_x&&!right_x&& - node_impl_type::black_count(x->impl(),root()->impl())!=len) - return false; - if(!AugmentPolicy::invariant(x->impl()))return false; - } - - if(leftmost()->impl()!=node_impl_type::minimum(root()->impl())) - return false; - if(rightmost()->impl()!=node_impl_type::maximum(root()->impl())) - return false; - } - - return super::invariant_(); - } - - - /* This forwarding function eases things for the boost::mem_fn construct - * in BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT. Actually, - * final_check_invariant is already an inherited member function of - * ordered_index_impl. - */ - void check_invariant_()const{this->final_check_invariant_();} -#endif - -protected: /* for the benefit of AugmentPolicy::augmented_interface */ - node_type* header()const{return this->final_header();} - node_type* root()const{return node_type::from_impl(header()->parent());} - node_type* leftmost()const{return node_type::from_impl(header()->left());} - node_type* rightmost()const{return node_type::from_impl(header()->right());} - -private: - void empty_initialize() - { - header()->color()=red; - /* used to distinguish header() from root, in iterator.operator++ */ - - header()->parent()=node_impl_pointer(0); - header()->left()=header()->impl(); - header()->right()=header()->impl(); - } - - struct link_info - { - /* coverity[uninit_ctor]: suppress warning */ - link_info():side(to_left){} - - ordered_index_side side; - node_impl_pointer pos; - }; - - bool link_point(key_param_type k,link_info& inf,ordered_unique_tag) - { - node_type* y=header(); - node_type* x=root(); - bool c=true; - while(x){ - y=x; - c=comp_(k,key(x->value())); - x=node_type::from_impl(c?x->left():x->right()); - } - node_type* yy=y; - if(c){ - if(yy==leftmost()){ - inf.side=to_left; - inf.pos=y->impl(); - return true; - } - else node_type::decrement(yy); - } - - if(comp_(key(yy->value()),k)){ - inf.side=c?to_left:to_right; - inf.pos=y->impl(); - return true; - } - else{ - inf.pos=yy->impl(); - return false; - } - } - - bool link_point(key_param_type k,link_info& inf,ordered_non_unique_tag) - { - node_type* y=header(); - node_type* x=root(); - bool c=true; - while (x){ - y=x; - c=comp_(k,key(x->value())); - x=node_type::from_impl(c?x->left():x->right()); - } - inf.side=c?to_left:to_right; - inf.pos=y->impl(); - return true; - } - - bool lower_link_point(key_param_type k,link_info& inf,ordered_non_unique_tag) - { - node_type* y=header(); - node_type* x=root(); - bool c=false; - while (x){ - y=x; - c=comp_(key(x->value()),k); - x=node_type::from_impl(c?x->right():x->left()); - } - inf.side=c?to_right:to_left; - inf.pos=y->impl(); - return true; - } - - bool hinted_link_point( - key_param_type k,node_type* position,link_info& inf,ordered_unique_tag) - { - if(position->impl()==header()->left()){ - if(size()>0&&comp_(k,key(position->value()))){ - inf.side=to_left; - inf.pos=position->impl(); - return true; - } - else return link_point(k,inf,ordered_unique_tag()); - } - else if(position==header()){ - if(comp_(key(rightmost()->value()),k)){ - inf.side=to_right; - inf.pos=rightmost()->impl(); - return true; - } - else return link_point(k,inf,ordered_unique_tag()); - } - else{ - node_type* before=position; - node_type::decrement(before); - if(comp_(key(before->value()),k)&&comp_(k,key(position->value()))){ - if(before->right()==node_impl_pointer(0)){ - inf.side=to_right; - inf.pos=before->impl(); - return true; - } - else{ - inf.side=to_left; - inf.pos=position->impl(); - return true; - } - } - else return link_point(k,inf,ordered_unique_tag()); - } - } - - bool hinted_link_point( - key_param_type k,node_type* position,link_info& inf,ordered_non_unique_tag) - { - if(position->impl()==header()->left()){ - if(size()>0&&!comp_(key(position->value()),k)){ - inf.side=to_left; - inf.pos=position->impl(); - return true; - } - else return lower_link_point(k,inf,ordered_non_unique_tag()); - } - else if(position==header()){ - if(!comp_(k,key(rightmost()->value()))){ - inf.side=to_right; - inf.pos=rightmost()->impl(); - return true; - } - else return link_point(k,inf,ordered_non_unique_tag()); - } - else{ - node_type* before=position; - node_type::decrement(before); - if(!comp_(k,key(before->value()))){ - if(!comp_(key(position->value()),k)){ - if(before->right()==node_impl_pointer(0)){ - inf.side=to_right; - inf.pos=before->impl(); - return true; - } - else{ - inf.side=to_left; - inf.pos=position->impl(); - return true; - } - } - else return lower_link_point(k,inf,ordered_non_unique_tag()); - } - else return link_point(k,inf,ordered_non_unique_tag()); - } - } - - void delete_all_nodes(node_type* x) - { - if(!x)return; - - delete_all_nodes(node_type::from_impl(x->left())); - delete_all_nodes(node_type::from_impl(x->right())); - this->final_delete_node_(static_cast(x)); - } - - bool in_place(value_param_type v,node_type* x,ordered_unique_tag) - { - node_type* y; - if(x!=leftmost()){ - y=x; - node_type::decrement(y); - if(!comp_(key(y->value()),key(v)))return false; - } - - y=x; - node_type::increment(y); - return y==header()||comp_(key(v),key(y->value())); - } - - bool in_place(value_param_type v,node_type* x,ordered_non_unique_tag) - { - node_type* y; - if(x!=leftmost()){ - y=x; - node_type::decrement(y); - if(comp_(key(v),key(y->value())))return false; - } - - y=x; - node_type::increment(y); - return y==header()||!comp_(key(y->value()),key(v)); - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - void detach_iterators(node_type* x) - { - iterator it=make_iterator(x); - safe_mode::detach_equivalent_iterators(it); - } -#endif - - template - std::pair emplace_impl(BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pairp= - this->final_emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - return std::pair(make_iterator(p.first),p.second); - } - - template - iterator emplace_hint_impl( - iterator position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT; - std::pairp= - this->final_emplace_hint_( - static_cast(position.get_node()), - BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - return make_iterator(p.first); - } - - template - std::pair - range(LowerBounder lower,UpperBounder upper,none_unbounded_tag)const - { - node_type* y=header(); - node_type* z=root(); - - while(z){ - if(!lower(key(z->value()))){ - z=node_type::from_impl(z->right()); - } - else if(!upper(key(z->value()))){ - y=z; - z=node_type::from_impl(z->left()); - } - else{ - return std::pair( - make_iterator( - lower_range(node_type::from_impl(z->left()),z,lower)), - make_iterator( - upper_range(node_type::from_impl(z->right()),y,upper))); - } - } - - return std::pair(make_iterator(y),make_iterator(y)); - } - - template - std::pair - range(LowerBounder,UpperBounder upper,lower_unbounded_tag)const - { - return std::pair( - begin(), - make_iterator(upper_range(root(),header(),upper))); - } - - template - std::pair - range(LowerBounder lower,UpperBounder,upper_unbounded_tag)const - { - return std::pair( - make_iterator(lower_range(root(),header(),lower)), - end()); - } - - template - std::pair - range(LowerBounder,UpperBounder,both_unbounded_tag)const - { - return std::pair(begin(),end()); - } - - template - node_type * lower_range(node_type* top,node_type* y,LowerBounder lower)const - { - while(top){ - if(lower(key(top->value()))){ - y=top; - top=node_type::from_impl(top->left()); - } - else top=node_type::from_impl(top->right()); - } - - return y; - } - - template - node_type * upper_range(node_type* top,node_type* y,UpperBounder upper)const - { - while(top){ - if(!upper(key(top->value()))){ - y=top; - top=node_type::from_impl(top->left()); - } - else top=node_type::from_impl(top->right()); - } - - return y; - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - template - void save_( - Archive& ar,const unsigned int version,const index_saver_type& sm, - ordered_unique_tag)const - { - super::save_(ar,version,sm); - } - - template - void load_( - Archive& ar,const unsigned int version,const index_loader_type& lm, - ordered_unique_tag) - { - super::load_(ar,version,lm); - } - - template - void save_( - Archive& ar,const unsigned int version,const index_saver_type& sm, - ordered_non_unique_tag)const - { - typedef duplicates_iterator dup_iterator; - - sm.save( - dup_iterator(begin().get_node(),end().get_node(),value_comp()), - dup_iterator(end().get_node(),value_comp()), - ar,version); - super::save_(ar,version,sm); - } - - template - void load_( - Archive& ar,const unsigned int version,const index_loader_type& lm, - ordered_non_unique_tag) - { - lm.load( - ::boost::bind( - &ordered_index_impl::rearranger,this, - ::boost::arg<1>(),::boost::arg<2>()), - ar,version); - super::load_(ar,version,lm); - } - - void rearranger(node_type* position,node_type *x) - { - if(!position||comp_(key(position->value()),key(x->value()))){ - position=lower_bound(key(x->value())).get_node(); - } - else if(comp_(key(x->value()),key(position->value()))){ - /* inconsistent rearrangement */ - throw_exception( - archive::archive_exception( - archive::archive_exception::other_exception)); - } - else node_type::increment(position); - - if(position!=x){ - node_impl_type::rebalance_for_erase( - x->impl(),header()->parent(),header()->left(),header()->right()); - node_impl_type::restore( - x->impl(),position->impl(),header()->impl()); - } - } -#endif /* serialization */ - -protected: /* for the benefit of AugmentPolicy::augmented_interface */ - key_from_value key; - key_compare comp_; - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -#pragma parse_mfunc_templ reset -#endif -}; - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -class ordered_index: - public AugmentPolicy::template augmented_interface< - ordered_index_impl< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy - > - >::type -{ - typedef typename AugmentPolicy::template - augmented_interface< - ordered_index_impl< - KeyFromValue,Compare, - SuperMeta,TagList,Category,AugmentPolicy - > - >::type super; -public: - typedef typename super::ctor_args_list ctor_args_list; - typedef typename super::allocator_type allocator_type; - typedef typename super::iterator iterator; - - /* construct/copy/destroy - * Default and copy ctors are in the protected section as indices are - * not supposed to be created on their own. No range ctor either. - */ - - ordered_index& operator=(const ordered_index& x) - { - this->final()=x.final(); - return *this; - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - ordered_index& operator=( - std::initializer_list list) - { - this->final()=list; - return *this; - } -#endif - -protected: - ordered_index( - const ctor_args_list& args_list,const allocator_type& al): - super(args_list,al){} - - ordered_index(const ordered_index& x):super(x){}; - - ordered_index(const ordered_index& x,do_not_copy_elements_tag): - super(x,do_not_copy_elements_tag()){}; -}; - -/* comparison */ - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator==( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) -{ - return x.size()==y.size()&&std::equal(x.begin(),x.end(),y.begin()); -} - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator<( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) -{ - return std::lexicographical_compare(x.begin(),x.end(),y.begin(),y.end()); -} - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator!=( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) -{ - return !(x==y); -} - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator>( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) -{ - return y -bool operator>=( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) -{ - return !(x -bool operator<=( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y) -{ - return !(x>y); -} - -/* specialized algorithms */ - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -void swap( - ordered_index< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x, - ordered_index< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -/* Boost.Foreach compatibility */ - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -inline boost::mpl::true_* boost_foreach_is_noncopyable( - boost::multi_index::detail::ordered_index< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>*&, - boost_foreach_argument_dependent_lookup_hack) -{ - return 0; -} - -#undef BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT -#undef BOOST_MULTI_INDEX_ORD_INDEX_CHECK_INVARIANT_OF - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl_fwd.hpp deleted file mode 100644 index 6590ef05fdd..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_impl_fwd.hpp +++ /dev/null @@ -1,128 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_IMPL_FWD_HPP -#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_IMPL_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -class ordered_index; - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator==( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator<( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator!=( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator>( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator>=( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); - -template< - typename KeyFromValue1,typename Compare1, - typename SuperMeta1,typename TagList1,typename Category1, - typename AugmentPolicy1, - typename KeyFromValue2,typename Compare2, - typename SuperMeta2,typename TagList2,typename Category2, - typename AugmentPolicy2 -> -bool operator<=( - const ordered_index< - KeyFromValue1,Compare1,SuperMeta1,TagList1,Category1,AugmentPolicy1>& x, - const ordered_index< - KeyFromValue2,Compare2,SuperMeta2,TagList2,Category2,AugmentPolicy2>& y); - -template< - typename KeyFromValue,typename Compare, - typename SuperMeta,typename TagList,typename Category,typename AugmentPolicy -> -void swap( - ordered_index< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& x, - ordered_index< - KeyFromValue,Compare,SuperMeta,TagList,Category,AugmentPolicy>& y); - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_node.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_node.hpp deleted file mode 100644 index e7af0377fb9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_node.hpp +++ /dev/null @@ -1,658 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - * - * The internal implementation of red-black trees is based on that of SGI STL - * stl_tree.h file: - * - * Copyright (c) 1996,1997 - * Silicon Graphics Computer Systems, Inc. - * - * Permission to use, copy, modify, distribute and sell this software - * and its documentation for any purpose is hereby granted without fee, - * provided that the above copyright notice appear in all copies and - * that both that copyright notice and this permission notice appear - * in supporting documentation. Silicon Graphics makes no - * representations about the suitability of this software for any - * purpose. It is provided "as is" without express or implied warranty. - * - * - * Copyright (c) 1994 - * Hewlett-Packard Company - * - * Permission to use, copy, modify, distribute and sell this software - * and its documentation for any purpose is hereby granted without fee, - * provided that the above copyright notice appear in all copies and - * that both that copyright notice and this permission notice appear - * in supporting documentation. Hewlett-Packard Company makes no - * representations about the suitability of this software for any - * purpose. It is provided "as is" without express or implied warranty. - * - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_NODE_HPP -#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_NODE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_COMPRESSED_ORDERED_INDEX_NODES) -#include -#include -#include -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* definition of red-black nodes for ordered_index */ - -enum ordered_index_color{red=false,black=true}; -enum ordered_index_side{to_left=false,to_right=true}; - -template -struct ordered_index_node_impl; /* fwd decl. */ - -template -struct ordered_index_node_std_base -{ - typedef typename - boost::detail::allocator::rebind_to< - Allocator, - ordered_index_node_impl - >::type::pointer pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator, - ordered_index_node_impl - >::type::const_pointer const_pointer; - typedef ordered_index_color& color_ref; - typedef pointer& parent_ref; - - ordered_index_color& color(){return color_;} - ordered_index_color color()const{return color_;} - pointer& parent(){return parent_;} - pointer parent()const{return parent_;} - pointer& left(){return left_;} - pointer left()const{return left_;} - pointer& right(){return right_;} - pointer right()const{return right_;} - -private: - ordered_index_color color_; - pointer parent_; - pointer left_; - pointer right_; -}; - -#if !defined(BOOST_MULTI_INDEX_DISABLE_COMPRESSED_ORDERED_INDEX_NODES) -/* If ordered_index_node_impl has even alignment, we can use the least - * significant bit of one of the ordered_index_node_impl pointers to - * store color information. This typically reduces the size of - * ordered_index_node_impl by 25%. - */ - -#if defined(BOOST_MSVC) -/* This code casts pointers to an integer type that has been computed - * to be large enough to hold the pointer, however the metaprogramming - * logic is not always spotted by the VC++ code analyser that issues a - * long list of warnings. - */ - -#pragma warning(push) -#pragma warning(disable:4312 4311) -#endif - -template -struct ordered_index_node_compressed_base -{ - typedef ordered_index_node_impl< - AugmentPolicy,Allocator>* pointer; - typedef const ordered_index_node_impl< - AugmentPolicy,Allocator>* const_pointer; - - struct color_ref - { - color_ref(uintptr_type* r_):r(r_){} - - operator ordered_index_color()const - { - return ordered_index_color(*r&uintptr_type(1)); - } - - color_ref& operator=(ordered_index_color c) - { - *r&=~uintptr_type(1); - *r|=uintptr_type(c); - return *this; - } - - color_ref& operator=(const color_ref& x) - { - return operator=(x.operator ordered_index_color()); - } - - private: - uintptr_type* r; - }; - - struct parent_ref - { - parent_ref(uintptr_type* r_):r(r_){} - - operator pointer()const - { - return (pointer)(void*)(*r&~uintptr_type(1)); - } - - parent_ref& operator=(pointer p) - { - *r=((uintptr_type)(void*)p)|(*r&uintptr_type(1)); - return *this; - } - - parent_ref& operator=(const parent_ref& x) - { - return operator=(x.operator pointer()); - } - - pointer operator->()const - { - return operator pointer(); - } - - private: - uintptr_type* r; - }; - - color_ref color(){return color_ref(&parentcolor_);} - ordered_index_color color()const - { - return ordered_index_color(parentcolor_&uintptr_type(1)); - } - - parent_ref parent(){return parent_ref(&parentcolor_);} - pointer parent()const - { - return (pointer)(void*)(parentcolor_&~uintptr_type(1)); - } - - pointer& left(){return left_;} - pointer left()const{return left_;} - pointer& right(){return right_;} - pointer right()const{return right_;} - -private: - uintptr_type parentcolor_; - pointer left_; - pointer right_; -}; -#if defined(BOOST_MSVC) -#pragma warning(pop) -#endif -#endif - -template -struct ordered_index_node_impl_base: - -#if !defined(BOOST_MULTI_INDEX_DISABLE_COMPRESSED_ORDERED_INDEX_NODES) - AugmentPolicy::template augmented_node< - typename mpl::if_c< - !(has_uintptr_type::value)|| - (alignment_of< - ordered_index_node_compressed_base - >::value%2)|| - !(is_same< - typename boost::detail::allocator::rebind_to< - Allocator, - ordered_index_node_impl - >::type::pointer, - ordered_index_node_impl*>::value), - ordered_index_node_std_base, - ordered_index_node_compressed_base - >::type - >::type -#else - AugmentPolicy::template augmented_node< - ordered_index_node_std_base - >::type -#endif - -{}; - -template -struct ordered_index_node_impl: - ordered_index_node_impl_base -{ -private: - typedef ordered_index_node_impl_base super; - -public: - typedef typename super::color_ref color_ref; - typedef typename super::parent_ref parent_ref; - typedef typename super::pointer pointer; - typedef typename super::const_pointer const_pointer; - - /* interoperability with bidir_node_iterator */ - - static void increment(pointer& x) - { - if(x->right()!=pointer(0)){ - x=x->right(); - while(x->left()!=pointer(0))x=x->left(); - } - else{ - pointer y=x->parent(); - while(x==y->right()){ - x=y; - y=y->parent(); - } - if(x->right()!=y)x=y; - } - } - - static void decrement(pointer& x) - { - if(x->color()==red&&x->parent()->parent()==x){ - x=x->right(); - } - else if(x->left()!=pointer(0)){ - pointer y=x->left(); - while(y->right()!=pointer(0))y=y->right(); - x=y; - }else{ - pointer y=x->parent(); - while(x==y->left()){ - x=y; - y=y->parent(); - } - x=y; - } - } - - /* algorithmic stuff */ - - static void rotate_left(pointer x,parent_ref root) - { - pointer y=x->right(); - x->right()=y->left(); - if(y->left()!=pointer(0))y->left()->parent()=x; - y->parent()=x->parent(); - - if(x==root) root=y; - else if(x==x->parent()->left())x->parent()->left()=y; - else x->parent()->right()=y; - y->left()=x; - x->parent()=y; - AugmentPolicy::rotate_left(x,y); - } - - static pointer minimum(pointer x) - { - while(x->left()!=pointer(0))x=x->left(); - return x; - } - - static pointer maximum(pointer x) - { - while(x->right()!=pointer(0))x=x->right(); - return x; - } - - static void rotate_right(pointer x,parent_ref root) - { - pointer y=x->left(); - x->left()=y->right(); - if(y->right()!=pointer(0))y->right()->parent()=x; - y->parent()=x->parent(); - - if(x==root) root=y; - else if(x==x->parent()->right())x->parent()->right()=y; - else x->parent()->left()=y; - y->right()=x; - x->parent()=y; - AugmentPolicy::rotate_right(x,y); - } - - static void rebalance(pointer x,parent_ref root) - { - x->color()=red; - while(x!=root&&x->parent()->color()==red){ - if(x->parent()==x->parent()->parent()->left()){ - pointer y=x->parent()->parent()->right(); - if(y!=pointer(0)&&y->color()==red){ - x->parent()->color()=black; - y->color()=black; - x->parent()->parent()->color()=red; - x=x->parent()->parent(); - } - else{ - if(x==x->parent()->right()){ - x=x->parent(); - rotate_left(x,root); - } - x->parent()->color()=black; - x->parent()->parent()->color()=red; - rotate_right(x->parent()->parent(),root); - } - } - else{ - pointer y=x->parent()->parent()->left(); - if(y!=pointer(0)&&y->color()==red){ - x->parent()->color()=black; - y->color()=black; - x->parent()->parent()->color()=red; - x=x->parent()->parent(); - } - else{ - if(x==x->parent()->left()){ - x=x->parent(); - rotate_right(x,root); - } - x->parent()->color()=black; - x->parent()->parent()->color()=red; - rotate_left(x->parent()->parent(),root); - } - } - } - root->color()=black; - } - - static void link( - pointer x,ordered_index_side side,pointer position,pointer header) - { - if(side==to_left){ - position->left()=x; /* also makes leftmost=x when parent==header */ - if(position==header){ - header->parent()=x; - header->right()=x; - } - else if(position==header->left()){ - header->left()=x; /* maintain leftmost pointing to min node */ - } - } - else{ - position->right()=x; - if(position==header->right()){ - header->right()=x; /* maintain rightmost pointing to max node */ - } - } - x->parent()=position; - x->left()=pointer(0); - x->right()=pointer(0); - AugmentPolicy::add(x,pointer(header->parent())); - ordered_index_node_impl::rebalance(x,header->parent()); - } - - static pointer rebalance_for_erase( - pointer z,parent_ref root,pointer& leftmost,pointer& rightmost) - { - pointer y=z; - pointer x=pointer(0); - pointer x_parent=pointer(0); - if(y->left()==pointer(0)){ /* z has at most one non-null child. y==z. */ - x=y->right(); /* x might be null */ - } - else{ - if(y->right()==pointer(0)){ /* z has exactly one non-null child. y==z. */ - x=y->left(); /* x is not null */ - } - else{ /* z has two non-null children. Set y to */ - y=y->right(); /* z's successor. x might be null. */ - while(y->left()!=pointer(0))y=y->left(); - x=y->right(); - } - } - AugmentPolicy::remove(y,pointer(root)); - if(y!=z){ - AugmentPolicy::copy(z,y); - z->left()->parent()=y; /* relink y in place of z. y is z's successor */ - y->left()=z->left(); - if(y!=z->right()){ - x_parent=y->parent(); - if(x!=pointer(0))x->parent()=y->parent(); - y->parent()->left()=x; /* y must be a child of left */ - y->right()=z->right(); - z->right()->parent()=y; - } - else{ - x_parent=y; - } - - if(root==z) root=y; - else if(z->parent()->left()==z)z->parent()->left()=y; - else z->parent()->right()=y; - y->parent()=z->parent(); - ordered_index_color c=y->color(); - y->color()=z->color(); - z->color()=c; - y=z; /* y now points to node to be actually deleted */ - } - else{ /* y==z */ - x_parent=y->parent(); - if(x!=pointer(0))x->parent()=y->parent(); - if(root==z){ - root=x; - } - else{ - if(z->parent()->left()==z)z->parent()->left()=x; - else z->parent()->right()=x; - } - if(leftmost==z){ - if(z->right()==pointer(0)){ /* z->left() must be null also */ - leftmost=z->parent(); - } - else{ - leftmost=minimum(x); /* makes leftmost==header if z==root */ - } - } - if(rightmost==z){ - if(z->left()==pointer(0)){ /* z->right() must be null also */ - rightmost=z->parent(); - } - else{ /* x==z->left() */ - rightmost=maximum(x); /* makes rightmost==header if z==root */ - } - } - } - if(y->color()!=red){ - while(x!=root&&(x==pointer(0)|| x->color()==black)){ - if(x==x_parent->left()){ - pointer w=x_parent->right(); - if(w->color()==red){ - w->color()=black; - x_parent->color()=red; - rotate_left(x_parent,root); - w=x_parent->right(); - } - if((w->left()==pointer(0)||w->left()->color()==black) && - (w->right()==pointer(0)||w->right()->color()==black)){ - w->color()=red; - x=x_parent; - x_parent=x_parent->parent(); - } - else{ - if(w->right()==pointer(0 ) - || w->right()->color()==black){ - if(w->left()!=pointer(0)) w->left()->color()=black; - w->color()=red; - rotate_right(w,root); - w=x_parent->right(); - } - w->color()=x_parent->color(); - x_parent->color()=black; - if(w->right()!=pointer(0))w->right()->color()=black; - rotate_left(x_parent,root); - break; - } - } - else{ /* same as above,with right <-> left */ - pointer w=x_parent->left(); - if(w->color()==red){ - w->color()=black; - x_parent->color()=red; - rotate_right(x_parent,root); - w=x_parent->left(); - } - if((w->right()==pointer(0)||w->right()->color()==black) && - (w->left()==pointer(0)||w->left()->color()==black)){ - w->color()=red; - x=x_parent; - x_parent=x_parent->parent(); - } - else{ - if(w->left()==pointer(0)||w->left()->color()==black){ - if(w->right()!=pointer(0))w->right()->color()=black; - w->color()=red; - rotate_left(w,root); - w=x_parent->left(); - } - w->color()=x_parent->color(); - x_parent->color()=black; - if(w->left()!=pointer(0))w->left()->color()=black; - rotate_right(x_parent,root); - break; - } - } - } - if(x!=pointer(0))x->color()=black; - } - return y; - } - - static void restore(pointer x,pointer position,pointer header) - { - if(position->left()==pointer(0)||position->left()==header){ - link(x,to_left,position,header); - } - else{ - decrement(position); - link(x,to_right,position,header); - } - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - static std::size_t black_count(pointer node,pointer root) - { - if(node==pointer(0))return 0; - std::size_t sum=0; - for(;;){ - if(node->color()==black)++sum; - if(node==root)break; - node=node->parent(); - } - return sum; - } -#endif -}; - -template -struct ordered_index_node_trampoline: - ordered_index_node_impl< - AugmentPolicy, - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > -{ - typedef ordered_index_node_impl< - AugmentPolicy, - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > impl_type; -}; - -template -struct ordered_index_node: - Super,ordered_index_node_trampoline -{ -private: - typedef ordered_index_node_trampoline trampoline; - -public: - typedef typename trampoline::impl_type impl_type; - typedef typename trampoline::color_ref impl_color_ref; - typedef typename trampoline::parent_ref impl_parent_ref; - typedef typename trampoline::pointer impl_pointer; - typedef typename trampoline::const_pointer const_impl_pointer; - - impl_color_ref color(){return trampoline::color();} - ordered_index_color color()const{return trampoline::color();} - impl_parent_ref parent(){return trampoline::parent();} - impl_pointer parent()const{return trampoline::parent();} - impl_pointer& left(){return trampoline::left();} - impl_pointer left()const{return trampoline::left();} - impl_pointer& right(){return trampoline::right();} - impl_pointer right()const{return trampoline::right();} - - impl_pointer impl() - { - return static_cast( - static_cast(static_cast(this))); - } - - const_impl_pointer impl()const - { - return static_cast( - static_cast(static_cast(this))); - } - - static ordered_index_node* from_impl(impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - static const ordered_index_node* from_impl(const_impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - /* interoperability with bidir_node_iterator */ - - static void increment(ordered_index_node*& x) - { - impl_pointer xi=x->impl(); - trampoline::increment(xi); - x=from_impl(xi); - } - - static void decrement(ordered_index_node*& x) - { - impl_pointer xi=x->impl(); - trampoline::decrement(xi); - x=from_impl(xi); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_ops.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_ops.hpp deleted file mode 100644 index 84d5cacae19..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/ord_index_ops.hpp +++ /dev/null @@ -1,266 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - * - * The internal implementation of red-black trees is based on that of SGI STL - * stl_tree.h file: - * - * Copyright (c) 1996,1997 - * Silicon Graphics Computer Systems, Inc. - * - * Permission to use, copy, modify, distribute and sell this software - * and its documentation for any purpose is hereby granted without fee, - * provided that the above copyright notice appear in all copies and - * that both that copyright notice and this permission notice appear - * in supporting documentation. Silicon Graphics makes no - * representations about the suitability of this software for any - * purpose. It is provided "as is" without express or implied warranty. - * - * - * Copyright (c) 1994 - * Hewlett-Packard Company - * - * Permission to use, copy, modify, distribute and sell this software - * and its documentation for any purpose is hereby granted without fee, - * provided that the above copyright notice appear in all copies and - * that both that copyright notice and this permission notice appear - * in supporting documentation. Hewlett-Packard Company makes no - * representations about the suitability of this software for any - * purpose. It is provided "as is" without express or implied warranty. - * - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_OPS_HPP -#define BOOST_MULTI_INDEX_DETAIL_ORD_INDEX_OPS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Common code for index memfuns having templatized and - * non-templatized versions. - * Implementation note: When CompatibleKey is consistently promoted to - * KeyFromValue::result_type for comparison, the promotion is made once in - * advance to increase efficiency. - */ - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline Node* ordered_index_find( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ordered_index_find( - top,y,key,x,comp, - mpl::and_< - promotes_1st_arg, - promotes_2nd_arg >()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline Node* ordered_index_find( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ordered_index_find(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline Node* ordered_index_find( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - Node* y0=y; - - while (top){ - if(!comp(key(top->value()),x)){ - y=top; - top=Node::from_impl(top->left()); - } - else top=Node::from_impl(top->right()); - } - - return (y==y0||comp(x,key(y->value())))?y0:y; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline Node* ordered_index_lower_bound( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ordered_index_lower_bound( - top,y,key,x,comp, - promotes_2nd_arg()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline Node* ordered_index_lower_bound( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ordered_index_lower_bound(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline Node* ordered_index_lower_bound( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - while(top){ - if(!comp(key(top->value()),x)){ - y=top; - top=Node::from_impl(top->left()); - } - else top=Node::from_impl(top->right()); - } - - return y; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline Node* ordered_index_upper_bound( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ordered_index_upper_bound( - top,y,key,x,comp, - promotes_1st_arg()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline Node* ordered_index_upper_bound( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ordered_index_upper_bound(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline Node* ordered_index_upper_bound( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - while(top){ - if(comp(x,key(top->value()))){ - y=top; - top=Node::from_impl(top->left()); - } - else top=Node::from_impl(top->right()); - } - - return y; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::pair ordered_index_equal_range( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ordered_index_equal_range( - top,y,key,x,comp, - mpl::and_< - promotes_1st_arg, - promotes_2nd_arg >()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline std::pair ordered_index_equal_range( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ordered_index_equal_range(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::pair ordered_index_equal_range( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - while(top){ - if(comp(key(top->value()),x)){ - top=Node::from_impl(top->right()); - } - else if(comp(x,key(top->value()))){ - y=top; - top=Node::from_impl(top->left()); - } - else{ - return std::pair( - ordered_index_lower_bound( - Node::from_impl(top->left()),top,key,x,comp,mpl::false_()), - ordered_index_upper_bound( - Node::from_impl(top->right()),y,key,x,comp,mpl::false_())); - } - } - - return std::pair(y,y); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/promotes_arg.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/promotes_arg.hpp deleted file mode 100644 index 7a11b6e9fbe..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/promotes_arg.hpp +++ /dev/null @@ -1,83 +0,0 @@ -/* Copyright 2003-2017 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_PROMOTES_ARG_HPP -#define BOOST_MULTI_INDEX_DETAIL_PROMOTES_ARG_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -/* Metafunctions to check if f(arg1,arg2) promotes either arg1 to the type of - * arg2 or viceversa. By default, (i.e. if it cannot be determined), no - * promotion is assumed. - */ - -#if BOOST_WORKAROUND(BOOST_MSVC,<1400) - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -struct promotes_1st_arg:mpl::false_{}; - -template -struct promotes_2nd_arg:mpl::false_{}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#else - -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -struct promotes_1st_arg: - mpl::and_< - mpl::not_ >, - is_convertible, - is_transparent - > -{}; - -template -struct promotes_2nd_arg: - mpl::and_< - mpl::not_ >, - is_convertible, - is_transparent - > -{}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/raw_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/raw_ptr.hpp deleted file mode 100644 index c32007435c0..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/raw_ptr.hpp +++ /dev/null @@ -1,52 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RAW_PTR_HPP -#define BOOST_MULTI_INDEX_DETAIL_RAW_PTR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* gets the underlying pointer of a pointer-like value */ - -template -inline RawPointer raw_ptr(RawPointer const& p,mpl::true_) -{ - return p; -} - -template -inline RawPointer raw_ptr(Pointer const& p,mpl::false_) -{ - return p==Pointer(0)?0:&*p; -} - -template -inline RawPointer raw_ptr(Pointer const& p) -{ - return raw_ptr(p,is_same()); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/restore_wstrict_aliasing.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/restore_wstrict_aliasing.hpp deleted file mode 100644 index ee2c799d5a8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/restore_wstrict_aliasing.hpp +++ /dev/null @@ -1,11 +0,0 @@ -/* Copyright 2003-2016 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#define BOOST_MULTI_INDEX_DETAIL_RESTORE_WSTRICT_ALIASING -#include -#undef BOOST_MULTI_INDEX_DETAIL_RESTORE_WSTRICT_ALIASING diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_loader.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_loader.hpp deleted file mode 100644 index 4b00345a6d9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_loader.hpp +++ /dev/null @@ -1,173 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RND_INDEX_LOADER_HPP -#define BOOST_MULTI_INDEX_DETAIL_RND_INDEX_LOADER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* This class implements a serialization rearranger for random access - * indices. In order to achieve O(n) performance, the following strategy - * is followed: the nodes of the index are handled as if in a bidirectional - * list, where the next pointers are stored in the original - * random_access_index_ptr_array and the prev pointers are stored in - * an auxiliary array. Rearranging of nodes in such a bidirectional list - * is constant time. Once all the arrangements are performed (on destruction - * time) the list is traversed in reverse order and - * pointers are swapped and set accordingly so that they recover its - * original semantics ( *(node->up())==node ) while retaining the - * new order. - */ - -template -class random_access_index_loader_base:private noncopyable -{ -protected: - typedef random_access_index_node_impl< - typename boost::detail::allocator::rebind_to< - Allocator, - char - >::type - > node_impl_type; - typedef typename node_impl_type::pointer node_impl_pointer; - typedef random_access_index_ptr_array ptr_array; - - random_access_index_loader_base(const Allocator& al_,ptr_array& ptrs_): - al(al_), - ptrs(ptrs_), - header(*ptrs.end()), - prev_spc(al,0), - preprocessed(false) - {} - - ~random_access_index_loader_base() - { - if(preprocessed) - { - node_impl_pointer n=header; - next(n)=n; - - for(std::size_t i=ptrs.size();i--;){ - n=prev(n); - std::size_t d=position(n); - if(d!=i){ - node_impl_pointer m=prev(next_at(i)); - std::swap(m->up(),n->up()); - next_at(d)=next_at(i); - std::swap(prev_at(d),prev_at(i)); - } - next(n)=n; - } - } - } - - void rearrange(node_impl_pointer position_,node_impl_pointer x) - { - preprocess(); /* only incur this penalty if rearrange() is ever called */ - if(position_==node_impl_pointer(0))position_=header; - next(prev(x))=next(x); - prev(next(x))=prev(x); - prev(x)=position_; - next(x)=next(position_); - next(prev(x))=prev(next(x))=x; - } - -private: - void preprocess() - { - if(!preprocessed){ - /* get space for the auxiliary prev array */ - auto_space tmp(al,ptrs.size()+1); - prev_spc.swap(tmp); - - /* prev_spc elements point to the prev nodes */ - std::rotate_copy( - &*ptrs.begin(),&*ptrs.end(),&*ptrs.end()+1,&*prev_spc.data()); - - /* ptrs elements point to the next nodes */ - std::rotate(&*ptrs.begin(),&*ptrs.begin()+1,&*ptrs.end()+1); - - preprocessed=true; - } - } - - std::size_t position(node_impl_pointer x)const - { - return (std::size_t)(x->up()-ptrs.begin()); - } - - node_impl_pointer& next_at(std::size_t n)const - { - return *ptrs.at(n); - } - - node_impl_pointer& prev_at(std::size_t n)const - { - return *(prev_spc.data()+n); - } - - node_impl_pointer& next(node_impl_pointer x)const - { - return *(x->up()); - } - - node_impl_pointer& prev(node_impl_pointer x)const - { - return prev_at(position(x)); - } - - Allocator al; - ptr_array& ptrs; - node_impl_pointer header; - auto_space prev_spc; - bool preprocessed; -}; - -template -class random_access_index_loader: - private random_access_index_loader_base -{ - typedef random_access_index_loader_base super; - typedef typename super::node_impl_pointer node_impl_pointer; - typedef typename super::ptr_array ptr_array; - -public: - random_access_index_loader(const Allocator& al_,ptr_array& ptrs_): - super(al_,ptrs_) - {} - - void rearrange(Node* position_,Node *x) - { - super::rearrange( - position_?position_->impl():node_impl_pointer(0),x->impl()); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_node.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_node.hpp deleted file mode 100644 index ad61ea25dda..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_node.hpp +++ /dev/null @@ -1,273 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RND_INDEX_NODE_HPP -#define BOOST_MULTI_INDEX_DETAIL_RND_INDEX_NODE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -struct random_access_index_node_impl -{ - typedef typename - boost::detail::allocator::rebind_to< - Allocator,random_access_index_node_impl - >::type::pointer pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator,random_access_index_node_impl - >::type::const_pointer const_pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator,pointer - >::type::pointer ptr_pointer; - - ptr_pointer& up(){return up_;} - ptr_pointer up()const{return up_;} - - /* interoperability with rnd_node_iterator */ - - static void increment(pointer& x) - { - x=*(x->up()+1); - } - - static void decrement(pointer& x) - { - x=*(x->up()-1); - } - - static void advance(pointer& x,std::ptrdiff_t n) - { - x=*(x->up()+n); - } - - static std::ptrdiff_t distance(pointer x,pointer y) - { - return y->up()-x->up(); - } - - /* algorithmic stuff */ - - static void relocate(ptr_pointer pos,ptr_pointer x) - { - pointer n=*x; - if(xup()=pos-1; - } - else{ - while(x!=pos){ - *x=*(x-1); - (*x)->up()=x; - --x; - } - *pos=n; - n->up()=pos; - } - }; - - static void relocate(ptr_pointer pos,ptr_pointer first,ptr_pointer last) - { - ptr_pointer begin,middle,end; - if(posup()=begin+j; - break; - } - else{ - *(begin+j)=*(begin+k); - (*(begin+j))->up()=begin+j; - } - - if(kup()=begin+k; - break; - } - else{ - *(begin+k)=*(begin+j); - (*(begin+k))->up()=begin+k; - } - } - } - }; - - static void extract(ptr_pointer x,ptr_pointer pend) - { - --pend; - while(x!=pend){ - *x=*(x+1); - (*x)->up()=x; - ++x; - } - } - - static void transfer( - ptr_pointer pbegin0,ptr_pointer pend0,ptr_pointer pbegin1) - { - while(pbegin0!=pend0){ - *pbegin1=*pbegin0++; - (*pbegin1)->up()=pbegin1; - ++pbegin1; - } - } - - static void reverse(ptr_pointer pbegin,ptr_pointer pend) - { - std::ptrdiff_t d=(pend-pbegin)/2; - for(std::ptrdiff_t i=0;iup()=pbegin; - (*pend)->up()=pend; - ++pbegin; - } - } - -private: - ptr_pointer up_; -}; - -template -struct random_access_index_node_trampoline: - random_access_index_node_impl< - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > -{ - typedef random_access_index_node_impl< - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > impl_type; -}; - -template -struct random_access_index_node: - Super,random_access_index_node_trampoline -{ -private: - typedef random_access_index_node_trampoline trampoline; - -public: - typedef typename trampoline::impl_type impl_type; - typedef typename trampoline::pointer impl_pointer; - typedef typename trampoline::const_pointer const_impl_pointer; - typedef typename trampoline::ptr_pointer impl_ptr_pointer; - - impl_ptr_pointer& up(){return trampoline::up();} - impl_ptr_pointer up()const{return trampoline::up();} - - impl_pointer impl() - { - return static_cast( - static_cast(static_cast(this))); - } - - const_impl_pointer impl()const - { - return static_cast( - static_cast(static_cast(this))); - } - - static random_access_index_node* from_impl(impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - static const random_access_index_node* from_impl(const_impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - /* interoperability with rnd_node_iterator */ - - static void increment(random_access_index_node*& x) - { - impl_pointer xi=x->impl(); - trampoline::increment(xi); - x=from_impl(xi); - } - - static void decrement(random_access_index_node*& x) - { - impl_pointer xi=x->impl(); - trampoline::decrement(xi); - x=from_impl(xi); - } - - static void advance(random_access_index_node*& x,std::ptrdiff_t n) - { - impl_pointer xi=x->impl(); - trampoline::advance(xi,n); - x=from_impl(xi); - } - - static std::ptrdiff_t distance( - random_access_index_node* x,random_access_index_node* y) - { - return trampoline::distance(x->impl(),y->impl()); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ops.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ops.hpp deleted file mode 100644 index f5e76e4441f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ops.hpp +++ /dev/null @@ -1,203 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RND_INDEX_OPS_HPP -#define BOOST_MULTI_INDEX_DETAIL_RND_INDEX_OPS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Common code for random_access_index memfuns having templatized and - * non-templatized versions. - */ - -template -Node* random_access_index_remove( - random_access_index_ptr_array& ptrs,Predicate pred) -{ - typedef typename Node::value_type value_type; - typedef typename Node::impl_ptr_pointer impl_ptr_pointer; - - impl_ptr_pointer first=ptrs.begin(), - res=first, - last=ptrs.end(); - for(;first!=last;++first){ - if(!pred( - const_cast(Node::from_impl(*first)->value()))){ - if(first!=res){ - std::swap(*first,*res); - (*first)->up()=first; - (*res)->up()=res; - } - ++res; - } - } - return Node::from_impl(*res); -} - -template -Node* random_access_index_unique( - random_access_index_ptr_array& ptrs,BinaryPredicate binary_pred) -{ - typedef typename Node::value_type value_type; - typedef typename Node::impl_ptr_pointer impl_ptr_pointer; - - impl_ptr_pointer first=ptrs.begin(), - res=first, - last=ptrs.end(); - if(first!=last){ - for(;++first!=last;){ - if(!binary_pred( - const_cast(Node::from_impl(*res)->value()), - const_cast(Node::from_impl(*first)->value()))){ - ++res; - if(first!=res){ - std::swap(*first,*res); - (*first)->up()=first; - (*res)->up()=res; - } - } - } - ++res; - } - return Node::from_impl(*res); -} - -template -void random_access_index_inplace_merge( - const Allocator& al, - random_access_index_ptr_array& ptrs, - BOOST_DEDUCED_TYPENAME Node::impl_ptr_pointer first1,Compare comp) -{ - typedef typename Node::value_type value_type; - typedef typename Node::impl_pointer impl_pointer; - typedef typename Node::impl_ptr_pointer impl_ptr_pointer; - - auto_space spc(al,ptrs.size()); - - impl_ptr_pointer first0=ptrs.begin(), - last0=first1, - last1=ptrs.end(), - out=spc.data(); - while(first0!=last0&&first1!=last1){ - if(comp( - const_cast(Node::from_impl(*first1)->value()), - const_cast(Node::from_impl(*first0)->value()))){ - *out++=*first1++; - } - else{ - *out++=*first0++; - } - } - std::copy(&*first0,&*last0,&*out); - std::copy(&*first1,&*last1,&*out); - - first1=ptrs.begin(); - out=spc.data(); - while(first1!=last1){ - *first1=*out++; - (*first1)->up()=first1; - ++first1; - } -} - -/* sorting */ - -/* auxiliary stuff */ - -template -struct random_access_index_sort_compare -{ - typedef typename Node::impl_pointer first_argument_type; - typedef typename Node::impl_pointer second_argument_type; - typedef bool result_type; - - random_access_index_sort_compare(Compare comp_=Compare()):comp(comp_){} - - bool operator()( - typename Node::impl_pointer x,typename Node::impl_pointer y)const - { - typedef typename Node::value_type value_type; - - return comp( - const_cast(Node::from_impl(x)->value()), - const_cast(Node::from_impl(y)->value())); - } - -private: - Compare comp; -}; - -template -void random_access_index_sort( - const Allocator& al, - random_access_index_ptr_array& ptrs, - Compare comp) -{ - /* The implementation is extremely simple: an auxiliary - * array of pointers is sorted using stdlib facilities and - * then used to rearrange the index. This is suboptimal - * in space and time, but has some advantages over other - * possible approaches: - * - Use std::stable_sort() directly on ptrs using some - * special iterator in charge of maintaining pointers - * and up() pointers in sync: we cannot guarantee - * preservation of the container invariants in the face of - * exceptions, if, for instance, std::stable_sort throws - * when ptrs transitorily contains duplicate elements. - * - Rewrite the internal algorithms of std::stable_sort - * adapted for this case: besides being a fair amount of - * work, making a stable sort compatible with Boost.MultiIndex - * invariants (basically, no duplicates or missing elements - * even if an exception is thrown) is complicated, error-prone - * and possibly won't perform much better than the - * solution adopted. - */ - - if(ptrs.size()<=1)return; - - typedef typename Node::impl_pointer impl_pointer; - typedef typename Node::impl_ptr_pointer impl_ptr_pointer; - typedef random_access_index_sort_compare< - Node,Compare> ptr_compare; - - impl_ptr_pointer first=ptrs.begin(); - impl_ptr_pointer last=ptrs.end(); - auto_space< - impl_pointer, - Allocator> spc(al,ptrs.size()); - impl_ptr_pointer buf=spc.data(); - - std::copy(&*first,&*last,&*buf); - std::stable_sort(&*buf,&*buf+ptrs.size(),ptr_compare(comp)); - - while(first!=last){ - *first=*buf++; - (*first)->up()=first; - ++first; - } -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ptr_array.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ptr_array.hpp deleted file mode 100644 index bae1c851b8e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_index_ptr_array.hpp +++ /dev/null @@ -1,144 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RND_INDEX_PTR_ARRAY_HPP -#define BOOST_MULTI_INDEX_DETAIL_RND_INDEX_PTR_ARRAY_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* pointer structure for use by random access indices */ - -template -class random_access_index_ptr_array:private noncopyable -{ - typedef random_access_index_node_impl< - typename boost::detail::allocator::rebind_to< - Allocator, - char - >::type - > node_impl_type; - -public: - typedef typename node_impl_type::pointer value_type; - typedef typename boost::detail::allocator::rebind_to< - Allocator,value_type - >::type::pointer pointer; - - random_access_index_ptr_array( - const Allocator& al,value_type end_,std::size_t sz): - size_(sz), - capacity_(sz), - spc(al,capacity_+1) - { - *end()=end_; - end_->up()=end(); - } - - std::size_t size()const{return size_;} - std::size_t capacity()const{return capacity_;} - - void room_for_one() - { - if(size_==capacity_){ - reserve(capacity_<=10?15:capacity_+capacity_/2); - } - } - - void reserve(std::size_t c) - { - if(c>capacity_)set_capacity(c); - } - - void shrink_to_fit() - { - if(capacity_>size_)set_capacity(size_); - } - - pointer begin()const{return ptrs();} - pointer end()const{return ptrs()+size_;} - pointer at(std::size_t n)const{return ptrs()+n;} - - void push_back(value_type x) - { - *(end()+1)=*end(); - (*(end()+1))->up()=end()+1; - *end()=x; - (*end())->up()=end(); - ++size_; - } - - void erase(value_type x) - { - node_impl_type::extract(x->up(),end()+1); - --size_; - } - - void clear() - { - *begin()=*end(); - (*begin())->up()=begin(); - size_=0; - } - - void swap(random_access_index_ptr_array& x) - { - std::swap(size_,x.size_); - std::swap(capacity_,x.capacity_); - spc.swap(x.spc); - } - -private: - std::size_t size_; - std::size_t capacity_; - auto_space spc; - - pointer ptrs()const - { - return spc.data(); - } - - void set_capacity(std::size_t c) - { - auto_space spc1(spc.get_allocator(),c+1); - node_impl_type::transfer(begin(),end()+1,spc1.data()); - spc.swap(spc1); - capacity_=c; - } -}; - -template -void swap( - random_access_index_ptr_array& x, - random_access_index_ptr_array& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_node_iterator.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_node_iterator.hpp deleted file mode 100644 index 48026132fb7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnd_node_iterator.hpp +++ /dev/null @@ -1,140 +0,0 @@ -/* Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RND_NODE_ITERATOR_HPP -#define BOOST_MULTI_INDEX_DETAIL_RND_NODE_ITERATOR_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Iterator class for node-based indices with random access iterators. */ - -template -class rnd_node_iterator: - public random_access_iterator_helper< - rnd_node_iterator, - typename Node::value_type, - std::ptrdiff_t, - const typename Node::value_type*, - const typename Node::value_type&> -{ -public: - /* coverity[uninit_ctor]: suppress warning */ - rnd_node_iterator(){} - explicit rnd_node_iterator(Node* node_):node(node_){} - - const typename Node::value_type& operator*()const - { - return node->value(); - } - - rnd_node_iterator& operator++() - { - Node::increment(node); - return *this; - } - - rnd_node_iterator& operator--() - { - Node::decrement(node); - return *this; - } - - rnd_node_iterator& operator+=(std::ptrdiff_t n) - { - Node::advance(node,n); - return *this; - } - - rnd_node_iterator& operator-=(std::ptrdiff_t n) - { - Node::advance(node,-n); - return *this; - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* Serialization. As for why the following is public, - * see explanation in safe_mode_iterator notes in safe_mode.hpp. - */ - - BOOST_SERIALIZATION_SPLIT_MEMBER() - - typedef typename Node::base_type node_base_type; - - template - void save(Archive& ar,const unsigned int)const - { - node_base_type* bnode=node; - ar< - void load(Archive& ar,const unsigned int) - { - node_base_type* bnode; - ar>>serialization::make_nvp("pointer",bnode); - node=static_cast(bnode); - } -#endif - - /* get_node is not to be used by the user */ - - typedef Node node_type; - - Node* get_node()const{return node;} - -private: - Node* node; -}; - -template -bool operator==( - const rnd_node_iterator& x, - const rnd_node_iterator& y) -{ - return x.get_node()==y.get_node(); -} - -template -bool operator<( - const rnd_node_iterator& x, - const rnd_node_iterator& y) -{ - return Node::distance(x.get_node(),y.get_node())>0; -} - -template -std::ptrdiff_t operator-( - const rnd_node_iterator& x, - const rnd_node_iterator& y) -{ - return Node::distance(y.get_node(),x.get_node()); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnk_index_ops.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnk_index_ops.hpp deleted file mode 100644 index fb233cf4973..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/rnk_index_ops.hpp +++ /dev/null @@ -1,300 +0,0 @@ -/* Copyright 2003-2017 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_RNK_INDEX_OPS_HPP -#define BOOST_MULTI_INDEX_DETAIL_RNK_INDEX_OPS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Common code for ranked_index memfuns having templatized and - * non-templatized versions. - */ - -template -inline std::size_t ranked_node_size(Pointer x) -{ - return x!=Pointer(0)?x->size:0; -} - -template -inline Pointer ranked_index_nth(std::size_t n,Pointer end_) -{ - Pointer top=end_->parent(); - if(top==Pointer(0)||n>=top->size)return end_; - - for(;;){ - std::size_t s=ranked_node_size(top->left()); - if(n==s)return top; - if(nleft(); - else{ - top=top->right(); - n-=s+1; - } - } -} - -template -inline std::size_t ranked_index_rank(Pointer x,Pointer end_) -{ - Pointer top=end_->parent(); - if(top==Pointer(0))return 0; - if(x==end_)return top->size; - - std::size_t s=ranked_node_size(x->left()); - while(x!=top){ - Pointer z=x->parent(); - if(x==z->right()){ - s+=ranked_node_size(z->left())+1; - } - x=z; - } - return s; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::size_t ranked_index_find_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ranked_index_find_rank( - top,y,key,x,comp, - mpl::and_< - promotes_1st_arg, - promotes_2nd_arg >()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline std::size_t ranked_index_find_rank( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ranked_index_find_rank(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::size_t ranked_index_find_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - if(!top)return 0; - - std::size_t s=top->impl()->size, - s0=s; - Node* y0=y; - - do{ - if(!comp(key(top->value()),x)){ - y=top; - s-=ranked_node_size(y->right())+1; - top=Node::from_impl(top->left()); - } - else top=Node::from_impl(top->right()); - }while(top); - - return (y==y0||comp(x,key(y->value())))?s0:s; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::size_t ranked_index_lower_bound_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ranked_index_lower_bound_rank( - top,y,key,x,comp, - promotes_2nd_arg()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline std::size_t ranked_index_lower_bound_rank( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ranked_index_lower_bound_rank(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::size_t ranked_index_lower_bound_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - if(!top)return 0; - - std::size_t s=top->impl()->size; - - do{ - if(!comp(key(top->value()),x)){ - y=top; - s-=ranked_node_size(y->right())+1; - top=Node::from_impl(top->left()); - } - else top=Node::from_impl(top->right()); - }while(top); - - return s; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::size_t ranked_index_upper_bound_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ranked_index_upper_bound_rank( - top,y,key,x,comp, - promotes_1st_arg()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline std::size_t ranked_index_upper_bound_rank( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ranked_index_upper_bound_rank(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::size_t ranked_index_upper_bound_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - if(!top)return 0; - - std::size_t s=top->impl()->size; - - do{ - if(comp(x,key(top->value()))){ - y=top; - s-=ranked_node_size(y->right())+1; - top=Node::from_impl(top->left()); - } - else top=Node::from_impl(top->right()); - }while(top); - - return s; -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::pair ranked_index_equal_range_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp) -{ - typedef typename KeyFromValue::result_type key_type; - - return ranked_index_equal_range_rank( - top,y,key,x,comp, - mpl::and_< - promotes_1st_arg, - promotes_2nd_arg >()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleCompare -> -inline std::pair ranked_index_equal_range_rank( - Node* top,Node* y,const KeyFromValue& key, - const BOOST_DEDUCED_TYPENAME KeyFromValue::result_type& x, - const CompatibleCompare& comp,mpl::true_) -{ - return ranked_index_equal_range_rank(top,y,key,x,comp,mpl::false_()); -} - -template< - typename Node,typename KeyFromValue, - typename CompatibleKey,typename CompatibleCompare -> -inline std::pair ranked_index_equal_range_rank( - Node* top,Node* y,const KeyFromValue& key,const CompatibleKey& x, - const CompatibleCompare& comp,mpl::false_) -{ - if(!top)return std::pair(0,0); - - std::size_t s=top->impl()->size; - - do{ - if(comp(key(top->value()),x)){ - top=Node::from_impl(top->right()); - } - else if(comp(x,key(top->value()))){ - y=top; - s-=ranked_node_size(y->right())+1; - top=Node::from_impl(top->left()); - } - else{ - return std::pair( - s-top->impl()->size+ - ranked_index_lower_bound_rank( - Node::from_impl(top->left()),top,key,x,comp,mpl::false_()), - s-ranked_node_size(top->right())+ - ranked_index_upper_bound_rank( - Node::from_impl(top->right()),y,key,x,comp,mpl::false_())); - } - }while(top); - - return std::pair(s,s); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/safe_mode.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/safe_mode.hpp deleted file mode 100644 index 905270e9fb3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/safe_mode.hpp +++ /dev/null @@ -1,588 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_SAFE_MODE_HPP -#define BOOST_MULTI_INDEX_DETAIL_SAFE_MODE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -/* Safe mode machinery, in the spirit of Cay Hortmann's "Safe STL" - * (http://www.horstmann.com/safestl.html). - * In this mode, containers of type Container are derived from - * safe_container, and their corresponding iterators - * are wrapped with safe_iterator. These classes provide - * an internal record of which iterators are at a given moment associated - * to a given container, and properly mark the iterators as invalid - * when the container gets destroyed. - * Iterators are chained in a single attached list, whose header is - * kept by the container. More elaborate data structures would yield better - * performance, but I decided to keep complexity to a minimum since - * speed is not an issue here. - * Safe mode iterators automatically check that only proper operations - * are performed on them: for instance, an invalid iterator cannot be - * dereferenced. Additionally, a set of utilty macros and functions are - * provided that serve to implement preconditions and cooperate with - * the framework within the container. - * Iterators can also be unchecked, i.e. they do not have info about - * which container they belong in. This situation arises when the iterator - * is restored from a serialization archive: only information on the node - * is available, and it is not possible to determine to which container - * the iterator is associated to. The only sensible policy is to assume - * unchecked iterators are valid, though this can certainly generate false - * positive safe mode checks. - * This is not a full-fledged safe mode framework, and is only intended - * for use within the limits of Boost.MultiIndex. - */ - -/* Assertion macros. These resolve to no-ops if - * !defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE). - */ - -#if !defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) -#undef BOOST_MULTI_INDEX_SAFE_MODE_ASSERT -#define BOOST_MULTI_INDEX_SAFE_MODE_ASSERT(expr,error_code) ((void)0) -#else -#if !defined(BOOST_MULTI_INDEX_SAFE_MODE_ASSERT) -#include -#define BOOST_MULTI_INDEX_SAFE_MODE_ASSERT(expr,error_code) BOOST_ASSERT(expr) -#endif -#endif - -#define BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_valid_iterator(it), \ - safe_mode::invalid_iterator); - -#define BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(it) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_dereferenceable_iterator(it), \ - safe_mode::not_dereferenceable_iterator); - -#define BOOST_MULTI_INDEX_CHECK_INCREMENTABLE_ITERATOR(it) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_incrementable_iterator(it), \ - safe_mode::not_incrementable_iterator); - -#define BOOST_MULTI_INDEX_CHECK_DECREMENTABLE_ITERATOR(it) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_decrementable_iterator(it), \ - safe_mode::not_decrementable_iterator); - -#define BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,cont) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_is_owner(it,cont), \ - safe_mode::not_owner); - -#define BOOST_MULTI_INDEX_CHECK_SAME_OWNER(it0,it1) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_same_owner(it0,it1), \ - safe_mode::not_same_owner); - -#define BOOST_MULTI_INDEX_CHECK_VALID_RANGE(it0,it1) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_valid_range(it0,it1), \ - safe_mode::invalid_range); - -#define BOOST_MULTI_INDEX_CHECK_OUTSIDE_RANGE(it,it0,it1) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_outside_range(it,it0,it1), \ - safe_mode::inside_range); - -#define BOOST_MULTI_INDEX_CHECK_IN_BOUNDS(it,n) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_in_bounds(it,n), \ - safe_mode::out_of_bounds); - -#define BOOST_MULTI_INDEX_CHECK_DIFFERENT_CONTAINER(cont0,cont1) \ - BOOST_MULTI_INDEX_SAFE_MODE_ASSERT( \ - safe_mode::check_different_container(cont0,cont1), \ - safe_mode::same_container); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#endif - -#if defined(BOOST_HAS_THREADS) -#include -#endif - -namespace boost{ - -namespace multi_index{ - -namespace safe_mode{ - -/* Checking routines. Assume the best for unchecked iterators - * (i.e. they pass the checking when there is not enough info - * to know.) - */ - -template -inline bool check_valid_iterator(const Iterator& it) -{ - return it.valid()||it.unchecked(); -} - -template -inline bool check_dereferenceable_iterator(const Iterator& it) -{ - return (it.valid()&&it!=it.owner()->end())||it.unchecked(); -} - -template -inline bool check_incrementable_iterator(const Iterator& it) -{ - return (it.valid()&&it!=it.owner()->end())||it.unchecked(); -} - -template -inline bool check_decrementable_iterator(const Iterator& it) -{ - return (it.valid()&&it!=it.owner()->begin())||it.unchecked(); -} - -template -inline bool check_is_owner( - const Iterator& it,const typename Iterator::container_type& cont) -{ - return (it.valid()&&it.owner()==&cont)||it.unchecked(); -} - -template -inline bool check_same_owner(const Iterator& it0,const Iterator& it1) -{ - return (it0.valid()&&it1.valid()&&it0.owner()==it1.owner())|| - it0.unchecked()||it1.unchecked(); -} - -template -inline bool check_valid_range(const Iterator& it0,const Iterator& it1) -{ - if(!check_same_owner(it0,it1))return false; - - if(it0.valid()){ - Iterator last=it0.owner()->end(); - if(it1==last)return true; - - for(Iterator first=it0;first!=last;++first){ - if(first==it1)return true; - } - return false; - } - return true; -} - -template -inline bool check_outside_range( - const Iterator& it,const Iterator& it0,const Iterator& it1) -{ - if(!check_same_owner(it0,it1))return false; - - if(it0.valid()){ - Iterator last=it0.owner()->end(); - bool found=false; - - Iterator first=it0; - for(;first!=last;++first){ - if(first==it1)break; - - /* crucial that this check goes after previous break */ - - if(first==it)found=true; - } - if(first!=it1)return false; - return !found; - } - return true; -} - -template -inline bool check_in_bounds(const Iterator& it,Difference n) -{ - if(it.unchecked())return true; - if(!it.valid()) return false; - if(n>0) return it.owner()->end()-it>=n; - else return it.owner()->begin()-it<=n; -} - -template -inline bool check_different_container( - const Container& cont0,const Container& cont1) -{ - return &cont0!=&cont1; -} - -/* Invalidates all iterators equivalent to that given. Safe containers - * must call this when deleting elements: the safe mode framework cannot - * perform this operation automatically without outside help. - */ - -template -inline void detach_equivalent_iterators(Iterator& it) -{ - if(it.valid()){ - { -#if defined(BOOST_HAS_THREADS) - boost::detail::lightweight_mutex::scoped_lock lock(it.cont->mutex); -#endif - - Iterator *prev_,*next_; - for( - prev_=static_cast(&it.cont->header); - (next_=static_cast(prev_->next))!=0;){ - if(next_!=&it&&*next_==it){ - prev_->next=next_->next; - next_->cont=0; - } - else prev_=next_; - } - } - it.detach(); - } -} - -template class safe_container; /* fwd decl. */ - -} /* namespace multi_index::safe_mode */ - -namespace detail{ - -class safe_container_base; /* fwd decl. */ - -class safe_iterator_base -{ -public: - bool valid()const{return cont!=0;} - bool unchecked()const{return unchecked_;} - - inline void detach(); - - void uncheck() - { - detach(); - unchecked_=true; - } - -protected: - safe_iterator_base():cont(0),next(0),unchecked_(false){} - - explicit safe_iterator_base(safe_container_base* cont_): - unchecked_(false) - { - attach(cont_); - } - - safe_iterator_base(const safe_iterator_base& it): - unchecked_(it.unchecked_) - { - attach(it.cont); - } - - safe_iterator_base& operator=(const safe_iterator_base& it) - { - unchecked_=it.unchecked_; - safe_container_base* new_cont=it.cont; - if(cont!=new_cont){ - detach(); - attach(new_cont); - } - return *this; - } - - ~safe_iterator_base() - { - detach(); - } - - const safe_container_base* owner()const{return cont;} - -BOOST_MULTI_INDEX_PRIVATE_IF_MEMBER_TEMPLATE_FRIENDS: - friend class safe_container_base; - -#if !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) - template friend class safe_mode::safe_container; - template friend - void safe_mode::detach_equivalent_iterators(Iterator&); -#endif - - inline void attach(safe_container_base* cont_); - - safe_container_base* cont; - safe_iterator_base* next; - bool unchecked_; -}; - -class safe_container_base:private noncopyable -{ -public: - safe_container_base(){} - -BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: - friend class safe_iterator_base; - -#if !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) - template friend - void safe_mode::detach_equivalent_iterators(Iterator&); -#endif - - ~safe_container_base() - { - /* Detaches all remaining iterators, which by now will - * be those pointing to the end of the container. - */ - - for(safe_iterator_base* it=header.next;it;it=it->next)it->cont=0; - header.next=0; - } - - void swap(safe_container_base& x) - { - for(safe_iterator_base* it0=header.next;it0;it0=it0->next)it0->cont=&x; - for(safe_iterator_base* it1=x.header.next;it1;it1=it1->next)it1->cont=this; - std::swap(header.cont,x.header.cont); - std::swap(header.next,x.header.next); - } - - safe_iterator_base header; - -#if defined(BOOST_HAS_THREADS) - boost::detail::lightweight_mutex mutex; -#endif -}; - -void safe_iterator_base::attach(safe_container_base* cont_) -{ - cont=cont_; - if(cont){ -#if defined(BOOST_HAS_THREADS) - boost::detail::lightweight_mutex::scoped_lock lock(cont->mutex); -#endif - - next=cont->header.next; - cont->header.next=this; - } -} - -void safe_iterator_base::detach() -{ - if(cont){ -#if defined(BOOST_HAS_THREADS) - boost::detail::lightweight_mutex::scoped_lock lock(cont->mutex); -#endif - - safe_iterator_base *prev_,*next_; - for(prev_=&cont->header;(next_=prev_->next)!=this;prev_=next_){} - prev_->next=next; - cont=0; - } -} - -} /* namespace multi_index::detail */ - -namespace safe_mode{ - -/* In order to enable safe mode on a container: - * - The container must derive from safe_container, - * - iterators must be generated via safe_iterator, which adapts a - * preexistent unsafe iterator class. - */ - -template -class safe_container; - -template -class safe_iterator: - public detail::iter_adaptor,Iterator>, - public detail::safe_iterator_base -{ - typedef detail::iter_adaptor super; - typedef detail::safe_iterator_base safe_super; - -public: - typedef Container container_type; - typedef typename Iterator::reference reference; - typedef typename Iterator::difference_type difference_type; - - safe_iterator(){} - explicit safe_iterator(safe_container* cont_): - safe_super(cont_){} - template - safe_iterator(const T0& t0,safe_container* cont_): - super(Iterator(t0)),safe_super(cont_){} - template - safe_iterator( - const T0& t0,const T1& t1,safe_container* cont_): - super(Iterator(t0,t1)),safe_super(cont_){} - - safe_iterator& operator=(const safe_iterator& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(x); - this->base_reference()=x.base_reference(); - safe_super::operator=(x); - return *this; - } - - const container_type* owner()const - { - return - static_cast( - static_cast*>( - this->safe_super::owner())); - } - - /* get_node is not to be used by the user */ - - typedef typename Iterator::node_type node_type; - - node_type* get_node()const{return this->base_reference().get_node();} - -private: - friend class boost::multi_index::detail::iter_adaptor_access; - - reference dereference()const - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(*this); - return *(this->base_reference()); - } - - bool equal(const safe_iterator& x)const - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(x); - BOOST_MULTI_INDEX_CHECK_SAME_OWNER(*this,x); - return this->base_reference()==x.base_reference(); - } - - void increment() - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - BOOST_MULTI_INDEX_CHECK_INCREMENTABLE_ITERATOR(*this); - ++(this->base_reference()); - } - - void decrement() - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - BOOST_MULTI_INDEX_CHECK_DECREMENTABLE_ITERATOR(*this); - --(this->base_reference()); - } - - void advance(difference_type n) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - BOOST_MULTI_INDEX_CHECK_IN_BOUNDS(*this,n); - this->base_reference()+=n; - } - - difference_type distance_to(const safe_iterator& x)const - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(x); - BOOST_MULTI_INDEX_CHECK_SAME_OWNER(*this,x); - return x.base_reference()-this->base_reference(); - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* Serialization. Note that Iterator::save and Iterator:load - * are assumed to be defined and public: at first sight it seems - * like we could have resorted to the public serialization interface - * for doing the forwarding to the adapted iterator class: - * ar<>base_reference(); - * but this would cause incompatibilities if a saving - * program is in safe mode and the loading program is not, or - * viceversa --in safe mode, the archived iterator data is one layer - * deeper, this is especially relevant with XML archives. - * It'd be nice if Boost.Serialization provided some forwarding - * facility for use by adaptor classes. - */ - - friend class boost::serialization::access; - - BOOST_SERIALIZATION_SPLIT_MEMBER() - - template - void save(Archive& ar,const unsigned int version)const - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(*this); - this->base_reference().save(ar,version); - } - - template - void load(Archive& ar,const unsigned int version) - { - this->base_reference().load(ar,version); - safe_super::uncheck(); - } -#endif -}; - -template -class safe_container:public detail::safe_container_base -{ - typedef detail::safe_container_base super; - -public: - void detach_dereferenceable_iterators() - { - typedef typename Container::iterator iterator; - - iterator end_=static_cast(this)->end(); - iterator *prev_,*next_; - for( - prev_=static_cast(&this->header); - (next_=static_cast(prev_->next))!=0;){ - if(*next_!=end_){ - prev_->next=next_->next; - next_->cont=0; - } - else prev_=next_; - } - } - - void swap(safe_container& x) - { - super::swap(x); - } -}; - -} /* namespace multi_index::safe_mode */ - -} /* namespace multi_index */ - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -namespace serialization{ -template -struct version< - boost::multi_index::safe_mode::safe_iterator -> -{ - BOOST_STATIC_CONSTANT( - int,value=boost::serialization::version::value); -}; -} /* namespace serialization */ -#endif - -} /* namespace boost */ - -#endif /* BOOST_MULTI_INDEX_ENABLE_SAFE_MODE */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/scope_guard.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/scope_guard.hpp deleted file mode 100644 index 116f8f50415..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/scope_guard.hpp +++ /dev/null @@ -1,453 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_SCOPE_GUARD_HPP -#define BOOST_MULTI_INDEX_DETAIL_SCOPE_GUARD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Until some official version of the ScopeGuard idiom makes it into Boost, - * we locally define our own. This is a merely reformated version of - * ScopeGuard.h as defined in: - * Alexandrescu, A., Marginean, P.:"Generic: Change the Way You - * Write Exception-Safe Code - Forever", C/C++ Users Jornal, Dec 2000, - * http://www.drdobbs.com/184403758 - * with the following modifications: - * - General pretty formatting (pretty to my taste at least.) - * - Naming style changed to standard C++ library requirements. - * - Added scope_guard_impl4 and obj_scope_guard_impl3, (Boost.MultiIndex - * needs them). A better design would provide guards for many more - * arguments through the Boost Preprocessor Library. - * - Added scope_guard_impl_base::touch (see below.) - * - Removed RefHolder and ByRef, whose functionality is provided - * already by Boost.Ref. - * - Removed static make_guard's and make_obj_guard's, so that the code - * will work even if BOOST_NO_MEMBER_TEMPLATES is defined. This forces - * us to move some private ctors to public, though. - * - * NB: CodeWarrior Pro 8 seems to have problems looking up safe_execute - * without an explicit qualification. - * - * We also define the following variants of the idiom: - * - * - make_guard_if_c( ... ) - * - make_guard_if( ... ) - * - make_obj_guard_if_c( ... ) - * - make_obj_guard_if( ... ) - * which may be used with a compile-time constant to yield - * a "null_guard" if the boolean compile-time parameter is false, - * or conversely, the guard is only constructed if the constant is true. - * This is useful to avoid extra tagging, because the returned - * null_guard can be optimzed comlpetely away by the compiler. - */ - -class scope_guard_impl_base -{ -public: - scope_guard_impl_base():dismissed_(false){} - void dismiss()const{dismissed_=true;} - - /* This helps prevent some "unused variable" warnings under, for instance, - * GCC 3.2. - */ - void touch()const{} - -protected: - ~scope_guard_impl_base(){} - - scope_guard_impl_base(const scope_guard_impl_base& other): - dismissed_(other.dismissed_) - { - other.dismiss(); - } - - template - static void safe_execute(J& j){ - BOOST_TRY{ - if(!j.dismissed_)j.execute(); - } - BOOST_CATCH(...){} - BOOST_CATCH_END - } - - mutable bool dismissed_; - -private: - scope_guard_impl_base& operator=(const scope_guard_impl_base&); -}; - -typedef const scope_guard_impl_base& scope_guard; - -struct null_guard : public scope_guard_impl_base -{ - template< class T1 > - null_guard( const T1& ) - { } - - template< class T1, class T2 > - null_guard( const T1&, const T2& ) - { } - - template< class T1, class T2, class T3 > - null_guard( const T1&, const T2&, const T3& ) - { } - - template< class T1, class T2, class T3, class T4 > - null_guard( const T1&, const T2&, const T3&, const T4& ) - { } - - template< class T1, class T2, class T3, class T4, class T5 > - null_guard( const T1&, const T2&, const T3&, const T4&, const T5& ) - { } -}; - -template< bool cond, class T > -struct null_guard_return -{ - typedef typename boost::mpl::if_c::type type; -}; - -template -class scope_guard_impl0:public scope_guard_impl_base -{ -public: - scope_guard_impl0(F fun):fun_(fun){} - ~scope_guard_impl0(){scope_guard_impl_base::safe_execute(*this);} - void execute(){fun_();} - -protected: - - F fun_; -}; - -template -inline scope_guard_impl0 make_guard(F fun) -{ - return scope_guard_impl0(fun); -} - -template -inline typename null_guard_return >::type -make_guard_if_c(F fun) -{ - return typename null_guard_return >::type(fun); -} - -template -inline typename null_guard_return >::type -make_guard_if(F fun) -{ - return make_guard_if(fun); -} - -template -class scope_guard_impl1:public scope_guard_impl_base -{ -public: - scope_guard_impl1(F fun,P1 p1):fun_(fun),p1_(p1){} - ~scope_guard_impl1(){scope_guard_impl_base::safe_execute(*this);} - void execute(){fun_(p1_);} - -protected: - F fun_; - const P1 p1_; -}; - -template -inline scope_guard_impl1 make_guard(F fun,P1 p1) -{ - return scope_guard_impl1(fun,p1); -} - -template -inline typename null_guard_return >::type -make_guard_if_c(F fun,P1 p1) -{ - return typename null_guard_return >::type(fun,p1); -} - -template -inline typename null_guard_return >::type -make_guard_if(F fun,P1 p1) -{ - return make_guard_if_c(fun,p1); -} - -template -class scope_guard_impl2:public scope_guard_impl_base -{ -public: - scope_guard_impl2(F fun,P1 p1,P2 p2):fun_(fun),p1_(p1),p2_(p2){} - ~scope_guard_impl2(){scope_guard_impl_base::safe_execute(*this);} - void execute(){fun_(p1_,p2_);} - -protected: - F fun_; - const P1 p1_; - const P2 p2_; -}; - -template -inline scope_guard_impl2 make_guard(F fun,P1 p1,P2 p2) -{ - return scope_guard_impl2(fun,p1,p2); -} - -template -inline typename null_guard_return >::type -make_guard_if_c(F fun,P1 p1,P2 p2) -{ - return typename null_guard_return >::type(fun,p1,p2); -} - -template -inline typename null_guard_return >::type -make_guard_if(F fun,P1 p1,P2 p2) -{ - return make_guard_if_c(fun,p1,p2); -} - -template -class scope_guard_impl3:public scope_guard_impl_base -{ -public: - scope_guard_impl3(F fun,P1 p1,P2 p2,P3 p3):fun_(fun),p1_(p1),p2_(p2),p3_(p3){} - ~scope_guard_impl3(){scope_guard_impl_base::safe_execute(*this);} - void execute(){fun_(p1_,p2_,p3_);} - -protected: - F fun_; - const P1 p1_; - const P2 p2_; - const P3 p3_; -}; - -template -inline scope_guard_impl3 make_guard(F fun,P1 p1,P2 p2,P3 p3) -{ - return scope_guard_impl3(fun,p1,p2,p3); -} - -template -inline typename null_guard_return >::type -make_guard_if_c(F fun,P1 p1,P2 p2,P3 p3) -{ - return typename null_guard_return >::type(fun,p1,p2,p3); -} - -template -inline typename null_guard_return< C::value,scope_guard_impl3 >::type -make_guard_if(F fun,P1 p1,P2 p2,P3 p3) -{ - return make_guard_if_c(fun,p1,p2,p3); -} - -template -class scope_guard_impl4:public scope_guard_impl_base -{ -public: - scope_guard_impl4(F fun,P1 p1,P2 p2,P3 p3,P4 p4): - fun_(fun),p1_(p1),p2_(p2),p3_(p3),p4_(p4){} - ~scope_guard_impl4(){scope_guard_impl_base::safe_execute(*this);} - void execute(){fun_(p1_,p2_,p3_,p4_);} - -protected: - F fun_; - const P1 p1_; - const P2 p2_; - const P3 p3_; - const P4 p4_; -}; - -template -inline scope_guard_impl4 make_guard( - F fun,P1 p1,P2 p2,P3 p3,P4 p4) -{ - return scope_guard_impl4(fun,p1,p2,p3,p4); -} - -template -inline typename null_guard_return >::type -make_guard_if_c( - F fun,P1 p1,P2 p2,P3 p3,P4 p4) -{ - return typename null_guard_return >::type(fun,p1,p2,p3,p4); -} - -template -inline typename null_guard_return >::type -make_guard_if( - F fun,P1 p1,P2 p2,P3 p3,P4 p4) -{ - return make_guard_if_c(fun,p1,p2,p3,p4); -} - -template -class obj_scope_guard_impl0:public scope_guard_impl_base -{ -public: - obj_scope_guard_impl0(Obj& obj,MemFun mem_fun):obj_(obj),mem_fun_(mem_fun){} - ~obj_scope_guard_impl0(){scope_guard_impl_base::safe_execute(*this);} - void execute(){(obj_.*mem_fun_)();} - -protected: - Obj& obj_; - MemFun mem_fun_; -}; - -template -inline obj_scope_guard_impl0 make_obj_guard(Obj& obj,MemFun mem_fun) -{ - return obj_scope_guard_impl0(obj,mem_fun); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if_c(Obj& obj,MemFun mem_fun) -{ - return typename null_guard_return >::type(obj,mem_fun); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if(Obj& obj,MemFun mem_fun) -{ - return make_obj_guard_if_c(obj,mem_fun); -} - -template -class obj_scope_guard_impl1:public scope_guard_impl_base -{ -public: - obj_scope_guard_impl1(Obj& obj,MemFun mem_fun,P1 p1): - obj_(obj),mem_fun_(mem_fun),p1_(p1){} - ~obj_scope_guard_impl1(){scope_guard_impl_base::safe_execute(*this);} - void execute(){(obj_.*mem_fun_)(p1_);} - -protected: - Obj& obj_; - MemFun mem_fun_; - const P1 p1_; -}; - -template -inline obj_scope_guard_impl1 make_obj_guard( - Obj& obj,MemFun mem_fun,P1 p1) -{ - return obj_scope_guard_impl1(obj,mem_fun,p1); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if_c( Obj& obj,MemFun mem_fun,P1 p1) -{ - return typename null_guard_return >::type(obj,mem_fun,p1); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if( Obj& obj,MemFun mem_fun,P1 p1) -{ - return make_obj_guard_if_c(obj,mem_fun,p1); -} - -template -class obj_scope_guard_impl2:public scope_guard_impl_base -{ -public: - obj_scope_guard_impl2(Obj& obj,MemFun mem_fun,P1 p1,P2 p2): - obj_(obj),mem_fun_(mem_fun),p1_(p1),p2_(p2) - {} - ~obj_scope_guard_impl2(){scope_guard_impl_base::safe_execute(*this);} - void execute(){(obj_.*mem_fun_)(p1_,p2_);} - -protected: - Obj& obj_; - MemFun mem_fun_; - const P1 p1_; - const P2 p2_; -}; - -template -inline obj_scope_guard_impl2 -make_obj_guard(Obj& obj,MemFun mem_fun,P1 p1,P2 p2) -{ - return obj_scope_guard_impl2(obj,mem_fun,p1,p2); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if_c(Obj& obj,MemFun mem_fun,P1 p1,P2 p2) -{ - return typename null_guard_return >::type(obj,mem_fun,p1,p2); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if(Obj& obj,MemFun mem_fun,P1 p1,P2 p2) -{ - return make_obj_guard_if_c(obj,mem_fun,p1,p2); -} - -template -class obj_scope_guard_impl3:public scope_guard_impl_base -{ -public: - obj_scope_guard_impl3(Obj& obj,MemFun mem_fun,P1 p1,P2 p2,P3 p3): - obj_(obj),mem_fun_(mem_fun),p1_(p1),p2_(p2),p3_(p3) - {} - ~obj_scope_guard_impl3(){scope_guard_impl_base::safe_execute(*this);} - void execute(){(obj_.*mem_fun_)(p1_,p2_,p3_);} - -protected: - Obj& obj_; - MemFun mem_fun_; - const P1 p1_; - const P2 p2_; - const P3 p3_; -}; - -template -inline obj_scope_guard_impl3 -make_obj_guard(Obj& obj,MemFun mem_fun,P1 p1,P2 p2,P3 p3) -{ - return obj_scope_guard_impl3(obj,mem_fun,p1,p2,p3); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if_c(Obj& obj,MemFun mem_fun,P1 p1,P2 p2,P3 p3) -{ - return typename null_guard_return >::type(obj,mem_fun,p1,p2,p3); -} - -template -inline typename null_guard_return >::type -make_obj_guard_if(Obj& obj,MemFun mem_fun,P1 p1,P2 p2,P3 p3) -{ - return make_obj_guard_if_c(obj,mem_fun,p1,p2,p3); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_node.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_node.hpp deleted file mode 100644 index 85b345af938..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_node.hpp +++ /dev/null @@ -1,217 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_SEQ_INDEX_NODE_HPP -#define BOOST_MULTI_INDEX_DETAIL_SEQ_INDEX_NODE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* doubly-linked node for use by sequenced_index */ - -template -struct sequenced_index_node_impl -{ - typedef typename - boost::detail::allocator::rebind_to< - Allocator,sequenced_index_node_impl - >::type::pointer pointer; - typedef typename - boost::detail::allocator::rebind_to< - Allocator,sequenced_index_node_impl - >::type::const_pointer const_pointer; - - pointer& prior(){return prior_;} - pointer prior()const{return prior_;} - pointer& next(){return next_;} - pointer next()const{return next_;} - - /* interoperability with bidir_node_iterator */ - - static void increment(pointer& x){x=x->next();} - static void decrement(pointer& x){x=x->prior();} - - /* algorithmic stuff */ - - static void link(pointer x,pointer header) - { - x->prior()=header->prior(); - x->next()=header; - x->prior()->next()=x->next()->prior()=x; - }; - - static void unlink(pointer x) - { - x->prior()->next()=x->next(); - x->next()->prior()=x->prior(); - } - - static void relink(pointer position,pointer x) - { - unlink(x); - x->prior()=position->prior(); - x->next()=position; - x->prior()->next()=x->next()->prior()=x; - } - - static void relink(pointer position,pointer x,pointer y) - { - /* position is assumed not to be in [x,y) */ - - if(x!=y){ - pointer z=y->prior(); - x->prior()->next()=y; - y->prior()=x->prior(); - x->prior()=position->prior(); - z->next()=position; - x->prior()->next()=x; - z->next()->prior()=z; - } - } - - static void reverse(pointer header) - { - pointer x=header; - do{ - pointer y=x->next(); - std::swap(x->prior(),x->next()); - x=y; - }while(x!=header); - } - - static void swap(pointer x,pointer y) - { - /* This swap function does not exchange the header nodes, - * but rather their pointers. This is *not* used for implementing - * sequenced_index::swap. - */ - - if(x->next()!=x){ - if(y->next()!=y){ - std::swap(x->next(),y->next()); - std::swap(x->prior(),y->prior()); - x->next()->prior()=x->prior()->next()=x; - y->next()->prior()=y->prior()->next()=y; - } - else{ - y->next()=x->next(); - y->prior()=x->prior(); - x->next()=x->prior()=x; - y->next()->prior()=y->prior()->next()=y; - } - } - else if(y->next()!=y){ - x->next()=y->next(); - x->prior()=y->prior(); - y->next()=y->prior()=y; - x->next()->prior()=x->prior()->next()=x; - } - } - -private: - pointer prior_; - pointer next_; -}; - -template -struct sequenced_index_node_trampoline: - sequenced_index_node_impl< - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > -{ - typedef sequenced_index_node_impl< - typename boost::detail::allocator::rebind_to< - typename Super::allocator_type, - char - >::type - > impl_type; -}; - -template -struct sequenced_index_node:Super,sequenced_index_node_trampoline -{ -private: - typedef sequenced_index_node_trampoline trampoline; - -public: - typedef typename trampoline::impl_type impl_type; - typedef typename trampoline::pointer impl_pointer; - typedef typename trampoline::const_pointer const_impl_pointer; - - impl_pointer& prior(){return trampoline::prior();} - impl_pointer prior()const{return trampoline::prior();} - impl_pointer& next(){return trampoline::next();} - impl_pointer next()const{return trampoline::next();} - - impl_pointer impl() - { - return static_cast( - static_cast(static_cast(this))); - } - - const_impl_pointer impl()const - { - return static_cast( - static_cast(static_cast(this))); - } - - static sequenced_index_node* from_impl(impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - static const sequenced_index_node* from_impl(const_impl_pointer x) - { - return - static_cast( - static_cast( - raw_ptr(x))); - } - - /* interoperability with bidir_node_iterator */ - - static void increment(sequenced_index_node*& x) - { - impl_pointer xi=x->impl(); - trampoline::increment(xi); - x=from_impl(xi); - } - - static void decrement(sequenced_index_node*& x) - { - impl_pointer xi=x->impl(); - trampoline::decrement(xi); - x=from_impl(xi); - } -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_ops.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_ops.hpp deleted file mode 100644 index 142bdd9dd9a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/seq_index_ops.hpp +++ /dev/null @@ -1,203 +0,0 @@ -/* Copyright 2003-2016 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_SEQ_INDEX_OPS_HPP -#define BOOST_MULTI_INDEX_DETAIL_SEQ_INDEX_OPS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Common code for sequenced_index memfuns having templatized and - * non-templatized versions. - */ - -template -void sequenced_index_remove(SequencedIndex& x,Predicate pred) -{ - typedef typename SequencedIndex::iterator iterator; - iterator first=x.begin(),last=x.end(); - while(first!=last){ - if(pred(*first))x.erase(first++); - else ++first; - } -} - -template -void sequenced_index_unique(SequencedIndex& x,BinaryPredicate binary_pred) -{ - typedef typename SequencedIndex::iterator iterator; - iterator first=x.begin(); - iterator last=x.end(); - if(first!=last){ - for(iterator middle=first;++middle!=last;middle=first){ - if(binary_pred(*middle,*first))x.erase(middle); - else first=middle; - } - } -} - -template -void sequenced_index_merge(SequencedIndex& x,SequencedIndex& y,Compare comp) -{ - typedef typename SequencedIndex::iterator iterator; - if(&x!=&y){ - iterator first0=x.begin(),last0=x.end(); - iterator first1=y.begin(),last1=y.end(); - while(first0!=last0&&first1!=last1){ - if(comp(*first1,*first0))x.splice(first0,y,first1++); - else ++first0; - } - x.splice(last0,y,first1,last1); - } -} - -/* sorting */ - -/* auxiliary stuff */ - -template -void sequenced_index_collate( - BOOST_DEDUCED_TYPENAME Node::impl_type* x, - BOOST_DEDUCED_TYPENAME Node::impl_type* y, - Compare comp) -{ - typedef typename Node::impl_type impl_type; - typedef typename Node::impl_pointer impl_pointer; - - impl_pointer first0=x->next(); - impl_pointer last0=x; - impl_pointer first1=y->next(); - impl_pointer last1=y; - while(first0!=last0&&first1!=last1){ - if(comp( - Node::from_impl(first1)->value(),Node::from_impl(first0)->value())){ - impl_pointer tmp=first1->next(); - impl_type::relink(first0,first1); - first1=tmp; - } - else first0=first0->next(); - } - impl_type::relink(last0,first1,last1); -} - -/* Some versions of CGG require a bogus typename in counter_spc - * inside sequenced_index_sort if the following is defined - * also inside sequenced_index_sort. - */ - -BOOST_STATIC_CONSTANT( - std::size_t, - sequenced_index_sort_max_fill= - (std::size_t)std::numeric_limits::digits+1); - -#include - -template -void sequenced_index_sort(Node* header,Compare comp) -{ - /* Musser's mergesort, see http://www.cs.rpi.edu/~musser/gp/List/lists1.html. - * The implementation is a little convoluted: in the original code - * counter elements and carry are std::lists: here we do not want - * to use multi_index instead, so we do things at a lower level, managing - * directly the internal node representation. - * Incidentally, the implementations I've seen of this algorithm (SGI, - * Dinkumware, STLPort) are not exception-safe: this is. Moreover, we do not - * use any dynamic storage. - */ - - if(header->next()==header->impl()|| - header->next()->next()==header->impl())return; - - typedef typename Node::impl_type impl_type; - typedef typename Node::impl_pointer impl_pointer; - - typedef typename aligned_storage< - sizeof(impl_type), - alignment_of::value - >::type carry_spc_type; - carry_spc_type carry_spc; - impl_type& carry= - *reinterpret_cast(&carry_spc); - typedef typename aligned_storage< - sizeof( - impl_type - [sequenced_index_sort_max_fill]), - alignment_of< - impl_type - [sequenced_index_sort_max_fill] - >::value - >::type counter_spc_type; - counter_spc_type counter_spc; - impl_type* counter= - reinterpret_cast(&counter_spc); - std::size_t fill=0; - - carry.prior()=carry.next()=static_cast(&carry); - counter[0].prior()=counter[0].next()=static_cast(&counter[0]); - - BOOST_TRY{ - while(header->next()!=header->impl()){ - impl_type::relink(carry.next(),header->next()); - std::size_t i=0; - while(i(&counter[i])){ - sequenced_index_collate(&carry,&counter[i++],comp); - } - impl_type::swap( - static_cast(&carry), - static_cast(&counter[i])); - if(i==fill){ - ++fill; - counter[fill].prior()=counter[fill].next()= - static_cast(&counter[fill]); - } - } - - for(std::size_t i=1;i(&counter[i],&counter[i-1],comp); - } - impl_type::swap( - header->impl(),static_cast(&counter[fill-1])); - } - BOOST_CATCH(...) - { - impl_type::relink( - header->impl(),carry.next(),static_cast(&carry)); - for(std::size_t i=0;i<=fill;++i){ - impl_type::relink( - header->impl(),counter[i].next(), - static_cast(&counter[i])); - } - BOOST_RETHROW; - } - BOOST_CATCH_END -} - -#include - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/serialization_version.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/serialization_version.hpp deleted file mode 100644 index ccd8bb4f791..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/serialization_version.hpp +++ /dev/null @@ -1,73 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_SERIALIZATION_VERSION_HPP -#define BOOST_MULTI_INDEX_DETAIL_SERIALIZATION_VERSION_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* Helper class for storing and retrieving a given type serialization class - * version while avoiding saving the number multiple times in the same - * archive. - * Behavior undefined if template partial specialization is not supported. - */ - -template -struct serialization_version -{ - serialization_version(): - value(boost::serialization::version::value){} - - serialization_version& operator=(unsigned int x){value=x;return *this;}; - - operator unsigned int()const{return value;} - -private: - friend class boost::serialization::access; - - BOOST_SERIALIZATION_SPLIT_MEMBER() - - template - void save(Archive&,const unsigned int)const{} - - template - void load(Archive&,const unsigned int version) - { - this->value=version; - } - - unsigned int value; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -namespace serialization { -template -struct version > -{ - BOOST_STATIC_CONSTANT(int,value=version::value); -}; -} /* namespace serialization */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/uintptr_type.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/uintptr_type.hpp deleted file mode 100644 index 9c92d01d4de..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/uintptr_type.hpp +++ /dev/null @@ -1,76 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_UINTPTR_TYPE_HPP -#define BOOST_MULTI_INDEX_DETAIL_UINTPTR_TYPE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* has_uintptr_type is an MPL integral constant determining whether - * there exists an unsigned integral type with the same size as - * void *. - * uintptr_type is such a type if has_uintptr is true, or unsigned int - * otherwise. - * Note that uintptr_type is more restrictive than C99 uintptr_t, - * where an integral type with size greater than that of void * - * would be conformant. - */ - -templatestruct uintptr_candidates; -template<>struct uintptr_candidates<-1>{typedef unsigned int type;}; -template<>struct uintptr_candidates<0> {typedef unsigned int type;}; -template<>struct uintptr_candidates<1> {typedef unsigned short type;}; -template<>struct uintptr_candidates<2> {typedef unsigned long type;}; - -#if defined(BOOST_HAS_LONG_LONG) -template<>struct uintptr_candidates<3> {typedef boost::ulong_long_type type;}; -#else -template<>struct uintptr_candidates<3> {typedef unsigned int type;}; -#endif - -#if defined(BOOST_HAS_MS_INT64) -template<>struct uintptr_candidates<4> {typedef unsigned __int64 type;}; -#else -template<>struct uintptr_candidates<4> {typedef unsigned int type;}; -#endif - -struct uintptr_aux -{ - BOOST_STATIC_CONSTANT(int,index= - sizeof(void*)==sizeof(uintptr_candidates<0>::type)?0: - sizeof(void*)==sizeof(uintptr_candidates<1>::type)?1: - sizeof(void*)==sizeof(uintptr_candidates<2>::type)?2: - sizeof(void*)==sizeof(uintptr_candidates<3>::type)?3: - sizeof(void*)==sizeof(uintptr_candidates<4>::type)?4:-1); - - BOOST_STATIC_CONSTANT(bool,has_uintptr_type=(index>=0)); - - typedef uintptr_candidates::type type; -}; - -typedef mpl::bool_ has_uintptr_type; -typedef uintptr_aux::type uintptr_type; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/unbounded.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/unbounded.hpp deleted file mode 100644 index dc09be1770d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/unbounded.hpp +++ /dev/null @@ -1,66 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_UNBOUNDED_HPP -#define BOOST_MULTI_INDEX_DETAIL_UNBOUNDED_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -namespace boost{ - -namespace multi_index{ - -/* dummy type and variable for use in ordered_index::range() */ - -/* ODR-abiding technique shown at the example attached to - * http://lists.boost.org/Archives/boost/2006/07/108355.php - */ - -namespace detail{class unbounded_helper;} - -detail::unbounded_helper unbounded(detail::unbounded_helper); - -namespace detail{ - -class unbounded_helper -{ - unbounded_helper(){} - unbounded_helper(const unbounded_helper&){} - friend unbounded_helper multi_index::unbounded(unbounded_helper); -}; - -typedef unbounded_helper (*unbounded_type)(unbounded_helper); - -} /* namespace multi_index::detail */ - -inline detail::unbounded_helper unbounded(detail::unbounded_helper) -{ - return detail::unbounded_helper(); -} - -/* tags used in the implementation of range */ - -namespace detail{ - -struct none_unbounded_tag{}; -struct lower_unbounded_tag{}; -struct upper_unbounded_tag{}; -struct both_unbounded_tag{}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/value_compare.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/value_compare.hpp deleted file mode 100644 index ac42e8779aa..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/value_compare.hpp +++ /dev/null @@ -1,56 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_VALUE_COMPARE_HPP -#define BOOST_MULTI_INDEX_DETAIL_VALUE_COMPARE_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -struct value_comparison -{ - typedef Value first_argument_type; - typedef Value second_argument_type; - typedef bool result_type; - - value_comparison( - const KeyFromValue& key_=KeyFromValue(),const Compare& comp_=Compare()): - key(key_),comp(comp_) - { - } - - bool operator()( - typename call_traits::param_type x, - typename call_traits::param_type y)const - { - return comp(key(x),key(y)); - } - -private: - KeyFromValue key; - Compare comp; -}; - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/vartempl_support.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/detail/vartempl_support.hpp deleted file mode 100644 index 06ff430f4be..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/detail/vartempl_support.hpp +++ /dev/null @@ -1,247 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_DETAIL_VARTEMPL_SUPPORT_HPP -#define BOOST_MULTI_INDEX_DETAIL_VARTEMPL_SUPPORT_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -/* Utilities for emulation of variadic template functions. Variadic packs are - * replaced by lists of BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS parameters: - * - * - typename... Args --> BOOST_MULTI_INDEX_TEMPLATE_PARAM_PACK - * - Args&&... args --> BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK - * - std::forward(args)... --> BOOST_MULTI_INDEX_FORWARD_PARAM_PACK - * - * Forwarding emulated with Boost.Move. A template functions foo_imp - * defined in such way accepts *exactly* BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS - * arguments: variable number of arguments is emulated by providing a set of - * overloads foo forwarding to foo_impl with - * - * BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL - * BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG (initial extra arg) - * - * which fill the extra args with boost::multi_index::detail::noarg's. - * boost::multi_index::detail::vartempl_placement_new works the opposite - * way: it acceps a full a pointer x to Value and a - * BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK and forwards to - * new(x) Value(args) where args is the argument pack after discarding - * noarg's. - * - * Emulation decays to the real thing when the compiler supports variadic - * templates and move semantics natively. - */ - -#include - -#if defined(BOOST_NO_CXX11_RVALUE_REFERENCES)||\ - defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS) -#define BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS 5 -#endif - -#define BOOST_MULTI_INDEX_TEMPLATE_PARAM_PACK \ -BOOST_PP_ENUM_PARAMS( \ - BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,typename T) - -#define BOOST_MULTI_INDEX_VARTEMPL_ARG(z,n,_) \ -BOOST_FWD_REF(BOOST_PP_CAT(T,n)) BOOST_PP_CAT(t,n) - -#define BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK \ -BOOST_PP_ENUM( \ - BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS, \ - BOOST_MULTI_INDEX_VARTEMPL_ARG,~) - -#define BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG(z,n,_) \ -boost::forward(BOOST_PP_CAT(t,n)) - -#define BOOST_MULTI_INDEX_FORWARD_PARAM_PACK \ -BOOST_PP_ENUM( \ - BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS, \ - BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG,~) - -namespace boost{namespace multi_index{namespace detail{ -struct noarg{}; -}}} - -/* call vartempl function without args */ - -#define BOOST_MULTI_INDEX_NULL_PARAM_PACK \ -BOOST_PP_ENUM_PARAMS( \ - BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS, \ - boost::multi_index::detail::noarg() BOOST_PP_INTERCEPT) - -#define BOOST_MULTI_INDEX_TEMPLATE_N(n) \ -template - -#define BOOST_MULTI_INDEX_TEMPLATE_0(n) - -#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_AUX(z,n,data) \ -BOOST_PP_IF(n, \ - BOOST_MULTI_INDEX_TEMPLATE_N, \ - BOOST_MULTI_INDEX_TEMPLATE_0)(n) \ -BOOST_PP_SEQ_ELEM(0,data) /* ret */ \ -BOOST_PP_SEQ_ELEM(1,data) /* name_from */ ( \ - BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_ARG,~)) \ -{ \ - return BOOST_PP_SEQ_ELEM(2,data) /* name_to */ ( \ - BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG,~) \ - BOOST_PP_COMMA_IF( \ - BOOST_PP_AND( \ - n,BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n))) \ - BOOST_PP_ENUM_PARAMS( \ - BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n), \ - boost::multi_index::detail::noarg() BOOST_PP_INTERCEPT) \ - ); \ -} - -#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( \ - ret,name_from,name_to) \ -BOOST_PP_REPEAT_FROM_TO( \ - 0,BOOST_PP_ADD(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,1), \ - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_AUX, \ - (ret)(name_from)(name_to)) - -#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG_AUX( \ - z,n,data) \ -BOOST_PP_IF(n, \ - BOOST_MULTI_INDEX_TEMPLATE_N, \ - BOOST_MULTI_INDEX_TEMPLATE_0)(n) \ -BOOST_PP_SEQ_ELEM(0,data) /* ret */ \ -BOOST_PP_SEQ_ELEM(1,data) /* name_from */ ( \ - BOOST_PP_SEQ_ELEM(3,data) BOOST_PP_SEQ_ELEM(4,data) /* extra arg */\ - BOOST_PP_COMMA_IF(n) \ - BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_ARG,~)) \ -{ \ - return BOOST_PP_SEQ_ELEM(2,data) /* name_to */ ( \ - BOOST_PP_SEQ_ELEM(4,data) /* extra_arg_name */ \ - BOOST_PP_COMMA_IF(n) \ - BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG,~) \ - BOOST_PP_COMMA_IF( \ - BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n)) \ - BOOST_PP_ENUM_PARAMS( \ - BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n), \ - boost::multi_index::detail::noarg() BOOST_PP_INTERCEPT) \ - ); \ -} - -#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( \ - ret,name_from,name_to,extra_arg_type,extra_arg_name) \ -BOOST_PP_REPEAT_FROM_TO( \ - 0,BOOST_PP_ADD(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,1), \ - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG_AUX, \ - (ret)(name_from)(name_to)(extra_arg_type)(extra_arg_name)) - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -#define BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW_AUX(z,n,name) \ -template< \ - typename Value \ - BOOST_PP_COMMA_IF(n) \ - BOOST_PP_ENUM_PARAMS(n,typename T) \ -> \ -Value* name( \ - Value* x \ - BOOST_PP_COMMA_IF(n) \ - BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_ARG,~) \ - BOOST_PP_COMMA_IF( \ - BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n)) \ - BOOST_PP_ENUM_PARAMS( \ - BOOST_PP_SUB(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,n), \ - BOOST_FWD_REF(noarg) BOOST_PP_INTERCEPT)) \ -{ \ - return new(x) Value( \ - BOOST_PP_ENUM(n,BOOST_MULTI_INDEX_VARTEMPL_FORWARD_ARG,~)); \ -} - -#define BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW(name) \ -BOOST_PP_REPEAT_FROM_TO( \ - 0,BOOST_PP_ADD(BOOST_MULTI_INDEX_LIMIT_VARTEMPL_ARGS,1), \ - BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW_AUX, \ - name) - -BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW(vartempl_placement_new) - -#undef BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW_AUX -#undef BOOST_MULTI_INDEX_VARTEMPL_TO_PLACEMENT_NEW - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#else - -/* native variadic templates support */ - -#include - -#define BOOST_MULTI_INDEX_TEMPLATE_PARAM_PACK typename... Args -#define BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK Args&&... args -#define BOOST_MULTI_INDEX_FORWARD_PARAM_PACK std::forward(args)... -#define BOOST_MULTI_INDEX_NULL_PARAM_PACK - -#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( \ - ret,name_from,name_to) \ -template ret name_from(Args&&... args) \ -{ \ - return name_to(std::forward(args)...); \ -} - -#define BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( \ - ret,name_from,name_to,extra_arg_type,extra_arg_name) \ -template ret name_from( \ - extra_arg_type extra_arg_name,Args&&... args) \ -{ \ - return name_to(extra_arg_name,std::forward(args)...); \ -} - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -Value* vartempl_placement_new(Value*x,Args&&... args) -{ - return new(x) Value(std::forward(args)...); -} - -} /* namespace multi_index::detail */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/global_fun.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/global_fun.hpp deleted file mode 100644 index 2c13769100c..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/global_fun.hpp +++ /dev/null @@ -1,185 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_GLOBAL_FUN_HPP -#define BOOST_MULTI_INDEX_GLOBAL_FUN_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_SFINAE) -#include -#endif - -namespace boost{ - -template class reference_wrapper; /* fwd decl. */ - -namespace multi_index{ - -namespace detail{ - -/* global_fun is a read-only key extractor from Value based on a given global - * (or static member) function with signature: - * - * Type f([const] Value [&]); - * - * Additionally, global_fun and const_global_fun are overloaded to support - * referece_wrappers of Value and "chained pointers" to Value's. By chained - * pointer to T we mean a type P such that, given a p of Type P - * *...n...*x is convertible to T&, for some n>=1. - * Examples of chained pointers are raw and smart pointers, iterators and - * arbitrary combinations of these (vg. T** or unique_ptr.) - */ - -template -struct const_ref_global_fun_base -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(Value x)const - { - return PtrToFunction(x); - } - - Type operator()( - const reference_wrapper< - typename remove_reference::type>& x)const - { - return operator()(x.get()); - } - - Type operator()( - const reference_wrapper< - typename remove_const< - typename remove_reference::type>::type>& x - -#if BOOST_WORKAROUND(BOOST_MSVC,==1310) -/* http://lists.boost.org/Archives/boost/2015/10/226135.php */ - ,int=0 -#endif - - )const - { - return operator()(x.get()); - } -}; - -template -struct non_const_ref_global_fun_base -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(Value x)const - { - return PtrToFunction(x); - } - - Type operator()( - const reference_wrapper< - typename remove_reference::type>& x)const - { - return operator()(x.get()); - } -}; - -template -struct non_ref_global_fun_base -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(const Value& x)const - { - return PtrToFunction(x); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type operator()( - const reference_wrapper::type>& x)const - { - return operator()(x.get()); - } -}; - -} /* namespace multi_index::detail */ - -template -struct global_fun: - mpl::if_c< - is_reference::value, - typename mpl::if_c< - is_const::type>::value, - detail::const_ref_global_fun_base, - detail::non_const_ref_global_fun_base - >::type, - detail::non_ref_global_fun_base - >::type -{ -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index.hpp deleted file mode 100644 index 352d0c13f17..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index.hpp +++ /dev/null @@ -1,1725 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_HASHED_INDEX_HPP -#define BOOST_MULTI_INDEX_HASHED_INDEX_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) -#include -#endif - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) -#define BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF(x) \ - detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ - detail::make_obj_guard(x,&hashed_index::check_invariant_); \ - BOOST_JOIN(check_invariant_,__LINE__).touch(); -#define BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT \ - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF(*this) -#else -#define BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF(x) -#define BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* hashed_index adds a layer of hashed indexing to a given Super */ - -/* Most of the implementation of unique and non-unique indices is - * shared. We tell from one another on instantiation time by using - * Category tags defined in hash_index_node.hpp. - */ - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -class hashed_index: - BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS SuperMeta::type - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - ,public safe_mode::safe_container< - hashed_index > -#endif - -{ -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the - * lifetime of const references bound to temporaries --precisely what - * scopeguards are. - */ - -#pragma parse_mfunc_templ off -#endif - - typedef typename SuperMeta::type super; - -protected: - typedef hashed_index_node< - typename super::node_type,Category> node_type; - -private: - typedef typename node_type::node_alg node_alg; - typedef typename node_type::impl_type node_impl_type; - typedef typename node_impl_type::pointer node_impl_pointer; - typedef typename node_impl_type::base_pointer node_impl_base_pointer; - typedef bucket_array< - typename super::final_allocator_type> bucket_array_type; - -public: - /* types */ - - typedef typename KeyFromValue::result_type key_type; - typedef typename node_type::value_type value_type; - typedef KeyFromValue key_from_value; - typedef Hash hasher; - typedef Pred key_equal; - typedef tuple ctor_args; - typedef typename super::final_allocator_type allocator_type; - typedef typename allocator_type::pointer pointer; - typedef typename allocator_type::const_pointer const_pointer; - typedef typename allocator_type::reference reference; - typedef typename allocator_type::const_reference const_reference; - typedef std::size_t size_type; - typedef std::ptrdiff_t difference_type; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_iterator< - hashed_index_iterator< - node_type,bucket_array_type, - hashed_index_global_iterator_tag>, - hashed_index> iterator; -#else - typedef hashed_index_iterator< - node_type,bucket_array_type, - hashed_index_global_iterator_tag> iterator; -#endif - - typedef iterator const_iterator; - - typedef hashed_index_iterator< - node_type,bucket_array_type, - hashed_index_local_iterator_tag> local_iterator; - typedef local_iterator const_local_iterator; - - typedef TagList tag_list; - -protected: - typedef typename super::final_node_type final_node_type; - typedef tuples::cons< - ctor_args, - typename super::ctor_args_list> ctor_args_list; - typedef typename mpl::push_front< - typename super::index_type_list, - hashed_index>::type index_type_list; - typedef typename mpl::push_front< - typename super::iterator_type_list, - iterator>::type iterator_type_list; - typedef typename mpl::push_front< - typename super::const_iterator_type_list, - const_iterator>::type const_iterator_type_list; - typedef typename super::copy_map_type copy_map_type; - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - typedef typename super::index_saver_type index_saver_type; - typedef typename super::index_loader_type index_loader_type; -#endif - -private: -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_container< - hashed_index> safe_super; -#endif - - typedef typename call_traits::param_type value_param_type; - typedef typename call_traits< - key_type>::param_type key_param_type; - - /* Needed to avoid commas in BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL - * expansion. - */ - - typedef std::pair emplace_return_type; - -public: - - /* construct/destroy/copy - * Default and copy ctors are in the protected section as indices are - * not supposed to be created on their own. No range ctor either. - */ - - hashed_index& operator=( - const hashed_index& x) - { - this->final()=x.final(); - return *this; - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - hashed_index& operator=( - std::initializer_list list) - { - this->final()=list; - return *this; - } -#endif - - allocator_type get_allocator()const BOOST_NOEXCEPT - { - return this->final().get_allocator(); - } - - /* size and capacity */ - - bool empty()const BOOST_NOEXCEPT{return this->final_empty_();} - size_type size()const BOOST_NOEXCEPT{return this->final_size_();} - size_type max_size()const BOOST_NOEXCEPT{return this->final_max_size_();} - - /* iterators */ - - iterator begin()BOOST_NOEXCEPT - {return make_iterator(node_type::from_impl(header()->next()->prior()));} - const_iterator begin()const BOOST_NOEXCEPT - {return make_iterator(node_type::from_impl(header()->next()->prior()));} - iterator end()BOOST_NOEXCEPT{return make_iterator(header());} - const_iterator end()const BOOST_NOEXCEPT{return make_iterator(header());} - const_iterator cbegin()const BOOST_NOEXCEPT{return begin();} - const_iterator cend()const BOOST_NOEXCEPT{return end();} - - iterator iterator_to(const value_type& x) - { - return make_iterator(node_from_value(&x)); - } - - const_iterator iterator_to(const value_type& x)const - { - return make_iterator(node_from_value(&x)); - } - - /* modifiers */ - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( - emplace_return_type,emplace,emplace_impl) - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( - iterator,emplace_hint,emplace_hint_impl,iterator,position) - - std::pair insert(const value_type& x) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_(x); - return std::pair(make_iterator(p.first),p.second); - } - - std::pair insert(BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_rv_(x); - return std::pair(make_iterator(p.first),p.second); - } - - iterator insert(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_( - x,static_cast(position.get_node())); - return make_iterator(p.first); - } - - iterator insert(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_rv_( - x,static_cast(position.get_node())); - return make_iterator(p.first); - } - - template - void insert(InputIterator first,InputIterator last) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - for(;first!=last;++first)this->final_insert_ref_(*first); - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - void insert(std::initializer_list list) - { - insert(list.begin(),list.end()); - } -#endif - - iterator erase(iterator position) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - this->final_erase_(static_cast(position++.get_node())); - return position; - } - - size_type erase(key_param_type k) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - - std::size_t buc=buckets.position(hash_(k)); - for(node_impl_pointer x=buckets.at(buc)->prior(); - x!=node_impl_pointer(0);x=node_alg::next_to_inspect(x)){ - if(eq_(k,key(node_type::from_impl(x)->value()))){ - node_impl_pointer y=end_of_range(x); - size_type s=0; - do{ - node_impl_pointer z=node_alg::after(x); - this->final_erase_( - static_cast(node_type::from_impl(x))); - x=z; - ++s; - }while(x!=y); - return s; - } - } - return 0; - } - - iterator erase(iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - while(first!=last){ - first=erase(first); - } - return first; - } - - bool replace(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - return this->final_replace_( - x,static_cast(position.get_node())); - } - - bool replace(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - return this->final_replace_rv_( - x,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod,Rollback back_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,back_,static_cast(position.get_node())); - } - - template - bool modify_key(iterator position,Modifier mod) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - return modify( - position,modify_key_adaptor(mod,key)); - } - - template - bool modify_key(iterator position,Modifier mod,Rollback back_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - return modify( - position, - modify_key_adaptor(mod,key), - modify_key_adaptor(back_,key)); - } - - void clear()BOOST_NOEXCEPT - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - this->final_clear_(); - } - - void swap(hashed_index& x) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF(x); - this->final_swap_(x.final()); - } - - /* observers */ - - key_from_value key_extractor()const{return key;} - hasher hash_function()const{return hash_;} - key_equal key_eq()const{return eq_;} - - /* lookup */ - - /* Internally, these ops rely on const_iterator being the same - * type as iterator. - */ - - /* Implementation note: When CompatibleKey is consistently promoted to - * KeyFromValue::result_type for equality comparison, the promotion is made - * once in advance to increase efficiency. - */ - - template - iterator find(const CompatibleKey& k)const - { - return find(k,hash_,eq_); - } - - template< - typename CompatibleKey,typename CompatibleHash,typename CompatiblePred - > - iterator find( - const CompatibleKey& k, - const CompatibleHash& hash,const CompatiblePred& eq)const - { - return find( - k,hash,eq,promotes_1st_arg()); - } - - template - size_type count(const CompatibleKey& k)const - { - return count(k,hash_,eq_); - } - - template< - typename CompatibleKey,typename CompatibleHash,typename CompatiblePred - > - size_type count( - const CompatibleKey& k, - const CompatibleHash& hash,const CompatiblePred& eq)const - { - return count( - k,hash,eq,promotes_1st_arg()); - } - - template - std::pair equal_range(const CompatibleKey& k)const - { - return equal_range(k,hash_,eq_); - } - - template< - typename CompatibleKey,typename CompatibleHash,typename CompatiblePred - > - std::pair equal_range( - const CompatibleKey& k, - const CompatibleHash& hash,const CompatiblePred& eq)const - { - return equal_range( - k,hash,eq,promotes_1st_arg()); - } - - /* bucket interface */ - - size_type bucket_count()const BOOST_NOEXCEPT{return buckets.size();} - size_type max_bucket_count()const BOOST_NOEXCEPT{return static_cast(-1);} - - size_type bucket_size(size_type n)const - { - size_type res=0; - for(node_impl_pointer x=buckets.at(n)->prior(); - x!=node_impl_pointer(0);x=node_alg::after_local(x)){ - ++res; - } - return res; - } - - size_type bucket(key_param_type k)const - { - return buckets.position(hash_(k)); - } - - local_iterator begin(size_type n) - { - return const_cast(this)->begin(n); - } - - const_local_iterator begin(size_type n)const - { - node_impl_pointer x=buckets.at(n)->prior(); - if(x==node_impl_pointer(0))return end(n); - return make_local_iterator(node_type::from_impl(x)); - } - - local_iterator end(size_type n) - { - return const_cast(this)->end(n); - } - - const_local_iterator end(size_type)const - { - return make_local_iterator(0); - } - - const_local_iterator cbegin(size_type n)const{return begin(n);} - const_local_iterator cend(size_type n)const{return end(n);} - - local_iterator local_iterator_to(const value_type& x) - { - return make_local_iterator(node_from_value(&x)); - } - - const_local_iterator local_iterator_to(const value_type& x)const - { - return make_local_iterator(node_from_value(&x)); - } - - /* hash policy */ - - float load_factor()const BOOST_NOEXCEPT - {return static_cast(size())/bucket_count();} - float max_load_factor()const BOOST_NOEXCEPT{return mlf;} - void max_load_factor(float z){mlf=z;calculate_max_load();} - - void rehash(size_type n) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - if(size()<=max_load&&n<=bucket_count())return; - - size_type bc =(std::numeric_limits::max)(); - float fbc=static_cast(1+size()/mlf); - if(bc>fbc){ - bc=static_cast(fbc); - if(bc(std::ceil(static_cast(n)/mlf))); - } - -BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: - hashed_index(const ctor_args_list& args_list,const allocator_type& al): - super(args_list.get_tail(),al), - key(tuples::get<1>(args_list.get_head())), - hash_(tuples::get<2>(args_list.get_head())), - eq_(tuples::get<3>(args_list.get_head())), - buckets(al,header()->impl(),tuples::get<0>(args_list.get_head())), - mlf(1.0f) - { - calculate_max_load(); - } - - hashed_index( - const hashed_index& x): - super(x), - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super(), -#endif - - key(x.key), - hash_(x.hash_), - eq_(x.eq_), - buckets(x.get_allocator(),header()->impl(),x.buckets.size()), - mlf(x.mlf), - max_load(x.max_load) - { - /* Copy ctor just takes the internal configuration objects from x. The rest - * is done in subsequent call to copy_(). - */ - } - - hashed_index( - const hashed_index& x, - do_not_copy_elements_tag): - super(x,do_not_copy_elements_tag()), - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super(), -#endif - - key(x.key), - hash_(x.hash_), - eq_(x.eq_), - buckets(x.get_allocator(),header()->impl(),0), - mlf(1.0f) - { - calculate_max_load(); - } - - ~hashed_index() - { - /* the container is guaranteed to be empty by now */ - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - iterator make_iterator(node_type* node) - { - return iterator(node,this); - } - - const_iterator make_iterator(node_type* node)const - { - return const_iterator(node,const_cast(this)); - } -#else - iterator make_iterator(node_type* node) - { - return iterator(node); - } - - const_iterator make_iterator(node_type* node)const - { - return const_iterator(node); - } -#endif - - local_iterator make_local_iterator(node_type* node) - { - return local_iterator(node); - } - - const_local_iterator make_local_iterator(node_type* node)const - { - return const_local_iterator(node); - } - - void copy_( - const hashed_index& x, - const copy_map_type& map) - { - copy_(x,map,Category()); - } - - void copy_( - const hashed_index& x, - const copy_map_type& map,hashed_unique_tag) - { - if(x.size()!=0){ - node_impl_pointer end_org=x.header()->impl(), - org=end_org, - cpy=header()->impl(); - do{ - node_impl_pointer prev_org=org->prior(), - prev_cpy= - static_cast(map.find(static_cast( - node_type::from_impl(prev_org))))->impl(); - cpy->prior()=prev_cpy; - if(node_alg::is_first_of_bucket(org)){ - node_impl_base_pointer buc_org=prev_org->next(), - buc_cpy= - buckets.begin()+(buc_org-x.buckets.begin()); - prev_cpy->next()=buc_cpy; - buc_cpy->prior()=cpy; - } - else{ - prev_cpy->next()=node_impl_type::base_pointer_from(cpy); - } - org=prev_org; - cpy=prev_cpy; - }while(org!=end_org); - } - - super::copy_(x,map); - } - - void copy_( - const hashed_index& x, - const copy_map_type& map,hashed_non_unique_tag) - { - if(x.size()!=0){ - node_impl_pointer end_org=x.header()->impl(), - org=end_org, - cpy=header()->impl(); - do{ - node_impl_pointer next_org=node_alg::after(org), - next_cpy= - static_cast(map.find(static_cast( - node_type::from_impl(next_org))))->impl(); - if(node_alg::is_first_of_bucket(next_org)){ - node_impl_base_pointer buc_org=org->next(), - buc_cpy= - buckets.begin()+(buc_org-x.buckets.begin()); - cpy->next()=buc_cpy; - buc_cpy->prior()=next_cpy; - next_cpy->prior()=cpy; - } - else{ - if(org->next()==node_impl_type::base_pointer_from(next_org)){ - cpy->next()=node_impl_type::base_pointer_from(next_cpy); - } - else{ - cpy->next()= - node_impl_type::base_pointer_from( - static_cast(map.find(static_cast( - node_type::from_impl( - node_impl_type::pointer_from(org->next())))))->impl()); - } - - if(next_org->prior()!=org){ - next_cpy->prior()= - static_cast(map.find(static_cast( - node_type::from_impl(next_org->prior()))))->impl(); - } - else{ - next_cpy->prior()=cpy; - } - } - org=next_org; - cpy=next_cpy; - }while(org!=end_org); - } - - super::copy_(x,map); - } - - template - final_node_type* insert_( - value_param_type v,final_node_type*& x,Variant variant) - { - reserve_for_insert(size()+1); - - std::size_t buc=find_bucket(v); - link_info pos(buckets.at(buc)); - if(!link_point(v,pos)){ - return static_cast( - node_type::from_impl(node_impl_type::pointer_from(pos))); - } - - final_node_type* res=super::insert_(v,x,variant); - if(res==x)link(static_cast(x),pos); - return res; - } - - template - final_node_type* insert_( - value_param_type v,node_type* position,final_node_type*& x,Variant variant) - { - reserve_for_insert(size()+1); - - std::size_t buc=find_bucket(v); - link_info pos(buckets.at(buc)); - if(!link_point(v,pos)){ - return static_cast( - node_type::from_impl(node_impl_type::pointer_from(pos))); - } - - final_node_type* res=super::insert_(v,position,x,variant); - if(res==x)link(static_cast(x),pos); - return res; - } - - void erase_(node_type* x) - { - unlink(x); - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - } - - void delete_all_nodes_() - { - delete_all_nodes_(Category()); - } - - void delete_all_nodes_(hashed_unique_tag) - { - for(node_impl_pointer x_end=header()->impl(),x=x_end->prior();x!=x_end;){ - node_impl_pointer y=x->prior(); - this->final_delete_node_( - static_cast(node_type::from_impl(x))); - x=y; - } - } - - void delete_all_nodes_(hashed_non_unique_tag) - { - for(node_impl_pointer x_end=header()->impl(),x=x_end->prior();x!=x_end;){ - node_impl_pointer y=x->prior(); - if(y->next()!=node_impl_type::base_pointer_from(x)&& - y->next()->prior()!=x){ /* n-1 of group */ - /* Make the second node prior() pointer back-linked so that it won't - * refer to a deleted node when the time for its own destruction comes. - */ - - node_impl_pointer first=node_impl_type::pointer_from(y->next()); - first->next()->prior()=first; - } - this->final_delete_node_( - static_cast(node_type::from_impl(x))); - x=y; - } - } - - void clear_() - { - super::clear_(); - buckets.clear(header()->impl()); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::detach_dereferenceable_iterators(); -#endif - } - - void swap_( - hashed_index& x) - { - std::swap(key,x.key); - std::swap(hash_,x.hash_); - std::swap(eq_,x.eq_); - buckets.swap(x.buckets); - std::swap(mlf,x.mlf); - std::swap(max_load,x.max_load); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_(x); - } - - void swap_elements_( - hashed_index& x) - { - buckets.swap(x.buckets); - std::swap(mlf,x.mlf); - std::swap(max_load,x.max_load); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_elements_(x); - } - - template - bool replace_(value_param_type v,node_type* x,Variant variant) - { - if(eq_(key(v),key(x->value()))){ - return super::replace_(v,x,variant); - } - - unlink_undo undo; - unlink(x,undo); - - BOOST_TRY{ - std::size_t buc=find_bucket(v); - link_info pos(buckets.at(buc)); - if(link_point(v,pos)&&super::replace_(v,x,variant)){ - link(x,pos); - return true; - } - undo(); - return false; - } - BOOST_CATCH(...){ - undo(); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - bool modify_(node_type* x) - { - std::size_t buc; - bool b; - BOOST_TRY{ - buc=find_bucket(x->value()); - b=in_place(x->impl(),key(x->value()),buc); - } - BOOST_CATCH(...){ - erase_(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - if(!b){ - unlink(x); - BOOST_TRY{ - link_info pos(buckets.at(buc)); - if(!link_point(x->value(),pos)){ - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - return false; - } - link(x,pos); - } - BOOST_CATCH(...){ - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - BOOST_TRY{ - if(!super::modify_(x)){ - unlink(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - return false; - } - else return true; - } - BOOST_CATCH(...){ - unlink(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - bool modify_rollback_(node_type* x) - { - std::size_t buc=find_bucket(x->value()); - if(in_place(x->impl(),key(x->value()),buc)){ - return super::modify_rollback_(x); - } - - unlink_undo undo; - unlink(x,undo); - - BOOST_TRY{ - link_info pos(buckets.at(buc)); - if(link_point(x->value(),pos)&&super::modify_rollback_(x)){ - link(x,pos); - return true; - } - undo(); - return false; - } - BOOST_CATCH(...){ - undo(); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - /* comparison */ - -#if !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) - /* defect macro refers to class, not function, templates, but anyway */ - - template - friend bool operator==( - const hashed_index&,const hashed_index& y); -#endif - - bool equals(const hashed_index& x)const{return equals(x,Category());} - - bool equals(const hashed_index& x,hashed_unique_tag)const - { - if(size()!=x.size())return false; - for(const_iterator it=begin(),it_end=end(),it2_end=x.end(); - it!=it_end;++it){ - const_iterator it2=x.find(key(*it)); - if(it2==it2_end||!(*it==*it2))return false; - } - return true; - } - - bool equals(const hashed_index& x,hashed_non_unique_tag)const - { - if(size()!=x.size())return false; - for(const_iterator it=begin(),it_end=end();it!=it_end;){ - const_iterator it2,it2_last; - boost::tie(it2,it2_last)=x.equal_range(key(*it)); - if(it2==it2_last)return false; - - const_iterator it_last=make_iterator( - node_type::from_impl(end_of_range(it.get_node()->impl()))); - if(std::distance(it,it_last)!=std::distance(it2,it2_last))return false; - - /* From is_permutation code in - * http://www.open-std.org/jtc1/sc22/wg21/docs/papers/2010/n3068.pdf - */ - - for(;it!=it_last;++it,++it2){ - if(!(*it==*it2))break; - } - if(it!=it_last){ - for(const_iterator scan=it;scan!=it_last;++scan){ - if(std::find(it,scan,*scan)!=scan)continue; - std::ptrdiff_t matches=std::count(it2,it2_last,*scan); - if(matches==0||matches!=std::count(scan,it_last,*scan))return false; - } - it=it_last; - } - } - return true; - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* serialization */ - - template - void save_( - Archive& ar,const unsigned int version,const index_saver_type& sm)const - { - ar< - void load_(Archive& ar,const unsigned int version,const index_loader_type& lm) - { - ar>>serialization::make_nvp("position",buckets); - super::load_(ar,version,lm); - } -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - bool invariant_()const - { - if(size()==0||begin()==end()){ - if(size()!=0||begin()!=end())return false; - } - else{ - size_type s0=0; - for(const_iterator it=begin(),it_end=end();it!=it_end;++it,++s0){} - if(s0!=size())return false; - - size_type s1=0; - for(size_type buc=0;bucfinal_check_invariant_();} -#endif - -private: - node_type* header()const{return this->final_header();} - - std::size_t find_bucket(value_param_type v)const - { - return bucket(key(v)); - } - - struct link_info_non_unique - { - link_info_non_unique(node_impl_base_pointer pos): - first(pos),last(node_impl_base_pointer(0)){} - - operator const node_impl_base_pointer&()const{return this->first;} - - node_impl_base_pointer first,last; - }; - - typedef typename mpl::if_< - is_same, - node_impl_base_pointer, - link_info_non_unique - >::type link_info; - - bool link_point(value_param_type v,link_info& pos) - { - return link_point(v,pos,Category()); - } - - bool link_point( - value_param_type v,node_impl_base_pointer& pos,hashed_unique_tag) - { - for(node_impl_pointer x=pos->prior();x!=node_impl_pointer(0); - x=node_alg::after_local(x)){ - if(eq_(key(v),key(node_type::from_impl(x)->value()))){ - pos=node_impl_type::base_pointer_from(x); - return false; - } - } - return true; - } - - bool link_point( - value_param_type v,link_info_non_unique& pos,hashed_non_unique_tag) - { - for(node_impl_pointer x=pos.first->prior();x!=node_impl_pointer(0); - x=node_alg::next_to_inspect(x)){ - if(eq_(key(v),key(node_type::from_impl(x)->value()))){ - pos.first=node_impl_type::base_pointer_from(x); - pos.last=node_impl_type::base_pointer_from(last_of_range(x)); - return true; - } - } - return true; - } - - node_impl_pointer last_of_range(node_impl_pointer x)const - { - return last_of_range(x,Category()); - } - - node_impl_pointer last_of_range(node_impl_pointer x,hashed_unique_tag)const - { - return x; - } - - node_impl_pointer last_of_range( - node_impl_pointer x,hashed_non_unique_tag)const - { - node_impl_base_pointer y=x->next(); - node_impl_pointer z=y->prior(); - if(z==x){ /* range of size 1 or 2 */ - node_impl_pointer yy=node_impl_type::pointer_from(y); - return - eq_( - key(node_type::from_impl(x)->value()), - key(node_type::from_impl(yy)->value()))?yy:x; - } - else if(z->prior()==x) /* last of bucket */ - return x; - else /* group of size>2 */ - return z; - } - - node_impl_pointer end_of_range(node_impl_pointer x)const - { - return end_of_range(x,Category()); - } - - node_impl_pointer end_of_range(node_impl_pointer x,hashed_unique_tag)const - { - return node_alg::after(last_of_range(x)); - } - - node_impl_pointer end_of_range( - node_impl_pointer x,hashed_non_unique_tag)const - { - node_impl_base_pointer y=x->next(); - node_impl_pointer z=y->prior(); - if(z==x){ /* range of size 1 or 2 */ - node_impl_pointer yy=node_impl_type::pointer_from(y); - if(!eq_( - key(node_type::from_impl(x)->value()), - key(node_type::from_impl(yy)->value())))yy=x; - return yy->next()->prior()==yy? - node_impl_type::pointer_from(yy->next()): - yy->next()->prior(); - } - else if(z->prior()==x) /* last of bucket */ - return z; - else /* group of size>2 */ - return z->next()->prior()==z? - node_impl_type::pointer_from(z->next()): - z->next()->prior(); - } - - void link(node_type* x,const link_info& pos) - { - link(x,pos,Category()); - } - - void link(node_type* x,node_impl_base_pointer pos,hashed_unique_tag) - { - node_alg::link(x->impl(),pos,header()->impl()); - } - - void link(node_type* x,const link_info_non_unique& pos,hashed_non_unique_tag) - { - if(pos.last==node_impl_base_pointer(0)){ - node_alg::link(x->impl(),pos.first,header()->impl()); - } - else{ - node_alg::link( - x->impl(), - node_impl_type::pointer_from(pos.first), - node_impl_type::pointer_from(pos.last)); - } - } - - void unlink(node_type* x) - { - node_alg::unlink(x->impl()); - } - - typedef typename node_alg::unlink_undo unlink_undo; - - void unlink(node_type* x,unlink_undo& undo) - { - node_alg::unlink(x->impl(),undo); - } - - void calculate_max_load() - { - float fml=static_cast(mlf*static_cast(bucket_count())); - max_load=(std::numeric_limits::max)(); - if(max_load>fml)max_load=static_cast(fml); - } - - void reserve_for_insert(size_type n) - { - if(n>max_load){ - size_type bc =(std::numeric_limits::max)(); - float fbc=static_cast(1+static_cast(n)/mlf); - if(bc>fbc)bc =static_cast(fbc); - unchecked_rehash(bc); - } - } - - void unchecked_rehash(size_type n){unchecked_rehash(n,Category());} - - void unchecked_rehash(size_type n,hashed_unique_tag) - { - node_impl_type cpy_end_node; - node_impl_pointer cpy_end=node_impl_pointer(&cpy_end_node), - end_=header()->impl(); - bucket_array_type buckets_cpy(get_allocator(),cpy_end,n); - - if(size()!=0){ - auto_space< - std::size_t,allocator_type> hashes(get_allocator(),size()); - auto_space< - node_impl_pointer,allocator_type> node_ptrs(get_allocator(),size()); - std::size_t i=0,size_=size(); - bool within_bucket=false; - BOOST_TRY{ - for(;i!=size_;++i){ - node_impl_pointer x=end_->prior(); - - /* only this can possibly throw */ - std::size_t h=hash_(key(node_type::from_impl(x)->value())); - - hashes.data()[i]=h; - node_ptrs.data()[i]=x; - within_bucket=!node_alg::unlink_last(end_); - node_alg::link(x,buckets_cpy.at(buckets_cpy.position(h)),cpy_end); - } - } - BOOST_CATCH(...){ - if(i!=0){ - std::size_t prev_buc=buckets.position(hashes.data()[i-1]); - if(!within_bucket)prev_buc=~prev_buc; - - for(std::size_t j=i;j--;){ - std::size_t buc=buckets.position(hashes.data()[j]); - node_impl_pointer x=node_ptrs.data()[j]; - if(buc==prev_buc)node_alg::append(x,end_); - else node_alg::link(x,buckets.at(buc),end_); - prev_buc=buc; - } - } - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - end_->prior()=cpy_end->prior()!=cpy_end?cpy_end->prior():end_; - end_->next()=cpy_end->next(); - end_->prior()->next()->prior()=end_->next()->prior()->prior()=end_; - buckets.swap(buckets_cpy); - calculate_max_load(); - } - - void unchecked_rehash(size_type n,hashed_non_unique_tag) - { - node_impl_type cpy_end_node; - node_impl_pointer cpy_end=node_impl_pointer(&cpy_end_node), - end_=header()->impl(); - bucket_array_type buckets_cpy(get_allocator(),cpy_end,n); - - if(size()!=0){ - auto_space< - std::size_t,allocator_type> hashes(get_allocator(),size()); - auto_space< - node_impl_pointer,allocator_type> node_ptrs(get_allocator(),size()); - std::size_t i=0; - bool within_bucket=false; - BOOST_TRY{ - for(;;++i){ - node_impl_pointer x=end_->prior(); - if(x==end_)break; - - /* only this can possibly throw */ - std::size_t h=hash_(key(node_type::from_impl(x)->value())); - - hashes.data()[i]=h; - node_ptrs.data()[i]=x; - std::pair p= - node_alg::unlink_last_group(end_); - node_alg::link_range( - p.first,x,buckets_cpy.at(buckets_cpy.position(h)),cpy_end); - within_bucket=!(p.second); - } - } - BOOST_CATCH(...){ - if(i!=0){ - std::size_t prev_buc=buckets.position(hashes.data()[i-1]); - if(!within_bucket)prev_buc=~prev_buc; - - for(std::size_t j=i;j--;){ - std::size_t buc=buckets.position(hashes.data()[j]); - node_impl_pointer x=node_ptrs.data()[j], - y= - x->prior()->next()!=node_impl_type::base_pointer_from(x)&& - x->prior()->next()->prior()!=x? - node_impl_type::pointer_from(x->prior()->next()):x; - node_alg::unlink_range(y,x); - if(buc==prev_buc)node_alg::append_range(y,x,end_); - else node_alg::link_range(y,x,buckets.at(buc),end_); - prev_buc=buc; - } - } - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - end_->prior()=cpy_end->prior()!=cpy_end?cpy_end->prior():end_; - end_->next()=cpy_end->next(); - end_->prior()->next()->prior()=end_->next()->prior()->prior()=end_; - buckets.swap(buckets_cpy); - calculate_max_load(); - } - - bool in_place(node_impl_pointer x,key_param_type k,std::size_t buc)const - { - return in_place(x,k,buc,Category()); - } - - bool in_place( - node_impl_pointer x,key_param_type k,std::size_t buc, - hashed_unique_tag)const - { - bool found=false; - for(node_impl_pointer y=buckets.at(buc)->prior(); - y!=node_impl_pointer(0);y=node_alg::after_local(y)){ - if(y==x)found=true; - else if(eq_(k,key(node_type::from_impl(y)->value())))return false; - } - return found; - } - - bool in_place( - node_impl_pointer x,key_param_type k,std::size_t buc, - hashed_non_unique_tag)const - { - bool found=false; - int range_size=0; - for(node_impl_pointer y=buckets.at(buc)->prior();y!=node_impl_pointer(0);){ - if(node_alg::is_first_of_group(y)){ /* group of 3 or more */ - if(y==x){ - /* in place <-> equal to some other member of the group */ - return eq_( - k, - key(node_type::from_impl( - node_impl_type::pointer_from(y->next()))->value())); - } - else{ - node_impl_pointer z= - node_alg::after_local(y->next()->prior()); /* end of range */ - if(eq_(k,key(node_type::from_impl(y)->value()))){ - if(found)return false; /* x lies outside */ - do{ - if(y==x)return true; - y=node_alg::after_local(y); - }while(y!=z); - return false; /* x not found */ - } - else{ - if(range_size==1&&!found)return false; - if(range_size==2)return found; - range_size=0; - y=z; /* skip range (and potentially x, too, which is fine) */ - } - } - } - else{ /* group of 1 or 2 */ - if(y==x){ - if(range_size==1)return true; - range_size=1; - found=true; - } - else if(eq_(k,key(node_type::from_impl(y)->value()))){ - if(range_size==0&&found)return false; - if(range_size==1&&!found)return false; - if(range_size==2)return false; - ++range_size; - } - else{ - if(range_size==1&&!found)return false; - if(range_size==2)return found; - range_size=0; - } - y=node_alg::after_local(y); - } - } - return found; - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - void detach_iterators(node_type* x) - { - iterator it=make_iterator(x); - safe_mode::detach_equivalent_iterators(it); - } -#endif - - template - std::pair emplace_impl(BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - std::pairp= - this->final_emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - return std::pair(make_iterator(p.first),p.second); - } - - template - iterator emplace_hint_impl( - iterator position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT; - std::pairp= - this->final_emplace_hint_( - static_cast(position.get_node()), - BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - return make_iterator(p.first); - } - - template< - typename CompatibleHash,typename CompatiblePred - > - iterator find( - const key_type& k, - const CompatibleHash& hash,const CompatiblePred& eq,mpl::true_)const - { - return find(k,hash,eq,mpl::false_()); - } - - template< - typename CompatibleKey,typename CompatibleHash,typename CompatiblePred - > - iterator find( - const CompatibleKey& k, - const CompatibleHash& hash,const CompatiblePred& eq,mpl::false_)const - { - std::size_t buc=buckets.position(hash(k)); - for(node_impl_pointer x=buckets.at(buc)->prior(); - x!=node_impl_pointer(0);x=node_alg::next_to_inspect(x)){ - if(eq(k,key(node_type::from_impl(x)->value()))){ - return make_iterator(node_type::from_impl(x)); - } - } - return end(); - } - - template< - typename CompatibleHash,typename CompatiblePred - > - size_type count( - const key_type& k, - const CompatibleHash& hash,const CompatiblePred& eq,mpl::true_)const - { - return count(k,hash,eq,mpl::false_()); - } - - template< - typename CompatibleKey,typename CompatibleHash,typename CompatiblePred - > - size_type count( - const CompatibleKey& k, - const CompatibleHash& hash,const CompatiblePred& eq,mpl::false_)const - { - std::size_t buc=buckets.position(hash(k)); - for(node_impl_pointer x=buckets.at(buc)->prior(); - x!=node_impl_pointer(0);x=node_alg::next_to_inspect(x)){ - if(eq(k,key(node_type::from_impl(x)->value()))){ - size_type res=0; - node_impl_pointer y=end_of_range(x); - do{ - ++res; - x=node_alg::after(x); - }while(x!=y); - return res; - } - } - return 0; - } - - template< - typename CompatibleHash,typename CompatiblePred - > - std::pair equal_range( - const key_type& k, - const CompatibleHash& hash,const CompatiblePred& eq,mpl::true_)const - { - return equal_range(k,hash,eq,mpl::false_()); - } - - template< - typename CompatibleKey,typename CompatibleHash,typename CompatiblePred - > - std::pair equal_range( - const CompatibleKey& k, - const CompatibleHash& hash,const CompatiblePred& eq,mpl::false_)const - { - std::size_t buc=buckets.position(hash(k)); - for(node_impl_pointer x=buckets.at(buc)->prior(); - x!=node_impl_pointer(0);x=node_alg::next_to_inspect(x)){ - if(eq(k,key(node_type::from_impl(x)->value()))){ - return std::pair( - make_iterator(node_type::from_impl(x)), - make_iterator(node_type::from_impl(end_of_range(x)))); - } - } - return std::pair(end(),end()); - } - - key_from_value key; - hasher hash_; - key_equal eq_; - bucket_array_type buckets; - float mlf; - size_type max_load; - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -#pragma parse_mfunc_templ reset -#endif -}; - -/* comparison */ - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -bool operator==( - const hashed_index& x, - const hashed_index& y) -{ - return x.equals(y); -} - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -bool operator!=( - const hashed_index& x, - const hashed_index& y) -{ - return !(x==y); -} - -/* specialized algorithms */ - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -void swap( - hashed_index& x, - hashed_index& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -/* hashed index specifiers */ - -template -struct hashed_unique -{ - typedef typename detail::hashed_index_args< - Arg1,Arg2,Arg3,Arg4> index_args; - typedef typename index_args::tag_list_type::type tag_list_type; - typedef typename index_args::key_from_value_type key_from_value_type; - typedef typename index_args::hash_type hash_type; - typedef typename index_args::pred_type pred_type; - - template - struct node_class - { - typedef detail::hashed_index_node type; - }; - - template - struct index_class - { - typedef detail::hashed_index< - key_from_value_type,hash_type,pred_type, - SuperMeta,tag_list_type,detail::hashed_unique_tag> type; - }; -}; - -template -struct hashed_non_unique -{ - typedef typename detail::hashed_index_args< - Arg1,Arg2,Arg3,Arg4> index_args; - typedef typename index_args::tag_list_type::type tag_list_type; - typedef typename index_args::key_from_value_type key_from_value_type; - typedef typename index_args::hash_type hash_type; - typedef typename index_args::pred_type pred_type; - - template - struct node_class - { - typedef detail::hashed_index_node< - Super,detail::hashed_non_unique_tag> type; - }; - - template - struct index_class - { - typedef detail::hashed_index< - key_from_value_type,hash_type,pred_type, - SuperMeta,tag_list_type,detail::hashed_non_unique_tag> type; - }; -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -/* Boost.Foreach compatibility */ - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -inline boost::mpl::true_* boost_foreach_is_noncopyable( - boost::multi_index::detail::hashed_index< - KeyFromValue,Hash,Pred,SuperMeta,TagList,Category>*&, - boost_foreach_argument_dependent_lookup_hack) -{ - return 0; -} - -#undef BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT -#undef BOOST_MULTI_INDEX_HASHED_INDEX_CHECK_INVARIANT_OF - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index_fwd.hpp deleted file mode 100644 index d77e36c321b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/hashed_index_fwd.hpp +++ /dev/null @@ -1,74 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_HASHED_INDEX_FWD_HPP -#define BOOST_MULTI_INDEX_HASHED_INDEX_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -class hashed_index; - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -bool operator==( - const hashed_index& x, - const hashed_index& y); - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -bool operator!=( - const hashed_index& x, - const hashed_index& y); - -template< - typename KeyFromValue,typename Hash,typename Pred, - typename SuperMeta,typename TagList,typename Category -> -void swap( - hashed_index& x, - hashed_index& y); - -} /* namespace multi_index::detail */ - -/* hashed_index specifiers */ - -template< - typename Arg1,typename Arg2=mpl::na, - typename Arg3=mpl::na,typename Arg4=mpl::na -> -struct hashed_unique; - -template< - typename Arg1,typename Arg2=mpl::na, - typename Arg3=mpl::na,typename Arg4=mpl::na -> -struct hashed_non_unique; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/identity.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/identity.hpp deleted file mode 100644 index 6c832ce1562..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/identity.hpp +++ /dev/null @@ -1,145 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_IDENTITY_HPP -#define BOOST_MULTI_INDEX_IDENTITY_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_SFINAE) -#include -#endif - -namespace boost{ - -template class reference_wrapper; /* fwd decl. */ - -namespace multi_index{ - -namespace detail{ - -/* identity is a do-nothing key extractor that returns the [const] Type& - * object passed. - * Additionally, identity is overloaded to support referece_wrappers - * of Type and "chained pointers" to Type's. By chained pointer to Type we - * mean a type P such that, given a p of type P - * *...n...*x is convertible to Type&, for some n>=1. - * Examples of chained pointers are raw and smart pointers, iterators and - * arbitrary combinations of these (vg. Type** or unique_ptr.) - */ - -template -struct const_identity_base -{ - typedef Type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if,Type&>::type -#else - Type& -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type& operator()(Type& x)const - { - return x; - } - - Type& operator()(const reference_wrapper& x)const - { - return x.get(); - } - - Type& operator()( - const reference_wrapper::type>& x - -#if BOOST_WORKAROUND(BOOST_MSVC,==1310) -/* http://lists.boost.org/Archives/boost/2015/10/226135.php */ - ,int=0 -#endif - - )const - { - return x.get(); - } -}; - -template -struct non_const_identity_base -{ - typedef Type result_type; - - /* templatized for pointer-like types */ - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type&>::type -#else - Type& -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - const Type& operator()(const Type& x)const - { - return x; - } - - Type& operator()(Type& x)const - { - return x; - } - - const Type& operator()(const reference_wrapper& x)const - { - return x.get(); - } - - Type& operator()(const reference_wrapper& x)const - { - return x.get(); - } -}; - -} /* namespace multi_index::detail */ - -template -struct identity: - mpl::if_c< - is_const::value, - detail::const_identity_base,detail::non_const_identity_base - >::type -{ -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/identity_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/identity_fwd.hpp deleted file mode 100644 index af6bd55ef5f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/identity_fwd.hpp +++ /dev/null @@ -1,26 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_IDENTITY_FWD_HPP -#define BOOST_MULTI_INDEX_IDENTITY_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -namespace boost{ - -namespace multi_index{ - -template struct identity; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/indexed_by.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/indexed_by.hpp deleted file mode 100644 index d2217e39166..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/indexed_by.hpp +++ /dev/null @@ -1,68 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_INDEXED_BY_HPP -#define BOOST_MULTI_INDEX_INDEXED_BY_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include - -/* An alias to mpl::vector used to hide MPL from the user. - * indexed_by contains the index specifiers for instantiation - * of a multi_index_container. - */ - -/* This user_definable macro limits the number of elements of an index list; - * useful for shortening resulting symbol names (MSVC++ 6.0, for instance, - * has problems coping with very long symbol names.) - */ - -#if !defined(BOOST_MULTI_INDEX_LIMIT_INDEXED_BY_SIZE) -#define BOOST_MULTI_INDEX_LIMIT_INDEXED_BY_SIZE BOOST_MPL_LIMIT_VECTOR_SIZE -#endif - -#if BOOST_MULTI_INDEX_LIMIT_INDEXED_BY_SIZE -struct indexed_by: - mpl::vector -{ -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#undef BOOST_MULTI_INDEX_INDEXED_BY_TEMPLATE_PARM -#undef BOOST_MULTI_INDEX_INDEXED_BY_SIZE - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/key_extractors.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/key_extractors.hpp deleted file mode 100644 index 60179ba2339..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/key_extractors.hpp +++ /dev/null @@ -1,22 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_KEY_EXTRACTORS_HPP -#define BOOST_MULTI_INDEX_KEY_EXTRACTORS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include -#include -#include -#include -#include - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/mem_fun.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/mem_fun.hpp deleted file mode 100644 index 111c386c5f5..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/mem_fun.hpp +++ /dev/null @@ -1,205 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_MEM_FUN_HPP -#define BOOST_MULTI_INDEX_MEM_FUN_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -#if !defined(BOOST_NO_SFINAE) -#include -#endif - -namespace boost{ - -template class reference_wrapper; /* fwd decl. */ - -namespace multi_index{ - -/* mem_fun implements a read-only key extractor based on a given non-const - * member function of a class. - * const_mem_fun does the same for const member functions. - * Additionally, mem_fun and const_mem_fun are overloaded to support - * referece_wrappers of T and "chained pointers" to T's. By chained pointer - * to T we mean a type P such that, given a p of Type P - * *...n...*x is convertible to T&, for some n>=1. - * Examples of chained pointers are raw and smart pointers, iterators and - * arbitrary combinations of these (vg. T** or unique_ptr.) - */ - -template -struct const_mem_fun -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(const Class& x)const - { - return (x.*PtrToMemberFunction)(); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -template -struct mem_fun -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(Class& x)const - { - return (x.*PtrToMemberFunction)(); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -/* MSVC++ 6.0 has problems with const member functions as non-type template - * parameters, somehow it takes them as non-const. const_mem_fun_explicit - * workarounds this deficiency by accepting an extra type parameter that - * specifies the signature of the member function. The workaround was found at: - * Daniel, C.:"Re: weird typedef problem in VC", - * news:microsoft.public.vc.language, 21st nov 2002, - * http://groups.google.com/groups? - * hl=en&lr=&ie=UTF-8&selm=ukwvg3O0BHA.1512%40tkmsftngp05 - * - * MSVC++ 6.0 support has been dropped and [const_]mem_fun_explicit is - * deprecated. - */ - -template< - class Class,typename Type, - typename PtrToMemberFunctionType,PtrToMemberFunctionType PtrToMemberFunction> -struct const_mem_fun_explicit -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(const Class& x)const - { - return (x.*PtrToMemberFunction)(); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -template< - class Class,typename Type, - typename PtrToMemberFunctionType,PtrToMemberFunctionType PtrToMemberFunction> -struct mem_fun_explicit -{ - typedef typename remove_reference::type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type>::type -#else - Type -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type operator()(Class& x)const - { - return (x.*PtrToMemberFunction)(); - } - - Type operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -/* BOOST_MULTI_INDEX_CONST_MEM_FUN and BOOST_MULTI_INDEX_MEM_FUN used to - * resolve to [const_]mem_fun_explicit for MSVC++ 6.0 and to - * [const_]mem_fun otherwise. Support for this compiler having been dropped, - * they are now just wrappers over [const_]mem_fun kept for backwards- - * compatibility reasons. - */ - -#define BOOST_MULTI_INDEX_CONST_MEM_FUN(Class,Type,MemberFunName) \ -::boost::multi_index::const_mem_fun< Class,Type,&Class::MemberFunName > -#define BOOST_MULTI_INDEX_MEM_FUN(Class,Type,MemberFunName) \ -::boost::multi_index::mem_fun< Class,Type,&Class::MemberFunName > - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/member.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/member.hpp deleted file mode 100644 index a8e645074a2..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/member.hpp +++ /dev/null @@ -1,262 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_MEMBER_HPP -#define BOOST_MULTI_INDEX_MEMBER_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include - -#if !defined(BOOST_NO_SFINAE) -#include -#endif - -namespace boost{ - -template class reference_wrapper; /* fwd decl. */ - -namespace multi_index{ - -namespace detail{ - -/* member is a read/write key extractor for accessing a given - * member of a class. - * Additionally, member is overloaded to support referece_wrappers - * of T and "chained pointers" to T's. By chained pointer to T we mean - * a type P such that, given a p of Type P - * *...n...*x is convertible to T&, for some n>=1. - * Examples of chained pointers are raw and smart pointers, iterators and - * arbitrary combinations of these (vg. T** or unique_ptr.) - */ - -template -struct const_member_base -{ - typedef Type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type&>::type -#else - Type& -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type& operator()(const Class& x)const - { - return x.*PtrToMember; - } - - Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -template -struct non_const_member_base -{ - typedef Type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type&>::type -#else - Type& -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - const Type& operator()(const Class& x)const - { - return x.*PtrToMember; - } - - Type& operator()(Class& x)const - { - return x.*PtrToMember; - } - - const Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -} /* namespace multi_index::detail */ - -template -struct member: - mpl::if_c< - is_const::value, - detail::const_member_base, - detail::non_const_member_base - >::type -{ -}; - -namespace detail{ - -/* MSVC++ 6.0 does not support properly pointers to members as - * non-type template arguments, as reported in - * http://support.microsoft.com/default.aspx?scid=kb;EN-US;249045 - * A similar problem (though not identical) is shown by MSVC++ 7.0. - * We provide an alternative to member<> accepting offsets instead - * of pointers to members. This happens to work even for non-POD - * types (although the standard forbids use of offsetof on these), - * so it serves as a workaround in this compiler for all practical - * purposes. - * Surprisingly enough, other compilers, like Intel C++ 7.0/7.1 and - * Visual Age 6.0, have similar bugs. This replacement of member<> - * can be used for them too. - * - * Support for such old compilers is dropped and - * [non_]const_member_offset_base is deprecated. - */ - -template -struct const_member_offset_base -{ - typedef Type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type&>::type -#else - Type& -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - Type& operator()(const Class& x)const - { - return *static_cast( - static_cast( - static_cast( - static_cast(&x))+OffsetOfMember)); - } - - Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -template -struct non_const_member_offset_base -{ - typedef Type result_type; - - template - -#if !defined(BOOST_NO_SFINAE) - typename disable_if< - is_convertible,Type&>::type -#else - Type& -#endif - - operator()(const ChainedPtr& x)const - { - return operator()(*x); - } - - const Type& operator()(const Class& x)const - { - return *static_cast( - static_cast( - static_cast( - static_cast(&x))+OffsetOfMember)); - } - - Type& operator()(Class& x)const - { - return *static_cast( - static_cast( - static_cast(static_cast(&x))+OffsetOfMember)); - } - - const Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } - - Type& operator()(const reference_wrapper& x)const - { - return operator()(x.get()); - } -}; - -} /* namespace multi_index::detail */ - -template -struct member_offset: - mpl::if_c< - is_const::value, - detail::const_member_offset_base, - detail::non_const_member_offset_base - >::type -{ -}; - -/* BOOST_MULTI_INDEX_MEMBER resolves to member in the normal cases, - * and to member_offset as a workaround in those defective compilers for - * which BOOST_NO_POINTER_TO_MEMBER_TEMPLATE_PARAMETERS is defined. - */ - -#if defined(BOOST_NO_POINTER_TO_MEMBER_TEMPLATE_PARAMETERS) -#define BOOST_MULTI_INDEX_MEMBER(Class,Type,MemberName) \ -::boost::multi_index::member_offset< Class,Type,offsetof(Class,MemberName) > -#else -#define BOOST_MULTI_INDEX_MEMBER(Class,Type,MemberName) \ -::boost::multi_index::member< Class,Type,&Class::MemberName > -#endif - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index.hpp deleted file mode 100644 index 5bcd69de8c9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index.hpp +++ /dev/null @@ -1,114 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_ORDERED_INDEX_HPP -#define BOOST_MULTI_INDEX_ORDERED_INDEX_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* no augment policy for plain ordered indices */ - -struct null_augment_policy -{ - template - struct augmented_interface - { - typedef OrderedIndexImpl type; - }; - - template - struct augmented_node - { - typedef OrderedIndexNodeImpl type; - }; - - template static void add(Pointer,Pointer){} - template static void remove(Pointer,Pointer){} - template static void copy(Pointer,Pointer){} - template static void rotate_left(Pointer,Pointer){} - template static void rotate_right(Pointer,Pointer){} - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - template static bool invariant(Pointer){return true;} - -#endif -}; - -} /* namespace multi_index::detail */ - -/* ordered_index specifiers */ - -template -struct ordered_unique -{ - typedef typename detail::ordered_index_args< - Arg1,Arg2,Arg3> index_args; - typedef typename index_args::tag_list_type::type tag_list_type; - typedef typename index_args::key_from_value_type key_from_value_type; - typedef typename index_args::compare_type compare_type; - - template - struct node_class - { - typedef detail::ordered_index_node type; - }; - - template - struct index_class - { - typedef detail::ordered_index< - key_from_value_type,compare_type, - SuperMeta,tag_list_type,detail::ordered_unique_tag, - detail::null_augment_policy> type; - }; -}; - -template -struct ordered_non_unique -{ - typedef detail::ordered_index_args< - Arg1,Arg2,Arg3> index_args; - typedef typename index_args::tag_list_type::type tag_list_type; - typedef typename index_args::key_from_value_type key_from_value_type; - typedef typename index_args::compare_type compare_type; - - template - struct node_class - { - typedef detail::ordered_index_node type; - }; - - template - struct index_class - { - typedef detail::ordered_index< - key_from_value_type,compare_type, - SuperMeta,tag_list_type,detail::ordered_non_unique_tag, - detail::null_augment_policy> type; - }; -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index_fwd.hpp deleted file mode 100644 index fe44aaf860d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/ordered_index_fwd.hpp +++ /dev/null @@ -1,35 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_ORDERED_INDEX_FWD_HPP -#define BOOST_MULTI_INDEX_ORDERED_INDEX_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include -#include - -namespace boost{ - -namespace multi_index{ - -/* ordered_index specifiers */ - -template -struct ordered_unique; - -template -struct ordered_non_unique; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index.hpp deleted file mode 100644 index fe1884ddd38..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index.hpp +++ /dev/null @@ -1,1167 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_RANDOM_ACCESS_INDEX_HPP -#define BOOST_MULTI_INDEX_RANDOM_ACCESS_INDEX_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) -#include -#endif - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) -#define BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF(x) \ - detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ - detail::make_obj_guard(x,&random_access_index::check_invariant_); \ - BOOST_JOIN(check_invariant_,__LINE__).touch(); -#define BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT \ - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF(*this) -#else -#define BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF(x) -#define BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* random_access_index adds a layer of random access indexing - * to a given Super - */ - -template -class random_access_index: - BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS SuperMeta::type - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - ,public safe_mode::safe_container< - random_access_index > -#endif - -{ -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the - * lifetime of const references bound to temporaries --precisely what - * scopeguards are. - */ - -#pragma parse_mfunc_templ off -#endif - - typedef typename SuperMeta::type super; - -protected: - typedef random_access_index_node< - typename super::node_type> node_type; - -private: - typedef typename node_type::impl_type node_impl_type; - typedef random_access_index_ptr_array< - typename super::final_allocator_type> ptr_array; - typedef typename ptr_array::pointer node_impl_ptr_pointer; - -public: - /* types */ - - typedef typename node_type::value_type value_type; - typedef tuples::null_type ctor_args; - typedef typename super::final_allocator_type allocator_type; - typedef typename allocator_type::reference reference; - typedef typename allocator_type::const_reference const_reference; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_iterator< - rnd_node_iterator, - random_access_index> iterator; -#else - typedef rnd_node_iterator iterator; -#endif - - typedef iterator const_iterator; - - typedef std::size_t size_type; - typedef std::ptrdiff_t difference_type; - typedef typename allocator_type::pointer pointer; - typedef typename allocator_type::const_pointer const_pointer; - typedef typename - boost::reverse_iterator reverse_iterator; - typedef typename - boost::reverse_iterator const_reverse_iterator; - typedef TagList tag_list; - -protected: - typedef typename super::final_node_type final_node_type; - typedef tuples::cons< - ctor_args, - typename super::ctor_args_list> ctor_args_list; - typedef typename mpl::push_front< - typename super::index_type_list, - random_access_index>::type index_type_list; - typedef typename mpl::push_front< - typename super::iterator_type_list, - iterator>::type iterator_type_list; - typedef typename mpl::push_front< - typename super::const_iterator_type_list, - const_iterator>::type const_iterator_type_list; - typedef typename super::copy_map_type copy_map_type; - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - typedef typename super::index_saver_type index_saver_type; - typedef typename super::index_loader_type index_loader_type; -#endif - -private: -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_container< - random_access_index> safe_super; -#endif - - typedef typename call_traits< - value_type>::param_type value_param_type; - - /* Needed to avoid commas in BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL - * expansion. - */ - - typedef std::pair emplace_return_type; - -public: - - /* construct/copy/destroy - * Default and copy ctors are in the protected section as indices are - * not supposed to be created on their own. No range ctor either. - */ - - random_access_index& operator=( - const random_access_index& x) - { - this->final()=x.final(); - return *this; - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - random_access_index& operator=( - std::initializer_list list) - { - this->final()=list; - return *this; - } -#endif - - template - void assign(InputIterator first,InputIterator last) - { - assign_iter(first,last,mpl::not_ >()); - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - void assign(std::initializer_list list) - { - assign(list.begin(),list.end()); - } -#endif - - void assign(size_type n,value_param_type value) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - clear(); - for(size_type i=0;ifinal().get_allocator(); - } - - /* iterators */ - - iterator begin()BOOST_NOEXCEPT - {return make_iterator(node_type::from_impl(*ptrs.begin()));} - const_iterator begin()const BOOST_NOEXCEPT - {return make_iterator(node_type::from_impl(*ptrs.begin()));} - iterator - end()BOOST_NOEXCEPT{return make_iterator(header());} - const_iterator - end()const BOOST_NOEXCEPT{return make_iterator(header());} - reverse_iterator - rbegin()BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} - const_reverse_iterator - rbegin()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} - reverse_iterator - rend()BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} - const_reverse_iterator - rend()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} - const_iterator - cbegin()const BOOST_NOEXCEPT{return begin();} - const_iterator - cend()const BOOST_NOEXCEPT{return end();} - const_reverse_iterator - crbegin()const BOOST_NOEXCEPT{return rbegin();} - const_reverse_iterator - crend()const BOOST_NOEXCEPT{return rend();} - - iterator iterator_to(const value_type& x) - { - return make_iterator(node_from_value(&x)); - } - - const_iterator iterator_to(const value_type& x)const - { - return make_iterator(node_from_value(&x)); - } - - /* capacity */ - - bool empty()const BOOST_NOEXCEPT{return this->final_empty_();} - size_type size()const BOOST_NOEXCEPT{return this->final_size_();} - size_type max_size()const BOOST_NOEXCEPT{return this->final_max_size_();} - size_type capacity()const BOOST_NOEXCEPT{return ptrs.capacity();} - - void reserve(size_type n) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - ptrs.reserve(n); - } - - void shrink_to_fit() - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - ptrs.shrink_to_fit(); - } - - void resize(size_type n) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - if(n>size()) - for(size_type m=n-size();m--;) - this->final_emplace_(BOOST_MULTI_INDEX_NULL_PARAM_PACK); - else if(nsize())for(size_type m=n-size();m--;)this->final_insert_(x); - else if(nvalue(); - } - - const_reference at(size_type n)const - { - if(n>=size())throw_exception(std::out_of_range("random access index")); - return node_type::from_impl(*ptrs.at(n))->value(); - } - - const_reference front()const{return operator[](0);} - const_reference back()const{return operator[](size()-1);} - - /* modifiers */ - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( - emplace_return_type,emplace_front,emplace_front_impl) - - std::pair push_front(const value_type& x) - {return insert(begin(),x);} - std::pair push_front(BOOST_RV_REF(value_type) x) - {return insert(begin(),boost::move(x));} - void pop_front(){erase(begin());} - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( - emplace_return_type,emplace_back,emplace_back_impl) - - std::pair push_back(const value_type& x) - {return insert(end(),x);} - std::pair push_back(BOOST_RV_REF(value_type) x) - {return insert(end(),boost::move(x));} - void pop_back(){erase(--end());} - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( - emplace_return_type,emplace,emplace_impl,iterator,position) - - std::pair insert(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_(x); - if(p.second&&position.get_node()!=header()){ - relocate(position.get_node(),p.first); - } - return std::pair(make_iterator(p.first),p.second); - } - - std::pair insert(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_rv_(x); - if(p.second&&position.get_node()!=header()){ - relocate(position.get_node(),p.first); - } - return std::pair(make_iterator(p.first),p.second); - } - - void insert(iterator position,size_type n,value_param_type x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - size_type s=0; - BOOST_TRY{ - while(n--){ - if(push_back(x).second)++s; - } - } - BOOST_CATCH(...){ - relocate(position,end()-s,end()); - BOOST_RETHROW; - } - BOOST_CATCH_END - relocate(position,end()-s,end()); - } - - template - void insert(iterator position,InputIterator first,InputIterator last) - { - insert_iter(position,first,last,mpl::not_ >()); - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - void insert(iterator position,std::initializer_list list) - { - insert(position,list.begin(),list.end()); - } -#endif - - iterator erase(iterator position) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - this->final_erase_(static_cast(position++.get_node())); - return position; - } - - iterator erase(iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - difference_type n=last-first; - relocate(end(),first,last); - while(n--)pop_back(); - return last; - } - - bool replace(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - return this->final_replace_( - x,static_cast(position.get_node())); - } - - bool replace(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - return this->final_replace_rv_( - x,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod,Rollback back_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,back_,static_cast(position.get_node())); - } - - void swap(random_access_index& x) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF(x); - this->final_swap_(x.final()); - } - - void clear()BOOST_NOEXCEPT - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - this->final_clear_(); - } - - /* list operations */ - - void splice(iterator position,random_access_index& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_DIFFERENT_CONTAINER(*this,x); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - iterator first=x.begin(),last=x.end(); - size_type n=0; - BOOST_TRY{ - while(first!=last){ - if(push_back(*first).second){ - first=x.erase(first); - ++n; - } - else ++first; - } - } - BOOST_CATCH(...){ - relocate(position,end()-n,end()); - BOOST_RETHROW; - } - BOOST_CATCH_END - relocate(position,end()-n,end()); - } - - void splice( - iterator position,random_access_index& x,iterator i) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(i,x); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - if(&x==this)relocate(position,i); - else{ - if(insert(position,*i).second){ - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer has a hard time with safe mode, and the following - * workaround is needed. Left it for all compilers as it does no - * harm. - */ - i.detach(); - x.erase(x.make_iterator(i.get_node())); -#else - x.erase(i); -#endif - - } - } - } - - void splice( - iterator position,random_access_index& x, - iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,x); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,x); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - if(&x==this)relocate(position,first,last); - else{ - size_type n=0; - BOOST_TRY{ - while(first!=last){ - if(push_back(*first).second){ - first=x.erase(first); - ++n; - } - else ++first; - } - } - BOOST_CATCH(...){ - relocate(position,end()-n,end()); - BOOST_RETHROW; - } - BOOST_CATCH_END - relocate(position,end()-n,end()); - } - } - - void remove(value_param_type value) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - difference_type n= - end()-make_iterator( - random_access_index_remove( - ptrs, - ::boost::bind(std::equal_to(),::boost::arg<1>(),value))); - while(n--)pop_back(); - } - - template - void remove_if(Predicate pred) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - difference_type n= - end()-make_iterator(random_access_index_remove(ptrs,pred)); - while(n--)pop_back(); - } - - void unique() - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - difference_type n= - end()-make_iterator( - random_access_index_unique( - ptrs,std::equal_to())); - while(n--)pop_back(); - } - - template - void unique(BinaryPredicate binary_pred) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - difference_type n= - end()-make_iterator( - random_access_index_unique(ptrs,binary_pred)); - while(n--)pop_back(); - } - - void merge(random_access_index& x) - { - if(this!=&x){ - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - size_type s=size(); - splice(end(),x); - random_access_index_inplace_merge( - get_allocator(),ptrs,ptrs.at(s),std::less()); - } - } - - template - void merge(random_access_index& x,Compare comp) - { - if(this!=&x){ - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - size_type s=size(); - splice(end(),x); - random_access_index_inplace_merge( - get_allocator(),ptrs,ptrs.at(s),comp); - } - } - - void sort() - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - random_access_index_sort( - get_allocator(),ptrs,std::less()); - } - - template - void sort(Compare comp) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - random_access_index_sort( - get_allocator(),ptrs,comp); - } - - void reverse()BOOST_NOEXCEPT - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - node_impl_type::reverse(ptrs.begin(),ptrs.end()); - } - - /* rearrange operations */ - - void relocate(iterator position,iterator i) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(i,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - if(position!=i)relocate(position.get_node(),i.get_node()); - } - - void relocate(iterator position,iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_CHECK_OUTSIDE_RANGE(position,first,last); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - if(position!=last)relocate( - position.get_node(),first.get_node(),last.get_node()); - } - - template - void rearrange(InputIterator first) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - for(node_impl_ptr_pointer p0=ptrs.begin(),p0_end=ptrs.end(); - p0!=p0_end;++first,++p0){ - const value_type& v1=*first; - node_impl_ptr_pointer p1=node_from_value(&v1)->up(); - - std::swap(*p0,*p1); - (*p0)->up()=p0; - (*p1)->up()=p1; - } - } - -BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: - random_access_index( - const ctor_args_list& args_list,const allocator_type& al): - super(args_list.get_tail(),al), - ptrs(al,header()->impl(),0) - { - } - - random_access_index(const random_access_index& x): - super(x), - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super(), -#endif - - ptrs(x.get_allocator(),header()->impl(),x.size()) - { - /* The actual copying takes place in subsequent call to copy_(). - */ - } - - random_access_index( - const random_access_index& x,do_not_copy_elements_tag): - super(x,do_not_copy_elements_tag()), - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super(), -#endif - - ptrs(x.get_allocator(),header()->impl(),0) - { - } - - ~random_access_index() - { - /* the container is guaranteed to be empty by now */ - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - iterator make_iterator(node_type* node){return iterator(node,this);} - const_iterator make_iterator(node_type* node)const - {return const_iterator(node,const_cast(this));} -#else - iterator make_iterator(node_type* node){return iterator(node);} - const_iterator make_iterator(node_type* node)const - {return const_iterator(node);} -#endif - - void copy_( - const random_access_index& x,const copy_map_type& map) - { - for(node_impl_ptr_pointer begin_org=x.ptrs.begin(), - begin_cpy=ptrs.begin(), - end_org=x.ptrs.end(); - begin_org!=end_org;++begin_org,++begin_cpy){ - *begin_cpy= - static_cast( - map.find( - static_cast( - node_type::from_impl(*begin_org))))->impl(); - (*begin_cpy)->up()=begin_cpy; - } - - super::copy_(x,map); - } - - template - final_node_type* insert_( - value_param_type v,final_node_type*& x,Variant variant) - { - ptrs.room_for_one(); - final_node_type* res=super::insert_(v,x,variant); - if(res==x)ptrs.push_back(static_cast(x)->impl()); - return res; - } - - template - final_node_type* insert_( - value_param_type v,node_type* position,final_node_type*& x,Variant variant) - { - ptrs.room_for_one(); - final_node_type* res=super::insert_(v,position,x,variant); - if(res==x)ptrs.push_back(static_cast(x)->impl()); - return res; - } - - void erase_(node_type* x) - { - ptrs.erase(x->impl()); - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - } - - void delete_all_nodes_() - { - for(node_impl_ptr_pointer x=ptrs.begin(),x_end=ptrs.end();x!=x_end;++x){ - this->final_delete_node_( - static_cast(node_type::from_impl(*x))); - } - } - - void clear_() - { - super::clear_(); - ptrs.clear(); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::detach_dereferenceable_iterators(); -#endif - } - - void swap_(random_access_index& x) - { - ptrs.swap(x.ptrs); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_(x); - } - - void swap_elements_(random_access_index& x) - { - ptrs.swap(x.ptrs); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_elements_(x); - } - - template - bool replace_(value_param_type v,node_type* x,Variant variant) - { - return super::replace_(v,x,variant); - } - - bool modify_(node_type* x) - { - BOOST_TRY{ - if(!super::modify_(x)){ - ptrs.erase(x->impl()); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - return false; - } - else return true; - } - BOOST_CATCH(...){ - ptrs.erase(x->impl()); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - bool modify_rollback_(node_type* x) - { - return super::modify_rollback_(x); - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* serialization */ - - template - void save_( - Archive& ar,const unsigned int version,const index_saver_type& sm)const - { - sm.save(begin(),end(),ar,version); - super::save_(ar,version,sm); - } - - template - void load_( - Archive& ar,const unsigned int version,const index_loader_type& lm) - { - { - typedef random_access_index_loader loader; - - loader ld(get_allocator(),ptrs); - lm.load( - ::boost::bind( - &loader::rearrange,&ld,::boost::arg<1>(),::boost::arg<2>()), - ar,version); - } /* exit scope so that ld frees its resources */ - super::load_(ar,version,lm); - } -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - bool invariant_()const - { - if(size()>capacity())return false; - if(size()==0||begin()==end()){ - if(size()!=0||begin()!=end())return false; - } - else{ - size_type s=0; - for(const_iterator it=begin(),it_end=end();;++it,++s){ - if(*(it.get_node()->up())!=it.get_node()->impl())return false; - if(it==it_end)break; - } - if(s!=size())return false; - } - - return super::invariant_(); - } - - /* This forwarding function eases things for the boost::mem_fn construct - * in BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT. Actually, - * final_check_invariant is already an inherited member function of index. - */ - void check_invariant_()const{this->final_check_invariant_();} -#endif - -private: - node_type* header()const{return this->final_header();} - - static void relocate(node_type* position,node_type* x) - { - node_impl_type::relocate(position->up(),x->up()); - } - - static void relocate(node_type* position,node_type* first,node_type* last) - { - node_impl_type::relocate( - position->up(),first->up(),last->up()); - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - void detach_iterators(node_type* x) - { - iterator it=make_iterator(x); - safe_mode::detach_equivalent_iterators(it); - } -#endif - - template - void assign_iter(InputIterator first,InputIterator last,mpl::true_) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - clear(); - for(;first!=last;++first)this->final_insert_ref_(*first); - } - - void assign_iter(size_type n,value_param_type value,mpl::false_) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - clear(); - for(size_type i=0;i - void insert_iter( - iterator position,InputIterator first,InputIterator last,mpl::true_) - { - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - size_type s=0; - BOOST_TRY{ - for(;first!=last;++first){ - if(this->final_insert_ref_(*first).second)++s; - } - } - BOOST_CATCH(...){ - relocate(position,end()-s,end()); - BOOST_RETHROW; - } - BOOST_CATCH_END - relocate(position,end()-s,end()); - } - - void insert_iter( - iterator position,size_type n,value_param_type x,mpl::false_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - size_type s=0; - BOOST_TRY{ - while(n--){ - if(push_back(x).second)++s; - } - } - BOOST_CATCH(...){ - relocate(position,end()-s,end()); - BOOST_RETHROW; - } - BOOST_CATCH_END - relocate(position,end()-s,end()); - } - - template - std::pair emplace_front_impl( - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - return emplace_impl(begin(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - } - - template - std::pair emplace_back_impl( - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - return emplace_impl(end(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - } - - template - std::pair emplace_impl( - iterator position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT; - std::pair p= - this->final_emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - if(p.second&&position.get_node()!=header()){ - relocate(position.get_node(),p.first); - } - return std::pair(make_iterator(p.first),p.second); - } - - ptr_array ptrs; - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -#pragma parse_mfunc_templ reset -#endif -}; - -/* comparison */ - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator==( - const random_access_index& x, - const random_access_index& y) -{ - return x.size()==y.size()&&std::equal(x.begin(),x.end(),y.begin()); -} - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator<( - const random_access_index& x, - const random_access_index& y) -{ - return std::lexicographical_compare(x.begin(),x.end(),y.begin(),y.end()); -} - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator!=( - const random_access_index& x, - const random_access_index& y) -{ - return !(x==y); -} - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator>( - const random_access_index& x, - const random_access_index& y) -{ - return y -bool operator>=( - const random_access_index& x, - const random_access_index& y) -{ - return !(x -bool operator<=( - const random_access_index& x, - const random_access_index& y) -{ - return !(x>y); -} - -/* specialized algorithms */ - -template -void swap( - random_access_index& x, - random_access_index& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -/* random access index specifier */ - -template -struct random_access -{ - BOOST_STATIC_ASSERT(detail::is_tag::value); - - template - struct node_class - { - typedef detail::random_access_index_node type; - }; - - template - struct index_class - { - typedef detail::random_access_index< - SuperMeta,typename TagList::type> type; - }; -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -/* Boost.Foreach compatibility */ - -template -inline boost::mpl::true_* boost_foreach_is_noncopyable( - boost::multi_index::detail::random_access_index*&, - boost_foreach_argument_dependent_lookup_hack) -{ - return 0; -} - -#undef BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT -#undef BOOST_MULTI_INDEX_RND_INDEX_CHECK_INVARIANT_OF - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index_fwd.hpp deleted file mode 100644 index 2ea19295426..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/random_access_index_fwd.hpp +++ /dev/null @@ -1,91 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_RANDOM_ACCESS_INDEX_FWD_HPP -#define BOOST_MULTI_INDEX_RANDOM_ACCESS_INDEX_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -class random_access_index; - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator==( - const random_access_index& x, - const random_access_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator<( - const random_access_index& x, - const random_access_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator!=( - const random_access_index& x, - const random_access_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator>( - const random_access_index& x, - const random_access_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator>=( - const random_access_index& x, - const random_access_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator<=( - const random_access_index& x, - const random_access_index& y); - -template -void swap( - random_access_index& x, - random_access_index& y); - -} /* namespace multi_index::detail */ - -/* index specifiers */ - -template > -struct random_access; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index.hpp deleted file mode 100644 index 4b24c4f5937..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index.hpp +++ /dev/null @@ -1,382 +0,0 @@ -/* Copyright 2003-2017 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_RANKED_INDEX_HPP -#define BOOST_MULTI_INDEX_RANKED_INDEX_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* ranked_index augments a given ordered index to provide rank operations */ - -template -struct ranked_node:OrderedIndexNodeImpl -{ - std::size_t size; -}; - -template -class ranked_index:public OrderedIndexImpl -{ - typedef OrderedIndexImpl super; - -protected: - typedef typename super::node_type node_type; - typedef typename super::node_impl_pointer node_impl_pointer; - -public: - typedef typename super::ctor_args_list ctor_args_list; - typedef typename super::allocator_type allocator_type; - typedef typename super::iterator iterator; - - /* rank operations */ - - iterator nth(std::size_t n)const - { - return this->make_iterator(node_type::from_impl( - ranked_index_nth(n,this->header()->impl()))); - } - - std::size_t rank(iterator position)const - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - - return ranked_index_rank( - position.get_node()->impl(),this->header()->impl()); - } - - template - std::size_t find_rank(const CompatibleKey& x)const - { - return ranked_index_find_rank( - this->root(),this->header(),this->key,x,this->comp_); - } - - template - std::size_t find_rank( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return ranked_index_find_rank( - this->root(),this->header(),this->key,x,comp); - } - - template - std::size_t lower_bound_rank(const CompatibleKey& x)const - { - return ranked_index_lower_bound_rank( - this->root(),this->header(),this->key,x,this->comp_); - } - - template - std::size_t lower_bound_rank( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return ranked_index_lower_bound_rank( - this->root(),this->header(),this->key,x,comp); - } - - template - std::size_t upper_bound_rank(const CompatibleKey& x)const - { - return ranked_index_upper_bound_rank( - this->root(),this->header(),this->key,x,this->comp_); - } - - template - std::size_t upper_bound_rank( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return ranked_index_upper_bound_rank( - this->root(),this->header(),this->key,x,comp); - } - - template - std::pair equal_range_rank( - const CompatibleKey& x)const - { - return ranked_index_equal_range_rank( - this->root(),this->header(),this->key,x,this->comp_); - } - - template - std::pair equal_range_rank( - const CompatibleKey& x,const CompatibleCompare& comp)const - { - return ranked_index_equal_range_rank( - this->root(),this->header(),this->key,x,comp); - } - - template - std::pair - range_rank(LowerBounder lower,UpperBounder upper)const - { - typedef typename mpl::if_< - is_same, - BOOST_DEDUCED_TYPENAME mpl::if_< - is_same, - both_unbounded_tag, - lower_unbounded_tag - >::type, - BOOST_DEDUCED_TYPENAME mpl::if_< - is_same, - upper_unbounded_tag, - none_unbounded_tag - >::type - >::type dispatch; - - return range_rank(lower,upper,dispatch()); - } - -protected: - ranked_index(const ranked_index& x):super(x){}; - - ranked_index(const ranked_index& x,do_not_copy_elements_tag): - super(x,do_not_copy_elements_tag()){}; - - ranked_index( - const ctor_args_list& args_list,const allocator_type& al): - super(args_list,al){} - -private: - template - std::pair - range_rank(LowerBounder lower,UpperBounder upper,none_unbounded_tag)const - { - node_type* y=this->header(); - node_type* z=this->root(); - - if(!z)return std::pair(0,0); - - std::size_t s=z->impl()->size; - - do{ - if(!lower(this->key(z->value()))){ - z=node_type::from_impl(z->right()); - } - else if(!upper(this->key(z->value()))){ - y=z; - s-=ranked_node_size(y->right())+1; - z=node_type::from_impl(z->left()); - } - else{ - return std::pair( - s-z->impl()->size+ - lower_range_rank(node_type::from_impl(z->left()),z,lower), - s-ranked_node_size(z->right())+ - upper_range_rank(node_type::from_impl(z->right()),y,upper)); - } - }while(z); - - return std::pair(s,s); - } - - template - std::pair - range_rank(LowerBounder,UpperBounder upper,lower_unbounded_tag)const - { - return std::pair( - 0, - upper_range_rank(this->root(),this->header(),upper)); - } - - template - std::pair - range_rank(LowerBounder lower,UpperBounder,upper_unbounded_tag)const - { - return std::pair( - lower_range_rank(this->root(),this->header(),lower), - this->size()); - } - - template - std::pair - range_rank(LowerBounder,UpperBounder,both_unbounded_tag)const - { - return std::pair(0,this->size()); - } - - template - std::size_t - lower_range_rank(node_type* top,node_type* y,LowerBounder lower)const - { - if(!top)return 0; - - std::size_t s=top->impl()->size; - - do{ - if(lower(this->key(top->value()))){ - y=top; - s-=ranked_node_size(y->right())+1; - top=node_type::from_impl(top->left()); - } - else top=node_type::from_impl(top->right()); - }while(top); - - return s; - } - - template - std::size_t - upper_range_rank(node_type* top,node_type* y,UpperBounder upper)const - { - if(!top)return 0; - - std::size_t s=top->impl()->size; - - do{ - if(!upper(this->key(top->value()))){ - y=top; - s-=ranked_node_size(y->right())+1; - top=node_type::from_impl(top->left()); - } - else top=node_type::from_impl(top->right()); - }while(top); - - return s; - } -}; - -/* augmenting policy for ordered_index */ - -struct rank_policy -{ - template - struct augmented_node - { - typedef ranked_node type; - }; - - template - struct augmented_interface - { - typedef ranked_index type; - }; - - /* algorithmic stuff */ - - template - static void add(Pointer x,Pointer root) - { - x->size=1; - while(x!=root){ - x=x->parent(); - ++(x->size); - } - } - - template - static void remove(Pointer x,Pointer root) - { - while(x!=root){ - x=x->parent(); - --(x->size); - } - } - - template - static void copy(Pointer x,Pointer y) - { - y->size=x->size; - } - - template - static void rotate_left(Pointer x,Pointer y) /* in: x==y->left() */ - { - y->size=x->size; - x->size=ranked_node_size(x->left())+ranked_node_size(x->right())+1; - } - - template - static void rotate_right(Pointer x,Pointer y) /* in: x==y->right() */ - { - rotate_left(x,y); - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - template - static bool invariant(Pointer x) - { - return x->size==ranked_node_size(x->left())+ranked_node_size(x->right())+1; - } -#endif -}; - -} /* namespace multi_index::detail */ - -/* ranked_index specifiers */ - -template -struct ranked_unique -{ - typedef typename detail::ordered_index_args< - Arg1,Arg2,Arg3> index_args; - typedef typename index_args::tag_list_type::type tag_list_type; - typedef typename index_args::key_from_value_type key_from_value_type; - typedef typename index_args::compare_type compare_type; - - template - struct node_class - { - typedef detail::ordered_index_node type; - }; - - template - struct index_class - { - typedef detail::ordered_index< - key_from_value_type,compare_type, - SuperMeta,tag_list_type,detail::ordered_unique_tag, - detail::rank_policy> type; - }; -}; - -template -struct ranked_non_unique -{ - typedef detail::ordered_index_args< - Arg1,Arg2,Arg3> index_args; - typedef typename index_args::tag_list_type::type tag_list_type; - typedef typename index_args::key_from_value_type key_from_value_type; - typedef typename index_args::compare_type compare_type; - - template - struct node_class - { - typedef detail::ordered_index_node type; - }; - - template - struct index_class - { - typedef detail::ordered_index< - key_from_value_type,compare_type, - SuperMeta,tag_list_type,detail::ordered_non_unique_tag, - detail::rank_policy> type; - }; -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index_fwd.hpp deleted file mode 100644 index 380d3480736..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/ranked_index_fwd.hpp +++ /dev/null @@ -1,35 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_RANKED_INDEX_FWD_HPP -#define BOOST_MULTI_INDEX_RANKED_INDEX_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include -#include - -namespace boost{ - -namespace multi_index{ - -/* ranked_index specifiers */ - -template -struct ranked_unique; - -template -struct ranked_non_unique; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/safe_mode_errors.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/safe_mode_errors.hpp deleted file mode 100644 index 1904706edec..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/safe_mode_errors.hpp +++ /dev/null @@ -1,48 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_SAFE_MODE_ERRORS_HPP -#define BOOST_MULTI_INDEX_SAFE_MODE_ERRORS_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -namespace boost{ - -namespace multi_index{ - -namespace safe_mode{ - -/* Error codes for Boost.MultiIndex safe mode. These go in a separate - * header so that the user can include it when redefining - * BOOST_MULTI_INDEX_SAFE_MODE_ASSERT prior to the inclusion of - * any other header of Boost.MultiIndex. - */ - -enum error_code -{ - invalid_iterator=0, - not_dereferenceable_iterator, - not_incrementable_iterator, - not_decrementable_iterator, - not_owner, - not_same_owner, - invalid_range, - inside_range, - out_of_bounds, - same_container -}; - -} /* namespace multi_index::safe_mode */ - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index.hpp deleted file mode 100644 index 424eebc376d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index.hpp +++ /dev/null @@ -1,1062 +0,0 @@ -/* Copyright 2003-2015 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_SEQUENCED_INDEX_HPP -#define BOOST_MULTI_INDEX_SEQUENCED_INDEX_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) -#include -#endif - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) -#define BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF(x) \ - detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ - detail::make_obj_guard(x,&sequenced_index::check_invariant_); \ - BOOST_JOIN(check_invariant_,__LINE__).touch(); -#define BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT \ - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF(*this) -#else -#define BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF(x) -#define BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT -#endif - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -/* sequenced_index adds a layer of sequenced indexing to a given Super */ - -template -class sequenced_index: - BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS SuperMeta::type - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - ,public safe_mode::safe_container< - sequenced_index > -#endif - -{ -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the - * lifetime of const references bound to temporaries --precisely what - * scopeguards are. - */ - -#pragma parse_mfunc_templ off -#endif - - typedef typename SuperMeta::type super; - -protected: - typedef sequenced_index_node< - typename super::node_type> node_type; - -private: - typedef typename node_type::impl_type node_impl_type; - -public: - /* types */ - - typedef typename node_type::value_type value_type; - typedef tuples::null_type ctor_args; - typedef typename super::final_allocator_type allocator_type; - typedef typename allocator_type::reference reference; - typedef typename allocator_type::const_reference const_reference; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_iterator< - bidir_node_iterator, - sequenced_index> iterator; -#else - typedef bidir_node_iterator iterator; -#endif - - typedef iterator const_iterator; - - typedef std::size_t size_type; - typedef std::ptrdiff_t difference_type; - typedef typename allocator_type::pointer pointer; - typedef typename allocator_type::const_pointer const_pointer; - typedef typename - boost::reverse_iterator reverse_iterator; - typedef typename - boost::reverse_iterator const_reverse_iterator; - typedef TagList tag_list; - -protected: - typedef typename super::final_node_type final_node_type; - typedef tuples::cons< - ctor_args, - typename super::ctor_args_list> ctor_args_list; - typedef typename mpl::push_front< - typename super::index_type_list, - sequenced_index>::type index_type_list; - typedef typename mpl::push_front< - typename super::iterator_type_list, - iterator>::type iterator_type_list; - typedef typename mpl::push_front< - typename super::const_iterator_type_list, - const_iterator>::type const_iterator_type_list; - typedef typename super::copy_map_type copy_map_type; - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - typedef typename super::index_saver_type index_saver_type; - typedef typename super::index_loader_type index_loader_type; -#endif - -private: -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef safe_mode::safe_container< - sequenced_index> safe_super; -#endif - - typedef typename call_traits::param_type value_param_type; - - /* Needed to avoid commas in BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL - * expansion. - */ - - typedef std::pair emplace_return_type; - -public: - - /* construct/copy/destroy - * Default and copy ctors are in the protected section as indices are - * not supposed to be created on their own. No range ctor either. - */ - - sequenced_index& operator=( - const sequenced_index& x) - { - this->final()=x.final(); - return *this; - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - sequenced_index& operator=( - std::initializer_list list) - { - this->final()=list; - return *this; - } -#endif - - template - void assign(InputIterator first,InputIterator last) - { - assign_iter(first,last,mpl::not_ >()); - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - void assign(std::initializer_list list) - { - assign(list.begin(),list.end()); - } -#endif - - void assign(size_type n,value_param_type value) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - clear(); - for(size_type i=0;ifinal().get_allocator(); - } - - /* iterators */ - - iterator begin()BOOST_NOEXCEPT - {return make_iterator(node_type::from_impl(header()->next()));} - const_iterator begin()const BOOST_NOEXCEPT - {return make_iterator(node_type::from_impl(header()->next()));} - iterator - end()BOOST_NOEXCEPT{return make_iterator(header());} - const_iterator - end()const BOOST_NOEXCEPT{return make_iterator(header());} - reverse_iterator - rbegin()BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} - const_reverse_iterator - rbegin()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(end());} - reverse_iterator - rend()BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} - const_reverse_iterator - rend()const BOOST_NOEXCEPT{return boost::make_reverse_iterator(begin());} - const_iterator - cbegin()const BOOST_NOEXCEPT{return begin();} - const_iterator - cend()const BOOST_NOEXCEPT{return end();} - const_reverse_iterator - crbegin()const BOOST_NOEXCEPT{return rbegin();} - const_reverse_iterator - crend()const BOOST_NOEXCEPT{return rend();} - - iterator iterator_to(const value_type& x) - { - return make_iterator(node_from_value(&x)); - } - - const_iterator iterator_to(const value_type& x)const - { - return make_iterator(node_from_value(&x)); - } - - /* capacity */ - - bool empty()const BOOST_NOEXCEPT{return this->final_empty_();} - size_type size()const BOOST_NOEXCEPT{return this->final_size_();} - size_type max_size()const BOOST_NOEXCEPT{return this->final_max_size_();} - - void resize(size_type n) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - if(n>size()){ - for(size_type m=n-size();m--;) - this->final_emplace_(BOOST_MULTI_INDEX_NULL_PARAM_PACK); - } - else if(nsize())insert(end(),n-size(),x); - else if(n push_front(const value_type& x) - {return insert(begin(),x);} - std::pair push_front(BOOST_RV_REF(value_type) x) - {return insert(begin(),boost::move(x));} - void pop_front(){erase(begin());} - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL( - emplace_return_type,emplace_back,emplace_back_impl) - - std::pair push_back(const value_type& x) - {return insert(end(),x);} - std::pair push_back(BOOST_RV_REF(value_type) x) - {return insert(end(),boost::move(x));} - void pop_back(){erase(--end());} - - BOOST_MULTI_INDEX_OVERLOADS_TO_VARTEMPL_EXTRA_ARG( - emplace_return_type,emplace,emplace_impl,iterator,position) - - std::pair insert(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_(x); - if(p.second&&position.get_node()!=header()){ - relink(position.get_node(),p.first); - } - return std::pair(make_iterator(p.first),p.second); - } - - std::pair insert(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - std::pair p=this->final_insert_rv_(x); - if(p.second&&position.get_node()!=header()){ - relink(position.get_node(),p.first); - } - return std::pair(make_iterator(p.first),p.second); - } - - void insert(iterator position,size_type n,value_param_type x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - for(size_type i=0;i - void insert(iterator position,InputIterator first,InputIterator last) - { - insert_iter(position,first,last,mpl::not_ >()); - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - void insert(iterator position,std::initializer_list list) - { - insert(position,list.begin(),list.end()); - } -#endif - - iterator erase(iterator position) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - this->final_erase_(static_cast(position++.get_node())); - return position; - } - - iterator erase(iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - while(first!=last){ - first=erase(first); - } - return first; - } - - bool replace(iterator position,const value_type& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - return this->final_replace_( - x,static_cast(position.get_node())); - } - - bool replace(iterator position,BOOST_RV_REF(value_type) x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - return this->final_replace_rv_( - x,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,static_cast(position.get_node())); - } - - template - bool modify(iterator position,Modifier mod,Rollback back_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer on safe mode code chokes if this - * this is not added. Left it for all compilers as it does no - * harm. - */ - - position.detach(); -#endif - - return this->final_modify_( - mod,back_,static_cast(position.get_node())); - } - - void swap(sequenced_index& x) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF(x); - this->final_swap_(x.final()); - } - - void clear()BOOST_NOEXCEPT - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - this->final_clear_(); - } - - /* list operations */ - - void splice(iterator position,sequenced_index& x) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_DIFFERENT_CONTAINER(*this,x); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - iterator first=x.begin(),last=x.end(); - while(first!=last){ - if(insert(position,*first).second)first=x.erase(first); - else ++first; - } - } - - void splice(iterator position,sequenced_index& x,iterator i) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(i,x); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - if(&x==this){ - if(position!=i)relink(position.get_node(),i.get_node()); - } - else{ - if(insert(position,*i).second){ - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - /* MSVC++ 6.0 optimizer has a hard time with safe mode, and the following - * workaround is needed. Left it for all compilers as it does no - * harm. - */ - i.detach(); - x.erase(x.make_iterator(i.get_node())); -#else - x.erase(i); -#endif - - } - } - } - - void splice( - iterator position,sequenced_index& x, - iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,x); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,x); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - if(&x==this){ - BOOST_MULTI_INDEX_CHECK_OUTSIDE_RANGE(position,first,last); - if(position!=last)relink( - position.get_node(),first.get_node(),last.get_node()); - } - else{ - while(first!=last){ - if(insert(position,*first).second)first=x.erase(first); - else ++first; - } - } - } - - void remove(value_param_type value) - { - sequenced_index_remove( - *this, - ::boost::bind(std::equal_to(),::boost::arg<1>(),value)); - } - - template - void remove_if(Predicate pred) - { - sequenced_index_remove(*this,pred); - } - - void unique() - { - sequenced_index_unique(*this,std::equal_to()); - } - - template - void unique(BinaryPredicate binary_pred) - { - sequenced_index_unique(*this,binary_pred); - } - - void merge(sequenced_index& x) - { - sequenced_index_merge(*this,x,std::less()); - } - - template - void merge(sequenced_index& x,Compare comp) - { - sequenced_index_merge(*this,x,comp); - } - - void sort() - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - sequenced_index_sort(header(),std::less()); - } - - template - void sort(Compare comp) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - sequenced_index_sort(header(),comp); - } - - void reverse()BOOST_NOEXCEPT - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - node_impl_type::reverse(header()->impl()); - } - - /* rearrange operations */ - - void relocate(iterator position,iterator i) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_DEREFERENCEABLE_ITERATOR(i); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(i,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - if(position!=i)relink(position.get_node(),i.get_node()); - } - - void relocate(iterator position,iterator first,iterator last) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(first); - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(last); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(first,*this); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(last,*this); - BOOST_MULTI_INDEX_CHECK_VALID_RANGE(first,last); - BOOST_MULTI_INDEX_CHECK_OUTSIDE_RANGE(position,first,last); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - if(position!=last)relink( - position.get_node(),first.get_node(),last.get_node()); - } - - template - void rearrange(InputIterator first) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - node_type* pos=header(); - for(size_type s=size();s--;){ - const value_type& v=*first++; - relink(pos,node_from_value(&v)); - } - } - -BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: - sequenced_index(const ctor_args_list& args_list,const allocator_type& al): - super(args_list.get_tail(),al) - { - empty_initialize(); - } - - sequenced_index(const sequenced_index& x): - super(x) - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - ,safe_super() -#endif - - { - /* the actual copying takes place in subsequent call to copy_() */ - } - - sequenced_index( - const sequenced_index& x,do_not_copy_elements_tag): - super(x,do_not_copy_elements_tag()) - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - ,safe_super() -#endif - - { - empty_initialize(); - } - - ~sequenced_index() - { - /* the container is guaranteed to be empty by now */ - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - iterator make_iterator(node_type* node){return iterator(node,this);} - const_iterator make_iterator(node_type* node)const - {return const_iterator(node,const_cast(this));} -#else - iterator make_iterator(node_type* node){return iterator(node);} - const_iterator make_iterator(node_type* node)const - {return const_iterator(node);} -#endif - - void copy_( - const sequenced_index& x,const copy_map_type& map) - { - node_type* org=x.header(); - node_type* cpy=header(); - do{ - node_type* next_org=node_type::from_impl(org->next()); - node_type* next_cpy=map.find(static_cast(next_org)); - cpy->next()=next_cpy->impl(); - next_cpy->prior()=cpy->impl(); - org=next_org; - cpy=next_cpy; - }while(org!=x.header()); - - super::copy_(x,map); - } - - template - final_node_type* insert_( - value_param_type v,final_node_type*& x,Variant variant) - { - final_node_type* res=super::insert_(v,x,variant); - if(res==x)link(static_cast(x)); - return res; - } - - template - final_node_type* insert_( - value_param_type v,node_type* position,final_node_type*& x,Variant variant) - { - final_node_type* res=super::insert_(v,position,x,variant); - if(res==x)link(static_cast(x)); - return res; - } - - void erase_(node_type* x) - { - unlink(x); - super::erase_(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - } - - void delete_all_nodes_() - { - for(node_type* x=node_type::from_impl(header()->next());x!=header();){ - node_type* y=node_type::from_impl(x->next()); - this->final_delete_node_(static_cast(x)); - x=y; - } - } - - void clear_() - { - super::clear_(); - empty_initialize(); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::detach_dereferenceable_iterators(); -#endif - } - - void swap_(sequenced_index& x) - { -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_(x); - } - - void swap_elements_(sequenced_index& x) - { -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - safe_super::swap(x); -#endif - - super::swap_elements_(x); - } - - template - bool replace_(value_param_type v,node_type* x,Variant variant) - { - return super::replace_(v,x,variant); - } - - bool modify_(node_type* x) - { - BOOST_TRY{ - if(!super::modify_(x)){ - unlink(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - return false; - } - else return true; - } - BOOST_CATCH(...){ - unlink(x); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - detach_iterators(x); -#endif - - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - bool modify_rollback_(node_type* x) - { - return super::modify_rollback_(x); - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* serialization */ - - template - void save_( - Archive& ar,const unsigned int version,const index_saver_type& sm)const - { - sm.save(begin(),end(),ar,version); - super::save_(ar,version,sm); - } - - template - void load_( - Archive& ar,const unsigned int version,const index_loader_type& lm) - { - lm.load( - ::boost::bind( - &sequenced_index::rearranger,this,::boost::arg<1>(),::boost::arg<2>()), - ar,version); - super::load_(ar,version,lm); - } -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - bool invariant_()const - { - if(size()==0||begin()==end()){ - if(size()!=0||begin()!=end()|| - header()->next()!=header()->impl()|| - header()->prior()!=header()->impl())return false; - } - else{ - size_type s=0; - for(const_iterator it=begin(),it_end=end();it!=it_end;++it,++s){ - if(it.get_node()->next()->prior()!=it.get_node()->impl())return false; - if(it.get_node()->prior()->next()!=it.get_node()->impl())return false; - } - if(s!=size())return false; - } - - return super::invariant_(); - } - - /* This forwarding function eases things for the boost::mem_fn construct - * in BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT. Actually, - * final_check_invariant is already an inherited member function of index. - */ - void check_invariant_()const{this->final_check_invariant_();} -#endif - -private: - node_type* header()const{return this->final_header();} - - void empty_initialize() - { - header()->prior()=header()->next()=header()->impl(); - } - - void link(node_type* x) - { - node_impl_type::link(x->impl(),header()->impl()); - }; - - static void unlink(node_type* x) - { - node_impl_type::unlink(x->impl()); - } - - static void relink(node_type* position,node_type* x) - { - node_impl_type::relink(position->impl(),x->impl()); - } - - static void relink(node_type* position,node_type* first,node_type* last) - { - node_impl_type::relink( - position->impl(),first->impl(),last->impl()); - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - void rearranger(node_type* position,node_type *x) - { - if(!position)position=header(); - node_type::increment(position); - if(position!=x)relink(position,x); - } -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - void detach_iterators(node_type* x) - { - iterator it=make_iterator(x); - safe_mode::detach_equivalent_iterators(it); - } -#endif - - template - void assign_iter(InputIterator first,InputIterator last,mpl::true_) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - clear(); - for(;first!=last;++first)this->final_insert_ref_(*first); - } - - void assign_iter(size_type n,value_param_type value,mpl::false_) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - clear(); - for(size_type i=0;i - void insert_iter( - iterator position,InputIterator first,InputIterator last,mpl::true_) - { - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - for(;first!=last;++first){ - std::pair p= - this->final_insert_ref_(*first); - if(p.second&&position.get_node()!=header()){ - relink(position.get_node(),p.first); - } - } - } - - void insert_iter( - iterator position,size_type n,value_param_type x,mpl::false_) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - for(size_type i=0;i - std::pair emplace_front_impl( - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - return emplace_impl(begin(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - } - - template - std::pair emplace_back_impl( - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - return emplace_impl(end(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - } - - template - std::pair emplace_impl( - iterator position,BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(position); - BOOST_MULTI_INDEX_CHECK_IS_OWNER(position,*this); - BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT; - std::pair p= - this->final_emplace_(BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - if(p.second&&position.get_node()!=header()){ - relink(position.get_node(),p.first); - } - return std::pair(make_iterator(p.first),p.second); - } - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -#pragma parse_mfunc_templ reset -#endif -}; - -/* comparison */ - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator==( - const sequenced_index& x, - const sequenced_index& y) -{ - return x.size()==y.size()&&std::equal(x.begin(),x.end(),y.begin()); -} - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator<( - const sequenced_index& x, - const sequenced_index& y) -{ - return std::lexicographical_compare(x.begin(),x.end(),y.begin(),y.end()); -} - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator!=( - const sequenced_index& x, - const sequenced_index& y) -{ - return !(x==y); -} - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator>( - const sequenced_index& x, - const sequenced_index& y) -{ - return y -bool operator>=( - const sequenced_index& x, - const sequenced_index& y) -{ - return !(x -bool operator<=( - const sequenced_index& x, - const sequenced_index& y) -{ - return !(x>y); -} - -/* specialized algorithms */ - -template -void swap( - sequenced_index& x, - sequenced_index& y) -{ - x.swap(y); -} - -} /* namespace multi_index::detail */ - -/* sequenced index specifier */ - -template -struct sequenced -{ - BOOST_STATIC_ASSERT(detail::is_tag::value); - - template - struct node_class - { - typedef detail::sequenced_index_node type; - }; - - template - struct index_class - { - typedef detail::sequenced_index type; - }; -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -/* Boost.Foreach compatibility */ - -template -inline boost::mpl::true_* boost_foreach_is_noncopyable( - boost::multi_index::detail::sequenced_index*&, - boost_foreach_argument_dependent_lookup_hack) -{ - return 0; -} - -#undef BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT -#undef BOOST_MULTI_INDEX_SEQ_INDEX_CHECK_INVARIANT_OF - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index_fwd.hpp deleted file mode 100644 index a019f2a6d2f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/sequenced_index_fwd.hpp +++ /dev/null @@ -1,91 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_SEQUENCED_INDEX_FWD_HPP -#define BOOST_MULTI_INDEX_SEQUENCED_INDEX_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include - -namespace boost{ - -namespace multi_index{ - -namespace detail{ - -template -class sequenced_index; - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator==( - const sequenced_index& x, - const sequenced_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator<( - const sequenced_index& x, - const sequenced_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator!=( - const sequenced_index& x, - const sequenced_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator>( - const sequenced_index& x, - const sequenced_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator>=( - const sequenced_index& x, - const sequenced_index& y); - -template< - typename SuperMeta1,typename TagList1, - typename SuperMeta2,typename TagList2 -> -bool operator<=( - const sequenced_index& x, - const sequenced_index& y); - -template -void swap( - sequenced_index& x, - sequenced_index& y); - -} /* namespace multi_index::detail */ - -/* index specifiers */ - -template > -struct sequenced; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index/tag.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index/tag.hpp deleted file mode 100644 index ce51f8241ee..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index/tag.hpp +++ /dev/null @@ -1,88 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_TAG_HPP -#define BOOST_MULTI_INDEX_TAG_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -/* A wrapper of mpl::vector used to hide MPL from the user. - * tag contains types used as tag names for indices in get() functions. - */ - -/* This user_definable macro limits the number of elements of a tag; - * useful for shortening resulting symbol names (MSVC++ 6.0, for instance, - * has problems coping with very long symbol names.) - */ - -#if !defined(BOOST_MULTI_INDEX_LIMIT_TAG_SIZE) -#define BOOST_MULTI_INDEX_LIMIT_TAG_SIZE BOOST_MPL_LIMIT_VECTOR_SIZE -#endif - -#if BOOST_MULTI_INDEX_LIMIT_TAG_SIZE -struct is_tag -{ - BOOST_STATIC_CONSTANT(bool,value=(is_base_and_derived::value)); -}; - -} /* namespace multi_index::detail */ - -template< - BOOST_PP_ENUM_BINARY_PARAMS( - BOOST_MULTI_INDEX_TAG_SIZE, - typename T, - =mpl::na BOOST_PP_INTERCEPT) -> -struct tag:private detail::tag_marker -{ - /* The mpl::transform pass produces shorter symbols (without - * trailing mpl::na's.) - */ - - typedef typename mpl::transform< - mpl::vector, - mpl::identity - >::type type; - - BOOST_STATIC_ASSERT(detail::no_duplicate_tags::value); -}; - -} /* namespace multi_index */ - -} /* namespace boost */ - -#undef BOOST_MULTI_INDEX_TAG_SIZE - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index_container.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index_container.hpp deleted file mode 100644 index 9993a8dfa10..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index_container.hpp +++ /dev/null @@ -1,1362 +0,0 @@ -/* Multiply indexed container. - * - * Copyright 2003-2014 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_HPP -#define BOOST_MULTI_INDEX_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) -#include -#endif - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -#include -#include -#include -#include -#include -#include -#include -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) -#include -#define BOOST_MULTI_INDEX_CHECK_INVARIANT_OF(x) \ - detail::scope_guard BOOST_JOIN(check_invariant_,__LINE__)= \ - detail::make_obj_guard(x,&multi_index_container::check_invariant_); \ - BOOST_JOIN(check_invariant_,__LINE__).touch(); -#define BOOST_MULTI_INDEX_CHECK_INVARIANT \ - BOOST_MULTI_INDEX_CHECK_INVARIANT_OF(*this) -#else -#define BOOST_MULTI_INDEX_CHECK_INVARIANT_OF(x) -#define BOOST_MULTI_INDEX_CHECK_INVARIANT -#endif - -namespace boost{ - -namespace multi_index{ - -#if BOOST_WORKAROUND(BOOST_MSVC,BOOST_TESTED_AT(1500)) -#pragma warning(push) -#pragma warning(disable:4522) /* spurious warning on multiple operator=()'s */ -#endif - -template -class multi_index_container: - private ::boost::base_from_member< - typename boost::detail::allocator::rebind_to< - Allocator, - typename detail::multi_index_node_type< - Value,IndexSpecifierList,Allocator>::type - >::type>, - BOOST_MULTI_INDEX_PRIVATE_IF_MEMBER_TEMPLATE_FRIENDS detail::header_holder< - typename boost::detail::allocator::rebind_to< - Allocator, - typename detail::multi_index_node_type< - Value,IndexSpecifierList,Allocator>::type - >::type::pointer, - multi_index_container >, - public detail::multi_index_base_type< - Value,IndexSpecifierList,Allocator>::type -{ -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -/* The "ISO C++ Template Parser" option in CW8.3 has a problem with the - * lifetime of const references bound to temporaries --precisely what - * scopeguards are. - */ - -#pragma parse_mfunc_templ off -#endif - -private: - BOOST_COPYABLE_AND_MOVABLE(multi_index_container) - -#if !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) - template friend class detail::index_base; - template friend struct detail::header_holder; - template friend struct detail::converter; -#endif - - typedef typename detail::multi_index_base_type< - Value,IndexSpecifierList,Allocator>::type super; - typedef typename - boost::detail::allocator::rebind_to< - Allocator, - typename super::node_type - >::type node_allocator; - typedef ::boost::base_from_member< - node_allocator> bfm_allocator; - typedef detail::header_holder< - typename node_allocator::pointer, - multi_index_container> bfm_header; - - -public: - /* All types are inherited from super, a few are explicitly - * brought forward here to save us some typename's. - */ - - typedef typename super::ctor_args_list ctor_args_list; - typedef IndexSpecifierList index_specifier_type_list; - - typedef typename super::index_type_list index_type_list; - - typedef typename super::iterator_type_list iterator_type_list; - typedef typename super::const_iterator_type_list const_iterator_type_list; - typedef typename super::value_type value_type; - typedef typename super::final_allocator_type allocator_type; - typedef typename super::iterator iterator; - typedef typename super::const_iterator const_iterator; - - BOOST_STATIC_ASSERT( - detail::no_duplicate_tags_in_index_list::value); - - /* global project() needs to see this publicly */ - - typedef typename super::node_type node_type; - - /* construct/copy/destroy */ - - explicit multi_index_container( - -#if BOOST_WORKAROUND(__IBMCPP__,<=600) - /* VisualAge seems to have an ETI issue with the default values - * for arguments args_list and al. - */ - - const ctor_args_list& args_list= - typename mpl::identity::type:: - ctor_args_list(), - const allocator_type& al= - typename mpl::identity::type:: - allocator_type()): -#else - const ctor_args_list& args_list=ctor_args_list(), - const allocator_type& al=allocator_type()): -#endif - - bfm_allocator(al), - super(args_list,bfm_allocator::member), - node_count(0) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - } - - explicit multi_index_container(const allocator_type& al): - bfm_allocator(al), - super(ctor_args_list(),bfm_allocator::member), - node_count(0) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - } - - template - multi_index_container( - InputIterator first,InputIterator last, - -#if BOOST_WORKAROUND(__IBMCPP__,<=600) - /* VisualAge seems to have an ETI issue with the default values - * for arguments args_list and al. - */ - - const ctor_args_list& args_list= - typename mpl::identity::type:: - ctor_args_list(), - const allocator_type& al= - typename mpl::identity::type:: - allocator_type()): -#else - const ctor_args_list& args_list=ctor_args_list(), - const allocator_type& al=allocator_type()): -#endif - - bfm_allocator(al), - super(args_list,bfm_allocator::member), - node_count(0) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - BOOST_TRY{ - iterator hint=super::end(); - for(;first!=last;++first){ - hint=super::make_iterator( - insert_ref_(*first,hint.get_node()).first); - ++hint; - } - } - BOOST_CATCH(...){ - clear_(); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - multi_index_container( - std::initializer_list list, - const ctor_args_list& args_list=ctor_args_list(), - const allocator_type& al=allocator_type()): - bfm_allocator(al), - super(args_list,bfm_allocator::member), - node_count(0) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - BOOST_TRY{ - typedef const Value* init_iterator; - - iterator hint=super::end(); - for(init_iterator first=list.begin(),last=list.end(); - first!=last;++first){ - hint=super::make_iterator(insert_(*first,hint.get_node()).first); - ++hint; - } - } - BOOST_CATCH(...){ - clear_(); - BOOST_RETHROW; - } - BOOST_CATCH_END - } -#endif - - multi_index_container( - const multi_index_container& x): - bfm_allocator(x.bfm_allocator::member), - bfm_header(), - super(x), - node_count(0) - { - copy_map_type map(bfm_allocator::member,x.size(),x.header(),header()); - for(const_iterator it=x.begin(),it_end=x.end();it!=it_end;++it){ - map.clone(it.get_node()); - } - super::copy_(x,map); - map.release(); - node_count=x.size(); - - /* Not until this point are the indices required to be consistent, - * hence the position of the invariant checker. - */ - - BOOST_MULTI_INDEX_CHECK_INVARIANT; - } - - multi_index_container(BOOST_RV_REF(multi_index_container) x): - bfm_allocator(x.bfm_allocator::member), - bfm_header(), - super(x,detail::do_not_copy_elements_tag()), - node_count(0) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - BOOST_MULTI_INDEX_CHECK_INVARIANT_OF(x); - swap_elements_(x); - } - - ~multi_index_container() - { - delete_all_nodes_(); - } - -#if defined(BOOST_NO_CXX11_RVALUE_REFERENCES) - /* As per http://www.boost.org/doc/html/move/emulation_limitations.html - * #move.emulation_limitations.assignment_operator - */ - - multi_index_container& operator=( - const multi_index_container& x) - { - multi_index_container y(x); - this->swap(y); - return *this; - } -#endif - - multi_index_container& operator=( - BOOST_COPY_ASSIGN_REF(multi_index_container) x) - { - multi_index_container y(x); - this->swap(y); - return *this; - } - - multi_index_container& operator=( - BOOST_RV_REF(multi_index_container) x) - { - this->swap(x); - return *this; - } - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - multi_index_container& operator=( - std::initializer_list list) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - typedef const Value* init_iterator; - - multi_index_container x(*this,detail::do_not_copy_elements_tag()); - iterator hint=x.end(); - for(init_iterator first=list.begin(),last=list.end(); - first!=last;++first){ - hint=x.make_iterator(x.insert_(*first,hint.get_node()).first); - ++hint; - } - x.swap_elements_(*this); - return*this; - } -#endif - - allocator_type get_allocator()const BOOST_NOEXCEPT - { - return allocator_type(bfm_allocator::member); - } - - /* retrieval of indices by number */ - -#if !defined(BOOST_NO_MEMBER_TEMPLATES) - template - struct nth_index - { - BOOST_STATIC_ASSERT(N>=0&&N::type::value); - typedef typename mpl::at_c::type type; - }; - - template - typename nth_index::type& get()BOOST_NOEXCEPT - { - BOOST_STATIC_ASSERT(N>=0&&N::type::value); - return *this; - } - - template - const typename nth_index::type& get()const BOOST_NOEXCEPT - { - BOOST_STATIC_ASSERT(N>=0&&N::type::value); - return *this; - } -#endif - - /* retrieval of indices by tag */ - -#if !defined(BOOST_NO_MEMBER_TEMPLATES) - template - struct index - { - typedef typename mpl::find_if< - index_type_list, - detail::has_tag - >::type iter; - - BOOST_STATIC_CONSTANT( - bool,index_found=!(is_same::type >::value)); - BOOST_STATIC_ASSERT(index_found); - - typedef typename mpl::deref::type type; - }; - - template - typename index::type& get()BOOST_NOEXCEPT - { - return *this; - } - - template - const typename index::type& get()const BOOST_NOEXCEPT - { - return *this; - } -#endif - - /* projection of iterators by number */ - -#if !defined(BOOST_NO_MEMBER_TEMPLATES) - template - struct nth_index_iterator - { - typedef typename nth_index::type::iterator type; - }; - - template - struct nth_index_const_iterator - { - typedef typename nth_index::type::const_iterator type; - }; - - template - typename nth_index_iterator::type project(IteratorType it) - { - typedef typename nth_index::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* fails in Sun C++ 5.7 */ - BOOST_STATIC_ASSERT( - (mpl::contains::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - BOOST_MULTI_INDEX_CHECK_IS_OWNER( - it,static_cast(*this)); - - return index_type::make_iterator(static_cast(it.get_node())); - } - - template - typename nth_index_const_iterator::type project(IteratorType it)const - { - typedef typename nth_index::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* fails in Sun C++ 5.7 */ - BOOST_STATIC_ASSERT(( - mpl::contains::value|| - mpl::contains::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - BOOST_MULTI_INDEX_CHECK_IS_OWNER( - it,static_cast(*this)); - return index_type::make_iterator(static_cast(it.get_node())); - } -#endif - - /* projection of iterators by tag */ - -#if !defined(BOOST_NO_MEMBER_TEMPLATES) - template - struct index_iterator - { - typedef typename index::type::iterator type; - }; - - template - struct index_const_iterator - { - typedef typename index::type::const_iterator type; - }; - - template - typename index_iterator::type project(IteratorType it) - { - typedef typename index::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* fails in Sun C++ 5.7 */ - BOOST_STATIC_ASSERT( - (mpl::contains::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - BOOST_MULTI_INDEX_CHECK_IS_OWNER( - it,static_cast(*this)); - return index_type::make_iterator(static_cast(it.get_node())); - } - - template - typename index_const_iterator::type project(IteratorType it)const - { - typedef typename index::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* fails in Sun C++ 5.7 */ - BOOST_STATIC_ASSERT(( - mpl::contains::value|| - mpl::contains::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - BOOST_MULTI_INDEX_CHECK_IS_OWNER( - it,static_cast(*this)); - return index_type::make_iterator(static_cast(it.get_node())); - } -#endif - -BOOST_MULTI_INDEX_PROTECTED_IF_MEMBER_TEMPLATE_FRIENDS: - typedef typename super::copy_map_type copy_map_type; - -#if !defined(BOOST_NO_CXX11_HDR_INITIALIZER_LIST) - multi_index_container( - const multi_index_container& x, - detail::do_not_copy_elements_tag): - bfm_allocator(x.bfm_allocator::member), - bfm_header(), - super(x,detail::do_not_copy_elements_tag()), - node_count(0) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - } -#endif - - node_type* header()const - { - return &*bfm_header::member; - } - - node_type* allocate_node() - { - return &*bfm_allocator::member.allocate(1); - } - - void deallocate_node(node_type* x) - { - typedef typename node_allocator::pointer node_pointer; - bfm_allocator::member.deallocate(static_cast(x),1); - } - - bool empty_()const - { - return node_count==0; - } - - std::size_t size_()const - { - return node_count; - } - - std::size_t max_size_()const - { - return static_cast(-1); - } - - template - std::pair insert_(const Value& v,Variant variant) - { - node_type* x=0; - node_type* res=super::insert_(v,x,variant); - if(res==x){ - ++node_count; - return std::pair(res,true); - } - else{ - return std::pair(res,false); - } - } - - std::pair insert_(const Value& v) - { - return insert_(v,detail::lvalue_tag()); - } - - std::pair insert_rv_(const Value& v) - { - return insert_(v,detail::rvalue_tag()); - } - - template - std::pair insert_ref_(T& t) - { - node_type* x=allocate_node(); - BOOST_TRY{ - new(&x->value()) value_type(t); - BOOST_TRY{ - node_type* res=super::insert_(x->value(),x,detail::emplaced_tag()); - if(res==x){ - ++node_count; - return std::pair(res,true); - } - else{ - boost::detail::allocator::destroy(&x->value()); - deallocate_node(x); - return std::pair(res,false); - } - } - BOOST_CATCH(...){ - boost::detail::allocator::destroy(&x->value()); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - BOOST_CATCH(...){ - deallocate_node(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - std::pair insert_ref_(const value_type& x) - { - return insert_(x); - } - - std::pair insert_ref_(value_type& x) - { - return insert_(x); - } - - template - std::pair emplace_( - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - node_type* x=allocate_node(); - BOOST_TRY{ - detail::vartempl_placement_new( - &x->value(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - BOOST_TRY{ - node_type* res=super::insert_(x->value(),x,detail::emplaced_tag()); - if(res==x){ - ++node_count; - return std::pair(res,true); - } - else{ - boost::detail::allocator::destroy(&x->value()); - deallocate_node(x); - return std::pair(res,false); - } - } - BOOST_CATCH(...){ - boost::detail::allocator::destroy(&x->value()); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - BOOST_CATCH(...){ - deallocate_node(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - template - std::pair insert_( - const Value& v,node_type* position,Variant variant) - { - node_type* x=0; - node_type* res=super::insert_(v,position,x,variant); - if(res==x){ - ++node_count; - return std::pair(res,true); - } - else{ - return std::pair(res,false); - } - } - - std::pair insert_(const Value& v,node_type* position) - { - return insert_(v,position,detail::lvalue_tag()); - } - - std::pair insert_rv_(const Value& v,node_type* position) - { - return insert_(v,position,detail::rvalue_tag()); - } - - template - std::pair insert_ref_( - T& t,node_type* position) - { - node_type* x=allocate_node(); - BOOST_TRY{ - new(&x->value()) value_type(t); - BOOST_TRY{ - node_type* res=super::insert_( - x->value(),position,x,detail::emplaced_tag()); - if(res==x){ - ++node_count; - return std::pair(res,true); - } - else{ - boost::detail::allocator::destroy(&x->value()); - deallocate_node(x); - return std::pair(res,false); - } - } - BOOST_CATCH(...){ - boost::detail::allocator::destroy(&x->value()); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - BOOST_CATCH(...){ - deallocate_node(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - std::pair insert_ref_( - const value_type& x,node_type* position) - { - return insert_(x,position); - } - - std::pair insert_ref_( - value_type& x,node_type* position) - { - return insert_(x,position); - } - - template - std::pair emplace_hint_( - node_type* position, - BOOST_MULTI_INDEX_FUNCTION_PARAM_PACK) - { - node_type* x=allocate_node(); - BOOST_TRY{ - detail::vartempl_placement_new( - &x->value(),BOOST_MULTI_INDEX_FORWARD_PARAM_PACK); - BOOST_TRY{ - node_type* res=super::insert_( - x->value(),position,x,detail::emplaced_tag()); - if(res==x){ - ++node_count; - return std::pair(res,true); - } - else{ - boost::detail::allocator::destroy(&x->value()); - deallocate_node(x); - return std::pair(res,false); - } - } - BOOST_CATCH(...){ - boost::detail::allocator::destroy(&x->value()); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - BOOST_CATCH(...){ - deallocate_node(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - void erase_(node_type* x) - { - --node_count; - super::erase_(x); - deallocate_node(x); - } - - void delete_node_(node_type* x) - { - super::delete_node_(x); - deallocate_node(x); - } - - void delete_all_nodes_() - { - super::delete_all_nodes_(); - } - - void clear_() - { - delete_all_nodes_(); - super::clear_(); - node_count=0; - } - - void swap_(multi_index_container& x) - { - if(bfm_allocator::member!=x.bfm_allocator::member){ - detail::adl_swap(bfm_allocator::member,x.bfm_allocator::member); - } - std::swap(bfm_header::member,x.bfm_header::member); - super::swap_(x); - std::swap(node_count,x.node_count); - } - - void swap_elements_( - multi_index_container& x) - { - std::swap(bfm_header::member,x.bfm_header::member); - super::swap_elements_(x); - std::swap(node_count,x.node_count); - } - - bool replace_(const Value& k,node_type* x) - { - return super::replace_(k,x,detail::lvalue_tag()); - } - - bool replace_rv_(const Value& k,node_type* x) - { - return super::replace_(k,x,detail::rvalue_tag()); - } - - template - bool modify_(Modifier& mod,node_type* x) - { - mod(const_cast(x->value())); - - BOOST_TRY{ - if(!super::modify_(x)){ - deallocate_node(x); - --node_count; - return false; - } - else return true; - } - BOOST_CATCH(...){ - deallocate_node(x); - --node_count; - BOOST_RETHROW; - } - BOOST_CATCH_END - } - - template - bool modify_(Modifier& mod,Rollback& back_,node_type* x) - { - mod(const_cast(x->value())); - - bool b; - BOOST_TRY{ - b=super::modify_rollback_(x); - } - BOOST_CATCH(...){ - BOOST_TRY{ - back_(const_cast(x->value())); - BOOST_RETHROW; - } - BOOST_CATCH(...){ - this->erase_(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - BOOST_CATCH_END - - BOOST_TRY{ - if(!b){ - back_(const_cast(x->value())); - return false; - } - else return true; - } - BOOST_CATCH(...){ - this->erase_(x); - BOOST_RETHROW; - } - BOOST_CATCH_END - } - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) - /* serialization */ - - friend class boost::serialization::access; - - BOOST_SERIALIZATION_SPLIT_MEMBER() - - typedef typename super::index_saver_type index_saver_type; - typedef typename super::index_loader_type index_loader_type; - - template - void save(Archive& ar,const unsigned int version)const - { - const serialization::collection_size_type s(size_()); - const detail::serialization_version value_version; - ar< - void load(Archive& ar,const unsigned int version) - { - BOOST_MULTI_INDEX_CHECK_INVARIANT; - - clear_(); - serialization::collection_size_type s; - detail::serialization_version value_version; - if(version<1){ - std::size_t sz; - ar>>serialization::make_nvp("count",sz); - s=static_cast(sz); - } - else{ - ar>>serialization::make_nvp("count",s); - } - if(version<2){ - value_version=0; - } - else{ - ar>>serialization::make_nvp("value_version",value_version); - } - - index_loader_type lm(bfm_allocator::member,s); - - for(std::size_t n=0;n value("item",ar,value_version); - std::pair p=insert_( - value.get(),super::end().get_node()); - if(!p.second)throw_exception( - archive::archive_exception( - archive::archive_exception::other_exception)); - ar.reset_object_address(&p.first->value(),&value.get()); - lm.add(p.first,ar,version); - } - lm.add_track(header(),ar,version); - - super::load_(ar,version,lm); - } -#endif - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING) - /* invariant stuff */ - - bool invariant_()const - { - return super::invariant_(); - } - - void check_invariant_()const - { - BOOST_MULTI_INDEX_INVARIANT_ASSERT(invariant_()); - } -#endif - -private: - std::size_t node_count; - -#if defined(BOOST_MULTI_INDEX_ENABLE_INVARIANT_CHECKING)&&\ - BOOST_WORKAROUND(__MWERKS__,<=0x3003) -#pragma parse_mfunc_templ reset -#endif -}; - -#if BOOST_WORKAROUND(BOOST_MSVC,BOOST_TESTED_AT(1500)) -#pragma warning(pop) /* C4522 */ -#endif - -/* retrieval of indices by number */ - -template -struct nth_index -{ - BOOST_STATIC_CONSTANT( - int, - M=mpl::size::type::value); - BOOST_STATIC_ASSERT(N>=0&&N::type type; -}; - -template -typename nth_index< - multi_index_container,N>::type& -get( - multi_index_container& m)BOOST_NOEXCEPT -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename nth_index< - multi_index_container< - Value,IndexSpecifierList,Allocator>, - N - >::type index_type; - - BOOST_STATIC_ASSERT(N>=0&& - N< - mpl::size< - BOOST_DEDUCED_TYPENAME multi_index_type::index_type_list - >::type::value); - - return detail::converter::index(m); -} - -template -const typename nth_index< - multi_index_container,N>::type& -get( - const multi_index_container& m -)BOOST_NOEXCEPT -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename nth_index< - multi_index_container< - Value,IndexSpecifierList,Allocator>, - N - >::type index_type; - - BOOST_STATIC_ASSERT(N>=0&& - N< - mpl::size< - BOOST_DEDUCED_TYPENAME multi_index_type::index_type_list - >::type::value); - - return detail::converter::index(m); -} - -/* retrieval of indices by tag */ - -template -struct index -{ - typedef typename MultiIndexContainer::index_type_list index_type_list; - - typedef typename mpl::find_if< - index_type_list, - detail::has_tag - >::type iter; - - BOOST_STATIC_CONSTANT( - bool,index_found=!(is_same::type >::value)); - BOOST_STATIC_ASSERT(index_found); - - typedef typename mpl::deref::type type; -}; - -template< - typename Tag,typename Value,typename IndexSpecifierList,typename Allocator -> -typename ::boost::multi_index::index< - multi_index_container,Tag>::type& -get( - multi_index_container& m)BOOST_NOEXCEPT -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename ::boost::multi_index::index< - multi_index_container< - Value,IndexSpecifierList,Allocator>, - Tag - >::type index_type; - - return detail::converter::index(m); -} - -template< - typename Tag,typename Value,typename IndexSpecifierList,typename Allocator -> -const typename ::boost::multi_index::index< - multi_index_container,Tag>::type& -get( - const multi_index_container& m -)BOOST_NOEXCEPT -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename ::boost::multi_index::index< - multi_index_container< - Value,IndexSpecifierList,Allocator>, - Tag - >::type index_type; - - return detail::converter::index(m); -} - -/* projection of iterators by number */ - -template -struct nth_index_iterator -{ - typedef typename nth_index::type::iterator type; -}; - -template -struct nth_index_const_iterator -{ - typedef typename nth_index::type::const_iterator type; -}; - -template< - int N,typename IteratorType, - typename Value,typename IndexSpecifierList,typename Allocator> -typename nth_index_iterator< - multi_index_container,N>::type -project( - multi_index_container& m, - IteratorType it) -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename nth_index::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* Sun C++ 5.7 fails */ - BOOST_STATIC_ASSERT(( - mpl::contains< - BOOST_DEDUCED_TYPENAME multi_index_type::iterator_type_list, - IteratorType>::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef detail::converter< - multi_index_type, - BOOST_DEDUCED_TYPENAME IteratorType::container_type> converter; - BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,converter::index(m)); -#endif - - return detail::converter::iterator( - m,static_cast(it.get_node())); -} - -template< - int N,typename IteratorType, - typename Value,typename IndexSpecifierList,typename Allocator> -typename nth_index_const_iterator< - multi_index_container,N>::type -project( - const multi_index_container& m, - IteratorType it) -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename nth_index::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* Sun C++ 5.7 fails */ - BOOST_STATIC_ASSERT(( - mpl::contains< - BOOST_DEDUCED_TYPENAME multi_index_type::iterator_type_list, - IteratorType>::value|| - mpl::contains< - BOOST_DEDUCED_TYPENAME multi_index_type::const_iterator_type_list, - IteratorType>::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef detail::converter< - multi_index_type, - BOOST_DEDUCED_TYPENAME IteratorType::container_type> converter; - BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,converter::index(m)); -#endif - - return detail::converter::const_iterator( - m,static_cast(it.get_node())); -} - -/* projection of iterators by tag */ - -template -struct index_iterator -{ - typedef typename ::boost::multi_index::index< - MultiIndexContainer,Tag>::type::iterator type; -}; - -template -struct index_const_iterator -{ - typedef typename ::boost::multi_index::index< - MultiIndexContainer,Tag>::type::const_iterator type; -}; - -template< - typename Tag,typename IteratorType, - typename Value,typename IndexSpecifierList,typename Allocator> -typename index_iterator< - multi_index_container,Tag>::type -project( - multi_index_container& m, - IteratorType it) -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename ::boost::multi_index::index< - multi_index_type,Tag>::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* Sun C++ 5.7 fails */ - BOOST_STATIC_ASSERT(( - mpl::contains< - BOOST_DEDUCED_TYPENAME multi_index_type::iterator_type_list, - IteratorType>::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef detail::converter< - multi_index_type, - BOOST_DEDUCED_TYPENAME IteratorType::container_type> converter; - BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,converter::index(m)); -#endif - - return detail::converter::iterator( - m,static_cast(it.get_node())); -} - -template< - typename Tag,typename IteratorType, - typename Value,typename IndexSpecifierList,typename Allocator> -typename index_const_iterator< - multi_index_container,Tag>::type -project( - const multi_index_container& m, - IteratorType it) -{ - typedef multi_index_container< - Value,IndexSpecifierList,Allocator> multi_index_type; - typedef typename ::boost::multi_index::index< - multi_index_type,Tag>::type index_type; - -#if !defined(__SUNPRO_CC)||!(__SUNPRO_CC<0x580) /* Sun C++ 5.7 fails */ - BOOST_STATIC_ASSERT(( - mpl::contains< - BOOST_DEDUCED_TYPENAME multi_index_type::iterator_type_list, - IteratorType>::value|| - mpl::contains< - BOOST_DEDUCED_TYPENAME multi_index_type::const_iterator_type_list, - IteratorType>::value)); -#endif - - BOOST_MULTI_INDEX_CHECK_VALID_ITERATOR(it); - -#if defined(BOOST_MULTI_INDEX_ENABLE_SAFE_MODE) - typedef detail::converter< - multi_index_type, - BOOST_DEDUCED_TYPENAME IteratorType::container_type> converter; - BOOST_MULTI_INDEX_CHECK_IS_OWNER(it,converter::index(m)); -#endif - - return detail::converter::const_iterator( - m,static_cast(it.get_node())); -} - -/* Comparison. Simple forward to first index. */ - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator==( - const multi_index_container& x, - const multi_index_container& y) -{ - return get<0>(x)==get<0>(y); -} - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator<( - const multi_index_container& x, - const multi_index_container& y) -{ - return get<0>(x)(y); -} - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator!=( - const multi_index_container& x, - const multi_index_container& y) -{ - return get<0>(x)!=get<0>(y); -} - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator>( - const multi_index_container& x, - const multi_index_container& y) -{ - return get<0>(x)>get<0>(y); -} - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator>=( - const multi_index_container& x, - const multi_index_container& y) -{ - return get<0>(x)>=get<0>(y); -} - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator<=( - const multi_index_container& x, - const multi_index_container& y) -{ - return get<0>(x)<=get<0>(y); -} - -/* specialized algorithms */ - -template -void swap( - multi_index_container& x, - multi_index_container& y) -{ - x.swap(y); -} - -} /* namespace multi_index */ - -#if !defined(BOOST_MULTI_INDEX_DISABLE_SERIALIZATION) -/* class version = 1 : we now serialize the size through - * boost::serialization::collection_size_type. - * class version = 2 : proper use of {save|load}_construct_data. - */ - -namespace serialization { -template -struct version< - boost::multi_index_container -> -{ - BOOST_STATIC_CONSTANT(int,value=2); -}; -} /* namespace serialization */ -#endif - -/* Associated global functions are promoted to namespace boost, except - * comparison operators and swap, which are meant to be Koenig looked-up. - */ - -using multi_index::get; -using multi_index::project; - -} /* namespace boost */ - -#undef BOOST_MULTI_INDEX_CHECK_INVARIANT -#undef BOOST_MULTI_INDEX_CHECK_INVARIANT_OF - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/multi_index_container_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/multi_index_container_fwd.hpp deleted file mode 100644 index b35acad407a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/multi_index_container_fwd.hpp +++ /dev/null @@ -1,121 +0,0 @@ -/* Copyright 2003-2013 Joaquin M Lopez Munoz. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/multi_index for library home page. - */ - -#ifndef BOOST_MULTI_INDEX_FWD_HPP -#define BOOST_MULTI_INDEX_FWD_HPP - -#if defined(_MSC_VER) -#pragma once -#endif - -#include /* keep it first to prevent nasty warns in MSVC */ -#include -#include -#include -#include - -namespace boost{ - -namespace multi_index{ - -/* Default value for IndexSpecifierList specifies a container - * equivalent to std::set. - */ - -template< - typename Value, - typename IndexSpecifierList=indexed_by > >, - typename Allocator=std::allocator > -class multi_index_container; - -template -struct nth_index; - -template -struct index; - -template -struct nth_index_iterator; - -template -struct nth_index_const_iterator; - -template -struct index_iterator; - -template -struct index_const_iterator; - -/* get and project functions not fwd declared due to problems - * with dependent typenames - */ - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator==( - const multi_index_container& x, - const multi_index_container& y); - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator<( - const multi_index_container& x, - const multi_index_container& y); - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator!=( - const multi_index_container& x, - const multi_index_container& y); - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator>( - const multi_index_container& x, - const multi_index_container& y); - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator>=( - const multi_index_container& x, - const multi_index_container& y); - -template< - typename Value1,typename IndexSpecifierList1,typename Allocator1, - typename Value2,typename IndexSpecifierList2,typename Allocator2 -> -bool operator<=( - const multi_index_container& x, - const multi_index_container& y); - -template -void swap( - multi_index_container& x, - multi_index_container& y); - -} /* namespace multi_index */ - -/* multi_index_container, being the main type of this library, is promoted to - * namespace boost. - */ - -using multi_index::multi_index_container; - -} /* namespace boost */ - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/access.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/access.hpp deleted file mode 100644 index f6581accc91..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/access.hpp +++ /dev/null @@ -1,145 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ACCESS_HPP -#define BOOST_SERIALIZATION_ACCESS_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// access.hpp: interface for serialization system. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -namespace boost { - -namespace archive { -namespace detail { - template - class iserializer; - template - class oserializer; -} // namespace detail -} // namespace archive - -namespace serialization { - -// forward declarations -template -inline void serialize_adl(Archive &, T &, const unsigned int); -namespace detail { - template - struct member_saver; - template - struct member_loader; -} // namespace detail - -// use an "accessor class so that we can use: -// "friend class boost::serialization::access;" -// in any serialized class to permit clean, safe access to private class members -// by the serialization system - -class access { -public: - // grant access to "real" serialization defaults -#ifdef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -public: -#else - template - friend struct detail::member_saver; - template - friend struct detail::member_loader; - template - friend class archive::detail::iserializer; - template - friend class archive::detail::oserializer; - template - friend inline void serialize( - Archive & ar, - T & t, - const unsigned int file_version - ); - template - friend inline void save_construct_data( - Archive & ar, - const T * t, - const unsigned int file_version - ); - template - friend inline void load_construct_data( - Archive & ar, - T * t, - const unsigned int file_version - ); -#endif - - // pass calls to users's class implementation - template - static void member_save( - Archive & ar, - //const T & t, - T & t, - const unsigned int file_version - ){ - t.save(ar, file_version); - } - template - static void member_load( - Archive & ar, - T & t, - const unsigned int file_version - ){ - t.load(ar, file_version); - } - template - static void serialize( - Archive & ar, - T & t, - const unsigned int file_version - ){ - // note: if you get a compile time error here with a - // message something like: - // cannot convert parameter 1 from to - // a likely possible cause is that the class T contains a - // serialize function - but that serialize function isn't - // a template and corresponds to a file type different than - // the class Archive. To resolve this, don't include an - // archive type other than that for which the serialization - // function is defined!!! - t.serialize(ar, file_version); - } - template - static void destroy( const T * t) // const appropriate here? - { - // the const business is an MSVC 6.0 hack that should be - // benign on everything else - delete const_cast(t); - } - template - static void construct(T * t){ - // default is inplace invocation of default constructor - // Note the :: before the placement new. Required if the - // class doesn't have a class-specific placement new defined. - ::new(t)T; - } - template - static T & cast_reference(U & u){ - return static_cast(u); - } - template - static T * cast_pointer(U * u){ - return static_cast(u); - } -}; - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_ACCESS_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_map.hpp deleted file mode 100644 index ccf806b1813..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_map.hpp +++ /dev/null @@ -1,85 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_MAP_HPP -#define BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/unordered_map.hpp: -// serialization for stl unordered_map templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { -namespace stl { - -// map input -template -struct archive_input_unordered_map -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - ar >> boost::serialization::make_nvp("item", t.reference()); - std::pair result = - s.insert(boost::move(t.reference())); - // note: the following presumes that the map::value_type was NOT tracked - // in the archive. This is the usual case, but here there is no way - // to determine that. - if(result.second){ - ar.reset_object_address( - & (result.first->second), - & t.reference().second - ); - } - } -}; - -// multimap input -template -struct archive_input_unordered_multimap -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - ar >> boost::serialization::make_nvp("item", t.reference()); - typename Container::const_iterator result = - s.insert(t.reference()); - // note: the following presumes that the map::value_type was NOT tracked - // in the archive. This is the usual case, but here there is no way - // to determine that. - ar.reset_object_address( - & result->second, - & t.reference() - ); - } -}; - -} // stl -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_set.hpp deleted file mode 100644 index 7f0003cc6a4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/archive_input_unordered_set.hpp +++ /dev/null @@ -1,72 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_SET_HPP -#define BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_SET_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// archive_input_unordered_set.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -namespace stl { - -// unordered_set input -template -struct archive_input_unordered_set -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - // borland fails silently w/o full namespace - ar >> boost::serialization::make_nvp("item", t.reference()); - std::pair result = - s.insert(boost::move(t.reference())); - if(result.second) - ar.reset_object_address(& (* result.first), & t.reference()); - } -}; - -// unordered_multiset input -template -struct archive_input_unordered_multiset -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - ar >> boost::serialization::make_nvp("item", t.reference()); - typename Container::const_iterator result = - s.insert(boost::move(t.reference())); - ar.reset_object_address(& (* result), & t.reference()); - } -}; - -} // stl -} // serialization -} // boost - -#endif // BOOST_SERIALIZATION_ARCHIVE_INPUT_UNORDERED_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/array.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/array.hpp deleted file mode 100644 index 612d1a61985..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/array.hpp +++ /dev/null @@ -1,48 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ARRAY_HPP -#define BOOST_SERIALIZATION_ARRAY_HPP - -// (C) Copyright 2005 Matthias Troyer and Dave Abrahams -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// for serialization of . If not supported by the standard -// library - this file becomes empty. This is to avoid breaking backward -// compatibiliy for applications which used this header to support -// serialization of native arrays. Code to serialize native arrays is -// now always include by default. RR - -#include // msvc 6.0 needs this for warning suppression - -#if defined(BOOST_NO_STDC_NAMESPACE) - -#include -#include // std::size_t -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include - -#ifndef BOOST_NO_CXX11_HDR_ARRAY - -#include -#include - -namespace boost { namespace serialization { - -template -void serialize(Archive& ar, std::array& a, const unsigned int /* version */) -{ - ar & boost::serialization::make_nvp( - "elems", - *static_cast(static_cast(a.data())) - ); - -} -} } // end namespace boost::serialization - -#endif // BOOST_NO_CXX11_HDR_ARRAY - -#endif //BOOST_SERIALIZATION_ARRAY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/array_optimization.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/array_optimization.hpp deleted file mode 100644 index 40dffba871a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/array_optimization.hpp +++ /dev/null @@ -1,37 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ARRAY_OPTIMIZATON_HPP -#define BOOST_SERIALIZATION_ARRAY_OPTIMIZATON_HPP - -// (C) Copyright 2005 Matthias Troyer and Dave Abrahams -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include // msvc 6.0 needs this for warning suppression - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include - -namespace boost { namespace serialization { - -template -struct use_array_optimization : boost::mpl::always {}; - -} } // end namespace boost::serialization - -#define BOOST_SERIALIZATION_USE_ARRAY_OPTIMIZATION(Archive) \ -namespace boost { namespace serialization { \ -template <> struct use_array_optimization { \ - template \ - struct apply : boost::mpl::apply1::type \ - >::type {}; \ -}; }} - -#endif //BOOST_SERIALIZATION_ARRAY_OPTIMIZATON_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/array_wrapper.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/array_wrapper.hpp deleted file mode 100644 index adf436e15b4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/array_wrapper.hpp +++ /dev/null @@ -1,121 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ARRAY_WRAPPER_HPP -#define BOOST_SERIALIZATION_ARRAY_WRAPPER_HPP - -// (C) Copyright 2005 Matthias Troyer and Dave Abrahams -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -//#include - -#include // msvc 6.0 needs this for warning suppression - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { namespace serialization { - -template -class array_wrapper : - public wrapper_traits > -{ -private: - array_wrapper & operator=(const array_wrapper & rhs); - // note: I would like to make the copy constructor private but this breaks - // make_array. So I make make_array a friend - template - friend const boost::serialization::array_wrapper make_array(Tx * t, S s); -public: - - array_wrapper(const array_wrapper & rhs) : - m_t(rhs.m_t), - m_element_count(rhs.m_element_count) - {} -public: - array_wrapper(T * t, std::size_t s) : - m_t(t), - m_element_count(s) - {} - - // default implementation - template - void serialize_optimized(Archive &ar, const unsigned int, mpl::false_ ) const - { - // default implemention does the loop - std::size_t c = count(); - T * t = address(); - while(0 < c--) - ar & boost::serialization::make_nvp("item", *t++); - } - - // optimized implementation - template - void serialize_optimized(Archive &ar, const unsigned int version, mpl::true_ ) - { - boost::serialization::split_member(ar, *this, version); - } - - // default implementation - template - void save(Archive &ar, const unsigned int version) const - { - ar.save_array(*this,version); - } - - // default implementation - template - void load(Archive &ar, const unsigned int version) - { - ar.load_array(*this,version); - } - - // default implementation - template - void serialize(Archive &ar, const unsigned int version) - { - typedef typename - boost::serialization::use_array_optimization::template apply< - typename remove_const< T >::type - >::type use_optimized; - serialize_optimized(ar,version,use_optimized()); - } - - T * address() const - { - return m_t; - } - - std::size_t count() const - { - return m_element_count; - } - -private: - T * const m_t; - const std::size_t m_element_count; -}; - -template -inline -const array_wrapper< T > make_array(T* t, S s){ - const array_wrapper< T > a(t, s); - return a; -} - -} } // end namespace boost::serialization - - -#endif //BOOST_SERIALIZATION_ARRAY_WRAPPER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/assume_abstract.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/assume_abstract.hpp deleted file mode 100644 index 632f9312f5f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/assume_abstract.hpp +++ /dev/null @@ -1,60 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ASSUME_ABSTRACT_HPP -#define BOOST_SERIALIZATION_ASSUME_ABSTRACT_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// assume_abstract_class.hpp: - -// (C) Copyright 2008 Robert Ramey -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// this is useful for compilers which don't support the boost::is_abstract - -#include -#include - -#ifndef BOOST_NO_IS_ABSTRACT - -// if there is an intrinsic is_abstract defined, we don't have to do anything -#define BOOST_SERIALIZATION_ASSUME_ABSTRACT(T) - -// but forward to the "official" is_abstract -namespace boost { -namespace serialization { - template - struct is_abstract : boost::is_abstract< T > {} ; -} // namespace serialization -} // namespace boost - -#else -// we have to "make" one - -namespace boost { -namespace serialization { - template - struct is_abstract : boost::false_type {}; -} // namespace serialization -} // namespace boost - -// define a macro to make explicit designation of this more transparent -#define BOOST_SERIALIZATION_ASSUME_ABSTRACT(T) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct is_abstract< T > : boost::true_type {}; \ -template<> \ -struct is_abstract< const T > : boost::true_type {}; \ -}} \ -/**/ - -#endif // BOOST_NO_IS_ABSTRACT - -#endif //BOOST_SERIALIZATION_ASSUME_ABSTRACT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/base_object.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/base_object.hpp deleted file mode 100644 index 1a82cecd4b5..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/base_object.hpp +++ /dev/null @@ -1,100 +0,0 @@ -#ifndef BOOST_SERIALIZATION_BASE_OBJECT_HPP -#define BOOST_SERIALIZATION_BASE_OBJECT_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// base_object.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// if no archive headers have been included this is a no op -// this is to permit BOOST_EXPORT etc to be included in a -// file declaration header - -#include -#include - -#include -#include -#include - -#include -#include -#include -#include - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -namespace detail -{ - // get the base type for a given derived type - // preserving the const-ness - template - struct base_cast - { - typedef typename - mpl::if_< - is_const, - const B, - B - >::type type; - BOOST_STATIC_ASSERT(is_const::value == is_const::value); - }; - - // only register void casts if the types are polymorphic - template - struct base_register - { - struct polymorphic { - static void const * invoke(){ - Base const * const b = 0; - Derived const * const d = 0; - return & void_cast_register(d, b); - } - }; - struct non_polymorphic { - static void const * invoke(){ - return 0; - } - }; - static void const * invoke(){ - typedef typename mpl::eval_if< - is_polymorphic, - mpl::identity, - mpl::identity - >::type type; - return type::invoke(); - } - }; - -} // namespace detail -template -typename detail::base_cast::type & -base_object(Derived &d) -{ - BOOST_STATIC_ASSERT(( is_base_and_derived::value)); - BOOST_STATIC_ASSERT(! is_pointer::value); - typedef typename detail::base_cast::type type; - detail::base_register::invoke(); - return access::cast_reference(d); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_BASE_OBJECT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/binary_object.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/binary_object.hpp deleted file mode 100644 index 5c9038e5a9f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/binary_object.hpp +++ /dev/null @@ -1,79 +0,0 @@ -#ifndef BOOST_SERIALIZATION_BINARY_OBJECT_HPP -#define BOOST_SERIALIZATION_BINARY_OBJECT_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// nvp.hpp: interface for serialization system. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include // std::size_t -#include -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -struct binary_object : - public wrapper_traits > -{ - void const * m_t; - std::size_t m_size; - template - void save(Archive & ar, const unsigned int /* file_version */) const { - ar.save_binary(m_t, m_size); - } - template - void load(Archive & ar, const unsigned int /* file_version */) const { - ar.load_binary(const_cast(m_t), m_size); - } - BOOST_SERIALIZATION_SPLIT_MEMBER() - binary_object & operator=(const binary_object & rhs) { - m_t = rhs.m_t; - m_size = rhs.m_size; - return *this; - } - binary_object(const void * const t, std::size_t size) : - m_t(t), - m_size(size) - {} - binary_object(const binary_object & rhs) : - m_t(rhs.m_t), - m_size(rhs.m_size) - {} -}; - -// just a little helper to support the convention that all serialization -// wrappers follow the naming convention make_xxxxx -inline -const binary_object -make_binary_object(const void * t, std::size_t size){ - return binary_object(t, size); -} - -} // namespace serialization -} // boost - -#endif // BOOST_SERIALIZATION_BINARY_OBJECT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/bitset.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/bitset.hpp deleted file mode 100644 index 78f9bd74336..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/bitset.hpp +++ /dev/null @@ -1,75 +0,0 @@ -/*! - * \file bitset.hpp - * \brief Provides Boost.Serialization support for std::bitset - * \author Brian Ravnsgaard Riis - * \author Kenneth Riddile - * \date 16.09.2004, updated 04.03.2009 - * \copyright 2004 Brian Ravnsgaard Riis - * \license Boost Software License 1.0 - */ -#ifndef BOOST_SERIALIZATION_BITSET_HPP -#define BOOST_SERIALIZATION_BITSET_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -#include -#include // size_t - -#include -#include -#include -#include - -namespace boost{ -namespace serialization{ - -template -inline void save( - Archive & ar, - std::bitset const & t, - const unsigned int /* version */ -){ - const std::string bits = t.template to_string< - std::string::value_type, - std::string::traits_type, - std::string::allocator_type - >(); - ar << BOOST_SERIALIZATION_NVP( bits ); -} - -template -inline void load( - Archive & ar, - std::bitset & t, - const unsigned int /* version */ -){ - std::string bits; - ar >> BOOST_SERIALIZATION_NVP( bits ); - t = std::bitset(bits); -} - -template -inline void serialize( - Archive & ar, - std::bitset & t, - const unsigned int version -){ - boost::serialization::split_free( ar, t, version ); -} - -// don't track bitsets since that would trigger tracking -// all over the program - which probably would be a surprise. -// also, tracking would be hard to implement since, we're -// serialization a representation of the data rather than -// the data itself. -template -struct tracking_level > - : mpl::int_ {} ; - -} //serialization -} //boost - -#endif // BOOST_SERIALIZATION_BITSET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/boost_array.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/boost_array.hpp deleted file mode 100644 index d564ff15de0..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/boost_array.hpp +++ /dev/null @@ -1,33 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ARRAY_HPP -#define BOOST_SERIALIZATION_ARRAY_HPP - -// (C) Copyright 2005 Matthias Troyer and Dave Abrahams -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -//#include - -#include // msvc 6.0 needs this for warning suppression - -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -#include -#include - -namespace boost { namespace serialization { -// implement serialization for boost::array -template -void serialize(Archive& ar, boost::array& a, const unsigned int /* version */) -{ - ar & boost::serialization::make_nvp("elems", a.elems); -} - -} } // end namespace boost::serialization - - -#endif //BOOST_SERIALIZATION_ARRAY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_map.hpp deleted file mode 100644 index 8913b31f9e6..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_map.hpp +++ /dev/null @@ -1,154 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UNORDERED_MAP_HPP -#define BOOST_SERIALIZATION_UNORDERED_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/unordered_map.hpp: -// serialization for stl unordered_map templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const boost::unordered_map &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - boost::unordered_map - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - boost::unordered_map &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - boost::unordered_map, - boost::serialization::stl::archive_input_unordered_map< - Archive, - boost::unordered_map - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - boost::unordered_map &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// unordered_multimap -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const boost::unordered_multimap &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - boost::unordered_multimap - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - boost::unordered_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - boost::unordered_multimap, - boost::serialization::stl::archive_input_unordered_multimap< - Archive, - boost::unordered_multimap - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - boost::unordered_multimap &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_UNORDERED_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_set.hpp deleted file mode 100644 index 307c7819cbd..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/boost_unordered_set.hpp +++ /dev/null @@ -1,150 +0,0 @@ -#ifndef BOOST_SERIALIZATION_BOOST_UNORDERED_SET_HPP -#define BOOST_SERIALIZATION_BOOST_UNORDERED_SET_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// unordered_set.hpp: serialization for boost unordered_set templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const boost::unordered_set &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - boost::unordered_set - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - boost::unordered_set &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - boost::unordered_set, - boost::serialization::stl::archive_input_unordered_set< - Archive, - boost::unordered_set - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - boost::unordered_set &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// unordered_multiset -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const boost::unordered_multiset &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - boost::unordered_multiset - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - boost::unordered_multiset &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - boost::unordered_multiset, - boost::serialization::stl::archive_input_unordered_multiset< - Archive, - boost::unordered_multiset - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - boost::unordered_multiset &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_BOOST_UNORDERED_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/collection_size_type.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/collection_size_type.hpp deleted file mode 100644 index 2dd8fa72584..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/collection_size_type.hpp +++ /dev/null @@ -1,62 +0,0 @@ -#ifndef BOOST_SERIALIZATION_COLLECTION_SIZE_TYPE_HPP -#define BOOST_SERIALIZATION_COLLECTION_SIZE_TYPE_HPP - -// (C) Copyright 2005 Matthias Troyer -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include // size_t -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -//BOOST_STRONG_TYPEDEF(std::size_t, collection_size_type) - -class collection_size_type { -private: - typedef std::size_t base_type; - base_type t; -public: - collection_size_type(): t(0) {}; - explicit collection_size_type(const std::size_t & t_) : - t(t_) - {} - collection_size_type(const collection_size_type & t_) : - t(t_.t) - {} - collection_size_type & operator=(const collection_size_type & rhs){ - t = rhs.t; - return *this; - } - collection_size_type & operator=(const unsigned int & rhs){ - t = rhs; - return *this; - } - // used for text output - operator base_type () const { - return t; - } - // used for text input - operator base_type & () { - return t; - } - bool operator==(const collection_size_type & rhs) const { - return t == rhs.t; - } - bool operator<(const collection_size_type & rhs) const { - return t < rhs.t; - } -}; - - -} } // end namespace boost::serialization - -BOOST_CLASS_IMPLEMENTATION(collection_size_type, primitive_type) -BOOST_IS_BITWISE_SERIALIZABLE(collection_size_type) - -#endif //BOOST_SERIALIZATION_COLLECTION_SIZE_TYPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/collection_traits.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/collection_traits.hpp deleted file mode 100644 index 3ec9401eff0..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/collection_traits.hpp +++ /dev/null @@ -1,79 +0,0 @@ -#ifndef BOOST_SERIALIZATION_COLLECTION_TRAITS_HPP -#define BOOST_SERIALIZATION_COLLECTION_TRAITS_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// collection_traits.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// This header assigns a level implemenation trait to a collection type -// for all primitives. It is needed so that archives which are meant to be -// portable don't write class information in the archive. Since, not all -// compiles recognize the same set of primitive types, the possibility -// exists for archives to be non-portable if class information for primitive -// types is included. This is addressed by the following macros. -#include -//#include -#include - -#include -#include -#include // ULONG_MAX -#include - -#define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(T, C) \ -template<> \ -struct implementation_level< C < T > > { \ - typedef mpl::integral_c_tag tag; \ - typedef mpl::int_ type; \ - BOOST_STATIC_CONSTANT(int, value = object_serializable); \ -}; \ -/**/ - -#if defined(BOOST_NO_CWCHAR) || defined(BOOST_NO_INTRINSIC_WCHAR_T) - #define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_WCHAR(C) -#else - #define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_WCHAR(C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(wchar_t, C) \ - /**/ -#endif - -#if defined(BOOST_HAS_LONG_LONG) - #define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_INT64(C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(boost::long_long_type, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(boost::ulong_long_type, C) \ - /**/ -#else - #define BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_INT64(C) -#endif - -#define BOOST_SERIALIZATION_COLLECTION_TRAITS(C) \ - namespace boost { namespace serialization { \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(bool, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(char, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(signed char, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(unsigned char, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(signed int, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(unsigned int, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(signed long, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(unsigned long, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(float, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(double, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(unsigned short, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER(signed short, C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_INT64(C) \ - BOOST_SERIALIZATION_COLLECTION_TRAITS_HELPER_WCHAR(C) \ - } } \ - /**/ - -#endif // BOOST_SERIALIZATION_COLLECTION_TRAITS diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/collections_load_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/collections_load_imp.hpp deleted file mode 100644 index e042c0c130d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/collections_load_imp.hpp +++ /dev/null @@ -1,106 +0,0 @@ -#ifndef BOOST_SERIALIZATION_COLLECTIONS_LOAD_IMP_HPP -#define BOOST_SERIALIZATION_COLLECTIONS_LOAD_IMP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -#if defined(_MSC_VER) && (_MSC_VER <= 1020) -# pragma warning (disable : 4786) // too long name, harmless warning -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// collections_load_imp.hpp: serialization for loading stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// helper function templates for serialization of collections - -#include -#include // size_t -#include // msvc 6.0 needs this for warning suppression -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost{ -namespace serialization { -namespace stl { - -////////////////////////////////////////////////////////////////////// -// implementation of serialization for STL containers -// - -template< - class Archive, - class T -> -typename boost::enable_if< - typename detail::is_default_constructible< - typename T::value_type - >, - void ->::type -collection_load_impl( - Archive & ar, - T & t, - collection_size_type count, - item_version_type /*item_version*/ -){ - t.resize(count); - typename T::iterator hint; - hint = t.begin(); - while(count-- > 0){ - ar >> boost::serialization::make_nvp("item", *hint++); - } -} - -template< - class Archive, - class T -> -typename boost::disable_if< - typename detail::is_default_constructible< - typename T::value_type - >, - void ->::type -collection_load_impl( - Archive & ar, - T & t, - collection_size_type count, - item_version_type item_version -){ - t.clear(); - while(count-- > 0){ - detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - t.push_back(boost::move(u.reference())); - ar.reset_object_address(& t.back() , & u.reference()); - } -} - -} // namespace stl -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_COLLECTIONS_LOAD_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/collections_save_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/collections_save_imp.hpp deleted file mode 100644 index f3cabfcf3f5..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/collections_save_imp.hpp +++ /dev/null @@ -1,82 +0,0 @@ -#ifndef BOOST_SERIALIZATION_COLLECTIONS_SAVE_IMP_HPP -#define BOOST_SERIALIZATION_COLLECTIONS_SAVE_IMP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// collections_save_imp.hpp: serialization for stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// helper function templates for serialization of collections - -#include -#include -#include -#include -#include -#include - -namespace boost{ -namespace serialization { -namespace stl { - -////////////////////////////////////////////////////////////////////// -// implementation of serialization for STL containers -// - -template -inline void save_collection( - Archive & ar, - const Container &s, - collection_size_type count) -{ - ar << BOOST_SERIALIZATION_NVP(count); - // record number of elements - const item_version_type item_version( - version::value - ); - #if 0 - boost::archive::library_version_type library_version( - ar.get_library_version() - ); - if(boost::archive::library_version_type(3) < library_version){ - ar << BOOST_SERIALIZATION_NVP(item_version); - } - #else - ar << BOOST_SERIALIZATION_NVP(item_version); - #endif - - typename Container::const_iterator it = s.begin(); - while(count-- > 0){ - // note borland emits a no-op without the explicit namespace - boost::serialization::save_construct_data_adl( - ar, - &(*it), - item_version - ); - ar << boost::serialization::make_nvp("item", *it++); - } -} - -template -inline void save_collection(Archive & ar, const Container &s) -{ - // record number of elements - collection_size_type count(s.size()); - save_collection(ar, s, count); -} - -} // namespace stl -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_COLLECTIONS_SAVE_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/complex.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/complex.hpp deleted file mode 100644 index b4ef44cf973..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/complex.hpp +++ /dev/null @@ -1,81 +0,0 @@ -#ifndef BOOST_SERIALIZATION_COMPLEX_HPP -#define BOOST_SERIALIZATION_COMPLEX_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/utility.hpp: -// serialization for stl utility templates - -// (C) Copyright 2007 Matthias Troyer . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include - -namespace boost { -namespace serialization { - -template -inline void serialize( - Archive & ar, - std::complex< T > & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -template -inline void save( - Archive & ar, - std::complex< T > const & t, - const unsigned int /* file_version */ -){ - const T re = t.real(); - const T im = t.imag(); - ar << boost::serialization::make_nvp("real", re); - ar << boost::serialization::make_nvp("imag", im); -} - -template -inline void load( - Archive & ar, - std::complex< T >& t, - const unsigned int /* file_version */ -){ - T re; - T im; - ar >> boost::serialization::make_nvp("real", re); - ar >> boost::serialization::make_nvp("imag", im); - t = std::complex< T >(re,im); -} - -// specialization of serialization traits for complex -template -struct is_bitwise_serializable > - : public is_bitwise_serializable< T > {}; - -template -struct implementation_level > - : mpl::int_ {} ; - -// treat complex just like builtin arithmetic types for tracking -template -struct tracking_level > - : mpl::int_ {} ; - -} // serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_COMPLEX_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/config.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/config.hpp deleted file mode 100644 index ea8cb9239ed..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/config.hpp +++ /dev/null @@ -1,74 +0,0 @@ -#ifndef BOOST_SERIALIZATION_CONFIG_HPP -#define BOOST_SERIALIZATION_CONFIG_HPP - -// config.hpp ---------------------------------------------// - -// (c) Copyright Robert Ramey 2004 -// Use, modification, and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See library home page at http://www.boost.org/libs/serialization - -//----------------------------------------------------------------------------// - -// This header implements separate compilation features as described in -// http://www.boost.org/more/separate_compilation.html - -#include -#include - -// note: this version incorporates the related code into the the -// the same library as BOOST_ARCHIVE. This could change some day in the -// future - -// if BOOST_SERIALIZATION_DECL is defined undefine it now: -#ifdef BOOST_SERIALIZATION_DECL - #undef BOOST_SERIALIZATION_DECL -#endif - -// we need to import/export our code only if the user has specifically -// asked for it by defining either BOOST_ALL_DYN_LINK if they want all boost -// libraries to be dynamically linked, or BOOST_SERIALIZATION_DYN_LINK -// if they want just this one to be dynamically liked: -#if defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) - #if !defined(BOOST_DYN_LINK) - #define BOOST_DYN_LINK - #endif - // export if this is our own source, otherwise import: - #if defined(BOOST_SERIALIZATION_SOURCE) - #define BOOST_SERIALIZATION_DECL BOOST_SYMBOL_EXPORT - #else - #define BOOST_SERIALIZATION_DECL BOOST_SYMBOL_IMPORT - #endif // defined(BOOST_SERIALIZATION_SOURCE) -#endif // defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) - -// if BOOST_SERIALIZATION_DECL isn't defined yet define it now: -#ifndef BOOST_SERIALIZATION_DECL - #define BOOST_SERIALIZATION_DECL -#endif - -// enable automatic library variant selection ------------------------------// - -#if !defined(BOOST_ALL_NO_LIB) && !defined(BOOST_SERIALIZATION_NO_LIB) \ -&& !defined(BOOST_ARCHIVE_SOURCE) && !defined(BOOST_WARCHIVE_SOURCE) \ -&& !defined(BOOST_SERIALIZATION_SOURCE) - // - // Set the name of our library, this will get undef'ed by auto_link.hpp - // once it's done with it: - // - #define BOOST_LIB_NAME boost_serialization - // - // If we're importing code from a dll, then tell auto_link.hpp about it: - // - #if defined(BOOST_ALL_DYN_LINK) || defined(BOOST_SERIALIZATION_DYN_LINK) - # define BOOST_DYN_LINK - #endif - // - // And include the header that does the work: - // - #include - -#endif - -#endif // BOOST_SERIALIZATION_CONFIG_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/deque.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/deque.hpp deleted file mode 100644 index bba81364ce2..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/deque.hpp +++ /dev/null @@ -1,80 +0,0 @@ -#ifndef BOOST_SERIALIZATION_DEQUE_HPP -#define BOOST_SERIALIZATION_DEQUE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// deque.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include - -#include -#include -#include - -namespace boost { -namespace serialization { - -template -inline void save( - Archive & ar, - const std::deque &t, - const unsigned int /* file_version */ -){ - boost::serialization::stl::save_collection< - Archive, std::deque - >(ar, t); -} - -template -inline void load( - Archive & ar, - std::deque &t, - const unsigned int /* file_version */ -){ - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - stl::collection_load_impl(ar, t, count, item_version); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::deque &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(std::deque) - -#endif // BOOST_SERIALIZATION_DEQUE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/is_default_constructible.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/is_default_constructible.hpp deleted file mode 100644 index 4d20b13bf3e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/detail/is_default_constructible.hpp +++ /dev/null @@ -1,54 +0,0 @@ -#ifndef BOOST_SERIALIZATION_DETAIL_IS_DEFAULT_CONSTRUCTIBLE_HPP -#define BOOST_SERIALIZATION_DETAIL_IS_DEFAULT_CONSTRUCTIBLE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// is_default_constructible.hpp: serialization for loading stl collections -// -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#if ! defined(BOOST_NO_CXX11_HDR_TYPE_TRAITS) - #include - namespace boost{ - namespace serialization { - namespace detail { - - template - struct is_default_constructible : public std::is_default_constructible {}; - - } // detail - } // serializaition - } // boost -#else - // we don't have standard library support for is_default_constructible - // so we fake it by using boost::has_trivial_construtor. But this is not - // actually correct because it's possible that a default constructor - // to be non trivial. So when using this, make sure you're not using your - // own definition of of T() but are using the actual default one! - #include - namespace boost{ - namespace serialization { - namespace detail { - - template - struct is_default_constructible : public boost::has_trivial_constructor {}; - - } // detail - } // serializaition - } // boost - -#endif - - -#endif // BOOST_SERIALIZATION_DETAIL_IS_DEFAULT_CONSTRUCTIBLE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_count_132.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_count_132.hpp deleted file mode 100644 index a5872557cf2..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_count_132.hpp +++ /dev/null @@ -1,551 +0,0 @@ -#ifndef BOOST_DETAIL_SHARED_COUNT_132_HPP_INCLUDED -#define BOOST_DETAIL_SHARED_COUNT_132_HPP_INCLUDED - -// MS compatible compilers support #pragma once - -#if defined(_MSC_VER) -# pragma once -#endif - -// -// detail/shared_count.hpp -// -// Copyright (c) 2001, 2002, 2003 Peter Dimov and Multi Media Ltd. -// -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// - -#include - -#if defined(BOOST_SP_USE_STD_ALLOCATOR) && defined(BOOST_SP_USE_QUICK_ALLOCATOR) -# error BOOST_SP_USE_STD_ALLOCATOR and BOOST_SP_USE_QUICK_ALLOCATOR are incompatible. -#endif - -#include -#include -#include - -#if defined(BOOST_SP_USE_QUICK_ALLOCATOR) -#include -#endif - -#include // std::auto_ptr, std::allocator -#include // std::less -#include // std::exception -#include // std::bad_alloc -#include // std::type_info in get_deleter -#include // std::size_t - -#include // msvc 6.0 needs this for warning suppression -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif - -namespace boost_132 { - -// Debug hooks - -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - -void sp_scalar_constructor_hook(void * px, std::size_t size, void * pn); -void sp_array_constructor_hook(void * px); -void sp_scalar_destructor_hook(void * px, std::size_t size, void * pn); -void sp_array_destructor_hook(void * px); - -#endif - - -// The standard library that comes with Borland C++ 5.5.1 -// defines std::exception and its members as having C calling -// convention (-pc). When the definition of bad_weak_ptr -// is compiled with -ps, the compiler issues an error. -// Hence, the temporary #pragma option -pc below. The version -// check is deliberately conservative. - -class bad_weak_ptr: public std::exception -{ -public: - - virtual char const * what() const throw() - { - return "boost::bad_weak_ptr"; - } -}; - -namespace detail{ - -class sp_counted_base -{ -//private: - - typedef boost::detail::lightweight_mutex mutex_type; - -public: - - sp_counted_base(): use_count_(1), weak_count_(1) - { - } - - virtual ~sp_counted_base() // nothrow - { - } - - // dispose() is called when use_count_ drops to zero, to release - // the resources managed by *this. - - virtual void dispose() = 0; // nothrow - - // destruct() is called when weak_count_ drops to zero. - - virtual void destruct() // nothrow - { - delete this; - } - - virtual void * get_deleter(std::type_info const & ti) = 0; - - void add_ref_copy() - { -#if defined(BOOST_HAS_THREADS) - mutex_type::scoped_lock lock(mtx_); -#endif - ++use_count_; - } - - void add_ref_lock() - { -#if defined(BOOST_HAS_THREADS) - mutex_type::scoped_lock lock(mtx_); -#endif - if(use_count_ == 0) boost::serialization::throw_exception(bad_weak_ptr()); - ++use_count_; - } - - void release() // nothrow - { - { -#if defined(BOOST_HAS_THREADS) - mutex_type::scoped_lock lock(mtx_); -#endif - long new_use_count = --use_count_; - - if(new_use_count != 0) return; - } - - dispose(); - weak_release(); - } - - void weak_add_ref() // nothrow - { -#if defined(BOOST_HAS_THREADS) - mutex_type::scoped_lock lock(mtx_); -#endif - ++weak_count_; - } - - void weak_release() // nothrow - { - long new_weak_count; - - { -#if defined(BOOST_HAS_THREADS) - mutex_type::scoped_lock lock(mtx_); -#endif - new_weak_count = --weak_count_; - } - - if(new_weak_count == 0) - { - destruct(); - } - } - - long use_count() const // nothrow - { -#if defined(BOOST_HAS_THREADS) - mutex_type::scoped_lock lock(mtx_); -#endif - return use_count_; - } - -//private: -public: - sp_counted_base(sp_counted_base const &); - sp_counted_base & operator= (sp_counted_base const &); - - long use_count_; // #shared - long weak_count_; // #weak + (#shared != 0) - -#if defined(BOOST_HAS_THREADS) || defined(BOOST_LWM_WIN32) - mutable mutex_type mtx_; -#endif -}; - -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - -template void cbi_call_constructor_hook(sp_counted_base * pn, T * px, boost::checked_deleter< T > const &) -{ - boost::sp_scalar_constructor_hook(px, sizeof(T), pn); -} - -template void cbi_call_constructor_hook(sp_counted_base *, T * px, boost::checked_array_deleter< T > const &) -{ - boost::sp_array_constructor_hook(px); -} - -template void cbi_call_constructor_hook(sp_counted_base *, P const &, D const &, long) -{ -} - -template void cbi_call_destructor_hook(sp_counted_base * pn, T * px, boost::checked_deleter< T > const &) -{ - boost::sp_scalar_destructor_hook(px, sizeof(T), pn); -} - -template void cbi_call_destructor_hook(sp_counted_base *, T * px, boost::checked_array_deleter< T > const &) -{ - boost::sp_array_destructor_hook(px); -} - -template void cbi_call_destructor_hook(sp_counted_base *, P const &, D const &, long) -{ -} - -#endif - -// -// Borland's Codeguard trips up over the -Vx- option here: -// -#ifdef __CODEGUARD__ -# pragma option push -Vx- -#endif - -template class sp_counted_base_impl: public sp_counted_base -{ -//private: -public: - P ptr; // copy constructor must not throw - D del; // copy constructor must not throw - - sp_counted_base_impl(sp_counted_base_impl const &); - sp_counted_base_impl & operator= (sp_counted_base_impl const &); - - typedef sp_counted_base_impl this_type; - -public: - - // pre: initial_use_count <= initial_weak_count, d(p) must not throw - - sp_counted_base_impl(P p, D d): ptr(p), del(d) - { -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - detail::cbi_call_constructor_hook(this, p, d, 0); -#endif - } - - virtual void dispose() // nothrow - { -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - detail::cbi_call_destructor_hook(this, ptr, del, 0); -#endif - del(ptr); - } - - virtual void * get_deleter(std::type_info const & ti) - { - return ti == typeid(D)? &del: 0; - } - -#if defined(BOOST_SP_USE_STD_ALLOCATOR) - - void * operator new(std::size_t) - { - return std::allocator().allocate(1, static_cast(0)); - } - - void operator delete(void * p) - { - std::allocator().deallocate(static_cast(p), 1); - } - -#endif - -#if defined(BOOST_SP_USE_QUICK_ALLOCATOR) - - void * operator new(std::size_t) - { - return boost::detail::quick_allocator::alloc(); - } - - void operator delete(void * p) - { - boost::detail::quick_allocator::dealloc(p); - } - -#endif -}; - -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - -int const shared_count_id = 0x2C35F101; -int const weak_count_id = 0x298C38A4; - -#endif - -class weak_count; - -class shared_count -{ -//private: -public: - sp_counted_base * pi_; - -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - int id_; -#endif - - friend class weak_count; - -public: - - shared_count(): pi_(0) // nothrow -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif - { - } - - template shared_count(P p, D d): pi_(0) -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif - { -#ifndef BOOST_NO_EXCEPTIONS - - try - { - pi_ = new sp_counted_base_impl(p, d); - } - catch(...) - { - d(p); // delete p - throw; - } - -#else - - pi_ = new sp_counted_base_impl(p, d); - - if(pi_ == 0) - { - d(p); // delete p - boost::serialization::throw_exception(std::bad_alloc()); - } - -#endif - } - -#ifndef BOOST_NO_AUTO_PTR - - // auto_ptr is special cased to provide the strong guarantee - - template - explicit shared_count(std::auto_ptr & r): pi_( - new sp_counted_base_impl< - Y *, - boost::checked_deleter - >(r.get(), boost::checked_deleter())) -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif - { - r.release(); - } - -#endif - - ~shared_count() // nothrow - { - if(pi_ != 0) pi_->release(); -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - id_ = 0; -#endif - } - - shared_count(shared_count const & r): pi_(r.pi_) // nothrow -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif - { - if(pi_ != 0) pi_->add_ref_copy(); - } - - explicit shared_count(weak_count const & r); // throws bad_weak_ptr when r.use_count() == 0 - - shared_count & operator= (shared_count const & r) // nothrow - { - sp_counted_base * tmp = r.pi_; - - if(tmp != pi_) - { - if(tmp != 0) tmp->add_ref_copy(); - if(pi_ != 0) pi_->release(); - pi_ = tmp; - } - - return *this; - } - - void swap(shared_count & r) // nothrow - { - sp_counted_base * tmp = r.pi_; - r.pi_ = pi_; - pi_ = tmp; - } - - long use_count() const // nothrow - { - return pi_ != 0? pi_->use_count(): 0; - } - - bool unique() const // nothrow - { - return use_count() == 1; - } - - friend inline bool operator==(shared_count const & a, shared_count const & b) - { - return a.pi_ == b.pi_; - } - - friend inline bool operator<(shared_count const & a, shared_count const & b) - { - return std::less()(a.pi_, b.pi_); - } - - void * get_deleter(std::type_info const & ti) const - { - return pi_? pi_->get_deleter(ti): 0; - } -}; - -#ifdef __CODEGUARD__ -# pragma option pop -#endif - - -class weak_count -{ -private: - - sp_counted_base * pi_; - -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - int id_; -#endif - - friend class shared_count; - -public: - - weak_count(): pi_(0) // nothrow -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(weak_count_id) -#endif - { - } - - weak_count(shared_count const & r): pi_(r.pi_) // nothrow -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif - { - if(pi_ != 0) pi_->weak_add_ref(); - } - - weak_count(weak_count const & r): pi_(r.pi_) // nothrow -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif - { - if(pi_ != 0) pi_->weak_add_ref(); - } - - ~weak_count() // nothrow - { - if(pi_ != 0) pi_->weak_release(); -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - id_ = 0; -#endif - } - - weak_count & operator= (shared_count const & r) // nothrow - { - sp_counted_base * tmp = r.pi_; - if(tmp != 0) tmp->weak_add_ref(); - if(pi_ != 0) pi_->weak_release(); - pi_ = tmp; - - return *this; - } - - weak_count & operator= (weak_count const & r) // nothrow - { - sp_counted_base * tmp = r.pi_; - if(tmp != 0) tmp->weak_add_ref(); - if(pi_ != 0) pi_->weak_release(); - pi_ = tmp; - - return *this; - } - - void swap(weak_count & r) // nothrow - { - sp_counted_base * tmp = r.pi_; - r.pi_ = pi_; - pi_ = tmp; - } - - long use_count() const // nothrow - { - return pi_ != 0? pi_->use_count(): 0; - } - - friend inline bool operator==(weak_count const & a, weak_count const & b) - { - return a.pi_ == b.pi_; - } - - friend inline bool operator<(weak_count const & a, weak_count const & b) - { - return std::less()(a.pi_, b.pi_); - } -}; - -inline shared_count::shared_count(weak_count const & r): pi_(r.pi_) -#if defined(BOOST_SP_ENABLE_DEBUG_HOOKS) - , id_(shared_count_id) -#endif -{ - if(pi_ != 0) - { - pi_->add_ref_lock(); - } - else - { - boost::serialization::throw_exception(bad_weak_ptr()); - } -} - -} // namespace detail - -} // namespace boost - -BOOST_SERIALIZATION_ASSUME_ABSTRACT(boost_132::detail::sp_counted_base) - -#endif // #ifndef BOOST_DETAIL_SHARED_COUNT_HPP_INCLUDED diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_132.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_132.hpp deleted file mode 100644 index ee98b7b9449..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_132.hpp +++ /dev/null @@ -1,443 +0,0 @@ -#ifndef BOOST_SHARED_PTR_132_HPP_INCLUDED -#define BOOST_SHARED_PTR_132_HPP_INCLUDED - -// -// shared_ptr.hpp -// -// (C) Copyright Greg Colvin and Beman Dawes 1998, 1999. -// Copyright (c) 2001, 2002, 2003 Peter Dimov -// -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// -// See http://www.boost.org/libs/smart_ptr/shared_ptr.htm for documentation. -// - -#include // for broken compiler workarounds - -#if defined(BOOST_NO_MEMBER_TEMPLATES) && !defined(BOOST_MSVC6_MEMBER_TEMPLATES) -#include -#else - -#include -#include -#include -#include - -#include -#include - -#include // for std::auto_ptr -#include // for std::swap -#include // for std::less -#include // for std::bad_cast -#include // for std::basic_ostream - -#ifdef BOOST_MSVC // moved here to work around VC++ compiler crash -# pragma warning(push) -# pragma warning(disable:4284) // odd return type for operator-> -#endif - -namespace boost_132 { - -template class weak_ptr; -template class enable_shared_from_this; - -namespace detail -{ - -struct static_cast_tag {}; -struct const_cast_tag {}; -struct dynamic_cast_tag {}; -struct polymorphic_cast_tag {}; - -template struct shared_ptr_traits -{ - typedef T & reference; -}; - -template<> struct shared_ptr_traits -{ - typedef void reference; -}; - -#if !defined(BOOST_NO_CV_VOID_SPECIALIZATIONS) - -template<> struct shared_ptr_traits -{ - typedef void reference; -}; - -template<> struct shared_ptr_traits -{ - typedef void reference; -}; - -template<> struct shared_ptr_traits -{ - typedef void reference; -}; - -#endif - -// enable_shared_from_this support - -template void sp_enable_shared_from_this( shared_count const & pn, enable_shared_from_this< T > const * pe, Y const * px ) -{ - if(pe != 0) pe->_internal_weak_this._internal_assign(const_cast(px), pn); -} - -inline void sp_enable_shared_from_this( shared_count const & /*pn*/, ... ) -{ -} - -} // namespace detail - - -// -// shared_ptr -// -// An enhanced relative of scoped_ptr with reference counted copy semantics. -// The object pointed to is deleted when the last shared_ptr pointing to it -// is destroyed or reset. -// - -template class shared_ptr -{ -private: - // Borland 5.5.1 specific workaround - typedef shared_ptr< T > this_type; - -public: - - typedef T element_type; - typedef T value_type; - typedef T * pointer; - typedef typename detail::shared_ptr_traits< T >::reference reference; - - shared_ptr(): px(0), pn() // never throws in 1.30+ - { - } - - template - explicit shared_ptr(Y * p): px(p), pn(p, boost::checked_deleter()) // Y must be complete - { - detail::sp_enable_shared_from_this( pn, p, p ); - } - - // - // Requirements: D's copy constructor must not throw - // - // shared_ptr will release p by calling d(p) - // - - template shared_ptr(Y * p, D d): px(p), pn(p, d) - { - detail::sp_enable_shared_from_this( pn, p, p ); - } - -// generated copy constructor, assignment, destructor are fine... - -// except that Borland C++ has a bug, and g++ with -Wsynth warns -#if defined(__GNUC__) - shared_ptr & operator=(shared_ptr const & r) // never throws - { - px = r.px; - pn = r.pn; // shared_count::op= doesn't throw - return *this; - } -#endif - - template - explicit shared_ptr(weak_ptr const & r): pn(r.pn) // may throw - { - // it is now safe to copy r.px, as pn(r.pn) did not throw - px = r.px; - } - - template - shared_ptr(shared_ptr const & r): px(r.px), pn(r.pn) // never throws - { - } - - template - shared_ptr(shared_ptr const & r, detail::static_cast_tag): px(static_cast(r.px)), pn(r.pn) - { - } - - template - shared_ptr(shared_ptr const & r, detail::const_cast_tag): px(const_cast(r.px)), pn(r.pn) - { - } - - template - shared_ptr(shared_ptr const & r, detail::dynamic_cast_tag): px(dynamic_cast(r.px)), pn(r.pn) - { - if(px == 0) // need to allocate new counter -- the cast failed - { - pn = detail::shared_count(); - } - } - - template - shared_ptr(shared_ptr const & r, detail::polymorphic_cast_tag): px(dynamic_cast(r.px)), pn(r.pn) - { - if(px == 0) - { - boost::serialization::throw_exception(std::bad_cast()); - } - } - -#ifndef BOOST_NO_AUTO_PTR - - template - explicit shared_ptr(std::auto_ptr & r): px(r.get()), pn() - { - Y * tmp = r.get(); - pn = detail::shared_count(r); - detail::sp_enable_shared_from_this( pn, tmp, tmp ); - } - -#endif - -#if !defined(BOOST_MSVC) || (BOOST_MSVC > 1200) - - template - shared_ptr & operator=(shared_ptr const & r) // never throws - { - px = r.px; - pn = r.pn; // shared_count::op= doesn't throw - return *this; - } - -#endif - -#ifndef BOOST_NO_AUTO_PTR - - template - shared_ptr & operator=(std::auto_ptr & r) - { - this_type(r).swap(*this); - return *this; - } - -#endif - - void reset() // never throws in 1.30+ - { - this_type().swap(*this); - } - - template void reset(Y * p) // Y must be complete - { - BOOST_ASSERT(p == 0 || p != px); // catch self-reset errors - this_type(p).swap(*this); - } - - template void reset(Y * p, D d) - { - this_type(p, d).swap(*this); - } - - reference operator* () const // never throws - { - BOOST_ASSERT(px != 0); - return *px; - } - - T * operator-> () const // never throws - { - BOOST_ASSERT(px != 0); - return px; - } - - T * get() const // never throws - { - return px; - } - - // implicit conversion to "bool" - -#if defined(__SUNPRO_CC) && BOOST_WORKAROUND(__SUNPRO_CC, <= 0x530) - - operator bool () const - { - return px != 0; - } - -#elif defined(__MWERKS__) && BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3003)) - typedef T * (this_type::*unspecified_bool_type)() const; - - operator unspecified_bool_type() const // never throws - { - return px == 0? 0: &this_type::get; - } - -#else - - typedef T * this_type::*unspecified_bool_type; - - operator unspecified_bool_type() const // never throws - { - return px == 0? 0: &this_type::px; - } - -#endif - - // operator! is redundant, but some compilers need it - - bool operator! () const // never throws - { - return px == 0; - } - - bool unique() const // never throws - { - return pn.unique(); - } - - long use_count() const // never throws - { - return pn.use_count(); - } - - void swap(shared_ptr< T > & other) // never throws - { - std::swap(px, other.px); - pn.swap(other.pn); - } - - template bool _internal_less(shared_ptr const & rhs) const - { - return pn < rhs.pn; - } - - void * _internal_get_deleter(std::type_info const & ti) const - { - return pn.get_deleter(ti); - } - -// Tasteless as this may seem, making all members public allows member templates -// to work in the absence of member template friends. (Matthew Langston) - -#ifndef BOOST_NO_MEMBER_TEMPLATE_FRIENDS - -private: - - template friend class shared_ptr; - template friend class weak_ptr; - - -#endif -public: // for serialization - T * px; // contained pointer - detail::shared_count pn; // reference counter - -}; // shared_ptr - -template inline bool operator==(shared_ptr< T > const & a, shared_ptr const & b) -{ - return a.get() == b.get(); -} - -template inline bool operator!=(shared_ptr< T > const & a, shared_ptr const & b) -{ - return a.get() != b.get(); -} - -template inline bool operator<(shared_ptr< T > const & a, shared_ptr const & b) -{ - return a._internal_less(b); -} - -template inline void swap(shared_ptr< T > & a, shared_ptr< T > & b) -{ - a.swap(b); -} - -template shared_ptr< T > static_pointer_cast(shared_ptr const & r) -{ - return shared_ptr< T >(r, detail::static_cast_tag()); -} - -template shared_ptr< T > const_pointer_cast(shared_ptr const & r) -{ - return shared_ptr< T >(r, detail::const_cast_tag()); -} - -template shared_ptr< T > dynamic_pointer_cast(shared_ptr const & r) -{ - return shared_ptr< T >(r, detail::dynamic_cast_tag()); -} - -// shared_*_cast names are deprecated. Use *_pointer_cast instead. - -template shared_ptr< T > shared_static_cast(shared_ptr const & r) -{ - return shared_ptr< T >(r, detail::static_cast_tag()); -} - -template shared_ptr< T > shared_dynamic_cast(shared_ptr const & r) -{ - return shared_ptr< T >(r, detail::dynamic_cast_tag()); -} - -template shared_ptr< T > shared_polymorphic_cast(shared_ptr const & r) -{ - return shared_ptr< T >(r, detail::polymorphic_cast_tag()); -} - -template shared_ptr< T > shared_polymorphic_downcast(shared_ptr const & r) -{ - BOOST_ASSERT(dynamic_cast(r.get()) == r.get()); - return shared_static_cast< T >(r); -} - -// get_pointer() enables boost::mem_fn to recognize shared_ptr - -template inline T * get_pointer(shared_ptr< T > const & p) -{ - return p.get(); -} - -// operator<< - - -template std::basic_ostream & operator<< (std::basic_ostream & os, shared_ptr const & p) -{ - os << p.get(); - return os; -} - -// get_deleter (experimental) - -#if defined(__EDG_VERSION__) && (__EDG_VERSION__ <= 238) - -// g++ 2.9x doesn't allow static_cast(void *) -// apparently EDG 2.38 also doesn't accept it - -template D * get_deleter(shared_ptr< T > const & p) -{ - void const * q = p._internal_get_deleter(typeid(D)); - return const_cast(static_cast(q)); -} - -#else - -template D * get_deleter(shared_ptr< T > const & p) -{ - return static_cast(p._internal_get_deleter(typeid(D))); -} - -#endif - -} // namespace boost - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -#endif // #if defined(BOOST_NO_MEMBER_TEMPLATES) && !defined(BOOST_MSVC6_MEMBER_TEMPLATES) - -#endif // #ifndef BOOST_SHARED_PTR_132_HPP_INCLUDED diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_nmt_132.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_nmt_132.hpp deleted file mode 100644 index 490e7ddd3d0..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/detail/shared_ptr_nmt_132.hpp +++ /dev/null @@ -1,182 +0,0 @@ -#ifndef BOOST_DETAIL_SHARED_PTR_NMT_132_HPP_INCLUDED -#define BOOST_DETAIL_SHARED_PTR_NMT_132_HPP_INCLUDED - -// -// detail/shared_ptr_nmt.hpp - shared_ptr.hpp without member templates -// -// (C) Copyright Greg Colvin and Beman Dawes 1998, 1999. -// Copyright (c) 2001, 2002 Peter Dimov -// -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// -// See http://www.boost.org/libs/smart_ptr/shared_ptr.htm for documentation. -// - -#include -#include -#include -#include - -#ifndef BOOST_NO_AUTO_PTR -# include // for std::auto_ptr -#endif - -#include // for std::swap -#include // for std::less -#include // for std::bad_alloc - -namespace boost -{ - -template class shared_ptr -{ -private: - - typedef detail::atomic_count count_type; - -public: - - typedef T element_type; - typedef T value_type; - - explicit shared_ptr(T * p = 0): px(p) - { -#ifndef BOOST_NO_EXCEPTIONS - - try // prevent leak if new throws - { - pn = new count_type(1); - } - catch(...) - { - boost::checked_delete(p); - throw; - } - -#else - - pn = new count_type(1); - - if(pn == 0) - { - boost::checked_delete(p); - boost::serialization::throw_exception(std::bad_alloc()); - } - -#endif - } - - ~shared_ptr() - { - if(--*pn == 0) - { - boost::checked_delete(px); - delete pn; - } - } - - shared_ptr(shared_ptr const & r): px(r.px) // never throws - { - pn = r.pn; - ++*pn; - } - - shared_ptr & operator=(shared_ptr const & r) - { - shared_ptr(r).swap(*this); - return *this; - } - -#ifndef BOOST_NO_AUTO_PTR - - explicit shared_ptr(std::auto_ptr< T > & r) - { - pn = new count_type(1); // may throw - px = r.release(); // fix: moved here to stop leak if new throws - } - - shared_ptr & operator=(std::auto_ptr< T > & r) - { - shared_ptr(r).swap(*this); - return *this; - } - -#endif - - void reset(T * p = 0) - { - BOOST_ASSERT(p == 0 || p != px); - shared_ptr(p).swap(*this); - } - - T & operator*() const // never throws - { - BOOST_ASSERT(px != 0); - return *px; - } - - T * operator->() const // never throws - { - BOOST_ASSERT(px != 0); - return px; - } - - T * get() const // never throws - { - return px; - } - - long use_count() const // never throws - { - return *pn; - } - - bool unique() const // never throws - { - return *pn == 1; - } - - void swap(shared_ptr< T > & other) // never throws - { - std::swap(px, other.px); - std::swap(pn, other.pn); - } - -private: - - T * px; // contained pointer - count_type * pn; // ptr to reference counter -}; - -template inline bool operator==(shared_ptr< T > const & a, shared_ptr const & b) -{ - return a.get() == b.get(); -} - -template inline bool operator!=(shared_ptr< T > const & a, shared_ptr const & b) -{ - return a.get() != b.get(); -} - -template inline bool operator<(shared_ptr< T > const & a, shared_ptr< T > const & b) -{ - return std::less()(a.get(), b.get()); -} - -template void swap(shared_ptr< T > & a, shared_ptr< T > & b) -{ - a.swap(b); -} - -// get_pointer() enables boost::mem_fn to recognize shared_ptr - -template inline T * get_pointer(shared_ptr< T > const & p) -{ - return p.get(); -} - -} // namespace boost - -#endif // #ifndef BOOST_DETAIL_SHARED_PTR_NMT_132_HPP_INCLUDED diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/detail/stack_constructor.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/detail/stack_constructor.hpp deleted file mode 100644 index ae14832c6db..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/detail/stack_constructor.hpp +++ /dev/null @@ -1,66 +0,0 @@ -#ifndef BOOST_SERIALIZATION_DETAIL_STACK_CONSTRUCTOR_HPP -#define BOOST_SERIALIZATION_DETAIL_STACK_CONSTRUCTOR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// stack_constructor.hpp: serialization for loading stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -namespace boost{ -namespace serialization { -namespace detail { - -// reserve space on stack for an object of type T without actually -// construction such an object -template -struct stack_allocate -{ - T * address() { - return static_cast(storage_.address()); - } - T & reference() { - return * address(); - } -private: - typedef typename boost::aligned_storage< - sizeof(T), - boost::alignment_of::value - > type; - type storage_; -}; - -// construct element on the stack -template -struct stack_construct : public stack_allocate -{ - stack_construct(Archive & ar, const unsigned int version){ - // note borland emits a no-op without the explicit namespace - boost::serialization::load_construct_data_adl( - ar, - this->address(), - version - ); - } - ~stack_construct(){ - this->address()->~T(); // undo load_construct_data above - } -}; - -} // detail -} // serializaition -} // boost - -#endif // BOOST_SERIALIZATION_DETAIL_STACH_CONSTRUCTOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/ephemeral.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/ephemeral.hpp deleted file mode 100644 index 3a422c30a35..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/ephemeral.hpp +++ /dev/null @@ -1,72 +0,0 @@ -#ifndef BOOST_SERIALIZATION_EPHEMERAL_HPP -#define BOOST_SERIALIZATION_EPHEMERAL_HPP - -// MS compatible compilers support -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// ephemeral_object.hpp: interface for serialization system. - -// (C) Copyright 2007 Matthias Troyer. -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -#include -#include - -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template -struct ephemeral_object : - public wrapper_traits > -{ - explicit ephemeral_object(T& t) : - val(t) - {} - - T & value() const { - return val; - } - - const T & const_value() const { - return val; - } - - template - void serialize(Archive &ar, const unsigned int) const - { - ar & val; - } - -private: - T & val; -}; - -template -inline -const ephemeral_object ephemeral(const char * name, T & t){ - return ephemeral_object(name, t); -} - -} // seralization -} // boost - -#endif // BOOST_SERIALIZATION_EPHEMERAL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/export.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/export.hpp deleted file mode 100644 index 9eef440df42..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/export.hpp +++ /dev/null @@ -1,225 +0,0 @@ -#ifndef BOOST_SERIALIZATION_EXPORT_HPP -#define BOOST_SERIALIZATION_EXPORT_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// export.hpp: set traits of classes to be serialized - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// (C) Copyright 2006 David Abrahams - http://www.boost.org. -// implementation of class export functionality. This is an alternative to -// "forward declaration" method to provoke instantiation of derived classes -// that are to be serialized through pointers. - -#include -#include // NULL - -#include -#include -#include -#include - -#include -#include -#include -#include - -#include // for guid_defined only -#include -#include -#include -#include - -#include - -#include - -namespace boost { -namespace archive { -namespace detail { - -class basic_pointer_iserializer; -class basic_pointer_oserializer; - -template -class pointer_iserializer; -template -class pointer_oserializer; - -template -struct export_impl -{ - static const basic_pointer_iserializer & - enable_load(mpl::true_){ - return boost::serialization::singleton< - pointer_iserializer - >::get_const_instance(); - } - - static const basic_pointer_oserializer & - enable_save(mpl::true_){ - return boost::serialization::singleton< - pointer_oserializer - >::get_const_instance(); - } - inline static void enable_load(mpl::false_) {} - inline static void enable_save(mpl::false_) {} -}; - -// On many platforms, naming a specialization of this template is -// enough to cause its argument to be instantiated. -template -struct instantiate_function {}; - -template -struct ptr_serialization_support -{ -# if defined(BOOST_MSVC) || defined(__SUNPRO_CC) - virtual BOOST_DLLEXPORT void instantiate() BOOST_USED; -# else - static BOOST_DLLEXPORT void instantiate() BOOST_USED; - typedef instantiate_function< - &ptr_serialization_support::instantiate - > x; -# endif -}; - -template -BOOST_DLLEXPORT void -ptr_serialization_support::instantiate() -{ - export_impl::enable_save( - typename Archive::is_saving() - ); - - export_impl::enable_load( - typename Archive::is_loading() - ); -} - -// Note INTENTIONAL usage of anonymous namespace in header. -// This was made this way so that export.hpp could be included -// in other headers. This is still under study. - -namespace extra_detail { - -template -struct guid_initializer -{ - void export_guid(mpl::false_) const { - // generates the statically-initialized objects whose constructors - // register the information allowing serialization of T objects - // through pointers to their base classes. - instantiate_ptr_serialization((T*)0, 0, adl_tag()); - } - void export_guid(mpl::true_) const { - } - guid_initializer const & export_guid() const { - BOOST_STATIC_WARNING(boost::is_polymorphic< T >::value); - // note: exporting an abstract base class will have no effect - // and cannot be used to instantitiate serialization code - // (one might be using this in a DLL to instantiate code) - //BOOST_STATIC_WARNING(! boost::serialization::is_abstract< T >::value); - export_guid(boost::serialization::is_abstract< T >()); - return *this; - } -}; - -template -struct init_guid; - -} // anonymous -} // namespace detail -} // namespace archive -} // namespace boost - -#define BOOST_CLASS_EXPORT_IMPLEMENT(T) \ - namespace boost { \ - namespace archive { \ - namespace detail { \ - namespace extra_detail { \ - template<> \ - struct init_guid< T > { \ - static guid_initializer< T > const & g; \ - }; \ - guid_initializer< T > const & init_guid< T >::g = \ - ::boost::serialization::singleton< \ - guid_initializer< T > \ - >::get_mutable_instance().export_guid(); \ - }}}} \ -/**/ - -#define BOOST_CLASS_EXPORT_KEY2(T, K) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct guid_defined< T > : boost::mpl::true_ {}; \ -template<> \ -inline const char * guid< T >(){ \ - return K; \ -} \ -} /* serialization */ \ -} /* boost */ \ -/**/ - -#define BOOST_CLASS_EXPORT_KEY(T) \ - BOOST_CLASS_EXPORT_KEY2(T, BOOST_PP_STRINGIZE(T)) \ -/**/ - -#define BOOST_CLASS_EXPORT_GUID(T, K) \ -BOOST_CLASS_EXPORT_KEY2(T, K) \ -BOOST_CLASS_EXPORT_IMPLEMENT(T) \ -/**/ - -#if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3205)) - -// CodeWarrior fails to construct static members of class templates -// when they are instantiated from within templates, so on that -// compiler we ask users to specifically register base/derived class -// relationships for exported classes. On all other compilers, use of -// this macro is entirely optional. -# define BOOST_SERIALIZATION_MWERKS_BASE_AND_DERIVED(Base,Derived) \ -namespace { \ - static int BOOST_PP_CAT(boost_serialization_mwerks_init_, __LINE__) = \ - (::boost::archive::detail::instantiate_ptr_serialization((Derived*)0,0), 3); \ - static int BOOST_PP_CAT(boost_serialization_mwerks_init2_, __LINE__) = ( \ - ::boost::serialization::void_cast_register((Derived*)0,(Base*)0) \ - , 3); \ -} - -#else - -# define BOOST_SERIALIZATION_MWERKS_BASE_AND_DERIVED(Base,Derived) - -#endif - -// check for unnecessary export. T isn't polymorphic so there is no -// need to export it. -#define BOOST_CLASS_EXPORT_CHECK(T) \ - BOOST_STATIC_WARNING( \ - boost::is_polymorphic::value \ - ); \ - /**/ - -// the default exportable class identifier is the class name -// the default list of archives types for which code id generated -// are the originally included with this serialization system -#define BOOST_CLASS_EXPORT(T) \ - BOOST_CLASS_EXPORT_GUID( \ - T, \ - BOOST_PP_STRINGIZE(T) \ - ) \ - /**/ - -#endif // BOOST_SERIALIZATION_EXPORT_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info.hpp deleted file mode 100644 index bb2a190d465..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info.hpp +++ /dev/null @@ -1,116 +0,0 @@ -#ifndef BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_HPP -#define BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// extended_type_info.hpp: interface for portable version of type_info - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// for now, extended type info is part of the serialization libraries -// this could change in the future. -#include -#include -#include // NULL -#include -#include -#include - -#include -#include // must be the last header -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4251 4231 4660 4275) -#endif - -#define BOOST_SERIALIZATION_MAX_KEY_SIZE 128 - -namespace boost { -namespace serialization { - -namespace void_cast_detail{ - class void_caster; -} - -class BOOST_SYMBOL_VISIBLE extended_type_info : - private boost::noncopyable -{ -private: - friend class boost::serialization::void_cast_detail::void_caster; - - // used to uniquely identify the type of class derived from this one - // so that different derivations of this class can be simultaneously - // included in implementation of sets and maps. - const unsigned int m_type_info_key; - virtual bool is_less_than(const extended_type_info & /*rhs*/) const = 0; - virtual bool is_equal(const extended_type_info & /*rhs*/) const = 0; - const char * m_key; - -protected: - BOOST_SERIALIZATION_DECL void key_unregister() const; - BOOST_SERIALIZATION_DECL void key_register() const; - // this class can't be used as is. It's just the - // common functionality for all type_info replacement - // systems. Hence, make these protected - BOOST_SERIALIZATION_DECL extended_type_info( - const unsigned int type_info_key, - const char * key - ); - virtual BOOST_SERIALIZATION_DECL ~extended_type_info(); -public: - const char * get_key() const { - return m_key; - } - virtual const char * get_debug_info() const = 0; - BOOST_SERIALIZATION_DECL bool operator<(const extended_type_info &rhs) const; - BOOST_SERIALIZATION_DECL bool operator==(const extended_type_info &rhs) const; - bool operator!=(const extended_type_info &rhs) const { - return !(operator==(rhs)); - } - // note explicit "export" of static function to work around - // gcc 4.5 mingw error - static BOOST_SERIALIZATION_DECL const extended_type_info * - find(const char *key); - // for plugins - virtual void * construct(unsigned int /*count*/ = 0, ...) const = 0; - virtual void destroy(void const * const /*p*/) const = 0; -}; - -template -struct guid_defined : boost::mpl::false_ {}; - -namespace ext { - template - struct guid_impl - { - static inline const char * call() - { - return NULL; - } - }; -} - -template -inline const char * guid(){ - return ext::guid_impl::call(); -} - -} // namespace serialization -} // namespace boost - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_no_rtti.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_no_rtti.hpp deleted file mode 100644 index aaa8b44459b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_no_rtti.hpp +++ /dev/null @@ -1,182 +0,0 @@ -#ifndef BOOST_EXTENDED_TYPE_INFO_NO_RTTI_HPP -#define BOOST_EXTENDED_TYPE_INFO_NO_RTTI_HPP - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -// extended_type_info_no_rtti.hpp: implementation for version that depends -// on runtime typing (rtti - typeid) but uses a user specified string -// as the portable class identifier. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include - -#include -#include - -#include -#include -#include - -#include -#include -#include -#include -#include - -#include -// hijack serialization access -#include - -#include // must be the last header -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4251 4231 4660 4275 4511 4512) -#endif - -namespace boost { -namespace serialization { -/////////////////////////////////////////////////////////////////////// -// define a special type_info that doesn't depend on rtti which is not -// available in all situations. - -namespace no_rtti_system { - -// common base class to share type_info_key. This is used to -// identify the method used to keep track of the extended type -class BOOST_SYMBOL_VISIBLE extended_type_info_no_rtti_0 : - public extended_type_info -{ -protected: - BOOST_SERIALIZATION_DECL extended_type_info_no_rtti_0(const char * key); - BOOST_SERIALIZATION_DECL ~extended_type_info_no_rtti_0(); -public: - virtual BOOST_SERIALIZATION_DECL bool - is_less_than(const boost::serialization::extended_type_info &rhs) const ; - virtual BOOST_SERIALIZATION_DECL bool - is_equal(const boost::serialization::extended_type_info &rhs) const ; -}; - -} // no_rtti_system - -template -class extended_type_info_no_rtti : - public no_rtti_system::extended_type_info_no_rtti_0, - public singleton > -{ - template - struct action { - struct defined { - static const char * invoke(){ - return guid< T >(); - } - }; - struct undefined { - // if your program traps here - you failed to - // export a guid for this type. the no_rtti - // system requires export for types serialized - // as pointers. - BOOST_STATIC_ASSERT(0 == sizeof(T)); - static const char * invoke(); - }; - static const char * invoke(){ - typedef - typename boost::mpl::if_c< - tf, - defined, - undefined - >::type type; - return type::invoke(); - } - }; -public: - extended_type_info_no_rtti() : - no_rtti_system::extended_type_info_no_rtti_0(get_key()) - { - key_register(); - } - ~extended_type_info_no_rtti(){ - key_unregister(); - } - const extended_type_info * - get_derived_extended_type_info(const T & t) const { - // find the type that corresponds to the most derived type. - // this implementation doesn't depend on typeid() but assumes - // that the specified type has a function of the following signature. - // A common implemention of such a function is to define as a virtual - // function. So if the is not a polymporphic type it's likely an error - BOOST_STATIC_WARNING(boost::is_polymorphic< T >::value); - const char * derived_key = t.get_key(); - BOOST_ASSERT(NULL != derived_key); - return boost::serialization::extended_type_info::find(derived_key); - } - const char * get_key() const{ - return action::value >::invoke(); - } - virtual const char * get_debug_info() const{ - return action::value >::invoke(); - } - virtual void * construct(unsigned int count, ...) const{ - // count up the arguments - std::va_list ap; - va_start(ap, count); - switch(count){ - case 0: - return factory::type, 0>(ap); - case 1: - return factory::type, 1>(ap); - case 2: - return factory::type, 2>(ap); - case 3: - return factory::type, 3>(ap); - case 4: - return factory::type, 4>(ap); - default: - BOOST_ASSERT(false); // too many arguments - // throw exception here? - return NULL; - } - } - virtual void destroy(void const * const p) const{ - boost::serialization::access::destroy( - static_cast(p) - ); - //delete static_cast(p) ; - } -}; - -} // namespace serialization -} // namespace boost - -/////////////////////////////////////////////////////////////////////////////// -// If no other implementation has been designated as default, -// use this one. To use this implementation as the default, specify it -// before any of the other headers. - -#ifndef BOOST_SERIALIZATION_DEFAULT_TYPE_INFO - #define BOOST_SERIALIZATION_DEFAULT_TYPE_INFO - namespace boost { - namespace serialization { - template - struct extended_type_info_impl { - typedef typename - boost::serialization::extended_type_info_no_rtti< T > type; - }; - } // namespace serialization - } // namespace boost -#endif - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_EXTENDED_TYPE_INFO_NO_RTTI_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_typeid.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_typeid.hpp deleted file mode 100644 index 8ee591b3169..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/extended_type_info_typeid.hpp +++ /dev/null @@ -1,167 +0,0 @@ -#ifndef BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_TYPEID_HPP -#define BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_TYPEID_HPP - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -// extended_type_info_typeid.hpp: implementation for version that depends -// on runtime typing (rtti - typeid) but uses a user specified string -// as the portable class identifier. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -#include -#include -#include -#include - -#include -#include -#include -#include - -// hijack serialization access -#include - -#include - -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4251 4231 4660 4275 4511 4512) -#endif - -namespace boost { -namespace serialization { -namespace typeid_system { - -class BOOST_SYMBOL_VISIBLE extended_type_info_typeid_0 : - public extended_type_info -{ - virtual const char * get_debug_info() const { - if(static_cast(0) == m_ti) - return static_cast(0); - return m_ti->name(); - } -protected: - const std::type_info * m_ti; - BOOST_SERIALIZATION_DECL extended_type_info_typeid_0(const char * key); - BOOST_SERIALIZATION_DECL ~extended_type_info_typeid_0(); - BOOST_SERIALIZATION_DECL void type_register(const std::type_info & ti); - BOOST_SERIALIZATION_DECL void type_unregister(); - BOOST_SERIALIZATION_DECL const extended_type_info * - get_extended_type_info(const std::type_info & ti) const; -public: - virtual BOOST_SERIALIZATION_DECL bool - is_less_than(const extended_type_info &rhs) const; - virtual BOOST_SERIALIZATION_DECL bool - is_equal(const extended_type_info &rhs) const; - const std::type_info & get_typeid() const { - return *m_ti; - } -}; - -} // typeid_system - -template -class extended_type_info_typeid : - public typeid_system::extended_type_info_typeid_0, - public singleton > -{ -public: - extended_type_info_typeid() : - typeid_system::extended_type_info_typeid_0( - boost::serialization::guid< T >() - ) - { - type_register(typeid(T)); - key_register(); - } - ~extended_type_info_typeid(){ - key_unregister(); - type_unregister(); - } - // get the eti record for the true type of this record - // relying upon standard type info implemenation (rtti) - const extended_type_info * - get_derived_extended_type_info(const T & t) const { - // note: this implementation - based on usage of typeid (rtti) - // only does something if the class has at least one virtual function. - BOOST_STATIC_WARNING(boost::is_polymorphic< T >::value); - return - typeid_system::extended_type_info_typeid_0::get_extended_type_info( - typeid(t) - ); - } - const char * get_key() const { - return boost::serialization::guid< T >(); - } - virtual void * construct(unsigned int count, ...) const{ - // count up the arguments - std::va_list ap; - va_start(ap, count); - switch(count){ - case 0: - return factory::type, 0>(ap); - case 1: - return factory::type, 1>(ap); - case 2: - return factory::type, 2>(ap); - case 3: - return factory::type, 3>(ap); - case 4: - return factory::type, 4>(ap); - default: - BOOST_ASSERT(false); // too many arguments - // throw exception here? - return NULL; - } - } - virtual void destroy(void const * const p) const { - boost::serialization::access::destroy( - static_cast(p) - ); - //delete static_cast(p); - } -}; - -} // namespace serialization -} // namespace boost - -/////////////////////////////////////////////////////////////////////////////// -// If no other implementation has been designated as default, -// use this one. To use this implementation as the default, specify it -// before any of the other headers. -#ifndef BOOST_SERIALIZATION_DEFAULT_TYPE_INFO - #define BOOST_SERIALIZATION_DEFAULT_TYPE_INFO - namespace boost { - namespace serialization { - template - struct extended_type_info_impl { - typedef typename - boost::serialization::extended_type_info_typeid< T > type; - }; - } // namespace serialization - } // namespace boost -#endif - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_SERIALIZATION_EXTENDED_TYPE_INFO_TYPEID_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/factory.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/factory.hpp deleted file mode 100644 index 2db7e7e36c3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/factory.hpp +++ /dev/null @@ -1,102 +0,0 @@ -#ifndef BOOST_SERIALIZATION_FACTORY_HPP -#define BOOST_SERIALIZATION_FACTORY_HPP - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -// factory.hpp: create an instance from an extended_type_info instance. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // valist -#include // NULL - -#include -#include -#include - -namespace std{ - #if defined(__LIBCOMO__) - using ::va_list; - #endif -} // namespace std - -namespace boost { -namespace serialization { - -// default implementation does nothing. -template -T * factory(std::va_list){ - BOOST_ASSERT(false); - // throw exception here? - return NULL; -} - -} // namespace serialization -} // namespace boost - -#define BOOST_SERIALIZATION_FACTORY(N, T, A0, A1, A2, A3) \ -namespace boost { \ -namespace serialization { \ - template<> \ - T * factory(std::va_list ap){ \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 0) \ - , A0 a0 = va_arg(ap, A0);, BOOST_PP_EMPTY()) \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 1) \ - , A1 a1 = va_arg(ap, A1);, BOOST_PP_EMPTY()) \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 2) \ - , A2 a2 = va_arg(ap, A2);, BOOST_PP_EMPTY()) \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 3) \ - , A3 a3 = va_arg(ap, A3);, BOOST_PP_EMPTY()) \ - return new T( \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 0) \ - , a0, BOOST_PP_EMPTY()) \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 1)) \ - , BOOST_PP_COMMA, BOOST_PP_EMPTY)() \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 1) \ - , a1, BOOST_PP_EMPTY()) \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 2)) \ - , BOOST_PP_COMMA, BOOST_PP_EMPTY)() \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 2) \ - , a2, BOOST_PP_EMPTY()) \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 3)) \ - , BOOST_PP_COMMA, BOOST_PP_EMPTY)() \ - BOOST_PP_IF(BOOST_PP_GREATER(N, 3) \ - , a3, BOOST_PP_EMPTY()) \ - ); \ - } \ -} \ -} /**/ - -#define BOOST_SERIALIZATION_FACTORY_4(T, A0, A1, A2, A3) \ - BOOST_SERIALIZATION_FACTORY(4, T, A0, A1, A2, A3) - -#define BOOST_SERIALIZATION_FACTORY_3(T, A0, A1, A2) \ - BOOST_SERIALIZATION_FACTORY(3, T, A0, A1, A2, 0) - -#define BOOST_SERIALIZATION_FACTORY_2(T, A0, A1) \ - BOOST_SERIALIZATION_FACTORY(2, T, A0, A1, 0, 0) - -#define BOOST_SERIALIZATION_FACTORY_1(T, A0) \ - BOOST_SERIALIZATION_FACTORY(1, T, A0, 0, 0, 0) - -#define BOOST_SERIALIZATION_FACTORY_0(T) \ -namespace boost { \ -namespace serialization { \ - template<> \ - T * factory(std::va_list){ \ - return new T(); \ - } \ -} \ -} \ -/**/ - -#endif // BOOST_SERIALIZATION_FACTORY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/force_include.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/force_include.hpp deleted file mode 100644 index 55ab79d0d58..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/force_include.hpp +++ /dev/null @@ -1,55 +0,0 @@ -#ifndef BOOST_SERIALIZATION_FORCE_INCLUDE_HPP -#define BOOST_SERIALIZATION_FORCE_INCLUDE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// force_include.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -// the following help macro is to guarentee that certain coded -// is not removed by over-eager linker optimiser. In certain cases -// we create static objects must be created but are actually never -// referenced - creation has a side-effect such as global registration -// which is important to us. We make an effort to refer these objects -// so that a smart linker won't remove them as being unreferenced. -// In microsoft compilers, inlining the code that does the referring -// means the code gets lost and the static object is not included -// in the library and hence never registered. This manifests itself -// in an ungraceful crash at runtime when (and only when) built in -// release mode. - -#if defined(BOOST_HAS_DECLSPEC) && !defined(__COMO__) -# define BOOST_DLLEXPORT __declspec(dllexport) -#elif ! defined(_WIN32) && ! defined(_WIN64) -# if defined(__MWERKS__) -# define BOOST_DLLEXPORT __declspec(dllexport) -# elif defined(__GNUC__) && (__GNUC__ >= 3) -# define BOOST_USED __attribute__ ((__used__)) -# elif defined(__IBMCPP__) && (__IBMCPP__ >= 1110) -# define BOOST_USED __attribute__ ((__used__)) -# elif defined(__INTEL_COMPILER) && (BOOST_INTEL_CXX_VERSION >= 800) -# define BOOST_USED __attribute__ ((__used__)) -# endif -#endif - -#ifndef BOOST_USED -# define BOOST_USED -#endif - -#ifndef BOOST_DLLEXPORT -# define BOOST_DLLEXPORT -#endif - -#endif // BOOST_SERIALIZATION_FORCE_INCLUDE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/forward_list.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/forward_list.hpp deleted file mode 100644 index b8a3c20a6ea..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/forward_list.hpp +++ /dev/null @@ -1,124 +0,0 @@ -#ifndef BOOST_SERIALIZATION_FORWARD_LIST_HPP -#define BOOST_SERIALIZATION_FORWARD_LIST_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// forward_list.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include // distance - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template -inline void save( - Archive & ar, - const std::forward_list &t, - const unsigned int /*file_version*/ -){ - const collection_size_type count(std::distance(t.cbegin(), t.cend())); - boost::serialization::stl::save_collection< - Archive, - std::forward_list - >(ar, t, count); -} - -namespace stl { - -template< - class Archive, - class T, - class Allocator -> -typename boost::disable_if< - typename detail::is_default_constructible< - typename std::forward_list::value_type - >, - void ->::type -collection_load_impl( - Archive & ar, - std::forward_list &t, - collection_size_type count, - item_version_type item_version -){ - t.clear(); - boost::serialization::detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - t.push_front(boost::move(u.reference())); - typename std::forward_list::iterator last; - last = t.begin(); - ar.reset_object_address(&(*t.begin()) , & u.reference()); - while(--count > 0){ - detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - last = t.insert_after(last, boost::move(u.reference())); - ar.reset_object_address(&(*last) , & u.reference()); - } -} - -} // stl - -template -inline void load( - Archive & ar, - std::forward_list &t, - const unsigned int /*file_version*/ -){ - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - stl::collection_load_impl(ar, t, count, item_version); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::forward_list &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(std::forward_list) - -#endif // BOOST_SERIALIZATION_FORWARD_LIST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_load_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_load_imp.hpp deleted file mode 100644 index 88def8f1aa4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_load_imp.hpp +++ /dev/null @@ -1,77 +0,0 @@ -#ifndef BOOST_SERIALIZATION_HASH_COLLECTIONS_LOAD_IMP_HPP -#define BOOST_SERIALIZATION_HASH_COLLECTIONS_LOAD_IMP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -# pragma warning (disable : 4786) // too long name, harmless warning -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// hash_collections_load_imp.hpp: serialization for loading stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// helper function templates for serialization of hashed collections -#include -#include -#include -#include -#include - -namespace boost{ -namespace serialization { -namespace stl { - -////////////////////////////////////////////////////////////////////// -// implementation of serialization for STL containers -// -template -inline void load_hash_collection(Archive & ar, Container &s) -{ - collection_size_type count; - collection_size_type bucket_count; - boost::serialization::item_version_type item_version(0); - boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - if(boost::archive::library_version_type(6) != library_version){ - ar >> BOOST_SERIALIZATION_NVP(count); - ar >> BOOST_SERIALIZATION_NVP(bucket_count); - } - else{ - // note: fixup for error in version 6. collection size was - // changed to size_t BUT for hashed collections it was implemented - // as an unsigned int. This should be a problem only on win64 machines - // but I'll leave it for everyone just in case. - unsigned int c; - unsigned int bc; - ar >> BOOST_SERIALIZATION_NVP(c); - count = c; - ar >> BOOST_SERIALIZATION_NVP(bc); - bucket_count = bc; - } - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - s.clear(); - #if ! defined(__MWERKS__) - s.resize(bucket_count); - #endif - InputFunction ifunc; - while(count-- > 0){ - ifunc(ar, s, item_version); - } -} - -} // namespace stl -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_HASH_COLLECTIONS_LOAD_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_save_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_save_imp.hpp deleted file mode 100644 index 65dfe83f16e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/hash_collections_save_imp.hpp +++ /dev/null @@ -1,97 +0,0 @@ -#ifndef BOOST_SERIALIZATION_HASH_COLLECTIONS_SAVE_IMP_HPP -#define BOOST_SERIALIZATION_HASH_COLLECTIONS_SAVE_IMP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// hash_collections_save_imp.hpp: serialization for stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// helper function templates for serialization of collections - -#include -#include -#include -#include -#include -#include - -namespace boost{ -namespace serialization { -namespace stl { - -////////////////////////////////////////////////////////////////////// -// implementation of serialization for STL containers -// - -template -inline void save_hash_collection(Archive & ar, const Container &s) -{ - collection_size_type count(s.size()); - const collection_size_type bucket_count(s.bucket_count()); - const item_version_type item_version( - version::value - ); - - #if 0 - /* should only be necessary to create archives of previous versions - * which is not currently supported. So for now comment this out - */ - boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - if(boost::archive::library_version_type(6) != library_version){ - ar << BOOST_SERIALIZATION_NVP(count); - ar << BOOST_SERIALIZATION_NVP(bucket_count); - } - else{ - // note: fixup for error in version 6. collection size was - // changed to size_t BUT for hashed collections it was implemented - // as an unsigned int. This should be a problem only on win64 machines - // but I'll leave it for everyone just in case. - const unsigned int c = count; - const unsigned int bc = bucket_count; - ar << BOOST_SERIALIZATION_NVP(c); - ar << BOOST_SERIALIZATION_NVP(bc); - } - if(boost::archive::library_version_type(3) < library_version){ - // record number of elements - // make sure the target type is registered so we can retrieve - // the version when we load - ar << BOOST_SERIALIZATION_NVP(item_version); - } - #else - ar << BOOST_SERIALIZATION_NVP(count); - ar << BOOST_SERIALIZATION_NVP(bucket_count); - ar << BOOST_SERIALIZATION_NVP(item_version); - #endif - - typename Container::const_iterator it = s.begin(); - while(count-- > 0){ - // note borland emits a no-op without the explicit namespace - boost::serialization::save_construct_data_adl( - ar, - &(*it), - boost::serialization::version< - typename Container::value_type - >::value - ); - ar << boost::serialization::make_nvp("item", *it++); - } -} - -} // namespace stl -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_HASH_COLLECTIONS_SAVE_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/hash_map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/hash_map.hpp deleted file mode 100644 index 22626db6838..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/hash_map.hpp +++ /dev/null @@ -1,232 +0,0 @@ -#ifndef BOOST_SERIALIZATION_HASH_MAP_HPP -#define BOOST_SERIALIZATION_HASH_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/hash_map.hpp: -// serialization for stl hash_map templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_HAS_HASH -#include BOOST_HASH_MAP_HEADER - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -namespace stl { - -// map input -template -struct archive_input_hash_map -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - // borland fails silently w/o full namespace - ar >> boost::serialization::make_nvp("item", t.reference()); - std::pair result = - s.insert(boost::move(t.reference())); - // note: the following presumes that the map::value_type was NOT tracked - // in the archive. This is the usual case, but here there is no way - // to determine that. - if(result.second){ - ar.reset_object_address( - & (result.first->second), - & t.reference().second - ); - } - } -}; - -// multimap input -template -struct archive_input_hash_multimap -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - // borland fails silently w/o full namespace - ar >> boost::serialization::make_nvp("item", t.reference()); - typename Container::const_iterator result - = s.insert(boost::move(t.reference())); - // note: the following presumes that the map::value_type was NOT tracked - // in the archive. This is the usual case, but here there is no way - // to determine that. - ar.reset_object_address( - & result->second, - & t.reference() - ); - } -}; - -} // stl - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const BOOST_STD_EXTENSION_NAMESPACE::hash_map< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::save_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_map< - Key, HashFcn, EqualKey, Allocator - > - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_map< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::load_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_map< - Key, HashFcn, EqualKey, Allocator - >, - boost::serialization::stl::archive_input_hash_map< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_map< - Key, HashFcn, EqualKey, Allocator - > - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_map< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// hash_multimap -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::save_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< - Key, HashFcn, EqualKey, Allocator - > - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::load_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< - Key, HashFcn, EqualKey, Allocator - >, - boost::serialization::stl::archive_input_hash_multimap< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< - Key, HashFcn, EqualKey, Allocator - > - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_HAS_HASH -#endif // BOOST_SERIALIZATION_HASH_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/hash_set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/hash_set.hpp deleted file mode 100644 index 0c72c18457e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/hash_set.hpp +++ /dev/null @@ -1,222 +0,0 @@ -#ifndef BOOST_SERIALIZATION_HASH_SET_HPP -#define BOOST_SERIALIZATION_HASH_SET_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// hash_set.hpp: serialization for stl hash_set templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_HAS_HASH -#include BOOST_HASH_SET_HEADER - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -namespace stl { - -// hash_set input -template -struct archive_input_hash_set -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - // borland fails silently w/o full namespace - ar >> boost::serialization::make_nvp("item", t.reference()); - std::pair result = - s.insert(boost::move(t.reference())); - if(result.second) - ar.reset_object_address(& (* result.first), & t.reference()); - } -}; - -// hash_multiset input -template -struct archive_input_hash_multiset -{ - inline void operator()( - Archive &ar, - Container &s, - const unsigned int v - ){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, v); - // borland fails silently w/o full namespace - ar >> boost::serialization::make_nvp("item", t.reference()); - typename Container::const_iterator result - = s.insert(boost::move(t.reference())); - ar.reset_object_address(& (* result), & t.reference()); - } -}; - -} // stl - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const BOOST_STD_EXTENSION_NAMESPACE::hash_set< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::save_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_set< - Key, HashFcn, EqualKey, Allocator - > - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_set< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::load_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_set< - Key, HashFcn, EqualKey, Allocator - >, - boost::serialization::stl::archive_input_hash_set< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_set< - Key, HashFcn, EqualKey, Allocator - > - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_set< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// hash_multiset -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::save_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< - Key, HashFcn, EqualKey, Allocator - > - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::stl::load_hash_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< - Key, HashFcn, EqualKey, Allocator - >, - boost::serialization::stl::archive_input_hash_multiset< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< - Key, HashFcn, EqualKey, Allocator - > - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::hash_multiset< - Key, HashFcn, EqualKey, Allocator - > & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(BOOST_STD_EXTENSION_NAMESPACE::hash_set) -BOOST_SERIALIZATION_COLLECTION_TRAITS(BOOST_STD_EXTENSION_NAMESPACE::hash_multiset) - -#endif // BOOST_HAS_HASH -#endif // BOOST_SERIALIZATION_HASH_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/is_bitwise_serializable.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/is_bitwise_serializable.hpp deleted file mode 100644 index 7e24a2cb6d8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/is_bitwise_serializable.hpp +++ /dev/null @@ -1,46 +0,0 @@ -// (C) Copyright 2007 Matthias Troyer - -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// Authors: Matthias Troyer - -/** @file is_bitwise_serializable.hpp - * - * This header provides a traits class for determining whether a class - * can be serialized (in a non-portable way) just by copying the bits. - */ - - -#ifndef BOOST_SERIALIZATION_IS_BITWISE_SERIALIZABLE_HPP -#define BOOST_SERIALIZATION_IS_BITWISE_SERIALIZABLE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -#include -#include - -namespace boost { -namespace serialization { - template - struct is_bitwise_serializable - : public is_arithmetic< T > - {}; -} // namespace serialization -} // namespace boost - - -// define a macro to make explicit designation of this more transparent -#define BOOST_IS_BITWISE_SERIALIZABLE(T) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct is_bitwise_serializable< T > : mpl::true_ {}; \ -}} \ -/**/ - -#endif //BOOST_SERIALIZATION_IS_BITWISE_SERIALIZABLE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/item_version_type.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/item_version_type.hpp deleted file mode 100644 index f3e5adac6f8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/item_version_type.hpp +++ /dev/null @@ -1,68 +0,0 @@ -#ifndef BOOST_SERIALIZATION_ITEM_VERSION_TYPE_HPP -#define BOOST_SERIALIZATION_ITEM_VERSION_TYPE_HPP - -// (C) Copyright 2010 Robert Ramey -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include // uint_least8_t -#include -#include -#include - -// fixes broken example build on x86_64-linux-gnu-gcc-4.6.0 -#include - -namespace boost { -namespace serialization { - -#if defined(_MSC_VER) -#pragma warning( push ) -#pragma warning( disable : 4244 4267 ) -#endif - -class item_version_type { -private: - typedef unsigned int base_type; - base_type t; -public: - // should be private - but MPI fails if it's not!!! - item_version_type(): t(0) {}; - explicit item_version_type(const unsigned int t_) : t(t_){ - BOOST_ASSERT(t_ <= boost::integer_traits::const_max); - } - item_version_type(const item_version_type & t_) : - t(t_.t) - {} - item_version_type & operator=(item_version_type rhs){ - t = rhs.t; - return *this; - } - // used for text output - operator base_type () const { - return t; - } - // used for text input - operator base_type & () { - return t; - } - bool operator==(const item_version_type & rhs) const { - return t == rhs.t; - } - bool operator<(const item_version_type & rhs) const { - return t < rhs.t; - } -}; - -#if defined(_MSC_VER) -#pragma warning( pop ) -#endif - -} } // end namespace boost::serialization - -BOOST_IS_BITWISE_SERIALIZABLE(item_version_type) - -BOOST_CLASS_IMPLEMENTATION(item_version_type, primitive_type) - -#endif //BOOST_SERIALIZATION_ITEM_VERSION_TYPE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/level.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/level.hpp deleted file mode 100644 index f6a84d10422..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/level.hpp +++ /dev/null @@ -1,116 +0,0 @@ -#ifndef BOOST_SERIALIZATION_LEVEL_HPP -#define BOOST_SERIALIZATION_LEVEL_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// level.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include - -#include - -namespace boost { -namespace serialization { - -struct basic_traits; - -// default serialization implementation level -template -struct implementation_level_impl { - template - struct traits_class_level { - typedef typename U::level type; - }; - - typedef mpl::integral_c_tag tag; - // note: at least one compiler complained w/o the full qualification - // on basic traits below - typedef - typename mpl::eval_if< - is_base_and_derived, - traits_class_level< T >, - //else - typename mpl::eval_if< - is_fundamental< T >, - mpl::int_, - //else - typename mpl::eval_if< - is_class< T >, - mpl::int_, - //else - typename mpl::eval_if< - is_array< T >, - mpl::int_, - //else - typename mpl::eval_if< - is_enum< T >, - mpl::int_, - //else - mpl::int_ - > - > - > - > - >::type type; - // vc 7.1 doesn't like enums here - BOOST_STATIC_CONSTANT(int, value = type::value); -}; - -template -struct implementation_level : - public implementation_level_impl -{ -}; - -template -inline bool operator>=(implementation_level< T > t, enum level_type l) -{ - return t.value >= (int)l; -} - -} // namespace serialization -} // namespace boost - -// specify the level of serialization implementation for the class -// require that class info saved when versioning is used -#define BOOST_CLASS_IMPLEMENTATION(T, E) \ - namespace boost { \ - namespace serialization { \ - template <> \ - struct implementation_level_impl< const T > \ - { \ - typedef mpl::integral_c_tag tag; \ - typedef mpl::int_< E > type; \ - BOOST_STATIC_CONSTANT( \ - int, \ - value = implementation_level_impl::type::value \ - ); \ - }; \ - } \ - } - /**/ - -#endif // BOOST_SERIALIZATION_LEVEL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/level_enum.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/level_enum.hpp deleted file mode 100644 index baf64e04f31..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/level_enum.hpp +++ /dev/null @@ -1,55 +0,0 @@ -#ifndef BOOST_SERIALIZATION_LEVEL_ENUM_HPP -#define BOOST_SERIALIZATION_LEVEL_ENUM_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// level_enum.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -namespace boost { -namespace serialization { - -// for each class used in the program, specify which level -// of serialization should be implemented - -// names for each level -enum level_type -{ - // Don't serialize this type. An attempt to do so should - // invoke a compile time assertion. - not_serializable = 0, - // write/read this type directly to the archive. In this case - // serialization code won't be called. This is the default - // case for fundamental types. It presumes a member function or - // template in the archive class that can handle this type. - // there is no runtime overhead associated reading/writing - // instances of this level - primitive_type = 1, - // Serialize the objects of this type using the objects "serialize" - // function or template. This permits values to be written/read - // to/from archives but includes no class or version information. - object_serializable = 2, - /////////////////////////////////////////////////////////////////// - // once an object is serialized at one of the above levels, the - // corresponding archives cannot be read if the implementation level - // for the archive object is changed. - /////////////////////////////////////////////////////////////////// - // Add class information to the archive. Class information includes - // implementation level, class version and class name if available - object_class_info = 3 -}; - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_LEVEL_ENUM_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/list.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/list.hpp deleted file mode 100644 index 5fdc114d7ed..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/list.hpp +++ /dev/null @@ -1,85 +0,0 @@ -#ifndef BOOST_SERIALIZATION_LIST_HPP -#define BOOST_SERIALIZATION_LIST_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// list.hpp: serialization for stl list templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include - -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template -inline void save( - Archive & ar, - const std::list &t, - const unsigned int /* file_version */ -){ - boost::serialization::stl::save_collection< - Archive, - std::list - >(ar, t); -} - -template -inline void load( - Archive & ar, - std::list &t, - const unsigned int /* file_version */ -){ - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - stl::collection_load_impl(ar, t, count, item_version); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::list & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(std::list) - -#endif // BOOST_SERIALIZATION_LIST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/map.hpp deleted file mode 100644 index 9209864c8cf..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/map.hpp +++ /dev/null @@ -1,139 +0,0 @@ -#ifndef BOOST_SERIALIZATION_MAP_HPP -#define BOOST_SERIALIZATION_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/map.hpp: -// serialization for stl map templates - -// (C) Copyright 2002-2014 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// implementation of serialization for map and mult-map STL containers - -template -inline void load_map_collection(Archive & ar, Container &s) -{ - s.clear(); - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - typename Container::iterator hint; - hint = s.begin(); - while(count-- > 0){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, item_version); - ar >> boost::serialization::make_nvp("item", t.reference()); - typename Container::iterator result = - s.insert(hint, boost::move(t.reference())); - ar.reset_object_address(& (result->second), & t.reference().second); - hint = result; - ++hint; - } -} - -// map -template -inline void save( - Archive & ar, - const std::map &t, - const unsigned int /* file_version */ -){ - boost::serialization::stl::save_collection< - Archive, - std::map - >(ar, t); -} - -template -inline void load( - Archive & ar, - std::map &t, - const unsigned int /* file_version */ -){ - load_map_collection(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::map &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// multimap -template -inline void save( - Archive & ar, - const std::multimap &t, - const unsigned int /* file_version */ -){ - boost::serialization::stl::save_collection< - Archive, - std::multimap - >(ar, t); -} - -template -inline void load( - Archive & ar, - std::multimap &t, - const unsigned int /* file_version */ -){ - load_map_collection(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::multimap &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/nvp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/nvp.hpp deleted file mode 100644 index 4e2297b3cc9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/nvp.hpp +++ /dev/null @@ -1,123 +0,0 @@ -#ifndef BOOST_SERIALIZATION_NVP_HPP -#define BOOST_SERIALIZATION_NVP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// nvp.hpp: interface for serialization system. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template -struct nvp : - public std::pair, - public wrapper_traits > -{ -//private: - nvp(const nvp & rhs) : - std::pair(rhs.first, rhs.second) - {} -public: - explicit nvp(const char * name_, T & t) : - // note: added _ to suppress useless gcc warning - std::pair(name_, & t) - {} - - const char * name() const { - return this->first; - } - T & value() const { - return *(this->second); - } - - const T & const_value() const { - return *(this->second); - } - - template - void save( - Archive & ar, - const unsigned int /* file_version */ - ) const { - ar.operator<<(const_value()); - } - template - void load( - Archive & ar, - const unsigned int /* file_version */ - ){ - ar.operator>>(value()); - } - BOOST_SERIALIZATION_SPLIT_MEMBER() -}; - -template -inline -const nvp< T > make_nvp(const char * name, T & t){ - return nvp< T >(name, t); -} - -// to maintain efficiency and portability, we want to assign -// specific serialization traits to all instances of this wrappers. -// we can't strait forward method below as it depends upon -// Partial Template Specialization and doing so would mean that wrappers -// wouldn't be treated the same on different platforms. This would -// break archive portability. Leave this here as reminder not to use it !!! - -template -struct implementation_level > -{ - typedef mpl::integral_c_tag tag; - typedef mpl::int_ type; - BOOST_STATIC_CONSTANT(int, value = implementation_level::type::value); -}; - -// nvp objects are generally created on the stack and are never tracked -template -struct tracking_level > -{ - typedef mpl::integral_c_tag tag; - typedef mpl::int_ type; - BOOST_STATIC_CONSTANT(int, value = tracking_level::type::value); -}; - -} // seralization -} // boost - -#include - -#define BOOST_SERIALIZATION_NVP(name) \ - boost::serialization::make_nvp(BOOST_PP_STRINGIZE(name), name) -/**/ - -#define BOOST_SERIALIZATION_BASE_OBJECT_NVP(name) \ - boost::serialization::make_nvp( \ - BOOST_PP_STRINGIZE(name), \ - boost::serialization::base_object(*this) \ - ) -/**/ - -#endif // BOOST_SERIALIZATION_NVP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/optional.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/optional.hpp deleted file mode 100644 index d6ff830a8c3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/optional.hpp +++ /dev/null @@ -1,107 +0,0 @@ -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 - -// (C) Copyright 2002-4 Pavel Vozenilek . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// Provides non-intrusive serialization for boost::optional. - -#ifndef BOOST_SERIALIZATION_OPTIONAL_HPP_ -#define BOOST_SERIALIZATION_OPTIONAL_HPP_ - -#if defined(_MSC_VER) -# pragma once -#endif - -#include - -#include - -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -// function specializations must be defined in the appropriate -// namespace - boost::serialization -namespace boost { -namespace serialization { - -template -void save( - Archive & ar, - const boost::optional< T > & t, - const unsigned int /*version*/ -){ - // It is an inherent limitation to the serialization of optional.hpp - // that the underlying type must be either a pointer or must have a - // default constructor. It's possible that this could change sometime - // in the future, but for now, one will have to work around it. This can - // be done by serialization the optional as optional - #if ! defined(BOOST_NO_CXX11_HDR_TYPE_TRAITS) - BOOST_STATIC_ASSERT( - boost::serialization::detail::is_default_constructible::value - || boost::is_pointer::value - ); - #endif - const bool tflag = t.is_initialized(); - ar << boost::serialization::make_nvp("initialized", tflag); - if (tflag){ - ar << boost::serialization::make_nvp("value", *t); - } -} - -template -void load( - Archive & ar, - boost::optional< T > & t, - const unsigned int version -){ - bool tflag; - ar >> boost::serialization::make_nvp("initialized", tflag); - if(! tflag){ - t.reset(); - return; - } - - if(0 == version){ - boost::serialization::item_version_type item_version(0); - boost::archive::library_version_type library_version( - ar.get_library_version() - ); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - } - if(! t.is_initialized()) - t = T(); - ar >> boost::serialization::make_nvp("value", *t); -} - -template -void serialize( - Archive & ar, - boost::optional< T > & t, - const unsigned int version -){ - boost::serialization::split_free(ar, t, version); -} - -template -struct version > { - BOOST_STATIC_CONSTANT(int, value = 1); -}; - -} // serialization -} // boost - -#endif // BOOST_SERIALIZATION_OPTIONAL_HPP_ diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/priority_queue.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/priority_queue.hpp deleted file mode 100644 index 5b08ffd1e82..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/priority_queue.hpp +++ /dev/null @@ -1,76 +0,0 @@ -#ifndef BOOST_SERIALIZATION_PRIORITY_QUEUE_HPP -#define BOOST_SERIALIZATION_PRIORITY_QUEUE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// priority_queue.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -// function specializations must be defined in the appropriate -// namespace - boost::serialization -#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) -#define STD _STLP_STD -#else -#define STD std -#endif - -namespace boost { -namespace serialization { -namespace detail{ - -template -struct priority_queue_save : public STD::priority_queue { - template - void operator()(Archive & ar, const unsigned int file_version) const { - save(ar, STD::priority_queue::c, file_version); - } -}; -template -struct priority_queue_load : public STD::priority_queue { - template - void operator()(Archive & ar, const unsigned int file_version) { - load(ar, STD::priority_queue::c, file_version); - } -}; - -} // detail - -template -inline void serialize( - Archive & ar, - std::priority_queue< T, Container, Compare> & t, - const unsigned int file_version -){ - typedef typename mpl::eval_if< - typename Archive::is_saving, - mpl::identity >, - mpl::identity > - >::type typex; - static_cast(t)(ar, file_version); -} - -} // namespace serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(STD::priority_queue) - -#undef STD - -#endif // BOOST_SERIALIZATION_PRIORITY_QUEUE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/queue.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/queue.hpp deleted file mode 100644 index b22745215d9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/queue.hpp +++ /dev/null @@ -1,76 +0,0 @@ -#ifndef BOOST_SERIALIZATION_QUEUE_HPP -#define BOOST_SERIALIZATION_QUEUE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// queue.hpp - -// (C) Copyright 2014 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -// function specializations must be defined in the appropriate -// namespace - boost::serialization -#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) -#define STD _STLP_STD -#else -#define STD std -#endif - -namespace boost { -namespace serialization { -namespace detail { - -template -struct queue_save : public STD::queue { - template - void operator()(Archive & ar, const unsigned int file_version) const { - save(ar, STD::queue::c, file_version); - } -}; -template -struct queue_load : public STD::queue { - template - void operator()(Archive & ar, const unsigned int file_version) { - load(ar, STD::queue::c, file_version); - } -}; - -} // detail - -template -inline void serialize( - Archive & ar, - std::queue< T, C> & t, - const unsigned int file_version -){ - typedef typename mpl::eval_if< - typename Archive::is_saving, - mpl::identity >, - mpl::identity > - >::type typex; - static_cast(t)(ar, file_version); -} - -} // namespace serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(STD::queue) - -#undef STD - -#endif // BOOST_SERIALIZATION_QUEUE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/scoped_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/scoped_ptr.hpp deleted file mode 100644 index 0d11f8436e0..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/scoped_ptr.hpp +++ /dev/null @@ -1,58 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SCOPED_PTR_HPP_VP_2003_10_30 -#define BOOST_SERIALIZATION_SCOPED_PTR_HPP_VP_2003_10_30 - -#if defined(_MSC_VER) -# pragma once -#endif - -// Copyright (c) 2003 Vladimir Prus. -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// Provides non-intrusive serialization for boost::scoped_ptr -// Does not allow to serialize scoped_ptr's to builtin types. - -#include - -#include -#include -#include - -namespace boost { -namespace serialization { - - template - void save( - Archive & ar, - const boost::scoped_ptr< T > & t, - const unsigned int /* version */ - ){ - T* r = t.get(); - ar << boost::serialization::make_nvp("scoped_ptr", r); - } - - template - void load( - Archive & ar, - boost::scoped_ptr< T > & t, - const unsigned int /* version */ - ){ - T* r; - ar >> boost::serialization::make_nvp("scoped_ptr", r); - t.reset(r); - } - - template - void serialize( - Archive& ar, - boost::scoped_ptr< T >& t, - const unsigned int version - ){ - boost::serialization::split_free(ar, t, version); - } - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_SCOPED_PTR_HPP_VP_2003_10_30 diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/serialization.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/serialization.hpp deleted file mode 100644 index a4d04723c75..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/serialization.hpp +++ /dev/null @@ -1,154 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SERIALIZATION_HPP -#define BOOST_SERIALIZATION_SERIALIZATION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -#if defined(_MSC_VER) -# pragma warning (disable : 4675) // suppress ADL warning -#endif - -#include -#include - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization.hpp: interface for serialization system. - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -////////////////////////////////////////////////////////////////////// -// public interface to serialization. - -///////////////////////////////////////////////////////////////////////////// -// layer 0 - intrusive verison -// declared and implemented for each user defined class to be serialized -// -// template -// serialize(Archive &ar, const unsigned int file_version){ -// ar & base_object(*this) & member1 & member2 ... ; -// } - -///////////////////////////////////////////////////////////////////////////// -// layer 1 - layer that routes member access through the access class. -// this is what permits us to grant access to private class member functions -// by specifying friend class boost::serialization::access - -#include - -///////////////////////////////////////////////////////////////////////////// -// layer 2 - default implementation of non-intrusive serialization. -// -// note the usage of function overloading to compensate that C++ does not -// currently support Partial Template Specialization for function templates -// We have declared the version number as "const unsigned long". -// Overriding templates for specific data types should declare the version -// number as "const unsigned int". Template matching will first be applied -// to functions with the same version types - that is the overloads. -// If there is no declared function prototype that matches, the second argument -// will be converted to "const unsigned long" and a match will be made with -// one of the default template functions below. - -namespace boost { -namespace serialization { - -BOOST_STRONG_TYPEDEF(unsigned int, version_type) - -// default implementation - call the member function "serialize" -template -inline void serialize( - Archive & ar, T & t, const unsigned int file_version -){ - access::serialize(ar, t, static_cast(file_version)); -} - -// save data required for construction -template -inline void save_construct_data( - Archive & /*ar*/, - const T * /*t*/, - const unsigned int /*file_version */ -){ - // default is to save no data because default constructor - // requires no arguments. -} - -// load data required for construction and invoke constructor in place -template -inline void load_construct_data( - Archive & /*ar*/, - T * t, - const unsigned int /*file_version*/ -){ - // default just uses the default constructor. going - // through access permits usage of otherwise private default - // constructor - access::construct(t); -} - -///////////////////////////////////////////////////////////////////////////// -// layer 3 - move call into serialization namespace so that ADL will function -// in the manner we desire. -// -// on compilers which don't implement ADL. only the current namespace -// i.e. boost::serialization will be searched. -// -// on compilers which DO implement ADL -// serialize overrides can be in any of the following -// -// 1) same namepace as Archive -// 2) same namespace as T -// 3) boost::serialization -// -// Due to Martin Ecker - -template -inline void serialize_adl( - Archive & ar, - T & t, - const unsigned int file_version -){ - // note usage of function overloading to delay final resolution - // until the point of instantiation. This works around the two-phase - // lookup "feature" which inhibits redefintion of a default function - // template implementation. Due to Robert Ramey - // - // Note that this trick generates problems for compiles which don't support - // PFTO, suppress it here. As far as we know, there are no compilers - // which fail to support PFTO while supporting two-phase lookup. - const version_type v(file_version); - serialize(ar, t, v); -} - -template -inline void save_construct_data_adl( - Archive & ar, - const T * t, - const unsigned int file_version -){ - // see above - const version_type v(file_version); - save_construct_data(ar, t, v); -} - -template -inline void load_construct_data_adl( - Archive & ar, - T * t, - const unsigned int file_version -){ - // see above comment - const version_type v(file_version); - load_construct_data(ar, t, v); -} - -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_SERIALIZATION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/set.hpp deleted file mode 100644 index 643906c5aac..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/set.hpp +++ /dev/null @@ -1,137 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SET_HPP -#define BOOST_SERIALIZATION_SET_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// set.hpp: serialization for stl set templates - -// (C) Copyright 2002-2014 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -namespace boost { -namespace serialization { - -template -inline void load_set_collection(Archive & ar, Container &s) -{ - s.clear(); - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - typename Container::iterator hint; - hint = s.begin(); - while(count-- > 0){ - typedef typename Container::value_type type; - detail::stack_construct t(ar, item_version); - // borland fails silently w/o full namespace - ar >> boost::serialization::make_nvp("item", t.reference()); - typename Container::iterator result = - s.insert(hint, boost::move(t.reference())); - ar.reset_object_address(& (* result), & t.reference()); - hint = result; - } -} - -template -inline void save( - Archive & ar, - const std::set &t, - const unsigned int /* file_version */ -){ - boost::serialization::stl::save_collection< - Archive, std::set - >(ar, t); -} - -template -inline void load( - Archive & ar, - std::set &t, - const unsigned int /* file_version */ -){ - load_set_collection(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::set & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// multiset -template -inline void save( - Archive & ar, - const std::multiset &t, - const unsigned int /* file_version */ -){ - boost::serialization::stl::save_collection< - Archive, - std::multiset - >(ar, t); -} - -template -inline void load( - Archive & ar, - std::multiset &t, - const unsigned int /* file_version */ -){ - load_set_collection(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::multiset & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(std::set) -BOOST_SERIALIZATION_COLLECTION_TRAITS(std::multiset) - -#endif // BOOST_SERIALIZATION_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr.hpp deleted file mode 100644 index 0d4c5ae6056..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr.hpp +++ /dev/null @@ -1,281 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SHARED_PTR_HPP -#define BOOST_SERIALIZATION_SHARED_PTR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// shared_ptr.hpp: serialization for boost shared pointer - -// (C) Copyright 2004 Robert Ramey and Martin Ecker -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include - -#include -#include -#include - -#include -#include - -#include -#include -#include -#include -#include - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// boost:: shared_ptr serialization traits -// version 1 to distinguish from boost 1.32 version. Note: we can only do this -// for a template when the compiler supports partial template specialization - -#ifndef BOOST_NO_TEMPLATE_PARTIAL_SPECIALIZATION - namespace boost { - namespace serialization{ - template - struct version< ::boost::shared_ptr< T > > { - typedef mpl::integral_c_tag tag; - #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3206)) - typedef typename mpl::int_<1> type; - #else - typedef mpl::int_<1> type; - #endif - BOOST_STATIC_CONSTANT(int, value = type::value); - }; - // don't track shared pointers - template - struct tracking_level< ::boost::shared_ptr< T > > { - typedef mpl::integral_c_tag tag; - #if BOOST_WORKAROUND(__MWERKS__, BOOST_TESTED_AT(0x3206)) - typedef typename mpl::int_< ::boost::serialization::track_never> type; - #else - typedef mpl::int_< ::boost::serialization::track_never> type; - #endif - BOOST_STATIC_CONSTANT(int, value = type::value); - }; - }} - #define BOOST_SERIALIZATION_SHARED_PTR(T) -#else - // define macro to let users of these compilers do this - #define BOOST_SERIALIZATION_SHARED_PTR(T) \ - BOOST_CLASS_VERSION( \ - ::boost::shared_ptr< T >, \ - 1 \ - ) \ - BOOST_CLASS_TRACKING( \ - ::boost::shared_ptr< T >, \ - ::boost::serialization::track_never \ - ) \ - /**/ -#endif - -namespace boost { -namespace serialization{ - -struct null_deleter { - void operator()(void const *) const {} -}; - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization for boost::shared_ptr - -// Using a constant means that all shared pointers are held in the same set. -// Thus we detect handle multiple pointers to the same value instances -// in the archive. -void * const shared_ptr_helper_id = 0; - -template -inline void save( - Archive & ar, - const boost::shared_ptr< T > &t, - const unsigned int /* file_version */ -){ - // The most common cause of trapping here would be serializing - // something like shared_ptr. This occurs because int - // is never tracked by default. Wrap int in a trackable type - BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); - const T * t_ptr = t.get(); - ar << boost::serialization::make_nvp("px", t_ptr); -} - -#ifdef BOOST_SERIALIZATION_SHARED_PTR_132_HPP -template -inline void load( - Archive & ar, - boost::shared_ptr< T > &t, - const unsigned int file_version -){ - // something like shared_ptr. This occurs because int - // is never tracked by default. Wrap int in a trackable type - BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); - T* r; - if(file_version < 1){ - ar.register_type(static_cast< - boost_132::detail::sp_counted_base_impl * - >(NULL)); - boost_132::shared_ptr< T > sp; - ar >> boost::serialization::make_nvp("px", sp.px); - ar >> boost::serialization::make_nvp("pn", sp.pn); - // got to keep the sps around so the sp.pns don't disappear - boost::serialization::shared_ptr_helper & h = - ar.template get_helper< shared_ptr_helper >( - shared_ptr_helper_id - ); - h.append(sp); - r = sp.get(); - } - else{ - ar >> boost::serialization::make_nvp("px", r); - } - shared_ptr_helper & h = - ar.template get_helper >( - shared_ptr_helper_id - ); - h.reset(t,r); -} -#else - -template -inline void load( - Archive & ar, - boost::shared_ptr< T > &t, - const unsigned int /*file_version*/ -){ - // The most common cause of trapping here would be serializing - // something like shared_ptr. This occurs because int - // is never tracked by default. Wrap int in a trackable type - BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); - T* r; - ar >> boost::serialization::make_nvp("px", r); - - boost::serialization::shared_ptr_helper & h = - ar.template get_helper >( - shared_ptr_helper_id - ); - h.reset(t,r); -} -#endif - -template -inline void serialize( - Archive & ar, - boost::shared_ptr< T > &t, - const unsigned int file_version -){ - // correct shared_ptr serialization depends upon object tracking - // being used. - BOOST_STATIC_ASSERT( - boost::serialization::tracking_level< T >::value - != boost::serialization::track_never - ); - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// std::shared_ptr serialization traits -// version 1 to distinguish from boost 1.32 version. Note: we can only do this -// for a template when the compiler supports partial template specialization - -#ifndef BOOST_NO_CXX11_SMART_PTR -#include - -// note: we presume that any compiler/library which supports C++11 -// std::pointers also supports template partial specialization -// trap here if such presumption were to turn out to wrong!!! -#ifdef BOOST_NO_TEMPLATE_PARTIAL_SPECIALIZATION - BOOST_STATIC_ASSERT(false); -#endif - -namespace boost { -namespace serialization{ - template - struct version< ::std::shared_ptr< T > > { - typedef mpl::integral_c_tag tag; - typedef mpl::int_<1> type; - BOOST_STATIC_CONSTANT(int, value = type::value); - }; - // don't track shared pointers - template - struct tracking_level< ::std::shared_ptr< T > > { - typedef mpl::integral_c_tag tag; - typedef mpl::int_< ::boost::serialization::track_never> type; - BOOST_STATIC_CONSTANT(int, value = type::value); - }; -}} -// the following just keeps older programs from breaking -#define BOOST_SERIALIZATION_SHARED_PTR(T) - -namespace boost { -namespace serialization{ - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization for std::shared_ptr - -template -inline void save( - Archive & ar, - const std::shared_ptr< T > &t, - const unsigned int /* file_version */ -){ - // The most common cause of trapping here would be serializing - // something like shared_ptr. This occurs because int - // is never tracked by default. Wrap int in a trackable type - BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); - const T * t_ptr = t.get(); - ar << boost::serialization::make_nvp("px", t_ptr); -} - -template -inline void load( - Archive & ar, - std::shared_ptr< T > &t, - const unsigned int /*file_version*/ -){ - // The most common cause of trapping here would be serializing - // something like shared_ptr. This occurs because int - // is never tracked by default. Wrap int in a trackable type - BOOST_STATIC_ASSERT((tracking_level< T >::value != track_never)); - T* r; - ar >> boost::serialization::make_nvp("px", r); - //void (* const id)(Archive &, std::shared_ptr< T > &, const unsigned int) = & load; - boost::serialization::shared_ptr_helper & h = - ar.template get_helper< - shared_ptr_helper - >( - shared_ptr_helper_id - ); - h.reset(t,r); -} - -template -inline void serialize( - Archive & ar, - std::shared_ptr< T > &t, - const unsigned int file_version -){ - // correct shared_ptr serialization depends upon object tracking - // being used. - BOOST_STATIC_ASSERT( - boost::serialization::tracking_level< T >::value - != boost::serialization::track_never - ); - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_NO_CXX11_SMART_PTR - -#endif // BOOST_SERIALIZATION_SHARED_PTR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_132.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_132.hpp deleted file mode 100644 index 3dfaba4d69a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_132.hpp +++ /dev/null @@ -1,222 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SHARED_PTR_132_HPP -#define BOOST_SERIALIZATION_SHARED_PTR_132_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// shared_ptr.hpp: serialization for boost shared pointer - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// note: totally unadvised hack to gain access to private variables -// in shared_ptr and shared_count. Unfortunately its the only way to -// do this without changing shared_ptr and shared_count -// the best we can do is to detect a conflict here -#include - -#include -#include // NULL - -#include -#include -#include -#include -#include - -// mark base class as an (uncreatable) base class -#include - -///////////////////////////////////////////////////////////// -// Maintain a couple of lists of loaded shared pointers of the old previous -// version (1.32) - -namespace boost_132 { -namespace serialization { -namespace detail { - -struct null_deleter { - void operator()(void const *) const {} -}; - -} // namespace detail -} // namespace serialization -} // namespace boost_132 - -///////////////////////////////////////////////////////////// -// sp_counted_base_impl serialization - -namespace boost { -namespace serialization { - -template -inline void serialize( - Archive & /* ar */, - boost_132::detail::sp_counted_base_impl & /* t */, - const unsigned int /*file_version*/ -){ - // register the relationship between each derived class - // its polymorphic base - boost::serialization::void_cast_register< - boost_132::detail::sp_counted_base_impl, - boost_132::detail::sp_counted_base - >( - static_cast *>(NULL), - static_cast(NULL) - ); -} - -template -inline void save_construct_data( - Archive & ar, - const - boost_132::detail::sp_counted_base_impl *t, - const unsigned int /* file_version */ -){ - // variables used for construction - ar << boost::serialization::make_nvp("ptr", t->ptr); -} - -template -inline void load_construct_data( - Archive & ar, - boost_132::detail::sp_counted_base_impl * t, - const unsigned int /* file_version */ -){ - P ptr_; - ar >> boost::serialization::make_nvp("ptr", ptr_); - // ::new(t)boost_132::detail::sp_counted_base_impl(ptr_, D()); - // placement - // note: the original ::new... above is replaced by the one here. This one - // creates all new objects with a null_deleter so that after the archive - // is finished loading and the shared_ptrs are destroyed - the underlying - // raw pointers are NOT deleted. This is necessary as they are used by the - // new system as well. - ::new(t)boost_132::detail::sp_counted_base_impl< - P, - boost_132::serialization::detail::null_deleter - >( - ptr_, boost_132::serialization::detail::null_deleter() - ); // placement new - // compensate for that fact that a new shared count always is - // initialized with one. the add_ref_copy below will increment it - // every time its serialized so without this adjustment - // the use and weak counts will be off by one. - t->use_count_ = 0; -} - -} // serialization -} // namespace boost - -///////////////////////////////////////////////////////////// -// shared_count serialization - -namespace boost { -namespace serialization { - -template -inline void save( - Archive & ar, - const boost_132::detail::shared_count &t, - const unsigned int /* file_version */ -){ - ar << boost::serialization::make_nvp("pi", t.pi_); -} - -template -inline void load( - Archive & ar, - boost_132::detail::shared_count &t, - const unsigned int /* file_version */ -){ - ar >> boost::serialization::make_nvp("pi", t.pi_); - if(NULL != t.pi_) - t.pi_->add_ref_copy(); -} - -} // serialization -} // namespace boost - -BOOST_SERIALIZATION_SPLIT_FREE(boost_132::detail::shared_count) - -///////////////////////////////////////////////////////////// -// implement serialization for shared_ptr< T > - -namespace boost { -namespace serialization { - -template -inline void save( - Archive & ar, - const boost_132::shared_ptr< T > &t, - const unsigned int /* file_version */ -){ - // only the raw pointer has to be saved - // the ref count is maintained automatically as shared pointers are loaded - ar.register_type(static_cast< - boost_132::detail::sp_counted_base_impl > * - >(NULL)); - ar << boost::serialization::make_nvp("px", t.px); - ar << boost::serialization::make_nvp("pn", t.pn); -} - -template -inline void load( - Archive & ar, - boost_132::shared_ptr< T > &t, - const unsigned int /* file_version */ -){ - // only the raw pointer has to be saved - // the ref count is maintained automatically as shared pointers are loaded - ar.register_type(static_cast< - boost_132::detail::sp_counted_base_impl > * - >(NULL)); - ar >> boost::serialization::make_nvp("px", t.px); - ar >> boost::serialization::make_nvp("pn", t.pn); -} - -template -inline void serialize( - Archive & ar, - boost_132::shared_ptr< T > &t, - const unsigned int file_version -){ - // correct shared_ptr serialization depends upon object tracking - // being used. - BOOST_STATIC_ASSERT( - boost::serialization::tracking_level< T >::value - != boost::serialization::track_never - ); - boost::serialization::split_free(ar, t, file_version); -} - -} // serialization -} // namespace boost - -// note: change below uses null_deleter -// This macro is used to export GUIDS for shared pointers to allow -// the serialization system to export them properly. David Tonge -#define BOOST_SHARED_POINTER_EXPORT_GUID(T, K) \ - typedef boost_132::detail::sp_counted_base_impl< \ - T *, \ - boost::checked_deleter< T > \ - > __shared_ptr_ ## T; \ - BOOST_CLASS_EXPORT_GUID(__shared_ptr_ ## T, "__shared_ptr_" K) \ - BOOST_CLASS_EXPORT_GUID(T, K) \ - /**/ - -#define BOOST_SHARED_POINTER_EXPORT(T) \ - BOOST_SHARED_POINTER_EXPORT_GUID( \ - T, \ - BOOST_PP_STRINGIZE(T) \ - ) \ - /**/ - -#endif // BOOST_SERIALIZATION_SHARED_PTR_132_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_helper.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_helper.hpp deleted file mode 100644 index 37c34d6b2c4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/shared_ptr_helper.hpp +++ /dev/null @@ -1,209 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SHARED_PTR_HELPER_HPP -#define BOOST_SERIALIZATION_SHARED_PTR_HELPER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// shared_ptr_helper.hpp: serialization for boost shared pointern - -// (C) Copyright 2004-2009 Robert Ramey, Martin Ecker and Takatoshi Kondo -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include // NULL - -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -namespace boost_132 { - template class shared_ptr; -} -namespace boost { -namespace serialization { - -#ifndef BOOST_NO_MEMBER_TEMPLATE_FRIENDS -template class SPT > -void load( - Archive & ar, - SPT< class U > &t, - const unsigned int file_version -); -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// a common class for holding various types of shared pointers - -template class SPT> -class shared_ptr_helper { - typedef std::map< - const void *, // address of object - SPT // address shared ptr to single instance - > object_shared_pointer_map; - - // list of shared_pointers create accessable by raw pointer. This - // is used to "match up" shared pointers loaded at different - // points in the archive. Note, we delay construction until - // it is actually used since this is by default included as - // a "mix-in" even if shared_ptr isn't used. - object_shared_pointer_map * m_o_sp; - - struct null_deleter { - void operator()(void const *) const {} - }; - -#if defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) \ -|| defined(BOOST_MSVC) \ -|| defined(__SUNPRO_CC) -public: -#else - template - friend void boost::serialization::load( - Archive & ar, - SPT< U > &t, - const unsigned int file_version - ); -#endif - - #ifdef BOOST_SERIALIZATION_SHARED_PTR_132_HPP - // list of loaded pointers. This is used to be sure that the pointers - // stay around long enough to be "matched" with other pointers loaded - // by the same archive. These are created with a "null_deleter" so that - // when this list is destroyed - the underlaying raw pointers are not - // destroyed. This has to be done because the pointers are also held by - // new system which is disjoint from this set. This is implemented - // by a change in load_construct_data below. It makes this file suitable - // only for loading pointers into a 1.33 or later boost system. - std::list > * m_pointers_132; - void - append(const boost_132::shared_ptr & t){ - if(NULL == m_pointers_132) - m_pointers_132 = new std::list >; - m_pointers_132->push_back(t); - } - #endif - - struct non_polymorphic { - template - static const boost::serialization::extended_type_info * - get_object_type(U & ){ - return & boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< U >::type - >::get_const_instance(); - } - }; - struct polymorphic { - template - static const boost::serialization::extended_type_info * - get_object_type(U & u){ - return boost::serialization::singleton< - typename - boost::serialization::type_info_implementation< U >::type - >::get_const_instance().get_derived_extended_type_info(u); - } - }; - -public: - template - void reset(SPT< T > & s, T * t){ - if(NULL == t){ - s.reset(); - return; - } - const boost::serialization::extended_type_info * this_type - = & boost::serialization::type_info_implementation< T >::type - ::get_const_instance(); - - // get pointer to the most derived object's eti. This is effectively - // the object type identifer - typedef typename mpl::if_< - is_polymorphic< T >, - polymorphic, - non_polymorphic - >::type type; - - const boost::serialization::extended_type_info * true_type - = type::get_object_type(*t); - - // note:if this exception is thrown, be sure that derived pointern - // is either registered or exported. - if(NULL == true_type) - boost::serialization::throw_exception( - boost::archive::archive_exception( - boost::archive::archive_exception::unregistered_class, - this_type->get_debug_info() - ) - ); - // get void pointer to the most derived type - // this uniquely identifies the object referred to - // oid = "object identifier" - const void * oid = void_downcast( - *true_type, - *this_type, - t - ); - if(NULL == oid) - boost::serialization::throw_exception( - boost::archive::archive_exception( - boost::archive::archive_exception::unregistered_cast, - true_type->get_debug_info(), - this_type->get_debug_info() - ) - ); - - // make tracking array if necessary - if(NULL == m_o_sp) - m_o_sp = new object_shared_pointer_map; - - typename object_shared_pointer_map::iterator i = m_o_sp->find(oid); - - // if it's a new object - if(i == m_o_sp->end()){ - s.reset(t); - std::pair result; - result = m_o_sp->insert(std::make_pair(oid, s)); - BOOST_ASSERT(result.second); - } - // if the object has already been seen - else{ - s = SPT(i->second, t); - } - } - - shared_ptr_helper() : - m_o_sp(NULL) - #ifdef BOOST_SERIALIZATION_SHARED_PTR_132_HPP - , m_pointers_132(NULL) - #endif - {} - virtual ~shared_ptr_helper(){ - if(NULL != m_o_sp) - delete m_o_sp; - #ifdef BOOST_SERIALIZATION_SHARED_PTR_132_HPP - if(NULL != m_pointers_132) - delete m_pointers_132; - #endif - } -}; - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_SHARED_PTR_HELPER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/singleton.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/singleton.hpp deleted file mode 100644 index b50afedbb92..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/singleton.hpp +++ /dev/null @@ -1,166 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SINGLETON_HPP -#define BOOST_SERIALIZATION_SINGLETON_HPP - -/////////1/////////2///////// 3/////////4/////////5/////////6/////////7/////////8 -// singleton.hpp -// -// Copyright David Abrahams 2006. Original version -// -// Copyright Robert Ramey 2007. Changes made to permit -// application throughout the serialization library. -// -// Distributed under the Boost -// Software License, Version 1.0. (See accompanying -// file LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) -// -// The intention here is to define a template which will convert -// any class into a singleton with the following features: -// -// a) initialized before first use. -// b) thread-safe for const access to the class -// c) non-locking -// -// In order to do this, -// a) Initialize dynamically when used. -// b) Require that all singletons be initialized before main -// is called or any entry point into the shared library is invoked. -// This guarentees no race condition for initialization. -// In debug mode, we assert that no non-const functions are called -// after main is invoked. -// - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -#include -#include -#include -#include - -#include -#include -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4511 4512) -#endif - -namespace boost { -namespace serialization { - -////////////////////////////////////////////////////////////////////// -// Provides a dynamically-initialized (singleton) instance of T in a -// way that avoids LNK1179 on vc6. See http://tinyurl.com/ljdp8 or -// http://lists.boost.org/Archives/boost/2006/05/105286.php for -// details. -// - -// singletons created by this code are guarenteed to be unique -// within the executable or shared library which creates them. -// This is sufficient and in fact ideal for the serialization library. -// The singleton is created when the module is loaded and destroyed -// when the module is unloaded. - -// This base class has two functions. - -// First it provides a module handle for each singleton indicating -// the executable or shared library in which it was created. This -// turns out to be necessary and sufficient to implement the tables -// used by serialization library. - -// Second, it provides a mechanism to detect when a non-const function -// is called after initialization. - -// make a singleton to lock/unlock all singletons for alteration. -// The intent is that all singletons created/used by this code -// are to be initialized before main is called. A test program -// can lock all the singletons when main is entereed. This any -// attempt to retieve a mutable instances while locked will -// generate a assertion if compiled for debug. - -// note usage of BOOST_DLLEXPORT. These functions are in danger of -// being eliminated by the optimizer when building an application in -// release mode. Usage of the macro is meant to signal the compiler/linker -// to avoid dropping these functions which seem to be unreferenced. -// This usage is not related to autolinking. - -class BOOST_SYMBOL_VISIBLE singleton_module : - public boost::noncopyable -{ -private: - BOOST_SERIALIZATION_DECL BOOST_DLLEXPORT static bool & get_lock() BOOST_USED; -public: - BOOST_DLLEXPORT static void lock(){ - get_lock() = true; - } - BOOST_DLLEXPORT static void unlock(){ - get_lock() = false; - } - BOOST_DLLEXPORT static bool is_locked(){ - return get_lock(); - } -}; - -template -class singleton : public singleton_module -{ -private: - static T & m_instance; - // include this to provoke instantiation at pre-execution time - static void use(T const *) {} - static T & get_instance() { - // use a wrapper so that types T with protected constructors - // can be used - class singleton_wrapper : public T {}; - static singleton_wrapper t; - // refer to instance, causing it to be instantiated (and - // initialized at startup on working compilers) - BOOST_ASSERT(! is_destroyed()); - // note that the following is absolutely essential. - // commenting out this statement will cause compilers to fail to - // construct the instance at pre-execution time. This would prevent - // our usage/implementation of "locking" and introduce uncertainty into - // the sequence of object initializaition. - use(& m_instance); - return static_cast(t); - } - static bool & get_is_destroyed(){ - static bool is_destroyed; - return is_destroyed; - } - -public: - BOOST_DLLEXPORT static T & get_mutable_instance(){ - BOOST_ASSERT(! is_locked()); - return get_instance(); - } - BOOST_DLLEXPORT static const T & get_const_instance(){ - return get_instance(); - } - BOOST_DLLEXPORT static bool is_destroyed(){ - return get_is_destroyed(); - } - BOOST_DLLEXPORT singleton(){ - get_is_destroyed() = false; - } - BOOST_DLLEXPORT ~singleton() { - get_is_destroyed() = true; - } -}; - -template -T & singleton< T >::m_instance = singleton< T >::get_instance(); - -} // namespace serialization -} // namespace boost - -#include // pops abi_suffix.hpp pragmas - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#endif // BOOST_SERIALIZATION_SINGLETON_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/slist.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/slist.hpp deleted file mode 100644 index d9b971bc4f1..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/slist.hpp +++ /dev/null @@ -1,145 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SLIST_HPP -#define BOOST_SERIALIZATION_SLIST_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// slist.hpp - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#ifdef BOOST_HAS_SLIST -#include BOOST_SLIST_HEADER - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template -inline void save( - Archive & ar, - const BOOST_STD_EXTENSION_NAMESPACE::slist &t, - const unsigned int file_version -){ - boost::serialization::stl::save_collection< - Archive, - BOOST_STD_EXTENSION_NAMESPACE::slist - >(ar, t); -} - -namespace stl { - -template< - class Archive, - class T, - class Allocator -> -typename boost::disable_if< - typename detail::is_default_constructible< - typename BOOST_STD_EXTENSION_NAMESPACE::slist::value_type - >, - void ->::type -collection_load_impl( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::slist &t, - collection_size_type count, - item_version_type item_version -){ - t.clear(); - boost::serialization::detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - t.push_front(boost::move(u.reference())); - typename BOOST_STD_EXTENSION_NAMESPACE::slist::iterator last; - last = t.begin(); - ar.reset_object_address(&(*t.begin()) , & u.reference()); - while(--count > 0){ - detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - last = t.insert_after(last, boost::move(u.reference())); - ar.reset_object_address(&(*last) , & u.reference()); - } -} - -} // stl - -template -inline void load( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::slist &t, - const unsigned int file_version -){ - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - if(detail::is_default_constructible()){ - t.resize(count); - typename BOOST_STD_EXTENSION_NAMESPACE::slist::iterator hint; - hint = t.begin(); - while(count-- > 0){ - ar >> boost::serialization::make_nvp("item", *hint++); - } - } - else{ - t.clear(); - boost::serialization::detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - t.push_front(boost::move(u.reference())); - typename BOOST_STD_EXTENSION_NAMESPACE::slist::iterator last; - last = t.begin(); - ar.reset_object_address(&(*t.begin()) , & u.reference()); - while(--count > 0){ - detail::stack_construct u(ar, item_version); - ar >> boost::serialization::make_nvp("item", u.reference()); - last = t.insert_after(last, boost::move(u.reference())); - ar.reset_object_address(&(*last) , & u.reference()); - } - } -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - BOOST_STD_EXTENSION_NAMESPACE::slist &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(BOOST_STD_EXTENSION_NAMESPACE::slist) - -#endif // BOOST_HAS_SLIST -#endif // BOOST_SERIALIZATION_SLIST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/smart_cast.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/smart_cast.hpp deleted file mode 100644 index 563f36aa20b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/smart_cast.hpp +++ /dev/null @@ -1,275 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SMART_CAST_HPP -#define BOOST_SERIALIZATION_SMART_CAST_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// smart_cast.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org/libs/serialization for updates, documentation, and revision history. - -// casting of pointers and references. - -// In casting between different C++ classes, there are a number of -// rules that have to be kept in mind in deciding whether to use -// static_cast or dynamic_cast. - -// a) dynamic casting can only be applied when one of the types is polymorphic -// Otherwise static_cast must be used. -// b) only dynamic casting can do runtime error checking -// use of static_cast is generally un checked even when compiled for debug -// c) static_cast would be considered faster than dynamic_cast. - -// If casting is applied to a template parameter, there is no apriori way -// to know which of the two casting methods will be permitted or convenient. - -// smart_cast uses C++ type_traits, and program debug mode to select the -// most convenient cast to use. - -#include -#include -#include // NULL - -#include -#include - -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace serialization { -namespace smart_cast_impl { - - template - struct reference { - - struct polymorphic { - - struct linear { - template - static T cast(U & u){ - return static_cast< T >(u); - } - }; - - struct cross { - template - static T cast(U & u){ - return dynamic_cast< T >(u); - } - }; - - template - static T cast(U & u){ - // if we're in debug mode - #if ! defined(NDEBUG) \ - || defined(__MWERKS__) - // do a checked dynamic cast - return cross::cast(u); - #else - // borland 5.51 chokes here so we can't use it - // note: if remove_reference isn't function for these types - // cross casting will be selected this will work but will - // not be the most efficient method. This will conflict with - // the original smart_cast motivation. - typedef typename mpl::eval_if< - typename mpl::and_< - mpl::not_::type, - U - > >, - mpl::not_::type - > > - >, - // borland chokes w/o full qualification here - mpl::identity, - mpl::identity - >::type typex; - // typex works around gcc 2.95 issue - return typex::cast(u); - #endif - } - }; - - struct non_polymorphic { - template - static T cast(U & u){ - return static_cast< T >(u); - } - }; - template - static T cast(U & u){ - typedef typename mpl::eval_if< - boost::is_polymorphic, - mpl::identity, - mpl::identity - >::type typex; - return typex::cast(u); - } - }; - - template - struct pointer { - - struct polymorphic { - // unfortunately, this below fails to work for virtual base - // classes. need has_virtual_base to do this. - // Subject for further study - #if 0 - struct linear { - template - static T cast(U * u){ - return static_cast< T >(u); - } - }; - - struct cross { - template - static T cast(U * u){ - T tmp = dynamic_cast< T >(u); - #ifndef NDEBUG - if ( tmp == 0 ) throw_exception(std::bad_cast()); - #endif - return tmp; - } - }; - - template - static T cast(U * u){ - typedef - typename mpl::eval_if< - typename mpl::and_< - mpl::not_::type, - U - > >, - mpl::not_::type - > > - >, - // borland chokes w/o full qualification here - mpl::identity, - mpl::identity - >::type typex; - return typex::cast(u); - } - #else - template - static T cast(U * u){ - T tmp = dynamic_cast< T >(u); - #ifndef NDEBUG - if ( tmp == 0 ) throw_exception(std::bad_cast()); - #endif - return tmp; - } - #endif - }; - - struct non_polymorphic { - template - static T cast(U * u){ - return static_cast< T >(u); - } - }; - - template - static T cast(U * u){ - typedef typename mpl::eval_if< - boost::is_polymorphic, - mpl::identity, - mpl::identity - >::type typex; - return typex::cast(u); - } - - }; - - template - struct void_pointer { - template - static TPtr cast(UPtr uptr){ - return static_cast(uptr); - } - }; - - template - struct error { - // if we get here, its because we are using one argument in the - // cast on a system which doesn't support partial template - // specialization - template - static T cast(U){ - BOOST_STATIC_ASSERT(sizeof(T)==0); - return * static_cast(NULL); - } - }; - -} // smart_cast_impl - -// this implements: -// smart_cast(Source * s) -// smart_cast(s) -// note that it will fail with -// smart_cast(s) -template -T smart_cast(U u) { - typedef - typename mpl::eval_if< - typename mpl::or_< - boost::is_same, - boost::is_same, - boost::is_same, - boost::is_same - >, - mpl::identity >, - // else - typename mpl::eval_if, - mpl::identity >, - // else - typename mpl::eval_if, - mpl::identity >, - // else - mpl::identity - > - > - > - >::type typex; - return typex::cast(u); -} - -// this implements: -// smart_cast_reference(Source & s) -template -T smart_cast_reference(U & u) { - return smart_cast_impl::reference< T >::cast(u); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_SMART_CAST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/split_free.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/split_free.hpp deleted file mode 100644 index 85e2f590fe4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/split_free.hpp +++ /dev/null @@ -1,93 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SPLIT_FREE_HPP -#define BOOST_SERIALIZATION_SPLIT_FREE_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// split_free.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -namespace boost { -namespace archive { - namespace detail { - template class interface_oarchive; - template class interface_iarchive; - } // namespace detail -} // namespace archive - -namespace serialization { - -//namespace detail { -template -struct free_saver { - static void invoke( - Archive & ar, - const T & t, - const unsigned int file_version - ){ - // use function overload (version_type) to workaround - // two-phase lookup issue - const version_type v(file_version); - save(ar, t, v); - } -}; -template -struct free_loader { - static void invoke( - Archive & ar, - T & t, - const unsigned int file_version - ){ - // use function overload (version_type) to workaround - // two-phase lookup issue - const version_type v(file_version); - load(ar, t, v); - } -}; -//} // namespace detail - -template -inline void split_free( - Archive & ar, - T & t, - const unsigned int file_version -){ - typedef typename mpl::eval_if< - typename Archive::is_saving, - mpl::identity >, - mpl::identity > - >::type typex; - typex::invoke(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#define BOOST_SERIALIZATION_SPLIT_FREE(T) \ -namespace boost { namespace serialization { \ -template \ -inline void serialize( \ - Archive & ar, \ - T & t, \ - const unsigned int file_version \ -){ \ - split_free(ar, t, file_version); \ -} \ -}} -/**/ - -#endif // BOOST_SERIALIZATION_SPLIT_FREE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/split_member.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/split_member.hpp deleted file mode 100644 index 5f32520559e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/split_member.hpp +++ /dev/null @@ -1,86 +0,0 @@ -#ifndef BOOST_SERIALIZATION_SPLIT_MEMBER_HPP -#define BOOST_SERIALIZATION_SPLIT_MEMBER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// split_member.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include - -#include - -namespace boost { -namespace archive { - namespace detail { - template class interface_oarchive; - template class interface_iarchive; - } // namespace detail -} // namespace archive - -namespace serialization { -namespace detail { - - template - struct member_saver { - static void invoke( - Archive & ar, - const T & t, - const unsigned int file_version - ){ - access::member_save(ar, t, file_version); - } - }; - - template - struct member_loader { - static void invoke( - Archive & ar, - T & t, - const unsigned int file_version - ){ - access::member_load(ar, t, file_version); - } - }; - -} // detail - -template -inline void split_member( - Archive & ar, T & t, const unsigned int file_version -){ - typedef typename mpl::eval_if< - typename Archive::is_saving, - mpl::identity >, - mpl::identity > - >::type typex; - typex::invoke(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -// split member function serialize funcition into save/load -#define BOOST_SERIALIZATION_SPLIT_MEMBER() \ -template \ -void serialize( \ - Archive &ar, \ - const unsigned int file_version \ -){ \ - boost::serialization::split_member(ar, *this, file_version); \ -} \ -/**/ - -#endif // BOOST_SERIALIZATION_SPLIT_MEMBER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/stack.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/stack.hpp deleted file mode 100644 index 96f90fe8767..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/stack.hpp +++ /dev/null @@ -1,76 +0,0 @@ -#ifndef BOOST_SERIALIZATION_STACK_HPP -#define BOOST_SERIALIZATION_STACK_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// stack.hpp - -// (C) Copyright 2014 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include - -// function specializations must be defined in the appropriate -// namespace - boost::serialization -#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) -#define STD _STLP_STD -#else -#define STD std -#endif - -namespace boost { -namespace serialization { -namespace detail{ - -template -struct stack_save : public STD::stack { - template - void operator()(Archive & ar, const unsigned int file_version) const { - save(ar, STD::stack::c, file_version); - } -}; -template -struct stack_load : public STD::stack { - template - void operator()(Archive & ar, const unsigned int file_version) { - load(ar, STD::stack::c, file_version); - } -}; - -} // detail - -template -inline void serialize( - Archive & ar, - std::stack< T, C> & t, - const unsigned int file_version -){ - typedef typename mpl::eval_if< - typename Archive::is_saving, - mpl::identity >, - mpl::identity > - >::type typex; - static_cast(t)(ar, file_version); -} - -} // namespace serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(STD::stack) - -#undef STD - -#endif // BOOST_SERIALIZATION_DEQUE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/state_saver.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/state_saver.hpp deleted file mode 100644 index 248b8d91556..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/state_saver.hpp +++ /dev/null @@ -1,96 +0,0 @@ -#ifndef BOOST_SERIALIZATION_STATE_SAVER_HPP -#define BOOST_SERIALIZATION_STATE_SAVER_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// state_saver.hpp: - -// (C) Copyright 2003-4 Pavel Vozenilek and Robert Ramey - http://www.rrsd.com. -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org/libs/serialization for updates, documentation, and revision history. - -// Inspired by Daryle Walker's iostate_saver concept. This saves the original -// value of a variable when a state_saver is constructed and restores -// upon destruction. Useful for being sure that state is restored to -// variables upon exit from scope. - - -#include -#ifndef BOOST_NO_EXCEPTIONS - #include -#endif - -#include -#include -#include -#include - -#include -#include - -namespace boost { -namespace serialization { - -template -// T requirements: -// - POD or object semantic (cannot be reference, function, ...) -// - copy constructor -// - operator = (no-throw one preferred) -class state_saver : private boost::noncopyable -{ -private: - const T previous_value; - T & previous_ref; - - struct restore { - static void invoke(T & previous_ref, const T & previous_value){ - previous_ref = previous_value; // won't throw - } - }; - - struct restore_with_exception { - static void invoke(T & previous_ref, const T & previous_value){ - BOOST_TRY{ - previous_ref = previous_value; - } - BOOST_CATCH(::std::exception &) { - // we must ignore it - we are in destructor - } - BOOST_CATCH_END - } - }; - -public: - state_saver( - T & object - ) : - previous_value(object), - previous_ref(object) - {} - - ~state_saver() { - #ifndef BOOST_NO_EXCEPTIONS - typedef typename mpl::eval_if< - has_nothrow_copy< T >, - mpl::identity, - mpl::identity - >::type typex; - typex::invoke(previous_ref, previous_value); - #else - previous_ref = previous_value; - #endif - } - -}; // state_saver<> - -} // serialization -} // boost - -#endif //BOOST_SERIALIZATION_STATE_SAVER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/static_warning.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/static_warning.hpp deleted file mode 100644 index 1d9238fc4d9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/static_warning.hpp +++ /dev/null @@ -1,103 +0,0 @@ -#ifndef BOOST_SERIALIZATION_STATIC_WARNING_HPP -#define BOOST_SERIALIZATION_STATIC_WARNING_HPP - -// (C) Copyright Robert Ramey 2003. Jonathan Turkanis 2004. -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org/libs/static_assert for documentation. - -/* - Revision history: - 15 June 2003 - Initial version. - 31 March 2004 - improved diagnostic messages and portability - (Jonathan Turkanis) - 03 April 2004 - works on VC6 at class and namespace scope - - ported to DigitalMars - - static warnings disabled by default; when enabled, - uses pragmas to enable required compiler warnings - on MSVC, Intel, Metrowerks and Borland 5.x. - (Jonathan Turkanis) - 30 May 2004 - tweaked for msvc 7.1 and gcc 3.3 - - static warnings ENabled by default; when enabled, - (Robert Ramey) -*/ - -#include - -// -// Implementation -// Makes use of the following warnings: -// 1. GCC prior to 3.3: division by zero. -// 2. BCC 6.0 preview: unreferenced local variable. -// 3. DigitalMars: returning address of local automatic variable. -// 4. VC6: class previously seen as struct (as in 'boost/mpl/print.hpp') -// 5. All others: deletion of pointer to incomplete type. -// -// The trick is to find code which produces warnings containing the name of -// a structure or variable. Details, with same numbering as above: -// 1. static_warning_impl::value is zero iff B is false, so diving an int -// by this value generates a warning iff B is false. -// 2. static_warning_impl::type has a constructor iff B is true, so an -// unreferenced variable of this type generates a warning iff B is false. -// 3. static_warning_impl::type overloads operator& to return a dynamically -// allocated int pointer only is B is true, so returning the address of an -// automatic variable of this type generates a warning iff B is fasle. -// 4. static_warning_impl::STATIC_WARNING is decalred as a struct iff B is -// false. -// 5. static_warning_impl::type is incomplete iff B is false, so deleting a -// pointer to this type generates a warning iff B is false. -// - -//------------------Enable selected warnings----------------------------------// - -// Enable the warnings relied on by BOOST_STATIC_WARNING, where possible. - -// 6. replaced implementation with one which depends solely on -// mpl::print<>. The previous one was found to fail for functions -// under recent versions of gcc and intel compilers - Robert Ramey - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template -struct BOOST_SERIALIZATION_STATIC_WARNING_LINE{}; - -template -struct static_warning_test{ - typename boost::mpl::eval_if_c< - B, - boost::mpl::true_, - typename boost::mpl::identity< - boost::mpl::print< - BOOST_SERIALIZATION_STATIC_WARNING_LINE - > - > - >::type type; -}; - -template -struct BOOST_SERIALIZATION_SS {}; - -} // serialization -} // boost - -#define BOOST_SERIALIZATION_BSW(B, L) \ - typedef boost::serialization::BOOST_SERIALIZATION_SS< \ - sizeof( boost::serialization::static_warning_test< B, L > ) \ - > BOOST_JOIN(STATIC_WARNING_LINE, L) BOOST_ATTRIBUTE_UNUSED; -#define BOOST_STATIC_WARNING(B) BOOST_SERIALIZATION_BSW(B, __LINE__) - -#endif // BOOST_SERIALIZATION_STATIC_WARNING_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/string.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/string.hpp deleted file mode 100644 index 76e695d4f3c..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/string.hpp +++ /dev/null @@ -1,30 +0,0 @@ -#ifndef BOOST_SERIALIZATION_STRING_HPP -#define BOOST_SERIALIZATION_STRING_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/string.hpp: -// serialization for stl string templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -BOOST_CLASS_IMPLEMENTATION(std::string, boost::serialization::primitive_type) -#ifndef BOOST_NO_STD_WSTRING -BOOST_CLASS_IMPLEMENTATION(std::wstring, boost::serialization::primitive_type) -#endif - -#endif // BOOST_SERIALIZATION_STRING_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/strong_typedef.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/strong_typedef.hpp deleted file mode 100644 index fdd1b24c9cb..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/strong_typedef.hpp +++ /dev/null @@ -1,50 +0,0 @@ -#ifndef BOOST_SERIALIZATION_STRONG_TYPEDEF_HPP -#define BOOST_SERIALIZATION_STRONG_TYPEDEF_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// strong_typedef.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2016 Ashish Sadanandan -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org/libs/serialization for updates, documentation, and revision history. - -// macro used to implement a strong typedef. strong typedef -// guarentees that two types are distinguised even though the -// share the same underlying implementation. typedef does not create -// a new type. BOOST_STRONG_TYPEDEF(T, D) creates a new type named D -// that operates as a type T. - -#include -#include -#include -#include -#include - -#define BOOST_STRONG_TYPEDEF(T, D) \ -struct D \ - : boost::totally_ordered1< D \ - , boost::totally_ordered2< D, T \ - > > \ -{ \ - T t; \ - explicit D(const T& t_) BOOST_NOEXCEPT_IF(boost::has_nothrow_copy_constructor::value) : t(t_) {} \ - D() BOOST_NOEXCEPT_IF(boost::has_nothrow_default_constructor::value) : t() {} \ - D(const D & t_) BOOST_NOEXCEPT_IF(boost::has_nothrow_copy_constructor::value) : t(t_.t) {} \ - D& operator=(const D& rhs) BOOST_NOEXCEPT_IF(boost::has_nothrow_assign::value) {t = rhs.t; return *this;} \ - D& operator=(const T& rhs) BOOST_NOEXCEPT_IF(boost::has_nothrow_assign::value) {t = rhs; return *this;} \ - operator const T&() const {return t;} \ - operator T&() {return t;} \ - bool operator==(const D& rhs) const {return t == rhs.t;} \ - bool operator<(const D& rhs) const {return t < rhs.t;} \ -}; - -#endif // BOOST_SERIALIZATION_STRONG_TYPEDEF_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/throw_exception.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/throw_exception.hpp deleted file mode 100644 index b67618adc92..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/throw_exception.hpp +++ /dev/null @@ -1,44 +0,0 @@ -#ifndef BOOST_SERIALIZATION_THROW_EXCEPTION_HPP_INCLUDED -#define BOOST_SERIALIZATION_THROW_EXCEPTION_HPP_INCLUDED - -// MS compatible compilers support #pragma once - -#if defined(_MSC_VER) -# pragma once -#endif - -// boost/throw_exception.hpp -// -// Copyright (c) 2002 Peter Dimov and Multi Media Ltd. -// -// Distributed under the Boost Software License, Version 1.0. (See -// accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include - -#ifndef BOOST_NO_EXCEPTIONS -#include -#endif - -namespace boost { -namespace serialization { - -#ifdef BOOST_NO_EXCEPTIONS - -inline void throw_exception(std::exception const & e) { - ::boost::throw_exception(e); -} - -#else - -template inline void throw_exception(E const & e){ - throw e; -} - -#endif - -} // namespace serialization -} // namespace boost - -#endif // #ifndef BOOST_SERIALIZATION_THROW_EXCEPTION_HPP_INCLUDED diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/tracking.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/tracking.hpp deleted file mode 100644 index d5c79b8409d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/tracking.hpp +++ /dev/null @@ -1,118 +0,0 @@ -#ifndef BOOST_SERIALIZATION_TRACKING_HPP -#define BOOST_SERIALIZATION_TRACKING_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// tracking.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -struct basic_traits; - -// default tracking level -template -struct tracking_level_impl { - template - struct traits_class_tracking { - typedef typename U::tracking type; - }; - typedef mpl::integral_c_tag tag; - // note: at least one compiler complained w/o the full qualification - // on basic traits below - typedef - typename mpl::eval_if< - is_base_and_derived, - traits_class_tracking< T >, - //else - typename mpl::eval_if< - is_pointer< T >, - // pointers are not tracked by default - mpl::int_, - //else - typename mpl::eval_if< - // for primitives - typename mpl::equal_to< - implementation_level< T >, - mpl::int_ - >, - // is never - mpl::int_, - // otherwise its selective - mpl::int_ - > > >::type type; - BOOST_STATIC_CONSTANT(int, value = type::value); -}; - -template -struct tracking_level : - public tracking_level_impl -{ -}; - -template -inline bool operator>=(tracking_level< T > t, enum tracking_type l) -{ - return t.value >= (int)l; -} - -} // namespace serialization -} // namespace boost - - -// The STATIC_ASSERT is prevents one from setting tracking for a primitive type. -// This almost HAS to be an error. Doing this will effect serialization of all -// char's in your program which is almost certainly what you don't want to do. -// If you want to track all instances of a given primitive type, You'll have to -// wrap it in your own type so its not a primitive anymore. Then it will compile -// without problem. -#define BOOST_CLASS_TRACKING(T, E) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct tracking_level< T > \ -{ \ - typedef mpl::integral_c_tag tag; \ - typedef mpl::int_< E> type; \ - BOOST_STATIC_CONSTANT( \ - int, \ - value = tracking_level::type::value \ - ); \ - /* tracking for a class */ \ - BOOST_STATIC_ASSERT(( \ - mpl::greater< \ - /* that is a prmitive */ \ - implementation_level< T >, \ - mpl::int_ \ - >::value \ - )); \ -}; \ -}} - -#endif // BOOST_SERIALIZATION_TRACKING_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/tracking_enum.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/tracking_enum.hpp deleted file mode 100644 index 278051e1baf..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/tracking_enum.hpp +++ /dev/null @@ -1,41 +0,0 @@ -#ifndef BOOST_SERIALIZATION_TRACKING_ENUM_HPP -#define BOOST_SERIALIZATION_TRACKING_ENUM_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// tracking_enum.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -namespace boost { -namespace serialization { - -// addresses of serialized objects may be tracked to avoid saving/loading -// redundant copies. This header defines a class trait that can be used -// to specify when objects should be tracked - -// names for each tracking level -enum tracking_type -{ - // never track this type - track_never = 0, - // track objects of this type if the object is serialized through a - // pointer. - track_selectively = 1, - // always track this type - track_always = 2 -}; - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_TRACKING_ENUM_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/traits.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/traits.hpp deleted file mode 100644 index 9e114fdd3df..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/traits.hpp +++ /dev/null @@ -1,65 +0,0 @@ -#ifndef BOOST_SERIALIZATION_TRAITS_HPP -#define BOOST_SERIALIZATION_TRAITS_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// traits.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// This header is used to apply serialization traits to templates. The -// standard system can't be used for platforms which don't support -// Partial Templlate Specialization. - -// The motivation for this is the Name-Value Pair (NVP) template. -// it has to work the same on all platforms in order for archives -// to be portable accross platforms. - -#include -#include - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -// common base class used to detect appended traits class -struct basic_traits {}; - -template -struct extended_type_info_impl; - -template< - class T, - int Level, - int Tracking, - unsigned int Version = 0, - class ETII = extended_type_info_impl< T >, - class Wrapper = mpl::false_ -> -struct traits : public basic_traits { - BOOST_STATIC_ASSERT(Version == 0 || Level >= object_class_info); - BOOST_STATIC_ASSERT(Tracking == track_never || Level >= object_serializable); - typedef typename mpl::int_ level; - typedef typename mpl::int_ tracking; - typedef typename mpl::int_ version; - typedef ETII type_info_implementation; - typedef Wrapper is_wrapper; -}; - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_TRAITS_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/type_info_implementation.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/type_info_implementation.hpp deleted file mode 100644 index 24637a8dbb3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/type_info_implementation.hpp +++ /dev/null @@ -1,73 +0,0 @@ -#ifndef BOOST_SERIALIZATION_TYPE_INFO_IMPLEMENTATION_HPP -#define BOOST_SERIALIZATION_TYPE_INFO_IMPLEMENTATION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// type_info_implementation.hpp: interface for portable version of type_info - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - - -#include -#include - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -// note that T and const T are folded into const T so that -// there is only one table entry per type -template -struct type_info_implementation { - template - struct traits_class_typeinfo_implementation { - typedef typename U::type_info_implementation::type type; - }; - // note: at least one compiler complained w/o the full qualification - // on basic traits below - typedef - typename mpl::eval_if< - is_base_and_derived, - traits_class_typeinfo_implementation< T >, - //else - mpl::identity< - typename extended_type_info_impl< T >::type - > - >::type type; -}; - -} // namespace serialization -} // namespace boost - -// define a macro to assign a particular derivation of extended_type_info -// to a specified a class. -#define BOOST_CLASS_TYPE_INFO(T, ETI) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct type_info_implementation< T > { \ - typedef ETI type; \ -}; \ -template<> \ -struct type_info_implementation< const T > { \ - typedef ETI type; \ -}; \ -} \ -} \ -/**/ - -#endif /// BOOST_SERIALIZATION_TYPE_INFO_IMPLEMENTATION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unique_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unique_ptr.hpp deleted file mode 100644 index 8d8703ef4f7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/unique_ptr.hpp +++ /dev/null @@ -1,68 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UNIQUE_PTR_HPP -#define BOOST_SERIALIZATION_UNIQUE_PTR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// unique_ptr.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. -#include -#include -#include - -namespace boost { -namespace serialization { - -///////////////////////////////////////////////////////////// -// implement serialization for unique_ptr< T > -// note: this must be added to the boost namespace in order to -// be called by the library -template -inline void save( - Archive & ar, - const std::unique_ptr< T > &t, - const unsigned int /*file_version*/ -){ - // only the raw pointer has to be saved - // the ref count is rebuilt automatically on load - const T * const tx = t.get(); - ar << BOOST_SERIALIZATION_NVP(tx); -} - -template -inline void load( - Archive & ar, - std::unique_ptr< T > &t, - const unsigned int /*file_version*/ -){ - T *tx; - ar >> BOOST_SERIALIZATION_NVP(tx); - // note that the reset automagically maintains the reference count - t.reset(tx); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::unique_ptr< T > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - - -#endif // BOOST_SERIALIZATION_UNIQUE_PTR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_load_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_load_imp.hpp deleted file mode 100644 index d56a423d180..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_load_imp.hpp +++ /dev/null @@ -1,73 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_LOAD_IMP_HPP -#define BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_LOAD_IMP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -# pragma warning (disable : 4786) // too long name, harmless warning -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// unordered_collections_load_imp.hpp: serialization for loading stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// helper function templates for serialization of collections - -#include -#include // size_t -#include // msvc 6.0 needs this for warning suppression -#if defined(BOOST_NO_STDC_NAMESPACE) -namespace std{ - using ::size_t; -} // namespace std -#endif -#include - -#include -#include -#include -#include -#include - -namespace boost{ -namespace serialization { -namespace stl { - -////////////////////////////////////////////////////////////////////// -// implementation of serialization for STL containers -// -template -inline void load_unordered_collection(Archive & ar, Container &s) -{ - collection_size_type count; - collection_size_type bucket_count; - boost::serialization::item_version_type item_version(0); - boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - ar >> BOOST_SERIALIZATION_NVP(count); - ar >> BOOST_SERIALIZATION_NVP(bucket_count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - s.clear(); - s.rehash(bucket_count); - InputFunction ifunc; - while(count-- > 0){ - ifunc(ar, s, item_version); - } -} - -} // namespace stl -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_LOAD_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_save_imp.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_save_imp.hpp deleted file mode 100644 index 56746ebeaa3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_collections_save_imp.hpp +++ /dev/null @@ -1,86 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_SAVE_IMP_HPP -#define BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_SAVE_IMP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// hash_collections_save_imp.hpp: serialization for stl collections - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -// helper function templates for serialization of collections - -#include -#include -#include -#include -#include -#include - -namespace boost{ -namespace serialization { -namespace stl { - -////////////////////////////////////////////////////////////////////// -// implementation of serialization for STL containers -// - -template -inline void save_unordered_collection(Archive & ar, const Container &s) -{ - collection_size_type count(s.size()); - const collection_size_type bucket_count(s.bucket_count()); - const item_version_type item_version( - version::value - ); - - #if 0 - /* should only be necessary to create archives of previous versions - * which is not currently supported. So for now comment this out - */ - boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - ar << BOOST_SERIALIZATION_NVP(count); - ar << BOOST_SERIALIZATION_NVP(bucket_count); - if(boost::archive::library_version_type(3) < library_version){ - // record number of elements - // make sure the target type is registered so we can retrieve - // the version when we load - ar << BOOST_SERIALIZATION_NVP(item_version); - } - #else - ar << BOOST_SERIALIZATION_NVP(count); - ar << BOOST_SERIALIZATION_NVP(bucket_count); - ar << BOOST_SERIALIZATION_NVP(item_version); - #endif - - typename Container::const_iterator it = s.begin(); - while(count-- > 0){ - // note borland emits a no-op without the explicit namespace - boost::serialization::save_construct_data_adl( - ar, - &(*it), - boost::serialization::version< - typename Container::value_type - >::value - ); - ar << boost::serialization::make_nvp("item", *it++); - } -} - -} // namespace stl -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_UNORDERED_COLLECTIONS_SAVE_IMP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_map.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_map.hpp deleted file mode 100644 index 4fdbddd7b65..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_map.hpp +++ /dev/null @@ -1,160 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UNORDERED_MAP_HPP -#define BOOST_SERIALIZATION_UNORDERED_MAP_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/unordered_map.hpp: -// serialization for stl unordered_map templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const std::unordered_map &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - std::unordered_map - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - std::unordered_map &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - std::unordered_map, - boost::serialization::stl::archive_input_unordered_map< - Archive, - std::unordered_map - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - std::unordered_map &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -// unordered_multimap -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const std::unordered_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - std::unordered_multimap - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - std::unordered_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - std::unordered_multimap< - Key, HashFcn, EqualKey, Allocator - >, - boost::serialization::stl::archive_input_unordered_multimap< - Archive, - std::unordered_multimap - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - std::unordered_multimap< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_UNORDERED_MAP_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_set.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/unordered_set.hpp deleted file mode 100644 index adfee609cbe..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/unordered_set.hpp +++ /dev/null @@ -1,162 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UNORDERED_SET_HPP -#define BOOST_SERIALIZATION_UNORDERED_SET_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) && (_MSC_VER >= 1020) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// unordered_set.hpp: serialization for stl unordered_set templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// (C) Copyright 2014 Jim Bell -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const std::unordered_set< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::save_unordered_collection< - Archive, - std::unordered_set - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - std::unordered_set< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - std::unordered_set, - stl::archive_input_unordered_set< - Archive, - std::unordered_set< - Key, HashFcn, EqualKey, Allocator - > - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - std::unordered_set< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int file_version -){ - split_free(ar, t, file_version); -} - -// unordered_multiset -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void save( - Archive & ar, - const std::unordered_multiset< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - stl::save_unordered_collection< - Archive, - std::unordered_multiset - >(ar, t); -} - -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void load( - Archive & ar, - std::unordered_multiset< - Key, HashFcn, EqualKey, Allocator - > &t, - const unsigned int /*file_version*/ -){ - boost::serialization::stl::load_unordered_collection< - Archive, - std::unordered_multiset, - boost::serialization::stl::archive_input_unordered_multiset< - Archive, - std::unordered_multiset - > - >(ar, t); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template< - class Archive, - class Key, - class HashFcn, - class EqualKey, - class Allocator -> -inline void serialize( - Archive & ar, - std::unordered_multiset &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_UNORDERED_SET_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/utility.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/utility.hpp deleted file mode 100644 index 4867a4a12d2..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/utility.hpp +++ /dev/null @@ -1,56 +0,0 @@ -#ifndef BOOST_SERIALIZATION_UTILITY_HPP -#define BOOST_SERIALIZATION_UTILITY_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// serialization/utility.hpp: -// serialization for stl utility templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include -#include - -namespace boost { -namespace serialization { - -// pair -template -inline void serialize( - Archive & ar, - std::pair & p, - const unsigned int /* file_version */ -){ - // note: we remove any const-ness on the first argument. The reason is that - // for stl maps, the type saved is pair::type typef; - ar & boost::serialization::make_nvp("first", const_cast(p.first)); - ar & boost::serialization::make_nvp("second", p.second); -} - -/// specialization of is_bitwise_serializable for pairs -template -struct is_bitwise_serializable > - : public mpl::and_,is_bitwise_serializable > -{ -}; - -} // serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_UTILITY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/valarray.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/valarray.hpp deleted file mode 100644 index 9eece5c1737..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/valarray.hpp +++ /dev/null @@ -1,86 +0,0 @@ -#ifndef BOOST_SERIALIZATION_VALARAY_HPP -#define BOOST_SERIALIZATION_VALARAY_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// valarray.hpp: serialization for stl vector templates - -// (C) Copyright 2005 Matthias Troyer . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -#include -#include -#include -#include -#include - -// function specializations must be defined in the appropriate -// namespace - boost::serialization -#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) -#define STD _STLP_STD -#else -#define STD std -#endif - -namespace boost { -namespace serialization { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// valarray< T > - -template -void save( Archive & ar, const STD::valarray &t, const unsigned int /*file_version*/ ) -{ - const collection_size_type count(t.size()); - ar << BOOST_SERIALIZATION_NVP(count); - if (t.size()){ - // explict template arguments to pass intel C++ compiler - ar << serialization::make_array( - static_cast(&t[0]), - count - ); - } -} - -template -void load( Archive & ar, STD::valarray &t, const unsigned int /*file_version*/ ) -{ - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - t.resize(count); - if (t.size()){ - // explict template arguments to pass intel C++ compiler - ar >> serialization::make_array( - static_cast(&t[0]), - count - ); - } -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( Archive & ar, STD::valarray & t, const unsigned int file_version) -{ - boost::serialization::split_free(ar, t, file_version); -} - -} } // end namespace boost::serialization - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(STD::valarray) -#undef STD - -#endif // BOOST_SERIALIZATION_VALARAY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/variant.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/variant.hpp deleted file mode 100644 index dce6f3d49e7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/variant.hpp +++ /dev/null @@ -1,158 +0,0 @@ -#ifndef BOOST_SERIALIZATION_VARIANT_HPP -#define BOOST_SERIALIZATION_VARIANT_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// variant.hpp - non-intrusive serialization of variant types -// -// copyright (c) 2005 -// troy d. straszheim -// http://www.resophonic.com -// -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// -// See http://www.boost.org for updates, documentation, and revision history. -// -// thanks to Robert Ramey, Peter Dimov, and Richard Crossley. -// - -#include -#include -#include -#include -#include -#include - -#include - -#include - -#include - -#include -#include -#include - -namespace boost { -namespace serialization { - -template -struct variant_save_visitor : - boost::static_visitor<> -{ - variant_save_visitor(Archive& ar) : - m_ar(ar) - {} - template - void operator()(T const & value) const - { - m_ar << BOOST_SERIALIZATION_NVP(value); - } -private: - Archive & m_ar; -}; - -template -void save( - Archive & ar, - boost::variant const & v, - unsigned int /*version*/ -){ - int which = v.which(); - ar << BOOST_SERIALIZATION_NVP(which); - variant_save_visitor visitor(ar); - v.apply_visitor(visitor); -} - -template -struct variant_impl { - - struct load_null { - template - static void invoke( - Archive & /*ar*/, - int /*which*/, - V & /*v*/, - const unsigned int /*version*/ - ){} - }; - - struct load_impl { - template - static void invoke( - Archive & ar, - int which, - V & v, - const unsigned int version - ){ - if(which == 0){ - // note: A non-intrusive implementation (such as this one) - // necessary has to copy the value. This wouldn't be necessary - // with an implementation that de-serialized to the address of the - // aligned storage included in the variant. - typedef typename mpl::front::type head_type; - head_type value; - ar >> BOOST_SERIALIZATION_NVP(value); - v = value; - ar.reset_object_address(& boost::get(v), & value); - return; - } - typedef typename mpl::pop_front::type type; - variant_impl::load(ar, which - 1, v, version); - } - }; - - template - static void load( - Archive & ar, - int which, - V & v, - const unsigned int version - ){ - typedef typename mpl::eval_if, - mpl::identity, - mpl::identity - >::type typex; - typex::invoke(ar, which, v, version); - } - -}; - -template -void load( - Archive & ar, - boost::variant& v, - const unsigned int version -){ - int which; - typedef typename boost::variant::types types; - ar >> BOOST_SERIALIZATION_NVP(which); - if(which >= mpl::size::value) - // this might happen if a type was removed from the list of variant types - boost::serialization::throw_exception( - boost::archive::archive_exception( - boost::archive::archive_exception::unsupported_version - ) - ); - variant_impl::load(ar, which, v, version); -} - -template -inline void serialize( - Archive & ar, - boost::variant & v, - const unsigned int file_version -){ - split_free(ar,v,file_version); -} - -} // namespace serialization -} // namespace boost - -#endif //BOOST_SERIALIZATION_VARIANT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/vector.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/vector.hpp deleted file mode 100644 index 9a114c00e20..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/vector.hpp +++ /dev/null @@ -1,233 +0,0 @@ -#ifndef BOOST_SERIALIZATION_VECTOR_HPP -#define BOOST_SERIALIZATION_VECTOR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// vector.hpp: serialization for stl vector templates - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// fast array serialization (C) Copyright 2005 Matthias Troyer -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include - -#include -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - -// default is being compatible with version 1.34.1 files, not 1.35 files -#ifndef BOOST_SERIALIZATION_VECTOR_VERSIONED -#define BOOST_SERIALIZATION_VECTOR_VERSIONED(V) (V==4 || V==5) -#endif - -// function specializations must be defined in the appropriate -// namespace - boost::serialization -#if defined(__SGI_STL_PORT) || defined(_STLPORT_VERSION) -#define STD _STLP_STD -#else -#define STD std -#endif - -namespace boost { -namespace serialization { - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// vector< T > - -// the default versions - -template -inline void save( - Archive & ar, - const std::vector &t, - const unsigned int /* file_version */, - mpl::false_ -){ - boost::serialization::stl::save_collection >( - ar, t - ); -} - -template -inline void load( - Archive & ar, - std::vector &t, - const unsigned int /* file_version */, - mpl::false_ -){ - const boost::archive::library_version_type library_version( - ar.get_library_version() - ); - // retrieve number of elements - item_version_type item_version(0); - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - if(boost::archive::library_version_type(3) < library_version){ - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - t.reserve(count); - stl::collection_load_impl(ar, t, count, item_version); -} - -// the optimized versions - -template -inline void save( - Archive & ar, - const std::vector &t, - const unsigned int /* file_version */, - mpl::true_ -){ - const collection_size_type count(t.size()); - ar << BOOST_SERIALIZATION_NVP(count); - if (!t.empty()) - // explict template arguments to pass intel C++ compiler - ar << serialization::make_array( - static_cast(&t[0]), - count - ); -} - -template -inline void load( - Archive & ar, - std::vector &t, - const unsigned int /* file_version */, - mpl::true_ -){ - collection_size_type count(t.size()); - ar >> BOOST_SERIALIZATION_NVP(count); - t.resize(count); - unsigned int item_version=0; - if(BOOST_SERIALIZATION_VECTOR_VERSIONED(ar.get_library_version())) { - ar >> BOOST_SERIALIZATION_NVP(item_version); - } - if (!t.empty()) - // explict template arguments to pass intel C++ compiler - ar >> serialization::make_array( - static_cast(&t[0]), - count - ); - } - -// dispatch to either default or optimized versions - -template -inline void save( - Archive & ar, - const std::vector &t, - const unsigned int file_version -){ - typedef typename - boost::serialization::use_array_optimization::template apply< - typename remove_const::type - >::type use_optimized; - save(ar,t,file_version, use_optimized()); -} - -template -inline void load( - Archive & ar, - std::vector &t, - const unsigned int file_version -){ -#ifdef BOOST_SERIALIZATION_VECTOR_135_HPP - if (ar.get_library_version()==boost::archive::library_version_type(5)) - { - load(ar,t,file_version, boost::is_arithmetic()); - return; - } -#endif - typedef typename - boost::serialization::use_array_optimization::template apply< - typename remove_const::type - >::type use_optimized; - load(ar,t,file_version, use_optimized()); -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::vector & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// vector -template -inline void save( - Archive & ar, - const std::vector &t, - const unsigned int /* file_version */ -){ - // record number of elements - collection_size_type count (t.size()); - ar << BOOST_SERIALIZATION_NVP(count); - std::vector::const_iterator it = t.begin(); - while(count-- > 0){ - bool tb = *it++; - ar << boost::serialization::make_nvp("item", tb); - } -} - -template -inline void load( - Archive & ar, - std::vector &t, - const unsigned int /* file_version */ -){ - // retrieve number of elements - collection_size_type count; - ar >> BOOST_SERIALIZATION_NVP(count); - t.resize(count); - for(collection_size_type i = collection_size_type(0); i < count; ++i){ - bool b; - ar >> boost::serialization::make_nvp("item", b); - t[i] = b; - } -} - -// split non-intrusive serialization function member into separate -// non intrusive save/load member functions -template -inline void serialize( - Archive & ar, - std::vector & t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // serialization -} // namespace boost - -#include - -BOOST_SERIALIZATION_COLLECTION_TRAITS(std::vector) -#undef STD - -#endif // BOOST_SERIALIZATION_VECTOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/vector_135.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/vector_135.hpp deleted file mode 100644 index fd1a7393d1b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/vector_135.hpp +++ /dev/null @@ -1,26 +0,0 @@ -////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// vector_135.hpp: serialization for stl vector templates for compatibility -// with release 1.35, which had a bug - -// (C) Copyright 2008 Matthias Troyer -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - - -#ifndef BOOST_SERIALIZATION_VECTOR_135_HPP -#define BOOST_SERIALIZATION_VECTOR_135_HPP - -#ifdef BOOST_SERIALIZATION_VECTOR_VERSIONED -#if BOOST_SERIALIZATION_VECTOR_VERSION != 4 -#error "Boost.Serialization cannot be compatible with both 1.35 and 1.36-1.40 files" -#endif -#else -#define BOOST_SERIALIZATION_VECTOR_VERSIONED(V) (V>4) -#endif - -#include - -#endif // BOOST_SERIALIZATION_VECTOR_135_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/version.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/version.hpp deleted file mode 100644 index 21a74d73daa..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/version.hpp +++ /dev/null @@ -1,107 +0,0 @@ -#ifndef BOOST_SERIALIZATION_VERSION_HPP -#define BOOST_SERIALIZATION_VERSION_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// version.hpp: - -// (C) Copyright 2002 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace serialization { - -struct basic_traits; - -// default version number is 0. Override with higher version -// when class definition changes. -template -struct version -{ - template - struct traits_class_version { - typedef typename U::version type; - }; - - typedef mpl::integral_c_tag tag; - // note: at least one compiler complained w/o the full qualification - // on basic traits below - typedef - typename mpl::eval_if< - is_base_and_derived, - traits_class_version< T >, - mpl::int_<0> - >::type type; - BOOST_STATIC_CONSTANT(int, value = version::type::value); -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -template -const int version::value; -#endif - -} // namespace serialization -} // namespace boost - -/* note: at first it seemed that this would be a good place to trap - * as an error an attempt to set a version # for a class which doesn't - * save its class information (including version #) in the archive. - * However, this imposes a requirement that the version be set after - * the implemention level which would be pretty confusing. If this - * is to be done, do this check in the input or output operators when - * ALL the serialization traits are available. Included the implementation - * here with this comment as a reminder not to do this! - */ -//#include -//#include - -#include -#include - -// specify the current version number for the class -// version numbers limited to 8 bits !!! -#define BOOST_CLASS_VERSION(T, N) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct version \ -{ \ - typedef mpl::int_ type; \ - typedef mpl::integral_c_tag tag; \ - BOOST_STATIC_CONSTANT(int, value = version::type::value); \ - BOOST_MPL_ASSERT(( \ - boost::mpl::less< \ - boost::mpl::int_, \ - boost::mpl::int_<256> \ - > \ - )); \ - /* \ - BOOST_MPL_ASSERT(( \ - mpl::equal_to< \ - :implementation_level, \ - mpl::int_ \ - >::value \ - )); \ - */ \ -}; \ -} \ -} - -#endif // BOOST_SERIALIZATION_VERSION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/void_cast.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/void_cast.hpp deleted file mode 100644 index f1b38286115..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/void_cast.hpp +++ /dev/null @@ -1,298 +0,0 @@ -#ifndef BOOST_SERIALIZATION_VOID_CAST_HPP -#define BOOST_SERIALIZATION_VOID_CAST_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// void_cast.hpp: interface for run-time casting of void pointers. - -// (C) Copyright 2002-2009 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// gennadiy.rozental@tfn.com - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // for ptrdiff_t -#include -#include - -#include -#include -#include -#include -#include -#include -#include - -#include -#include // must be the last header - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4251 4231 4660 4275) -#endif - -namespace boost { -namespace serialization { - -class extended_type_info; - -// Given a void *, assume that it really points to an instance of one type -// and alter it so that it would point to an instance of a related type. -// Return the altered pointer. If there exists no sequence of casts that -// can transform from_type to to_type, return a NULL. - -BOOST_SERIALIZATION_DECL void const * -void_upcast( - extended_type_info const & derived, - extended_type_info const & base, - void const * const t -); - -inline void * -void_upcast( - extended_type_info const & derived, - extended_type_info const & base, - void * const t -){ - return const_cast(void_upcast( - derived, - base, - const_cast(t) - )); -} - -BOOST_SERIALIZATION_DECL void const * -void_downcast( - extended_type_info const & derived, - extended_type_info const & base, - void const * const t -); - -inline void * -void_downcast( - extended_type_info const & derived, - extended_type_info const & base, - void * const t -){ - return const_cast(void_downcast( - derived, - base, - const_cast(t) - )); -} - -namespace void_cast_detail { - -class BOOST_SYMBOL_VISIBLE void_caster : - private boost::noncopyable -{ - friend - BOOST_SERIALIZATION_DECL void const * - boost::serialization::void_upcast( - extended_type_info const & derived, - extended_type_info const & base, - void const * const - ); - friend - BOOST_SERIALIZATION_DECL void const * - boost::serialization::void_downcast( - extended_type_info const & derived, - extended_type_info const & base, - void const * const - ); -protected: - BOOST_SERIALIZATION_DECL void recursive_register(bool includes_virtual_base = false) const; - BOOST_SERIALIZATION_DECL void recursive_unregister() const; - virtual bool has_virtual_base() const = 0; -public: - // Data members - const extended_type_info * m_derived; - const extended_type_info * m_base; - /*const*/ std::ptrdiff_t m_difference; - void_caster const * const m_parent; - - // note that void_casters are keyed on value of - // member extended type info records - NOT their - // addresses. This is necessary in order for the - // void cast operations to work across dll and exe - // module boundries. - bool operator<(const void_caster & rhs) const; - - const void_caster & operator*(){ - return *this; - } - // each derived class must re-implement these; - virtual void const * upcast(void const * const t) const = 0; - virtual void const * downcast(void const * const t) const = 0; - // Constructor - void_caster( - extended_type_info const * derived, - extended_type_info const * base, - std::ptrdiff_t difference = 0, - void_caster const * const parent = 0 - ) : - m_derived(derived), - m_base(base), - m_difference(difference), - m_parent(parent) - {} - virtual ~void_caster(){} -}; - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable : 4251 4231 4660 4275 4511 4512) -#endif - -template -class BOOST_SYMBOL_VISIBLE void_caster_primitive : - public void_caster -{ - virtual void const * downcast(void const * const t) const { - const Derived * d = - boost::serialization::smart_cast( - static_cast(t) - ); - return d; - } - virtual void const * upcast(void const * const t) const { - const Base * b = - boost::serialization::smart_cast( - static_cast(t) - ); - return b; - } - virtual bool has_virtual_base() const { - return false; - } -public: - void_caster_primitive(); - virtual ~void_caster_primitive(); -}; - -template -void_caster_primitive::void_caster_primitive() : - void_caster( - & type_info_implementation::type::get_const_instance(), - & type_info_implementation::type::get_const_instance(), - // note:I wanted to displace from 0 here, but at least one compiler - // treated 0 by not shifting it at all. - reinterpret_cast( - static_cast( - reinterpret_cast(8) - ) - ) - 8 - ) -{ - recursive_register(); -} - -template -void_caster_primitive::~void_caster_primitive(){ - recursive_unregister(); -} - -template -class BOOST_SYMBOL_VISIBLE void_caster_virtual_base : - public void_caster -{ - virtual bool has_virtual_base() const { - return true; - } -public: - virtual void const * downcast(void const * const t) const { - const Derived * d = - dynamic_cast( - static_cast(t) - ); - return d; - } - virtual void const * upcast(void const * const t) const { - const Base * b = - dynamic_cast( - static_cast(t) - ); - return b; - } - void_caster_virtual_base(); - virtual ~void_caster_virtual_base(); -}; - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -template -void_caster_virtual_base::void_caster_virtual_base() : - void_caster( - & (type_info_implementation::type::get_const_instance()), - & (type_info_implementation::type::get_const_instance()) - ) -{ - recursive_register(true); -} - -template -void_caster_virtual_base::~void_caster_virtual_base(){ - recursive_unregister(); -} - -template -struct BOOST_SYMBOL_VISIBLE void_caster_base : - public void_caster -{ - typedef - typename mpl::eval_if, - mpl::identity< - void_cast_detail::void_caster_virtual_base - > - ,// else - mpl::identity< - void_cast_detail::void_caster_primitive - > - >::type type; -}; - -} // void_cast_detail - -template -BOOST_DLLEXPORT -inline const void_cast_detail::void_caster & void_cast_register( - Derived const * /* dnull = NULL */, - Base const * /* bnull = NULL */ -){ - typedef - typename mpl::eval_if, - mpl::identity< - void_cast_detail::void_caster_virtual_base - > - ,// else - mpl::identity< - void_cast_detail::void_caster_primitive - > - >::type typex; - return singleton::get_const_instance(); -} - -template -class BOOST_SYMBOL_VISIBLE void_caster : - public void_cast_detail::void_caster_base::type -{ -}; - -} // namespace serialization -} // namespace boost - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -#include // pops abi_suffix.hpp pragmas - -#endif // BOOST_SERIALIZATION_VOID_CAST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/void_cast_fwd.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/void_cast_fwd.hpp deleted file mode 100644 index def61d52bb7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/void_cast_fwd.hpp +++ /dev/null @@ -1,37 +0,0 @@ -#ifndef BOOST_SERIALIZATION_VOID_CAST_FWD_HPP -#define BOOST_SERIALIZATION_VOID_CAST_FWD_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// void_cast_fwd.hpp: interface for run-time casting of void pointers. - -// (C) Copyright 2005 Robert Ramey - http://www.rrsd.com . -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) -// gennadiy.rozental@tfn.com - -// See http://www.boost.org for updates, documentation, and revision history. - -#include // NULL -#include - -namespace boost { -namespace serialization { -namespace void_cast_detail{ -class void_caster; -} // namespace void_cast_detail -template -BOOST_DLLEXPORT -inline const void_cast_detail::void_caster & void_cast_register( - const Derived * dnull = NULL, - const Base * bnull = NULL -) BOOST_USED; -} // namespace serialization -} // namespace boost - -#endif // BOOST_SERIALIZATION_VOID_CAST_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/weak_ptr.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/weak_ptr.hpp deleted file mode 100644 index 6952d24cb37..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/weak_ptr.hpp +++ /dev/null @@ -1,99 +0,0 @@ -#ifndef BOOST_SERIALIZATION_WEAK_PTR_HPP -#define BOOST_SERIALIZATION_WEAK_PTR_HPP - -// MS compatible compilers support #pragma once -#if defined(_MSC_VER) -# pragma once -#endif - -/////////1/////////2/////////3/////////4/////////5/////////6/////////7/////////8 -// weak_ptr.hpp: serialization for boost weak pointer - -// (C) Copyright 2004 Robert Ramey and Martin Ecker -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -// See http://www.boost.org for updates, documentation, and revision history. - -#include -#include - -namespace boost { -namespace serialization{ - -template -inline void save( - Archive & ar, - const boost::weak_ptr< T > &t, - const unsigned int /* file_version */ -){ - const boost::shared_ptr< T > sp = t.lock(); - ar << boost::serialization::make_nvp("weak_ptr", sp); -} - -template -inline void load( - Archive & ar, - boost::weak_ptr< T > &t, - const unsigned int /* file_version */ -){ - boost::shared_ptr< T > sp; - ar >> boost::serialization::make_nvp("weak_ptr", sp); - t = sp; -} - -template -inline void serialize( - Archive & ar, - boost::weak_ptr< T > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#ifndef BOOST_NO_CXX11_SMART_PTR -#include - -namespace boost { -namespace serialization{ - -template -inline void save( - Archive & ar, - const std::weak_ptr< T > &t, - const unsigned int /* file_version */ -){ - const std::shared_ptr< T > sp = t.lock(); - ar << boost::serialization::make_nvp("weak_ptr", sp); -} - -template -inline void load( - Archive & ar, - std::weak_ptr< T > &t, - const unsigned int /* file_version */ -){ - std::shared_ptr< T > sp; - ar >> boost::serialization::make_nvp("weak_ptr", sp); - t = sp; -} - -template -inline void serialize( - Archive & ar, - std::weak_ptr< T > &t, - const unsigned int file_version -){ - boost::serialization::split_free(ar, t, file_version); -} - -} // namespace serialization -} // namespace boost - -#endif // BOOST_NO_CXX11_SMART_PTR - -#endif // BOOST_SERIALIZATION_WEAK_PTR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/serialization/wrapper.hpp b/contrib/libboost/boost_1_65_0/boost/serialization/wrapper.hpp deleted file mode 100644 index 60d7910b17a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/serialization/wrapper.hpp +++ /dev/null @@ -1,60 +0,0 @@ -#ifndef BOOST_SERIALIZATION_WRAPPER_HPP -#define BOOST_SERIALIZATION_WRAPPER_HPP - -// (C) Copyright 2005-2006 Matthias Troyer -// Use, modification and distribution is subject to the Boost Software -// License, Version 1.0. (See accompanying file LICENSE_1_0.txt or copy at -// http://www.boost.org/LICENSE_1_0.txt) - -#include -#include -#include -#include - -namespace boost { namespace serialization { - -/// the base class for serialization wrappers -/// -/// wrappers need to be treated differently at various places in the serialization library, -/// e.g. saving of non-const wrappers has to be possible. Since partial specialization -// is not supported by all compilers, we derive all wrappers from wrapper_traits. - -template< - class T, - int Level = object_serializable, - int Tracking = track_never, - unsigned int Version = 0, - class ETII = extended_type_info_impl< T > -> -struct wrapper_traits : - public traits -{}; - -template -struct is_wrapper_impl : - boost::mpl::eval_if< - boost::is_base_and_derived, - boost::mpl::true_, - boost::mpl::false_ - >::type -{}; - -template -struct is_wrapper { - typedef typename is_wrapper_impl::type type; -}; - -} // serialization -} // boost - -// A macro to define that a class is a wrapper -#define BOOST_CLASS_IS_WRAPPER(T) \ -namespace boost { \ -namespace serialization { \ -template<> \ -struct is_wrapper_impl : boost::mpl::true_ {}; \ -} \ -} \ -/**/ - -#endif //BOOST_SERIALIZATION_WRAPPER_HPP diff --git a/contrib/poco b/contrib/poco new file mode 160000 index 00000000000..29439cf7fa3 --- /dev/null +++ b/contrib/poco @@ -0,0 +1 @@ +Subproject commit 29439cf7fa32c1a2d62d925bb6d6a3f14668a4a2 diff --git a/contrib/zlib-ng b/contrib/zlib-ng new file mode 160000 index 00000000000..9173b89d467 --- /dev/null +++ b/contrib/zlib-ng @@ -0,0 +1 @@ +Subproject commit 9173b89d46799582d20a30578e0aa9788bc7d6e1 diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index b7ad890b47c..07b95dbb91a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -2266,8 +2266,6 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart( { auto committed_parts_range = getDataPartsStateRange(state); - auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed); - /// The part can be covered only by the previous or the next one in data_parts. auto it = data_parts_by_state_and_info.lower_bound(DataPartStateAndInfo{state, part_info}); @@ -2600,7 +2598,6 @@ MergeTreeData::getDetachedParts() const part.prefix = dir_name.substr(0, first_separator); } - return res; } From 03d659ea1abd57f8c908f3a2d3c83332f3cdd519 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 13 Jun 2019 12:41:40 -0400 Subject: [PATCH 076/509] * Sync with yandex/master so that the git diff yandex/master is empty --- .gitmodules | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/.gitmodules b/.gitmodules index 336c2a892e2..0fda654f07c 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,3 +1,6 @@ +[submodule "contrib/poco"] + path = contrib/poco + url = https://github.com/ClickHouse-Extras/poco [submodule "contrib/zstd"] path = contrib/zstd url = https://github.com/facebook/zstd.git @@ -10,6 +13,12 @@ [submodule "contrib/cctz"] path = contrib/cctz url = https://github.com/google/cctz.git +[submodule "contrib/zlib-ng"] + path = contrib/zlib-ng + url = https://github.com/ClickHouse-Extras/zlib-ng.git +[submodule "contrib/googletest"] + path = contrib/googletest + url = https://github.com/google/googletest.git [submodule "contrib/capnproto"] path = contrib/capnproto url = https://github.com/capnproto/capnproto.git From 6cca6d359dcb410ee30e9d30cf52e92819e2c265 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 13 Jun 2019 21:02:15 -0400 Subject: [PATCH 077/509] * Reverting changes to clickhouse-test as it seems to cause some tests to fail * Updating client.py to use CLICKHOUSE_BINARY env variable instead of CLICKHOUSE_CLIENT --- dbms/tests/clickhouse-test | 2 +- dbms/tests/queries/0_stateless/helpers/client.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 4ca549889d6..e64b04f2db5 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -258,7 +258,7 @@ def main(args): # Keep same default values as in queries/shell_config.sh os.environ.setdefault("CLICKHOUSE_BINARY", args.binary) - os.environ.setdefault("CLICKHOUSE_CLIENT", args.client) + #os.environ.setdefault("CLICKHOUSE_CLIENT", args.client) os.environ.setdefault("CLICKHOUSE_CONFIG", args.configserver) if args.configclient: os.environ.setdefault("CLICKHOUSE_CONFIG_CLIENT", args.configclient) diff --git a/dbms/tests/queries/0_stateless/helpers/client.py b/dbms/tests/queries/0_stateless/helpers/client.py index 59ea3d898ea..dec7eca7d78 100644 --- a/dbms/tests/queries/0_stateless/helpers/client.py +++ b/dbms/tests/queries/0_stateless/helpers/client.py @@ -12,7 +12,7 @@ end_of_block = r'.*\r\n.*\r\n' def client(command=None, name='', log=None): if command is None: - client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT')) + client = uexpect.spawn(os.environ.get('CLICKHOUSE_BINARY', 'clickhouse') + '-client') else: client = uexpect.spawn(command) client.eol('\r') From 13978c03f382ce44e404ecfe830bb62490aca02c Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Fri, 14 Jun 2019 10:06:15 -0400 Subject: [PATCH 078/509] * Fixing a bug in StorageLiveView.cpp in the getHeader() method when header block would not match blocks in the stream when Const column is present. * Updating 00963_temporary_live_view_watch_live_timeout.py and removed timeout=5 after WATCH query is aborted with Ctrl-C so that the default timeout of 20sec is used * Small style fixes in tests --- dbms/src/Storages/StorageLiveView.cpp | 25 ++++++++++++++----- .../0_stateless/00958_live_view_watch_live.py | 2 +- .../00960_live_view_watch_events_live.py | 2 +- .../00962_temporary_live_view_watch_live.py | 2 +- ..._temporary_live_view_watch_live_timeout.py | 4 +-- .../00964_live_view_watch_events_heartbeat.py | 2 +- .../00965_live_view_watch_heartbeat.py | 2 +- 7 files changed, 26 insertions(+), 13 deletions(-) diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index 77e95ac2e48..917d62baaa4 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -159,12 +159,25 @@ Block StorageLiveView::getHeader() const { if (!sample_block) { - auto storage = global_context.getTable(select_database_name, select_table_name); - sample_block = InterpreterSelectQuery(inner_query, global_context, storage).getSampleBlock(); - sample_block.insert({DataTypeUInt64().createColumnConst( - sample_block.rows(), 0)->convertToFullColumnIfConst(), - std::make_shared(), - "_version"}); + if (!(*blocks_ptr) || (*blocks_ptr)->empty()) + { + auto storage = global_context.getTable(select_database_name, select_table_name); + sample_block = InterpreterSelectQuery(inner_query, global_context, storage, SelectQueryOptions(QueryProcessingStage::Complete)).getSampleBlock(); + sample_block.insert({DataTypeUInt64().createColumnConst( + sample_block.rows(), 0)->convertToFullColumnIfConst(), + std::make_shared(), + "_version"}); + /// convert all columns to full columns + /// in case some of them are constant + for (size_t i = 0; i < sample_block.columns(); ++i) + { + sample_block.safeGetByPosition(i).column = sample_block.safeGetByPosition(i).column->convertToFullColumnIfConst(); + } + } + else + { + sample_block = (*blocks_ptr)->front().cloneEmpty(); + } } return sample_block; diff --git a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py index cb6f1e95f7e..7cbea5fbff6 100755 --- a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py @@ -31,7 +31,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') client1.expect(r'21.*3' + end_of_block) # send Ctrl-C - os.kill(client1.process.pid,signal.SIGINT) + os.kill(client1.process.pid, signal.SIGINT) client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py index 414f9c1ad96..becf0335600 100755 --- a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py +++ b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py @@ -31,7 +31,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') client1.expect('3.*2186dbea325ee4c56b67e9b792e993a3' + end_of_block) # send Ctrl-C - os.kill(client1.process.pid,signal.SIGINT) + os.kill(client1.process.pid, signal.SIGINT) client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py index 1967284c38b..d4d35548d68 100755 --- a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py @@ -31,7 +31,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') client1.expect(r'21.*3' + end_of_block) # send Ctrl-C - os.kill(client1.process.pid,signal.SIGINT) + os.kill(client1.process.pid, signal.SIGINT) client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py index 581363cd796..2008368f12e 100755 --- a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py +++ b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py @@ -35,8 +35,8 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client2.expect(prompt) client1.expect(r'21.*3' + end_of_block) # send Ctrl-C - os.kill(client1.process.pid,signal.SIGINT) - client1.expect(prompt, timeout=5) + os.kill(client1.process.pid, signal.SIGINT) + client1.expect(prompt) client1.send('SELECT sleep(1)') client1.expect(prompt) client1.send('DROP TABLE test.lv') diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py index b624e0b7080..3ced5f6f315 100755 --- a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -33,7 +33,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo # wait for heartbeat client1.expect('Progress: 2.00 rows.*\)') # send Ctrl-C - os.kill(client1.process.pid,signal.SIGINT) + os.kill(client1.process.pid, signal.SIGINT) client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py index dfb46273f7c..b2bd84e0742 100755 --- a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py @@ -34,7 +34,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo # wait for heartbeat client1.expect('Progress: 2.00 rows.*\)') # send Ctrl-C - os.kill(client1.process.pid,signal.SIGINT) + os.kill(client1.process.pid, signal.SIGINT) client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) From d6d0404c48af9cf56441a8def682e9fc47b3349b Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Fri, 14 Jun 2019 16:27:43 -0400 Subject: [PATCH 079/509] * Fixing bug in StorageLiveView.cpp getHeader() method that sometimes addressed invalid pointer when getHeader() method was called without holding the mutex --- dbms/src/Storages/StorageLiveView.cpp | 28 +++++++++++---------------- 1 file changed, 11 insertions(+), 17 deletions(-) diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index 917d62baaa4..eecc7f6d1dc 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -159,24 +159,18 @@ Block StorageLiveView::getHeader() const { if (!sample_block) { - if (!(*blocks_ptr) || (*blocks_ptr)->empty()) + auto storage = global_context.getTable(select_database_name, select_table_name); + sample_block = InterpreterSelectQuery(inner_query, global_context, storage, + SelectQueryOptions(QueryProcessingStage::Complete)).getSampleBlock(); + sample_block.insert({DataTypeUInt64().createColumnConst( + sample_block.rows(), 0)->convertToFullColumnIfConst(), + std::make_shared(), + "_version"}); + /// convert all columns to full columns + /// in case some of them are constant + for (size_t i = 0; i < sample_block.columns(); ++i) { - auto storage = global_context.getTable(select_database_name, select_table_name); - sample_block = InterpreterSelectQuery(inner_query, global_context, storage, SelectQueryOptions(QueryProcessingStage::Complete)).getSampleBlock(); - sample_block.insert({DataTypeUInt64().createColumnConst( - sample_block.rows(), 0)->convertToFullColumnIfConst(), - std::make_shared(), - "_version"}); - /// convert all columns to full columns - /// in case some of them are constant - for (size_t i = 0; i < sample_block.columns(); ++i) - { - sample_block.safeGetByPosition(i).column = sample_block.safeGetByPosition(i).column->convertToFullColumnIfConst(); - } - } - else - { - sample_block = (*blocks_ptr)->front().cloneEmpty(); + sample_block.safeGetByPosition(i).column = sample_block.safeGetByPosition(i).column->convertToFullColumnIfConst(); } } From 282ff6bfda42c949b9d1a4921ad7f42435436526 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Fri, 14 Jun 2019 21:08:46 -0400 Subject: [PATCH 080/509] * Fixing a bug in writeIntoLiveView method where blocks->front() is called before checking if blocks vector is empty. --- dbms/src/Storages/StorageLiveView.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index 01dd82ff1fd..69410ec4e8b 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -202,7 +202,7 @@ public: } /// Need make new mergeable block structure match the other mergeable blocks - if (!mergeable_blocks->front()->empty() && !new_mergeable_blocks->empty()) + if (!mergeable_blocks->front()->empty()) { auto sample_block = mergeable_blocks->front()->front(); auto sample_new_block = new_mergeable_blocks->front(); @@ -237,6 +237,9 @@ public: blocks->push_back(this_block); } + if (blocks->empty()) + return; + auto sample_block = blocks->front().cloneEmpty(); BlockInputStreamPtr new_data = std::make_shared(std::make_shared(blocks), sample_block); From 103bf9d068eb047d2f1dc789401265b18fdbd13f Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sat, 15 Jun 2019 23:34:28 -0400 Subject: [PATCH 081/509] * Updating tests as SHOW TABLES 'lv' does not return any tables as test database is not selected by default. --- .../queries/0_stateless/00952_live_view_create.reference | 1 - dbms/tests/queries/0_stateless/00952_live_view_create.sql | 2 -- .../0_stateless/00954_live_view_select_version.reference | 1 - .../queries/0_stateless/00954_live_view_select_version.sql | 2 -- .../00955_live_view_select_with_aggregation.reference | 1 - .../0_stateless/00955_live_view_select_with_aggregation.sql | 2 -- .../0_stateless/00956_live_view_watch_events.reference | 1 - .../queries/0_stateless/00956_live_view_watch_events.sql | 2 -- .../tests/queries/0_stateless/00957_live_view_watch.reference | 1 - dbms/tests/queries/0_stateless/00957_live_view_watch.sql | 2 -- .../0_stateless/00959_create_temporary_live_view.reference | 1 - .../0_stateless/00961_temporary_live_view_watch.reference | 1 - .../queries/0_stateless/00961_temporary_live_view_watch.sql | 2 -- ..._live_view_select_format_jsoneachrowwithprogress.reference | 1 - .../00968_live_view_select_format_jsoneachrowwithprogress.sql | 2 -- ...9_live_view_watch_format_jsoneachrowwithprogress.reference | 1 - .../00969_live_view_watch_format_jsoneachrowwithprogress.sql | 2 -- .../queries/0_stateless/00972_live_view_select_1.reference | 1 - dbms/tests/queries/0_stateless/00972_live_view_select_1.sql | 4 +++- .../queries/0_stateless/00973_live_view_select.reference | 1 - dbms/tests/queries/0_stateless/00973_live_view_select.sql | 2 -- 21 files changed, 3 insertions(+), 30 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00952_live_view_create.reference b/dbms/tests/queries/0_stateless/00952_live_view_create.reference index c39f21eaf5e..e69de29bb2d 100644 --- a/dbms/tests/queries/0_stateless/00952_live_view_create.reference +++ b/dbms/tests/queries/0_stateless/00952_live_view_create.reference @@ -1 +0,0 @@ -lv diff --git a/dbms/tests/queries/0_stateless/00952_live_view_create.sql b/dbms/tests/queries/0_stateless/00952_live_view_create.sql index 0b64ed3a876..1c929b15b00 100644 --- a/dbms/tests/queries/0_stateless/00952_live_view_create.sql +++ b/dbms/tests/queries/0_stateless/00952_live_view_create.sql @@ -3,7 +3,5 @@ DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; -SHOW TABLES LIKE 'lv'; - DROP TABLE test.lv; DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00954_live_view_select_version.reference b/dbms/tests/queries/0_stateless/00954_live_view_select_version.reference index 07b5ce41af1..453bd800469 100644 --- a/dbms/tests/queries/0_stateless/00954_live_view_select_version.reference +++ b/dbms/tests/queries/0_stateless/00954_live_view_select_version.reference @@ -1,4 +1,3 @@ -lv 1 1 2 1 3 1 diff --git a/dbms/tests/queries/0_stateless/00954_live_view_select_version.sql b/dbms/tests/queries/0_stateless/00954_live_view_select_version.sql index 569cf5b0ab2..5f3ab1f7546 100644 --- a/dbms/tests/queries/0_stateless/00954_live_view_select_version.sql +++ b/dbms/tests/queries/0_stateless/00954_live_view_select_version.sql @@ -4,8 +4,6 @@ DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; -SHOW TABLES LIKE 'lv'; - INSERT INTO test.mt VALUES (1),(2),(3); SELECT *,_version FROM test.lv; diff --git a/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.reference b/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.reference index cb865431ffd..6d50f0e9c3a 100644 --- a/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.reference +++ b/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.reference @@ -1,3 +1,2 @@ -lv 6 21 diff --git a/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.sql b/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.sql index cdac382cae2..3c11f855c9d 100644 --- a/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.sql +++ b/dbms/tests/queries/0_stateless/00955_live_view_select_with_aggregation.sql @@ -4,8 +4,6 @@ DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; -SHOW TABLES LIKE 'lv'; - INSERT INTO test.mt VALUES (1),(2),(3); SELECT sum(a) FROM test.lv; diff --git a/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference b/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference index b67c49182e9..57e3d59ecad 100644 --- a/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference +++ b/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference @@ -1,4 +1,3 @@ -lv 1 c9d39b11cce79112219a73aaa319b475 2 4cd0592103888d4682de9a32a23602e3 3 2186dbea325ee4c56b67e9b792e993a3 diff --git a/dbms/tests/queries/0_stateless/00956_live_view_watch_events.sql b/dbms/tests/queries/0_stateless/00956_live_view_watch_events.sql index 1dbc201bdca..a3b84e8d4c1 100644 --- a/dbms/tests/queries/0_stateless/00956_live_view_watch_events.sql +++ b/dbms/tests/queries/0_stateless/00956_live_view_watch_events.sql @@ -4,8 +4,6 @@ DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; -SHOW TABLES LIKE 'lv'; - WATCH test.lv EVENTS LIMIT 0; INSERT INTO test.mt VALUES (1),(2),(3); diff --git a/dbms/tests/queries/0_stateless/00957_live_view_watch.reference b/dbms/tests/queries/0_stateless/00957_live_view_watch.reference index 65500578a69..6fbbedf1b21 100644 --- a/dbms/tests/queries/0_stateless/00957_live_view_watch.reference +++ b/dbms/tests/queries/0_stateless/00957_live_view_watch.reference @@ -1,4 +1,3 @@ -lv 0 1 6 2 21 3 diff --git a/dbms/tests/queries/0_stateless/00957_live_view_watch.sql b/dbms/tests/queries/0_stateless/00957_live_view_watch.sql index ac304fec0b9..abe4a6c32ae 100644 --- a/dbms/tests/queries/0_stateless/00957_live_view_watch.sql +++ b/dbms/tests/queries/0_stateless/00957_live_view_watch.sql @@ -4,8 +4,6 @@ DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; -SHOW TABLES LIKE 'lv'; - WATCH test.lv LIMIT 0; INSERT INTO test.mt VALUES (1),(2),(3); diff --git a/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.reference b/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.reference index 49d86fc2fbf..7f9fcbb2e9c 100644 --- a/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.reference +++ b/dbms/tests/queries/0_stateless/00959_create_temporary_live_view.reference @@ -1,4 +1,3 @@ temporary_live_view_timeout 5 live_view_heartbeat_interval 15 -lv 0 diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference index 65500578a69..6fbbedf1b21 100644 --- a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference +++ b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference @@ -1,4 +1,3 @@ -lv 0 1 6 2 21 3 diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql index eb99e054514..c3e2ab8d102 100644 --- a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql +++ b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql @@ -4,8 +4,6 @@ DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; -SHOW TABLES LIKE 'lv'; - WATCH test.lv LIMIT 0; INSERT INTO test.mt VALUES (1),(2),(3); diff --git a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference index 0f6a0405cda..5ae423d90d1 100644 --- a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference +++ b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference @@ -1,4 +1,3 @@ -lv {"row":{"a":1}} {"row":{"a":2}} {"row":{"a":3}} diff --git a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql index 748d901b3bf..8c6f4197d54 100644 --- a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql +++ b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql @@ -4,8 +4,6 @@ DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; -SHOW TABLES LIKE 'lv'; - INSERT INTO test.mt VALUES (1),(2),(3); SELECT * FROM test.lv FORMAT JSONEachRowWithProgress; diff --git a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference index 8510bebad77..287a1ced92d 100644 --- a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference +++ b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference @@ -1,4 +1,3 @@ -lv {"row":{"sum(a)":"0","_version":"1"}} {"progress":{"read_rows":"1","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} {"row":{"sum(a)":"6","_version":"2"}} diff --git a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql index 9f2e0384dd8..725a4ad00ed 100644 --- a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql +++ b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql @@ -4,8 +4,6 @@ DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; -SHOW TABLES LIKE 'lv'; - WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; INSERT INTO test.mt VALUES (1),(2),(3); diff --git a/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference b/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference index f2f8c69d020..d00491fd7e5 100644 --- a/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference +++ b/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference @@ -1,2 +1 @@ -lv 1 diff --git a/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql b/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql index 01bfeb08fd5..661080b577b 100644 --- a/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql +++ b/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql @@ -1,5 +1,7 @@ DROP TABLE IF EXISTS test.lv; + CREATE LIVE VIEW test.lv AS SELECT 1; -SHOW TABLES LIKE 'lv'; + SELECT * FROM test.lv; + DROP TABLE test.lv; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_select.reference b/dbms/tests/queries/0_stateless/00973_live_view_select.reference index 7beda18cd83..75236c0daf7 100644 --- a/dbms/tests/queries/0_stateless/00973_live_view_select.reference +++ b/dbms/tests/queries/0_stateless/00973_live_view_select.reference @@ -1,4 +1,3 @@ -lv 6 1 6 1 12 2 diff --git a/dbms/tests/queries/0_stateless/00973_live_view_select.sql b/dbms/tests/queries/0_stateless/00973_live_view_select.sql index 492fa265b7d..ff4a45ffcc1 100644 --- a/dbms/tests/queries/0_stateless/00973_live_view_select.sql +++ b/dbms/tests/queries/0_stateless/00973_live_view_select.sql @@ -4,8 +4,6 @@ DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; -SHOW TABLES LIKE 'lv'; - INSERT INTO test.mt VALUES (1),(2),(3); SELECT *,_version FROM test.lv; From ea83a4cafa7199f6473db6b7fd35407af54e9cd0 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sun, 16 Jun 2019 07:09:43 -0400 Subject: [PATCH 082/509] * Removing 'hash' column from WATCH [db.live_view] EVENTS query result as hash of the result are not usually provided in ClickHouse. This is also done to match the implementation of the SELECT query that only returns '_version' virtual column when using LIVE VIEW tables. --- dbms/src/DataStreams/LiveViewEventsBlockInputStream.h | 9 ++------- .../0_stateless/00956_live_view_watch_events.reference | 6 +++--- .../0_stateless/00960_live_view_watch_events_live.py | 4 ++-- .../00964_live_view_watch_events_heartbeat.py | 2 +- .../0_stateless/00966_live_view_watch_events_http.py | 4 ++-- .../00970_live_view_watch_events_http_heartbeat.py | 10 +++++----- 6 files changed, 15 insertions(+), 20 deletions(-) diff --git a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h index c7911074176..75e68b15d92 100644 --- a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h @@ -66,7 +66,7 @@ public: Block getHeader() const override { - return {ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "version"), ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "hash")}; + return {ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "version")}; } void refresh() @@ -106,12 +106,7 @@ public: ColumnWithTypeAndName( DataTypeUInt64().createColumnConst(1, blocks_metadata->version)->convertToFullColumnIfConst(), std::make_shared(), - "version"), - ColumnWithTypeAndName( - DataTypeString().createColumnConst(1, blocks_metadata->hash)->convertToFullColumnIfConst(), - std::make_shared(), - "hash"), - + "version") }; return res; } diff --git a/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference b/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference index 57e3d59ecad..01e79c32a8c 100644 --- a/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference +++ b/dbms/tests/queries/0_stateless/00956_live_view_watch_events.reference @@ -1,3 +1,3 @@ -1 c9d39b11cce79112219a73aaa319b475 -2 4cd0592103888d4682de9a32a23602e3 -3 2186dbea325ee4c56b67e9b792e993a3 +1 +2 +3 diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py index becf0335600..44b8df185b1 100755 --- a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py +++ b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py @@ -27,9 +27,9 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('WATCH test.lv EVENTS') client1.expect('1.*' + end_of_block) client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect('2.*4cd0592103888d4682de9a32a23602e3' + end_of_block) + client1.expect('2.*' + end_of_block) client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') - client1.expect('3.*2186dbea325ee4c56b67e9b792e993a3' + end_of_block) + client1.expect('3.*' + end_of_block) # send Ctrl-C os.kill(client1.process.pid, signal.SIGINT) client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py index 3ced5f6f315..cecae7c5a72 100755 --- a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -28,7 +28,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('WATCH test.lv EVENTS') client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect('2.*4cd0592103888d4682de9a32a23602e3' + end_of_block) + client1.expect('2.*' + end_of_block) client1.expect('Progress: 2.00 rows.*\)') # wait for heartbeat client1.expect('Progress: 2.00 rows.*\)') diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py index a1b6f2418ea..bb9d6152200 100755 --- a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py +++ b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py @@ -26,10 +26,10 @@ with client(name='client1>', log=log) as client1: with http_client({'method':'GET', 'url': '/?query=WATCH%20test.lv%20EVENTS'}, name='client2>', log=log) as client2: - client2.expect('.*1\tc9d39b11cce79112219a73aaa319b475\n') + client2.expect('.*1\n') client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') client1.expect(prompt) - client2.expect('.*2\t.*\n') + client2.expect('.*2\n') client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py index 29ea2142d5c..63628c4a76f 100755 --- a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py @@ -26,16 +26,16 @@ with client(name='client1>', log=log) as client1: with http_client({'method':'GET', 'url': '/?live_view_heartbeat_interval=1&query=WATCH%20test.lv%20EVENTS%20FORMAT%20JSONEachRowWithProgress'}, name='client2>', log=log) as client2: - client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}\n', escape=True) - client2.expect('{"row":{"version":"1","hash":"c9d39b11cce79112219a73aaa319b475"}}', escape=True) - client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) + client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}\n', escape=True) + client2.expect('{"row":{"version":"1"}', escape=True) + client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) # heartbeat is provided by progress message - client2.expect('{"progress":{"read_rows":"1","read_bytes":"49","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) + client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') client1.expect(prompt) - client2.expect('{"row":{"version":"2","hash":"4cd0592103888d4682de9a32a23602e3"}}\n', escape=True) + client2.expect('{"row":{"version":"2"}}\n', escape=True) client1.send('DROP TABLE test.lv') client1.expect(prompt) From 8df3e48a7b997f423840af6ec1642f2699272ccd Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Mon, 17 Jun 2019 12:39:01 -0400 Subject: [PATCH 083/509] * Fixing bug in writeIntoLiveView method where mergeable blocks would always be reobtained after 64 inserts --- dbms/src/Storages/StorageLiveView.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index 69410ec4e8b..84a7ea80bca 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -141,6 +141,7 @@ public: std::shared_ptr getBlocksPtr() { return blocks_ptr; } BlocksPtrs getMergeableBlocks() { return mergeable_blocks; } + void setMergeableBlocks(BlocksPtrs blocks) { mergeable_blocks = blocks; } std::shared_ptr getActivePtr() { return active_ptr; } /// Read new data blocks that store query result @@ -193,12 +194,13 @@ public: { mergeable_blocks = std::make_shared>(); BlocksPtr base_mergeable_blocks = std::make_shared(); - InterpreterSelectQuery interpreter(live_view.getInnerQuery(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names{}); + InterpreterSelectQuery interpreter(live_view.getInnerQuery(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); auto view_mergeable_stream = std::make_shared( interpreter.execute().in); while (Block this_block = view_mergeable_stream->read()) base_mergeable_blocks->push_back(this_block); mergeable_blocks->push_back(base_mergeable_blocks); + live_view.setMergeableBlocks(mergeable_blocks); } /// Need make new mergeable block structure match the other mergeable blocks From d87ed7f267becec342366b432613ef31932fb082 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sun, 23 Jun 2019 09:51:29 -0400 Subject: [PATCH 084/509] * Adding max_live_view_insert_blocks_before_refresh setting with default value of 64 * Updating writeIntoLiveView method to use max_live_view_insert_blocks_before_refresh from global context instead of hard coded number * Adding squashing of the result blocks similar how it is done in materialized views * Fixing bug in writeIntoLiveView that caused incoming block to be processed twice when maximum number of insert blocks was reached --- dbms/src/Core/Settings.h | 3 +- dbms/src/Storages/StorageLiveView.cpp | 7 ++ dbms/src/Storages/StorageLiveView.h | 104 +++++++++--------- .../0_stateless/00958_live_view_watch_live.py | 6 + 4 files changed, 68 insertions(+), 52 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 5d29c50a9ac..0196e1e35ce 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -330,7 +330,8 @@ struct Settings : public SettingsCollection M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.") \ M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.") \ M(SettingSeconds, temporary_live_channel_timeout, DEFAULT_TEMPORARY_LIVE_CHANNEL_TIMEOUT_SEC, "Timeout after which temporary live channel is deleted.") \ - M(SettingMilliseconds, alter_channel_wait_ms, DEFAULT_ALTER_LIVE_CHANNEL_WAIT_MS, "The wait time for alter channel request.") + M(SettingMilliseconds, alter_channel_wait_ms, DEFAULT_ALTER_LIVE_CHANNEL_WAIT_MS, "The wait time for alter channel request.") \ + M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.") DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index eecc7f6d1dc..0eb31acf1ae 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -197,6 +197,13 @@ bool StorageLiveView::getNewBlocks() auto proxy_storage = ProxyStorage::createProxyStorage(global_context.getTable(select_database_name, select_table_name), {from}, QueryProcessingStage::WithMergeableState); InterpreterSelectQuery select(inner_query->clone(), global_context, proxy_storage, SelectQueryOptions(QueryProcessingStage::Complete)); BlockInputStreamPtr data = std::make_shared(select.execute().in); + + /// Squashing is needed here because the view query can generate a lot of blocks + /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY + /// and two-level aggregation is triggered). + data = std::make_shared( + data, global_context.getSettingsRef().min_insert_block_size_rows, global_context.getSettingsRef().min_insert_block_size_bytes); + while (Block block = data->read()) { /// calculate hash before virtual column is added diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index 84a7ea80bca..3f2732647b3 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -17,6 +17,7 @@ limitations under the License. */ #include #include #include +#include #include #include #include @@ -166,10 +167,42 @@ public: } } + bool is_block_processed = false; BlockInputStreams from; - BlocksPtr blocks = std::make_shared(); BlocksPtrs mergeable_blocks; BlocksPtr new_mergeable_blocks = std::make_shared(); + + { + Poco::FastMutex::ScopedLock lock(live_view.mutex); + + mergeable_blocks = live_view.getMergeableBlocks(); + if (!mergeable_blocks || mergeable_blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh) + { + mergeable_blocks = std::make_shared>(); + BlocksPtr base_mergeable_blocks = std::make_shared(); + InterpreterSelectQuery interpreter(live_view.getInnerQuery(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); + auto view_mergeable_stream = std::make_shared( + interpreter.execute().in); + while (Block this_block = view_mergeable_stream->read()) + base_mergeable_blocks->push_back(this_block); + mergeable_blocks->push_back(base_mergeable_blocks); + live_view.setMergeableBlocks(mergeable_blocks); + + /// Create from streams + for (auto & blocks_ : *mergeable_blocks) + { + if (blocks_->empty()) + continue; + auto sample_block = blocks_->front().cloneEmpty(); + BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); + from.push_back(std::move(stream)); + } + + is_block_processed = true; + } + } + + if (!is_block_processed) { auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); BlockInputStreams streams = {std::make_shared(block)}; @@ -181,52 +214,26 @@ public: select_block.execute().in); while (Block this_block = data_mergeable_stream->read()) new_mergeable_blocks->push_back(this_block); - } - if (new_mergeable_blocks->empty()) - return; + if (new_mergeable_blocks->empty()) + return; - { - Poco::FastMutex::ScopedLock lock(live_view.mutex); - - mergeable_blocks = live_view.getMergeableBlocks(); - if (!mergeable_blocks || mergeable_blocks->size() >= 64) { - mergeable_blocks = std::make_shared>(); - BlocksPtr base_mergeable_blocks = std::make_shared(); - InterpreterSelectQuery interpreter(live_view.getInnerQuery(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); - auto view_mergeable_stream = std::make_shared( - interpreter.execute().in); - while (Block this_block = view_mergeable_stream->read()) - base_mergeable_blocks->push_back(this_block); - mergeable_blocks->push_back(base_mergeable_blocks); - live_view.setMergeableBlocks(mergeable_blocks); - } + Poco::FastMutex::ScopedLock lock(live_view.mutex); - /// Need make new mergeable block structure match the other mergeable blocks - if (!mergeable_blocks->front()->empty()) - { - auto sample_block = mergeable_blocks->front()->front(); - auto sample_new_block = new_mergeable_blocks->front(); - for (auto col : sample_new_block) + mergeable_blocks = live_view.getMergeableBlocks(); + mergeable_blocks->push_back(new_mergeable_blocks); + + /// Create from streams + for (auto & blocks_ : *mergeable_blocks) { - for (auto & new_block : *new_mergeable_blocks) - { - if (!sample_block.has(col.name)) - new_block.erase(col.name); - } + if (blocks_->empty()) + continue; + auto sample_block = blocks_->front().cloneEmpty(); + BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); + from.push_back(std::move(stream)); } } - - mergeable_blocks->push_back(new_mergeable_blocks); - - /// Create from blocks streams - for (auto & blocks_ : *mergeable_blocks) - { - auto sample_block = mergeable_blocks->front()->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); - from.push_back(std::move(stream)); - } } auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); @@ -234,18 +241,13 @@ public: InterpreterSelectQuery select(live_view.getInnerQuery(), context, proxy_storage, QueryProcessingStage::Complete); BlockInputStreamPtr data = std::make_shared(select.execute().in); - while (Block this_block = data->read()) - { - blocks->push_back(this_block); - } + /// Squashing is needed here because the view query can generate a lot of blocks + /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY + /// and two-level aggregation is triggered). + data = std::make_shared( + data, context.getGlobalContext().getSettingsRef().min_insert_block_size_rows, context.getGlobalContext().getSettingsRef().min_insert_block_size_bytes); - if (blocks->empty()) - return; - - auto sample_block = blocks->front().cloneEmpty(); - BlockInputStreamPtr new_data = std::make_shared(std::make_shared(blocks), sample_block); - - copyData(*new_data, *output); + copyData(*data, *output); } private: diff --git a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py index 7cbea5fbff6..33cc9db3ccc 100755 --- a/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00958_live_view_watch_live.py @@ -28,8 +28,14 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(r'0.*1' + end_of_block) client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') client1.expect(r'6.*2' + end_of_block) + client2.expect(prompt) client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') client1.expect(r'21.*3' + end_of_block) + client2.expect(prompt) + for i in range(1,129): + client2.send('INSERT INTO test.mt VALUES (1)') + client1.expect(r'%d.*%d' % (21+i, 3+i) + end_of_block) + client2.expect(prompt) # send Ctrl-C os.kill(client1.process.pid, signal.SIGINT) client1.expect(prompt) From bb78012cf91e21d7394d001eba6480918ab4fb01 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 30 Jun 2019 12:09:06 +0200 Subject: [PATCH 085/509] Removed unused method and assertions --- .../CheckConstraintsBlockOutputStream.cpp | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 75067b6afa7..ec4a7bd45b8 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -38,26 +38,11 @@ bool CheckConstraintsBlockOutputStream::checkImplMemory(const Block & block, con { Block res = block; constraint->execute(res); - assert(block.columns() == res.columns() - 1); ColumnWithTypeAndName res_column = res.safeGetByPosition(res.columns() - 1); auto res_column_uint8 = checkAndGetColumn(res_column.column.get()); return memoryIsByte(res_column_uint8->getRawDataBegin<1>(), res_column_uint8->byteSize(), 0x1); } -bool CheckConstraintsBlockOutputStream::checkImplBool(const Block & block, const ExpressionActionsPtr & constraint) -{ - Block res = block; - constraint->execute(res); - assert(block.columns() == res.columns() - 1); - ColumnWithTypeAndName res_column = res.safeGetByPosition(res.columns() - 1); - size_t column_size = res_column.column->size(); - // std::cerr << "Sizes of constraints: " << res_column.column->size() << ' ' << res_column.column->get << '\n'; - for (size_t i = 0; i < column_size; ++i) - if (!res_column.column->getBool(i)) - return false; - return true; -} - bool CheckConstraintsBlockOutputStream::checkConstraintOnBlock(const Block & block, const ExpressionActionsPtr & constraint) { return checkImplMemory(block, constraint); From ea4d42c3aca78eb6d114ea3703042649cc0db110 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 30 Jun 2019 12:09:58 +0200 Subject: [PATCH 086/509] Merged DROP_(COLUMN|INDEX|CONSTRAINT) AST parsing into one block --- dbms/src/Storages/AlterCommands.cpp | 39 ++++++++++------------------- 1 file changed, 13 insertions(+), 26 deletions(-) diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 80c01a0028e..1ce70723238 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -74,17 +74,6 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ return command; } - else if (command_ast->type == ASTAlterCommand::DROP_COLUMN && !command_ast->partition) - { - if (command_ast->clear_column) - throw Exception("\"ALTER TABLE table CLEAR COLUMN column\" queries are not supported yet. Use \"CLEAR COLUMN column IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED); - - AlterCommand command; - command.type = AlterCommand::DROP_COLUMN; - command.column_name = *getIdentifierName(command_ast->column); - command.if_exists = command_ast->if_exists; - return command; - } else if (command_ast->type == ASTAlterCommand::MODIFY_COLUMN) { AlterCommand command; @@ -154,18 +143,6 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ return command; } - else if (command_ast->type == ASTAlterCommand::DROP_INDEX) - { - if (command_ast->clear_column) - throw Exception("\"ALTER TABLE table CLEAR COLUMN column\" queries are not supported yet. Use \"CLEAR COLUMN column IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED); - - AlterCommand command; - command.type = AlterCommand::DROP_INDEX; - command.index_name = command_ast->index->as().name; - command.if_exists = command_ast->if_exists; - - return command; - } else if (command_ast->type == ASTAlterCommand::ADD_CONSTRAINT) { AlterCommand command; @@ -180,15 +157,25 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ return command; } - else if (command_ast->type == ASTAlterCommand::DROP_CONSTRAINT) + else if (command_ast->type == ASTAlterCommand::DROP_CONSTRAINT + || command_ast->type == ASTAlterCommand::DROP_INDEX + || (command_ast->type == ASTAlterCommand::DROP_COLUMN && !command_ast->partition)) { if (command_ast->clear_column) throw Exception("\"ALTER TABLE table CLEAR COLUMN column\" queries are not supported yet. Use \"CLEAR COLUMN column IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED); AlterCommand command; - command.type = AlterCommand::DROP_CONSTRAINT; - command.constraint_name = command_ast->constraint->as().name; command.if_exists = command_ast->if_exists; + if (command_ast->type == ASTAlterCommand::DROP_INDEX) { + command.type = AlterCommand::DROP_INDEX; + command.index_name = command_ast->index->as().name; + } else if (command_ast->type == ASTAlterCommand::DROP_CONSTRAINT) { + command.type = AlterCommand::DROP_CONSTRAINT; + command.constraint_name = command_ast->constraint->as().name; + } else if (command_ast->type == ASTAlterCommand::DROP_COLUMN) { + command.type = AlterCommand::DROP_COLUMN; + command.column_name = *getIdentifierName(command_ast->column); + } return command; } From 06e92e14b828c1457410d04aa8682acbda11d182 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Fri, 5 Jul 2019 10:10:15 +0300 Subject: [PATCH 087/509] alterTable fix in MySQL --- dbms/src/Databases/DatabaseMySQL.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Databases/DatabaseMySQL.h b/dbms/src/Databases/DatabaseMySQL.h index 7ce836d6a64..01a8c1df0d2 100644 --- a/dbms/src/Databases/DatabaseMySQL.h +++ b/dbms/src/Databases/DatabaseMySQL.h @@ -71,7 +71,7 @@ public: throw Exception("MySQL database engine does not support create table.", ErrorCodes::NOT_IMPLEMENTED); } - void alterTable(const Context &, const String &, const ColumnsDescription &, const IndicesDescription &, const ASTModifier &) override + void alterTable(const Context &, const String &, const ColumnsDescription &, const IndicesDescription &, const ConstraintsDescription &, const ASTModifier &) override { throw Exception("MySQL database engine does not support alter table.", ErrorCodes::NOT_IMPLEMENTED); } From 430400c3c5206c383a1e0870f00a77f5923147a3 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Fri, 5 Jul 2019 10:16:34 +0300 Subject: [PATCH 088/509] Style fix --- dbms/src/Storages/AlterCommands.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 8b3713ae31f..c90d5ca2c25 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -166,13 +166,16 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ AlterCommand command; command.if_exists = command_ast->if_exists; - if (command_ast->type == ASTAlterCommand::DROP_INDEX) { + if (command_ast->type == ASTAlterCommand::DROP_INDEX) + { command.type = AlterCommand::DROP_INDEX; command.index_name = command_ast->index->as().name; - } else if (command_ast->type == ASTAlterCommand::DROP_CONSTRAINT) { + } else if (command_ast->type == ASTAlterCommand::DROP_CONSTRAINT) + { command.type = AlterCommand::DROP_CONSTRAINT; command.constraint_name = command_ast->constraint->as().name; - } else if (command_ast->type == ASTAlterCommand::DROP_COLUMN) { + } else if (command_ast->type == ASTAlterCommand::DROP_COLUMN) + { command.type = AlterCommand::DROP_COLUMN; command.column_name = *getIdentifierName(command_ast->column); } From 3757aa9d020046d0a825a27546b43066ced2ffec Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sat, 6 Jul 2019 23:30:48 +0300 Subject: [PATCH 089/509] More style fix --- dbms/src/Storages/AlterCommands.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index c90d5ca2c25..b250452c683 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -170,11 +170,13 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ { command.type = AlterCommand::DROP_INDEX; command.index_name = command_ast->index->as().name; - } else if (command_ast->type == ASTAlterCommand::DROP_CONSTRAINT) + } + else if (command_ast->type == ASTAlterCommand::DROP_CONSTRAINT) { command.type = AlterCommand::DROP_CONSTRAINT; command.constraint_name = command_ast->constraint->as().name; - } else if (command_ast->type == ASTAlterCommand::DROP_COLUMN) + } + else if (command_ast->type == ASTAlterCommand::DROP_COLUMN) { command.type = AlterCommand::DROP_COLUMN; command.column_name = *getIdentifierName(command_ast->column); From 259a95a3da5ca6ad05d0cbc11f82682258f00576 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sun, 7 Jul 2019 19:57:58 -0400 Subject: [PATCH 090/509] * Updates to fix issues after the merge --- dbms/src/Interpreters/ExternalLoader.cpp | 7 +++++++ dbms/src/Storages/IStorage.h | 4 ++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 018565e0a2c..f00fa2bfa90 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -400,6 +400,10 @@ public: return count; } +#if !__clang__ +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-variable" +#endif bool hasCurrentlyLoadedObjects() const { std::lock_guard lock{mutex}; @@ -408,6 +412,9 @@ public: return true; return false; } +#if !__clang__ +#pragma GCC diagnostic pop +#endif /// Starts loading of a specified object. void load(const String & name) diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 8541def8753..b7358c3fc6e 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -84,7 +84,8 @@ public: public: /// thread-unsafe part. lockStructure must be acquired virtual const ColumnsDescription & getColumns() const; /// returns combined set of columns - + virtual void setColumns(ColumnsDescription columns_); /// sets only real columns, possibly overwrites virtual ones. + const IndicesDescription & getIndices() const; /// NOTE: these methods should include virtual columns, @@ -113,7 +114,6 @@ public: /// thread-unsafe part. lockStructure must be acquired void check(const Block & block, bool need_all = false) const; protected: /// still thread-unsafe part. - virtual void setColumns(ColumnsDescription columns_); /// sets only real columns, possibly overwrites virtual ones. void setIndices(IndicesDescription indices_); /// Returns whether the column is virtual - by default all columns are real. From a80af666adbcbeb173b5dd125a72439b55b92c25 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Tue, 9 Jul 2019 08:41:55 -0400 Subject: [PATCH 091/509] * Updates that incorporate comments from the review --- dbms/src/DataStreams/BlocksBlockInputStream.h | 6 +-- .../DataStreams/LiveViewBlockInputStream.h | 47 +++++++++---------- .../LiveViewEventsBlockInputStream.h | 46 ++++++++++-------- dbms/src/Storages/IStorage.h | 2 +- dbms/src/Storages/StorageLiveView.cpp | 13 +++-- dbms/src/Storages/StorageLiveView.h | 2 +- 6 files changed, 62 insertions(+), 54 deletions(-) diff --git a/dbms/src/DataStreams/BlocksBlockInputStream.h b/dbms/src/DataStreams/BlocksBlockInputStream.h index ad0d37da622..58a63e5b85e 100644 --- a/dbms/src/DataStreams/BlocksBlockInputStream.h +++ b/dbms/src/DataStreams/BlocksBlockInputStream.h @@ -23,8 +23,8 @@ class BlocksBlockInputStream : public IBlockInputStream { public: /// Acquires shared ownership of the blocks vector - BlocksBlockInputStream(std::shared_ptr blocks_ptr_, Block header) - : blocks_ptr(blocks_ptr_), it((*blocks_ptr_)->begin()), end((*blocks_ptr_)->end()), header(header) {} + BlocksBlockInputStream(const std::shared_ptr & blocks_ptr_, Block header) + : blocks(*blocks_ptr_), it((*blocks_ptr_)->begin()), end((*blocks_ptr_)->end()), header(std::move(header)) {} String getName() const override { return "Blocks"; } @@ -42,7 +42,7 @@ protected: } private: - std::shared_ptr blocks_ptr; + BlocksPtr blocks; Blocks::iterator it; const Blocks::iterator end; Block header; diff --git a/dbms/src/DataStreams/LiveViewBlockInputStream.h b/dbms/src/DataStreams/LiveViewBlockInputStream.h index e9bb599a1d6..89ac5e4f9f3 100644 --- a/dbms/src/DataStreams/LiveViewBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewBlockInputStream.h @@ -23,9 +23,10 @@ limitations under the License. */ namespace DB { -/** - */ - +/** Implements LIVE VIEW table WATCH input stream. + * Keeps stream alive by outputing blocks with no rows + * based on period specified by the heartbeat interval. + */ class LiveViewBlockInputStream : public IBlockInputStream { @@ -37,18 +38,17 @@ public: /// Start storage no users thread /// if we are the last active user if (!storage->is_dropped && blocks_ptr.use_count() < 3) - storage->startNoUsersThread(temporary_live_view_timeout); + storage->startNoUsersThread(temporary_live_view_timeout_sec); } - /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update - /// and LIMIT 0 just returns data without waiting for any updates + LiveViewBlockInputStream(std::shared_ptr storage_, std::shared_ptr blocks_ptr_, std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, - int64_t length_, const UInt64 & heartbeat_interval_, - const UInt64 & temporary_live_view_timeout_) - : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), length(length_ + 1), heartbeat_interval(heartbeat_interval_ * 1000000), temporary_live_view_timeout(temporary_live_view_timeout_), - blocks_hash("") + const bool has_limit_, const UInt64 limit_, + const UInt64 heartbeat_interval_sec_, + const UInt64 temporary_live_view_timeout_sec_) + : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000), temporary_live_view_timeout_sec(temporary_live_view_timeout_sec_) { /// grab active pointer active = active_ptr.lock(); @@ -70,7 +70,7 @@ public: void refresh() { if (active && blocks && it == end) - it = blocks->begin(); + it = blocks->begin(); } void suspend() @@ -111,7 +111,7 @@ protected: { Block res; - if (length == 0) + if (has_limit && num_updates == (Int64)limit) { return { Block(), true }; } @@ -162,7 +162,7 @@ protected: while (true) { UInt64 timestamp_usec = static_cast(timestamp.epochMicroseconds()); - bool signaled = storage->condition.tryWait(storage->mutex, std::max(static_cast(0), heartbeat_interval - (timestamp_usec - last_event_timestamp)) / 1000); + bool signaled = storage->condition.tryWait(storage->mutex, std::max(static_cast(0), heartbeat_interval_usec - (timestamp_usec - last_event_timestamp_usec)) / 1000); if (isCancelled() || storage->is_dropped) { @@ -175,7 +175,7 @@ protected: else { // heartbeat - last_event_timestamp = static_cast(timestamp.epochMicroseconds()); + last_event_timestamp_usec = static_cast(timestamp.epochMicroseconds()); return { getHeader(), true }; } } @@ -191,11 +191,10 @@ protected: if (it == end) { end_of_blocks = false; - if (length > 0) - --length; + num_updates += 1; } - last_event_timestamp = static_cast(timestamp.epochMicroseconds()); + last_event_timestamp_usec = static_cast(timestamp.epochMicroseconds()); return { res, true }; } @@ -210,13 +209,13 @@ private: Blocks::iterator it; Blocks::iterator end; Blocks::iterator begin; - /// Length specifies number of updates to send, default -1 (no limit) - int64_t length; - bool end_of_blocks{0}; - UInt64 heartbeat_interval; - UInt64 temporary_live_view_timeout; - String blocks_hash; - UInt64 last_event_timestamp{0}; + const bool has_limit; + const UInt64 limit; + Int64 num_updates = -1; + bool end_of_blocks = false; + UInt64 heartbeat_interval_usec; + UInt64 temporary_live_view_timeout_sec; + UInt64 last_event_timestamp_usec = 0; Poco::Timestamp timestamp; }; diff --git a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h index 75e68b15d92..fa97ae2e7f5 100644 --- a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h @@ -27,9 +27,10 @@ limitations under the License. */ namespace DB { -/** - */ - +/** Implements LIVE VIEW table WATCH EVENTS input stream. + * Keeps stream alive by outputing blocks with no rows + * based on period specified by the heartbeat interval. + */ class LiveViewEventsBlockInputStream : public IBlockInputStream { @@ -41,13 +42,18 @@ public: /// Start storage no users thread /// if we are the last active user if (!storage->is_dropped && blocks_ptr.use_count() < 3) - storage->startNoUsersThread(temporary_live_view_timeout); + storage->startNoUsersThread(temporary_live_view_timeout_sec); } /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update /// and LIMIT 0 just returns data without waiting for any updates - LiveViewEventsBlockInputStream(std::shared_ptr storage_, std::shared_ptr blocks_ptr_, std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, - int64_t length_, const UInt64 & heartbeat_interval_, const UInt64 & temporary_live_view_timeout_) - : storage(storage_), blocks_ptr(blocks_ptr_), blocks_metadata_ptr(blocks_metadata_ptr_), active_ptr(active_ptr_), length(length_ + 1), heartbeat_interval(heartbeat_interval_ * 1000000), temporary_live_view_timeout(temporary_live_view_timeout_) + LiveViewEventsBlockInputStream(std::shared_ptr storage_, + std::shared_ptr blocks_ptr_, + std::shared_ptr blocks_metadata_ptr_, + std::shared_ptr active_ptr_, + const bool has_limit_, const UInt64 limit_, + const UInt64 heartbeat_interval_sec_, + const UInt64 temporary_live_view_timeout_sec_) + : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000), temporary_live_view_timeout_sec(temporary_live_view_timeout_sec_) { /// grab active pointer active = active_ptr.lock(); @@ -72,7 +78,7 @@ public: void refresh() { if (active && blocks && it == end) - it = blocks->begin(); + it = blocks->begin(); } void suspend() @@ -124,7 +130,7 @@ protected: */ NonBlockingResult tryRead_(bool blocking) { - if (length == 0) + if (has_limit && num_updates == (Int64)limit) { return { Block(), true }; } @@ -177,7 +183,7 @@ protected: while (true) { UInt64 timestamp_usec = static_cast(timestamp.epochMicroseconds()); - bool signaled = storage->condition.tryWait(storage->mutex, std::max(static_cast(0), heartbeat_interval - (timestamp_usec - last_event_timestamp)) / 1000); + bool signaled = storage->condition.tryWait(storage->mutex, std::max(static_cast(0), heartbeat_interval_usec - (timestamp_usec - last_event_timestamp_usec)) / 1000); if (isCancelled() || storage->is_dropped) { @@ -190,7 +196,7 @@ protected: else { // repeat the event block as a heartbeat - last_event_timestamp = static_cast(timestamp.epochMicroseconds()); + last_event_timestamp_usec = static_cast(timestamp.epochMicroseconds()); return { getHeader(), true }; } } @@ -205,11 +211,10 @@ protected: if (it == end) { end_of_blocks = false; - if (length > 0) - --length; + num_updates += 1; } - last_event_timestamp = static_cast(timestamp.epochMicroseconds()); + last_event_timestamp_usec = static_cast(timestamp.epochMicroseconds()); return { getEventBlock(), true }; } @@ -225,12 +230,13 @@ private: Blocks::iterator it; Blocks::iterator end; Blocks::iterator begin; - /// Length specifies number of updates to send, default -1 (no limit) - int64_t length; - bool end_of_blocks{0}; - UInt64 heartbeat_interval; - UInt64 temporary_live_view_timeout; - UInt64 last_event_timestamp{0}; + const bool has_limit; + const UInt64 limit; + Int64 num_updates = -1; + bool end_of_blocks = false; + UInt64 heartbeat_interval_usec; + UInt64 temporary_live_view_timeout_sec; + UInt64 last_event_timestamp_usec = 0; Poco::Timestamp timestamp; }; diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index b7358c3fc6e..0d13a63a214 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -85,7 +85,7 @@ public: public: /// thread-unsafe part. lockStructure must be acquired virtual const ColumnsDescription & getColumns() const; /// returns combined set of columns virtual void setColumns(ColumnsDescription columns_); /// sets only real columns, possibly overwrites virtual ones. - + const IndicesDescription & getIndices() const; /// NOTE: these methods should include virtual columns, diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index 0eb31acf1ae..df7f8c896b3 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -407,15 +407,18 @@ BlockInputStreams StorageLiveView::watch( { ASTWatchQuery & query = typeid_cast(*query_info.query); - /// By default infinite stream of updates - int64_t length = -2; + bool has_limit = false; + UInt64 limit = 0; if (query.limit_length) - length = static_cast(safeGet(typeid_cast(*query.limit_length).value)); + { + has_limit = true; + limit = safeGet(typeid_cast(*query.limit_length).value); + } if (query.is_watch_events) { - auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, length, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), + auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), context.getSettingsRef().temporary_live_view_timeout.totalSeconds()); if (no_users_thread.joinable()) @@ -440,7 +443,7 @@ BlockInputStreams StorageLiveView::watch( } else { - auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, length, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), + auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), context.getSettingsRef().temporary_live_view_timeout.totalSeconds()); if (no_users_thread.joinable()) diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index 3f2732647b3..6bff9c2dc85 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -78,7 +78,7 @@ public: return blocks_ptr.use_count() > 1; } - /// Check we we have any active readers + /// Check we have any active readers /// must be called with mutex locked bool hasActiveUsers() { From c84fd80345d76f648e9c8f1463c9b4210d08c5be Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Tue, 9 Jul 2019 22:06:29 -0400 Subject: [PATCH 092/509] * Fixing clang build by converting old style cast (Int64) to static_cast. --- dbms/src/DataStreams/LiveViewBlockInputStream.h | 2 +- dbms/src/DataStreams/LiveViewEventsBlockInputStream.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataStreams/LiveViewBlockInputStream.h b/dbms/src/DataStreams/LiveViewBlockInputStream.h index 89ac5e4f9f3..b3756c9ff6d 100644 --- a/dbms/src/DataStreams/LiveViewBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewBlockInputStream.h @@ -111,7 +111,7 @@ protected: { Block res; - if (has_limit && num_updates == (Int64)limit) + if (has_limit && num_updates == static_cast(limit)) { return { Block(), true }; } diff --git a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h index fa97ae2e7f5..93fb6a76372 100644 --- a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h +++ b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h @@ -130,7 +130,7 @@ protected: */ NonBlockingResult tryRead_(bool blocking) { - if (has_limit && num_updates == (Int64)limit) + if (has_limit && num_updates == static_cast(limit)) { return { Block(), true }; } From 9c5514135d689c77036850afed38f48f81ad0134 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Tue, 9 Jul 2019 23:27:19 -0400 Subject: [PATCH 093/509] * Updating tests to send Ctrl-C instead of SIGINT signal to abort WATCH query * Updating client.py to start commands inside shell * Removing test code from uexpect.py --- .../00960_live_view_watch_events_live.py | 2 +- .../00962_temporary_live_view_watch_live.py | 2 +- ...3_temporary_live_view_watch_live_timeout.py | 2 +- .../00964_live_view_watch_events_heartbeat.py | 2 +- .../00965_live_view_watch_heartbeat.py | 2 +- .../0_stateless/00979_live_view_watch_live.py | 2 +- .../queries/0_stateless/helpers/client.py | 7 ++++--- .../queries/0_stateless/helpers/uexpect.py | 18 ------------------ 8 files changed, 10 insertions(+), 27 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py index 44b8df185b1..5f2d7c90da7 100755 --- a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py +++ b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py @@ -31,7 +31,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') client1.expect('3.*' + end_of_block) # send Ctrl-C - os.kill(client1.process.pid, signal.SIGINT) + client1.send('\x03', eol='') client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py index d4d35548d68..d3c6e4cb6cf 100755 --- a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py @@ -31,7 +31,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') client1.expect(r'21.*3' + end_of_block) # send Ctrl-C - os.kill(client1.process.pid, signal.SIGINT) + client1.send('\x03', eol='') client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py index 2008368f12e..28c1c027017 100755 --- a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py +++ b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py @@ -35,7 +35,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client2.expect(prompt) client1.expect(r'21.*3' + end_of_block) # send Ctrl-C - os.kill(client1.process.pid, signal.SIGINT) + client1.send('\x03', eol='') client1.expect(prompt) client1.send('SELECT sleep(1)') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py index cecae7c5a72..5db6a6c13aa 100755 --- a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -33,7 +33,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo # wait for heartbeat client1.expect('Progress: 2.00 rows.*\)') # send Ctrl-C - os.kill(client1.process.pid, signal.SIGINT) + client1.send('\x03', eol='') client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py index b2bd84e0742..f024f3f1008 100755 --- a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py @@ -34,7 +34,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo # wait for heartbeat client1.expect('Progress: 2.00 rows.*\)') # send Ctrl-C - os.kill(client1.process.pid, signal.SIGINT) + client1.send('\x03', eol='') client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py index 33cc9db3ccc..e5e44284959 100755 --- a/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py @@ -37,7 +37,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(r'%d.*%d' % (21+i, 3+i) + end_of_block) client2.expect(prompt) # send Ctrl-C - os.kill(client1.process.pid, signal.SIGINT) + client1.send('\x03', eol='') client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/helpers/client.py b/dbms/tests/queries/0_stateless/helpers/client.py index dec7eca7d78..b8f0dec46ad 100644 --- a/dbms/tests/queries/0_stateless/helpers/client.py +++ b/dbms/tests/queries/0_stateless/helpers/client.py @@ -11,11 +11,12 @@ prompt = ':\) ' end_of_block = r'.*\r\n.*\r\n' def client(command=None, name='', log=None): + client = uexpect.spawn(['/bin/bash','--noediting']) if command is None: - client = uexpect.spawn(os.environ.get('CLICKHOUSE_BINARY', 'clickhouse') + '-client') - else: - client = uexpect.spawn(command) + command = os.environ.get('CLICKHOUSE_BINARY', 'clickhouse') + '-client' client.eol('\r') client.logger(log, prefix=name) client.timeout(20) + client.expect('[#\$] ', timeout=2) + client.send(command) return client diff --git a/dbms/tests/queries/0_stateless/helpers/uexpect.py b/dbms/tests/queries/0_stateless/helpers/uexpect.py index 2f323cf6ca6..d65190e2b29 100644 --- a/dbms/tests/queries/0_stateless/helpers/uexpect.py +++ b/dbms/tests/queries/0_stateless/helpers/uexpect.py @@ -202,21 +202,3 @@ def reader(process, out, queue, kill_event): if e.errno == 5 and kill_event.is_set(): break raise - -if __name__ == '__main__': - io = spawn(['/bin/bash','--noediting']) - prompt = '\$ ' - io.logger(sys.stdout) - io.timeout(2) - io.eol('\r') - - io.expect(prompt) - io.send('clickhouse-client') - prompt = ':\) ' - io.expect(prompt) - io.send('SELECT 1') - io.expect(prompt) - io.send('SHOW TABLES') - io.expect('.*\r\n.*') - io.expect(prompt) - io.close() From 280025d5b4931205d3186d055ed69c2ccfe9685e Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Wed, 10 Jul 2019 08:22:31 -0400 Subject: [PATCH 094/509] * Updating live view tests to workaround an issue with clickhouse-client exiting when Ctrl-C is pressed during WATCH query execution instead of going to the client prompt. --- .../queries/0_stateless/00960_live_view_watch_events_live.py | 5 ++++- .../0_stateless/00962_temporary_live_view_watch_live.py | 5 ++++- .../00963_temporary_live_view_watch_live_timeout.py | 5 ++++- .../0_stateless/00964_live_view_watch_events_heartbeat.py | 5 ++++- .../queries/0_stateless/00965_live_view_watch_heartbeat.py | 5 ++++- dbms/tests/queries/0_stateless/00979_live_view_watch_live.py | 5 ++++- dbms/tests/queries/0_stateless/helpers/client.py | 1 + 7 files changed, 25 insertions(+), 6 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py index 5f2d7c90da7..b7fc3f4e3a6 100755 --- a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py +++ b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py @@ -32,7 +32,10 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect('3.*' + end_of_block) # send Ctrl-C client1.send('\x03', eol='') - client1.expect(prompt) + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) client1.send('DROP TABLE test.mt') diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py index d3c6e4cb6cf..f27b1213c70 100755 --- a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py @@ -32,7 +32,10 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(r'21.*3' + end_of_block) # send Ctrl-C client1.send('\x03', eol='') - client1.expect(prompt) + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) client1.send('DROP TABLE test.mt') diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py index 28c1c027017..df627c84e49 100755 --- a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py +++ b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py @@ -36,7 +36,10 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(r'21.*3' + end_of_block) # send Ctrl-C client1.send('\x03', eol='') - client1.expect(prompt) + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('SELECT sleep(1)') client1.expect(prompt) client1.send('DROP TABLE test.lv') diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py index 5db6a6c13aa..5664c0e6c6d 100755 --- a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -34,7 +34,10 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect('Progress: 2.00 rows.*\)') # send Ctrl-C client1.send('\x03', eol='') - client1.expect(prompt) + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) client1.send('DROP TABLE test.mt') diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py index f024f3f1008..03e22175dff 100755 --- a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py @@ -35,7 +35,10 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect('Progress: 2.00 rows.*\)') # send Ctrl-C client1.send('\x03', eol='') - client1.expect(prompt) + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) client1.send('DROP TABLE test.mt') diff --git a/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py index e5e44284959..948e4c93662 100755 --- a/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py @@ -38,7 +38,10 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client2.expect(prompt) # send Ctrl-C client1.send('\x03', eol='') - client1.expect(prompt) + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('DROP TABLE test.lv') client1.expect(prompt) client1.send('DROP TABLE test.mt') diff --git a/dbms/tests/queries/0_stateless/helpers/client.py b/dbms/tests/queries/0_stateless/helpers/client.py index b8f0dec46ad..de4da794805 100644 --- a/dbms/tests/queries/0_stateless/helpers/client.py +++ b/dbms/tests/queries/0_stateless/helpers/client.py @@ -14,6 +14,7 @@ def client(command=None, name='', log=None): client = uexpect.spawn(['/bin/bash','--noediting']) if command is None: command = os.environ.get('CLICKHOUSE_BINARY', 'clickhouse') + '-client' + client.command = command client.eol('\r') client.logger(log, prefix=name) client.timeout(20) From c1d91222d50e368529e0c6675b4a6a6bb6681103 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sat, 13 Jul 2019 13:42:52 +0300 Subject: [PATCH 095/509] Ranamed setPrimaryKeyIndicesAndColumns to setProperties --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 6 +++--- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 3c30e7ce778..c2aa0a61855 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -121,7 +121,7 @@ MergeTreeData::MergeTreeData( data_parts_by_info(data_parts_indexes.get()), data_parts_by_state_and_info(data_parts_indexes.get()) { - setPrimaryKeyIndicesAndColumns(order_by_ast_, primary_key_ast_, columns_, indices_, constraints_); + setProperties(order_by_ast_, primary_key_ast_, columns_, indices_, constraints_); setConstraints(constraints_); /// NOTE: using the same columns list as is read when performing actual merges. @@ -232,7 +232,7 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam } -void MergeTreeData::setPrimaryKeyIndicesAndColumns( +void MergeTreeData::setProperties( const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast, const ColumnsDescription & new_columns, const IndicesDescription & indices_description, const ConstraintsDescription & constraints_description, bool only_check) @@ -1262,7 +1262,7 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c } } - setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, + setProperties(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints, /* only_check = */ true); setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast, /* only_check = */ true); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 7572e9f2856..d4848462df0 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -757,7 +757,7 @@ protected: /// The same for clearOldTemporaryDirectories. std::mutex clear_old_temporary_directories_mutex; - void setPrimaryKeyIndicesAndColumns(const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast, + void setProperties(const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast, const ColumnsDescription & new_columns, const IndicesDescription & indices_description, const ConstraintsDescription & constraints_description, bool only_check = false); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 7cf8facb5f2..b7adc8c5bff 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -282,7 +282,7 @@ void StorageMergeTree::alter( context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, storage_modifier); /// Reinitialize primary key because primary key column types might have changed. - setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints); + setProperties(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints); setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index f1f4a4049f8..ae0ccfaaf1e 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -484,7 +484,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column /// Even if the primary/sorting keys didn't change we must reinitialize it /// because primary key column types might have changed. - setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints); + setProperties(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints); setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast); } From 1feb20d9e01b17876e768c8db2672f9d28e4eac9 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 23 Jul 2019 11:01:08 +0300 Subject: [PATCH 096/509] DOCAPI-7460: The histogram function docs. --- .../agg_functions/parametric_functions.md | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/docs/en/query_language/agg_functions/parametric_functions.md b/docs/en/query_language/agg_functions/parametric_functions.md index c6a9694ed0c..cefc9e6777f 100644 --- a/docs/en/query_language/agg_functions/parametric_functions.md +++ b/docs/en/query_language/agg_functions/parametric_functions.md @@ -2,6 +2,42 @@ Some aggregate functions can accept not only argument columns (used for compression), but a set of parameters – constants for initialization. The syntax is two pairs of brackets instead of one. The first is for parameters, and the second is for arguments. +## histogram + +Calculates a histogram. + +``` +histogram(number_of_bins)(values) +``` + +**Parameters** + +`number_of_bins` — Number of bins for the histogram. +`values` — [Expression](../syntax.md#expressions) resulting in a data sample. + +**Returned values** + +- [Array](../../data_types/array.md) of [Tuples](../../data_types/tuple.md) of the following format: + + ``` + [(lower_1, upper_1, height_1), ... (lower_N, upper_N, height_N)] + ``` + + - `lower` — Lower bound of the bin. + - `upper` — Upper bound of the bin. + - `height` — Calculated height of the bin. + +**Example** + +```sql +SELECT histogram(5)(number + 1) FROM (SELECT * FROM system.numbers LIMIT 20) +``` +```text +┌─histogram(5)(plus(number, 1))───────────────────────────────────────────┐ +│ [(1,4.5,4),(4.5,8.5,4),(8.5,12.75,4.125),(12.75,17,4.625),(17,20,3.25)] │ +└─────────────────────────────────────────────────────────────────────────┘ +``` + ## sequenceMatch(pattern)(time, cond1, cond2, ...) Pattern matching for event chains. From 221ab6a04f32b3be40cf80c123b8b67d5609fd01 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 23 Jul 2019 11:18:09 +0300 Subject: [PATCH 097/509] DOCAPI-7460: Link fix. --- docs/en/query_language/agg_functions/parametric_functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/query_language/agg_functions/parametric_functions.md b/docs/en/query_language/agg_functions/parametric_functions.md index cefc9e6777f..da6052545dc 100644 --- a/docs/en/query_language/agg_functions/parametric_functions.md +++ b/docs/en/query_language/agg_functions/parametric_functions.md @@ -13,7 +13,7 @@ histogram(number_of_bins)(values) **Parameters** `number_of_bins` — Number of bins for the histogram. -`values` — [Expression](../syntax.md#expressions) resulting in a data sample. +`values` — [Expression](../syntax.md#syntax-expressions) resulting in a data sample. **Returned values** From 3c2172b750989b72d9290615aeb669a6b90dc096 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 22 Jul 2019 14:23:11 +0300 Subject: [PATCH 098/509] parse and interpret query --- dbms/src/Core/Settings.h | 3 ++- dbms/src/Interpreters/InterpreterAlterQuery.cpp | 7 +++++++ dbms/src/Parsers/ASTAlterQuery.cpp | 6 ++++++ dbms/src/Parsers/ASTAlterQuery.h | 3 ++- dbms/src/Parsers/ParserAlterQuery.cpp | 17 +++++++++++++++++ dbms/src/Storages/PartitionCommands.cpp | 11 +++++++++++ dbms/src/Storages/PartitionCommands.h | 3 ++- dbms/src/Storages/StorageMergeTree.cpp | 4 ++++ .../src/Storages/StorageReplicatedMergeTree.cpp | 4 ++++ 9 files changed, 55 insertions(+), 3 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index b1182cae9bf..4bb76039cab 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -336,7 +336,8 @@ struct Settings : public SettingsCollection \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ - M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") + M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") \ + M(SettingBool, allow_drop_detached_part, false, "Allow ALTER TABLE ... DROP DETACHED PART ... queries") DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index 8751ff067b1..074fbb7d4c2 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int ILLEGAL_COLUMN; + extern const int SUPPORT_IS_DISABLED; } @@ -53,7 +54,13 @@ BlockIO InterpreterAlterQuery::execute() if (auto alter_command = AlterCommand::parse(command_ast)) alter_commands.emplace_back(std::move(*alter_command)); else if (auto partition_command = PartitionCommand::parse(command_ast)) + { + if (partition_command->type == PartitionCommand::DROP_DETACHED_PARTITION + && !context.getSettingsRef().allow_drop_detached_part) + throw DB::Exception("Cannot execute query: DROP DETACHED PART is disabled " + "(see allow_drop_detached setting)", ErrorCodes::SUPPORT_IS_DISABLED); partition_commands.emplace_back(std::move(*partition_command)); + } else if (auto mut_command = MutationCommand::parse(command_ast)) mutation_commands.emplace_back(std::move(*mut_command)); else diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index c7cd100b415..6d87156a19b 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -118,6 +118,12 @@ void ASTAlterCommand::formatImpl( << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::DROP_DETACHED_PARTITION) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DROP DETACHED" << (part ? " PART " : " PARTITION ") + << (settings.hilite ? hilite_none : ""); + partition->formatImpl(settings, state, frame); + } else if (type == ASTAlterCommand::ATTACH_PARTITION) { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ATTACH " diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index 2c4b3ddbaf1..d6a54812960 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -33,6 +33,7 @@ public: DROP_INDEX, DROP_PARTITION, + DROP_DETACHED_PARTITION, ATTACH_PARTITION, REPLACE_PARTITION, FETCH_PARTITION, @@ -90,7 +91,7 @@ public: bool detach = false; /// true for DETACH PARTITION - bool part = false; /// true for ATTACH PART + bool part = false; /// true for ATTACH PART and DROP DETACHED PART bool clear_column = false; /// for CLEAR COLUMN (do not drop column from metadata) diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 98891bbdf5f..75c6f6291a8 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -35,6 +35,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_attach_partition("ATTACH PARTITION"); ParserKeyword s_detach_partition("DETACH PARTITION"); ParserKeyword s_drop_partition("DROP PARTITION"); + ParserKeyword s_drop_detached_partition("DROP DETACHED PARTITION"); + ParserKeyword s_drop_detached_part("DROP DETACHED PART"); ParserKeyword s_attach_part("ATTACH PART"); ParserKeyword s_fetch_partition("FETCH PARTITION"); ParserKeyword s_replace_partition("REPLACE PARTITION"); @@ -87,6 +89,21 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::DROP_PARTITION; } + else if (s_drop_detached_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + + command->type = ASTAlterCommand::DROP_DETACHED_PARTITION; + } + else if (s_drop_detached_part.ignore(pos, expected)) + { + if (!parser_string_literal.parse(pos, command->partition, expected)) + return false; + + command->type = ASTAlterCommand::DROP_DETACHED_PARTITION; + command->part = true; + } else if (s_drop_column.ignore(pos, expected)) { if (s_if_exists.ignore(pos, expected)) diff --git a/dbms/src/Storages/PartitionCommands.cpp b/dbms/src/Storages/PartitionCommands.cpp index f6aaee4c70e..bab3f6ced24 100644 --- a/dbms/src/Storages/PartitionCommands.cpp +++ b/dbms/src/Storages/PartitionCommands.cpp @@ -23,6 +23,17 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.detach = command_ast->detach; return res; } + else if (command_ast->type == ASTAlterCommand::DROP_DETACHED_PARTITION) + { + if (!command_ast->part) // TODO + throw DB::Exception("Not implemented yet", ErrorCodes::NOT_IMPLEMENTED); + + PartitionCommand res; + res.type = DROP_DETACHED_PARTITION; + res.partition = command_ast->partition; + res.part = command_ast->part; + return res; + } else if (command_ast->type == ASTAlterCommand::ATTACH_PARTITION) { PartitionCommand res; diff --git a/dbms/src/Storages/PartitionCommands.h b/dbms/src/Storages/PartitionCommands.h index 1f66c3f0c30..cb71a02548c 100644 --- a/dbms/src/Storages/PartitionCommands.h +++ b/dbms/src/Storages/PartitionCommands.h @@ -21,6 +21,7 @@ struct PartitionCommand ATTACH_PARTITION, CLEAR_COLUMN, DROP_PARTITION, + DROP_DETACHED_PARTITION, FETCH_PARTITION, FREEZE_ALL_PARTITIONS, FREEZE_PARTITION, @@ -35,7 +36,7 @@ struct PartitionCommand /// true for DETACH PARTITION. bool detach = false; - /// true for ATTACH PART (and false for PARTITION) + /// true for ATTACH PART and DROP DETACHED PART (and false for PARTITION) bool part = false; /// For ATTACH PARTITION partition FROM db.table diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index d021866487c..6e527c0c6c1 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -920,6 +920,10 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma dropPartition(command.partition, command.detach, context); break; + case PartitionCommand::DROP_DETACHED_PARTITION: + // TODO + throw DB::Exception("Not implemented yet", ErrorCodes::NOT_IMPLEMENTED); + case PartitionCommand::ATTACH_PARTITION: attachPartition(command.partition, command.part, context); break; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index b51da168192..5f91c304e98 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3348,6 +3348,10 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part dropPartition(query, command.partition, command.detach, query_context); break; + case PartitionCommand::DROP_DETACHED_PARTITION: + // TODO + throw DB::Exception("Not implemented yet", ErrorCodes::NOT_IMPLEMENTED); + case PartitionCommand::ATTACH_PARTITION: attachPartition(command.partition, command.part, query_context); break; From ad787938f5fa247901da7003c5717fed6a838445 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 23 Jul 2019 22:43:33 +0300 Subject: [PATCH 099/509] better detached part name parsing --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 17 +------------- .../Storages/MergeTree/MergeTreePartInfo.cpp | 22 +++++++++++++++++++ .../Storages/MergeTree/MergeTreePartInfo.h | 6 +++++ .../System/StorageSystemDetachedParts.cpp | 1 + 4 files changed, 30 insertions(+), 16 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index b32470f9f77..6a7b6d5405e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -2579,22 +2579,7 @@ MergeTreeData::getDetachedParts() const res.emplace_back(); auto & part = res.back(); - /// First, try to parse as . - if (MergeTreePartInfo::tryParsePartName(dir_name, &part, format_version)) - continue; - - /// Next, as _. Use entire name as prefix if it fails. - part.prefix = dir_name; - const auto first_separator = dir_name.find_first_of('_'); - if (first_separator == String::npos) - continue; - - const auto part_name = dir_name.substr(first_separator + 1, - dir_name.size() - first_separator - 1); - if (!MergeTreePartInfo::tryParsePartName(part_name, &part, format_version)) - continue; - - part.prefix = dir_name.substr(0, first_separator); + DetachedPartInfo::tryParseDetachedPartName(dir_name, &part, format_version); } return res; } diff --git a/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp b/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp index 19f77448110..732cc3436f4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -188,4 +188,26 @@ String MergeTreePartInfo::getPartNameV0(DayNum left_date, DayNum right_date) con return wb.str(); } +bool DetachedPartInfo::tryParseDetachedPartName(const String & dir_name, DetachedPartInfo * part_info, + MergeTreeDataFormatVersion format_version) +{ + /// First, try to parse as . + if (MergeTreePartInfo::tryParsePartName(dir_name, part_info, format_version)) + return part_info->valid_name = true; + + /// Next, as _. Use entire name as prefix if it fails. + part_info->prefix = dir_name; + const auto first_separator = dir_name.find_first_of('_'); + if (first_separator == String::npos) + return part_info->valid_name = false; + + // TODO what if contains '_'? + const auto part_name = dir_name.substr(first_separator + 1, + dir_name.size() - first_separator - 1); + if (!MergeTreePartInfo::tryParsePartName(part_name, part_info, format_version)) + return part_info->valid_name = false; + + part_info->prefix = dir_name.substr(0, first_separator); + return part_info->valid_name = true; +} } diff --git a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h index e80664c3dd9..2a168086a1c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h @@ -93,6 +93,12 @@ struct MergeTreePartInfo struct DetachedPartInfo : public MergeTreePartInfo { String prefix; + + /// If false, prefix contains full directory name and MergeTreePartInfo may be in invalid state + /// (directory name was not successfully parsed). + bool valid_name; + + static bool tryParseDetachedPartName(const String & dir_name, DetachedPartInfo * part_info, MergeTreeDataFormatVersion format_version); }; } diff --git a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp index 9ae6f7b607a..9f33a60b84a 100644 --- a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp @@ -28,6 +28,7 @@ public: protected: explicit StorageSystemDetachedParts() { + // TODO add column "directory_name" or "is_valid_name" setColumns(ColumnsDescription{{ {"database", std::make_shared()}, {"table", std::make_shared()}, From 6e4aabbb1a4074d1a979190b063ffbe959894ebc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 23 Jul 2019 23:18:18 +0300 Subject: [PATCH 100/509] draft for StorageMergeTree --- dbms/src/Storages/StorageMergeTree.cpp | 33 ++++++++++++++++++++++++-- dbms/src/Storages/StorageMergeTree.h | 1 + 2 files changed, 32 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 6e527c0c6c1..bd09588981b 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -921,8 +921,8 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma break; case PartitionCommand::DROP_DETACHED_PARTITION: - // TODO - throw DB::Exception("Not implemented yet", ErrorCodes::NOT_IMPLEMENTED); + dropDetached(command.partition, command.part, context); + break; case PartitionCommand::ATTACH_PARTITION: attachPartition(command.partition, command.part, context); @@ -993,6 +993,34 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons } +void StorageMergeTree::dropDetached(const ASTPtr & partition, bool part, const Context & /*context*/) +{ + if (!part) // TODO + throw DB::Exception("DROP DETACHED PARTITION is not implemented, use DROP DETACHED PART", ErrorCodes::NOT_IMPLEMENTED); + + String part_id = partition->as().value.safeGet(); + Poco::Path part_path(part_id); + const bool file_zero_depth = part_path.isFile() && part_path.depth() == 0 && part_path.getFileName() != ".."; + const bool dir_zero_depth = part_path.isDirectory() && part_path.depth() == 1 && part_path.directory(0) != ".."; + const bool zero_depth = file_zero_depth || dir_zero_depth; + if (!part_path.isRelative() || !zero_depth) + throw DB::Exception("Part name must contain exactly one path component: name of detached part", ErrorCodes::INCORRECT_FILE_NAME); + + part_id = part_path.isFile() ? part_path.getFileName() : part_path.directory(0); + Poco::Path base_dir(full_path + "detached"); + Poco::File detached_part_dir(Poco::Path(base_dir, part_id)); + if (!detached_part_dir.exists()) + throw DB::Exception("Detached part \"" + part_id + "\" not found" , ErrorCodes::INCORRECT_FILE_NAME); + + DetachedPartInfo info; + DetachedPartInfo::tryParseDetachedPartName(part_id, &info, format_version); + MergeTreeDataPart detached_part(*this, part_id, info); + detached_part.relative_path = "detached/" + part_id; + + // TODO make sure it's ok + detached_part.remove(); +} + void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context) { // TODO: should get some locks to prevent race with 'alter … modify column' @@ -1039,6 +1067,7 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par LOG_DEBUG(log, "Checking data"); MutableDataPartPtr part = loadPartAndFixMetadata(source_path); + // TODO fix race with DROP DETACHED PARTITION LOG_INFO(log, "Attaching part " << source_part_name << " from " << source_path); renameTempPartAndAdd(part, &increment); diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 0de9618d915..fa2561e4ab2 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -120,6 +120,7 @@ private: // Partition helpers void dropPartition(const ASTPtr & partition, bool detach, const Context & context); + void dropDetached(const ASTPtr & partition, bool part, const Context & context); void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context); void attachPartition(const ASTPtr & partition, bool part, const Context & context); void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context); From 8a00ce5ff1c413cfa22185795682ccf084cdb533 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Jul 2019 15:56:39 +0300 Subject: [PATCH 101/509] Intermediate step --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Parsers/ASTAlterQuery.cpp | 10 +++++ dbms/src/Parsers/ASTAlterQuery.h | 6 ++- dbms/src/Storages/AlterCommands.cpp | 42 +++++++++++++++---- dbms/src/Storages/AlterCommands.h | 16 ++++--- dbms/src/Storages/IStorage.cpp | 8 +++- dbms/src/Storages/IStorage.h | 5 ++- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 11 ++++- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 +- dbms/src/Storages/StorageBuffer.cpp | 2 +- dbms/src/Storages/StorageDistributed.cpp | 2 +- dbms/src/Storages/StorageMerge.cpp | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 29 +++++++++++-- dbms/src/Storages/StorageMergeTree.h | 3 ++ dbms/src/Storages/StorageNull.cpp | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 6 ++- 16 files changed, 121 insertions(+), 26 deletions(-) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index c472a336d73..d0a5edab1d3 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -437,6 +437,7 @@ namespace ErrorCodes extern const int CANNOT_CREATE_TIMER = 460; extern const int CANNOT_SET_TIMER_PERIOD = 461; extern const int CANNOT_DELETE_TIMER = 462; + extern const int SETTINGS_ARE_NOT_SUPPORTED = 463; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index c7cd100b415..52aa95d65e5 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -45,6 +45,11 @@ ASTPtr ASTAlterCommand::clone() const res->ttl = ttl->clone(); res->children.push_back(res->ttl); } + if (settings_changes) + { + res->settings_changes = settings_changes->clone(); + res->children.push_back(res->settings_changes); + } return res; } @@ -184,6 +189,11 @@ void ASTAlterCommand::formatImpl( settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY TTL " << (settings.hilite ? hilite_none : ""); ttl->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::MODIFY_SETTING) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY SETTING " << (settings.hilite ? hilite_none : ""); + settings_changes->formatImpl(settings, state, frame); + } else throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE); } diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index 2c4b3ddbaf1..96ec8d91255 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -28,6 +28,7 @@ public: COMMENT_COLUMN, MODIFY_ORDER_BY, MODIFY_TTL, + MODIFY_SETTING, ADD_INDEX, DROP_INDEX, @@ -69,7 +70,7 @@ public: /** The ADD INDEX query stores the name of the index following AFTER. * The DROP INDEX query stores the name for deletion. */ - ASTPtr index; + ASTPtr index; /** Used in DROP PARTITION and ATTACH PARTITION FROM queries. * The value or ID of the partition is stored here. @@ -88,6 +89,9 @@ public: /// For MODIFY TTL query ASTPtr ttl; + /// FOR MODIFY_SETTING + ASTPtr settings_changes; + bool detach = false; /// true for DETACH PARTITION bool part = false; /// true for ATTACH PART diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 7814f1a6ba0..4b733f86d22 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -28,6 +29,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; + extern const int UNKNOWN_SETTING; } @@ -172,13 +174,20 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.ttl = command_ast->ttl; return command; } + else if (command_ast->type == ASTAlterCommand::MODIFY_SETTING) + { + AlterCommand command; + command.type = AlterCommand::MODIFY_SETTING; + command.settings_changes = command_ast->settings_changes->as().changes; + return command; + } else return {}; } void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, - ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const + ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast, SettingsChanges & changes) const { if (type == ADD_COLUMN) { @@ -306,38 +315,43 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri { ttl_table_ast = ttl; } + else if (type == MODIFY_SETTING) + { + changes.insert(changes.begin(), settings_changes.begin(), settings_changes.end()); + } else throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR); } bool AlterCommand::isMutable() const { - if (type == COMMENT_COLUMN) + if (type == COMMENT_COLUMN || type == MODIFY_SETTING) return false; if (type == MODIFY_COLUMN) return data_type.get() || default_expression; - // TODO: возможно, здесь нужно дополнить return true; } void AlterCommands::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, - ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const + ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast, SettingsChanges & changes) const { auto new_columns_description = columns_description; auto new_indices_description = indices_description; auto new_order_by_ast = order_by_ast; auto new_primary_key_ast = primary_key_ast; auto new_ttl_table_ast = ttl_table_ast; + auto new_changes = changes; for (const AlterCommand & command : *this) if (!command.ignore) - command.apply(new_columns_description, new_indices_description, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast); + command.apply(new_columns_description, new_indices_description, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, changes); columns_description = std::move(new_columns_description); indices_description = std::move(new_indices_description); order_by_ast = std::move(new_order_by_ast); primary_key_ast = std::move(new_primary_key_ast); ttl_table_ast = std::move(new_ttl_table_ast); + changes = std::move(new_changes); } void AlterCommands::validate(const IStorage & table, const Context & context) @@ -453,6 +467,16 @@ void AlterCommands::validate(const IStorage & table, const Context & context) throw Exception{"Wrong column name. Cannot find column " + command.column_name + " to comment", ErrorCodes::ILLEGAL_COLUMN}; } } + else if (command.type == AlterCommand::MODIFY_SETTING) + { + for (const auto & change : command.settings_changes) + { + if (!table.hasSetting(change.name)) + { + throw Exception{"Storage '" + table.getName() + "' doesn't have setting '" + change.name + "'", ErrorCodes::UNKNOWN_SETTING}; + } + } + } } /** Existing defaulted columns may require default expression extensions with a type conversion, @@ -518,14 +542,15 @@ void AlterCommands::validate(const IStorage & table, const Context & context) } } -void AlterCommands::apply(ColumnsDescription & columns_description) const +void AlterCommands::applyForColumnsOnly(ColumnsDescription & columns_description) const { auto out_columns_description = columns_description; IndicesDescription indices_description; ASTPtr out_order_by; ASTPtr out_primary_key; ASTPtr out_ttl_table; - apply(out_columns_description, indices_description, out_order_by, out_primary_key, out_ttl_table); + SettingsChanges out_changes; + apply(out_columns_description, indices_description, out_order_by, out_primary_key, out_ttl_table, out_changes); if (out_order_by) throw Exception("Storage doesn't support modifying ORDER BY expression", ErrorCodes::NOT_IMPLEMENTED); @@ -535,6 +560,9 @@ void AlterCommands::apply(ColumnsDescription & columns_description) const throw Exception("Storage doesn't support modifying indices", ErrorCodes::NOT_IMPLEMENTED); if (out_ttl_table) throw Exception("Storage doesn't support modifying TTL expression", ErrorCodes::NOT_IMPLEMENTED); + if (!out_changes.empty()) + throw Exception("Storage doesn't support modifying settings", ErrorCodes::NOT_IMPLEMENTED); + columns_description = std::move(out_columns_description); } diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index 4905b80f92f..6daafe5cd7f 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -27,6 +28,7 @@ struct AlterCommand DROP_INDEX, MODIFY_TTL, UKNOWN_TYPE, + MODIFY_SETTING, }; Type type = UKNOWN_TYPE; @@ -71,6 +73,9 @@ struct AlterCommand /// For ADD and MODIFY CompressionCodecPtr codec; + /// For MODIFY SETTING + SettingsChanges settings_changes; + AlterCommand() = default; AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type, const ColumnDefaultKind default_kind, const ASTPtr & default_expression, @@ -84,7 +89,7 @@ struct AlterCommand static std::optional parse(const ASTAlterCommand * command); void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, - ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const; + ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast, SettingsChanges & changes) const; /// Checks that not only metadata touched by that command bool isMutable() const; @@ -95,11 +100,12 @@ class Context; class AlterCommands : public std::vector { public: - void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, ASTPtr & order_by_ast, - ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const; - /// For storages that don't support MODIFY_ORDER_BY. - void apply(ColumnsDescription & columns_description) const; + void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, ASTPtr & order_by_ast, + ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast, SettingsChanges & changes) const; + + /// Used for primiteive table engines, where only columns metadata can be changed + void applyForColumnsOnly(ColumnsDescription & columns_description) const; void validate(const IStorage & table, const Context & context); bool isMutable() const; diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 687ca970311..0fd1994f5ce 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -18,6 +18,7 @@ namespace ErrorCodes extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int NOT_FOUND_COLUMN_IN_BLOCK; extern const int TYPE_MISMATCH; + extern const int SETTINGS_ARE_NOT_SUPPORTED; } IStorage::IStorage(ColumnsDescription columns_) @@ -292,6 +293,11 @@ bool IStorage::isVirtualColumn(const String & column_name) const return getColumns().get(column_name).is_virtual; } +bool IStorage::hasSetting(const String & /* setting_name */) const +{ + throw Exception("Storage '" + getName() + "' doesn't support settings.", ErrorCodes::SETTINGS_ARE_NOT_SUPPORTED); +} + TableStructureReadLockHolder IStorage::lockStructureForShare(bool will_add_new_data, const String & query_id) { TableStructureReadLockHolder result; @@ -362,7 +368,7 @@ void IStorage::alter( lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); auto new_columns = getColumns(); auto new_indices = getIndices(); - params.apply(new_columns); + params.applyForColumnsOnly(new_columns); context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, {}); setColumns(std::move(new_columns)); } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 477c4456b17..f269dce220f 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -12,6 +12,7 @@ #include #include #include +#include #include #include @@ -129,6 +130,9 @@ public: /// thread-unsafe part. lockStructure must be acquired /// If |need_all| is set, then checks that all the columns of the table are in the block. void check(const Block & block, bool need_all = false) const; + /// Check storage has setting + virtual bool hasSetting(const String & setting_name) const; + protected: /// still thread-unsafe part. void setColumns(ColumnsDescription columns_); /// sets only real columns, possibly overwrites virtual ones. void setIndices(IndicesDescription indices_); @@ -136,7 +140,6 @@ protected: /// still thread-unsafe part. /// Returns whether the column is virtual - by default all columns are real. /// Initially reserved virtual column name may be shadowed by real column. virtual bool isVirtualColumn(const String & column_name) const; - private: ColumnsDescription columns; /// combined real and virtual columns const ColumnsDescription virtuals = {}; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index b32470f9f77..25e4246a414 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -87,6 +87,7 @@ namespace ErrorCodes extern const int CANNOT_MUNMAP; extern const int CANNOT_MREMAP; extern const int BAD_TTL_EXPRESSION; + extern const int UNKNOWN_SETTING; } @@ -1174,7 +1175,8 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c ASTPtr new_order_by_ast = order_by_ast; ASTPtr new_primary_key_ast = primary_key_ast; ASTPtr new_ttl_table_ast = ttl_table_ast; - commands.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast); + SettingsChanges new_changes; + commands.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes); if (getIndices().empty() && !new_indices.empty() && !context.getSettingsRef().allow_experimental_data_skipping_indices) @@ -1262,6 +1264,13 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast, /* only_check = */ true); + for (const auto & setting : new_changes) + { + if (!hasSetting(setting.name)) + throw Exception{"Storage '" + getName() + "' doesn't have setting '" + setting.name + "'", ErrorCodes::UNKNOWN_SETTING}; + + } + /// Check that type conversions are possible. ExpressionActionsPtr unused_expression; NameToNameMap unused_map; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 29962382749..33200500667 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -629,7 +629,7 @@ public: String sampling_expr_column_name; Names columns_required_for_sampling; - const MergeTreeSettings settings; + MergeTreeSettings settings; /// Limiting parallel sends per one table, used in DataPartsExchange std::atomic_uint current_table_sends {0}; diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 3c334b2a48b..6a857675018 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -701,7 +701,7 @@ void StorageBuffer::alter(const AlterCommands & params, const String & database_ auto new_columns = getColumns(); auto new_indices = getIndices(); - params.apply(new_columns); + params.applyForColumnsOnly(new_columns); context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, {}); setColumns(std::move(new_columns)); } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 27ceb1f45db..e8aed6d5702 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -350,7 +350,7 @@ void StorageDistributed::alter( auto new_columns = getColumns(); auto new_indices = getIndices(); - params.apply(new_columns); + params.applyForColumnsOnly(new_columns); context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {}); setColumns(std::move(new_columns)); } diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 4c029fab677..a0454185307 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -401,7 +401,7 @@ void StorageMerge::alter( auto new_columns = getColumns(); auto new_indices = getIndices(); - params.apply(new_columns); + params.applyForColumnsOnly(new_columns); context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, {}); setColumns(new_columns); } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 0536423101d..d8360cb61e4 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -36,6 +37,7 @@ namespace ErrorCodes extern const int INCORRECT_FILE_NAME; extern const int CANNOT_ASSIGN_OPTIMIZE; extern const int INCOMPATIBLE_COLUMNS; + extern const int UNKNOWN_SETTING; } namespace ActionLocks @@ -245,9 +247,21 @@ void StorageMergeTree::alter( lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); auto new_columns = getColumns(); auto new_indices = getIndices(); - params.apply(new_columns); + ASTPtr new_order_by_ast = order_by_ast; + ASTPtr new_primary_key_ast = primary_key_ast; + ASTPtr new_ttl_table_ast = ttl_table_ast; + SettingsChanges new_changes; + params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes); + IDatabase::ASTModifier storage_modifier = [&] (IAST & ast) + { + auto & storage_ast = ast.as(); + if (!new_changes.empty()) + storage_ast.settings->changes.insert(storage_ast.settings->changes.begin(), new_changes.begin(), new_changes.end()); + }; + context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {}); setColumns(std::move(new_columns)); + settings.applyChanges(new_changes); return; } @@ -263,7 +277,8 @@ void StorageMergeTree::alter( ASTPtr new_order_by_ast = order_by_ast; ASTPtr new_primary_key_ast = primary_key_ast; ASTPtr new_ttl_table_ast = ttl_table_ast; - params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast); + SettingsChanges new_changes; + params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes); auto transactions = prepareAlterTransactions(new_columns, new_indices, context); @@ -844,7 +859,9 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi ASTPtr ignored_order_by_ast; ASTPtr ignored_primary_key_ast; ASTPtr ignored_ttl_table_ast; - alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast); + SettingsChanges ignore_settings_changes; + + alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast, ignore_settings_changes); auto columns_for_parts = new_columns.getAllPhysical(); for (const auto & part : parts) @@ -1009,6 +1026,12 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons } + +bool StorageMergeTree::hasSetting(const String & setting_name) const +{ + return settings.findIndex(setting_name) != MergeTreeSettings::npos; +} + void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context) { // TODO: should get some locks to prevent race with 'alter … modify column' diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 0de9618d915..e2d24ceecdd 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -128,7 +128,10 @@ private: friend class MergeTreeData; friend struct CurrentlyMergingPartsTagger; + bool hasSetting(const String & setting_name) const override; + protected: + /** Attach the table with the appropriate name, along the appropriate path (with / at the end), * (correctness of names and paths are not checked) * consisting of the specified columns. diff --git a/dbms/src/Storages/StorageNull.cpp b/dbms/src/Storages/StorageNull.cpp index f6de6e87e37..f5b463ff972 100644 --- a/dbms/src/Storages/StorageNull.cpp +++ b/dbms/src/Storages/StorageNull.cpp @@ -38,7 +38,7 @@ void StorageNull::alter( ColumnsDescription new_columns = getColumns(); IndicesDescription new_indices = getIndices(); - params.apply(new_columns); + params.applyForColumnsOnly(new_columns); context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {}); setColumns(std::move(new_columns)); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 934d651fead..87ab8815c57 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -1510,7 +1510,8 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & ASTPtr ignored_order_by_ast; ASTPtr ignored_primary_key_ast; ASTPtr ignored_ttl_table_ast; - alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast); + SettingsChanges ignored_changes; + alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast, ignored_changes); size_t modified_parts = 0; auto parts = getDataParts(); @@ -3119,7 +3120,8 @@ void StorageReplicatedMergeTree::alter( ASTPtr new_order_by_ast = order_by_ast; ASTPtr new_primary_key_ast = primary_key_ast; ASTPtr new_ttl_table_ast = ttl_table_ast; - params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast); + SettingsChanges new_changes; + params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes); String new_columns_str = new_columns.toString(); if (new_columns_str != getColumns().toString()) From f7e0d17490ddace60010895d0609e01ff9717dd9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Jul 2019 18:22:16 +0300 Subject: [PATCH 102/509] Fix some bugs --- dbms/src/Parsers/ParserAlterQuery.cpp | 9 +++++++++ dbms/src/Parsers/ParserAlterQuery.h | 1 + dbms/src/Storages/AlterCommands.cpp | 4 ++-- dbms/src/Storages/StorageMergeTree.cpp | 4 ++-- 4 files changed, 14 insertions(+), 4 deletions(-) diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 98891bbdf5f..b0ff5bf6c8c 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -28,6 +29,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_comment_column("COMMENT COLUMN"); ParserKeyword s_modify_order_by("MODIFY ORDER BY"); ParserKeyword s_modify_ttl("MODIFY TTL"); + ParserKeyword s_modify_setting("MODIFY SETTING"); ParserKeyword s_add_index("ADD INDEX"); ParserKeyword s_drop_index("DROP INDEX"); @@ -63,6 +65,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserList parser_assignment_list( std::make_unique(), std::make_unique(TokenType::Comma), /* allow_empty = */ false); + ParserSetQuery parser_settings(true); if (s_add_column.ignore(pos, expected)) { @@ -289,6 +292,12 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; command->type = ASTAlterCommand::MODIFY_TTL; } + else if (s_modify_setting.ignore(pos, expected)) + { + if (!parser_settings.parse(pos, command->settings_changes, expected)) + return false; + command->type = ASTAlterCommand::MODIFY_SETTING; + } else return false; diff --git a/dbms/src/Parsers/ParserAlterQuery.h b/dbms/src/Parsers/ParserAlterQuery.h index 282a4277e17..49d10eac34b 100644 --- a/dbms/src/Parsers/ParserAlterQuery.h +++ b/dbms/src/Parsers/ParserAlterQuery.h @@ -13,6 +13,7 @@ namespace DB * [CLEAR COLUMN [IF EXISTS] col_to_clear [IN PARTITION partition],] * [MODIFY COLUMN [IF EXISTS] col_to_modify type, ...] * [MODIFY PRIMARY KEY (a, b, c...)] + * [MODIFY SETTING setting_name=setting_value, ...] * [COMMENT COLUMN [IF EXISTS] col_name string] * [DROP|DETACH|ATTACH PARTITION|PART partition, ...] * [FETCH PARTITION partition FROM ...] diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 4b733f86d22..789b66b271a 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -317,7 +317,7 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri } else if (type == MODIFY_SETTING) { - changes.insert(changes.begin(), settings_changes.begin(), settings_changes.end()); + changes.insert(changes.end(), settings_changes.begin(), settings_changes.end()); } else throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR); @@ -344,7 +344,7 @@ void AlterCommands::apply(ColumnsDescription & columns_description, IndicesDescr for (const AlterCommand & command : *this) if (!command.ignore) - command.apply(new_columns_description, new_indices_description, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, changes); + command.apply(new_columns_description, new_indices_description, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes); columns_description = std::move(new_columns_description); indices_description = std::move(new_indices_description); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index d8360cb61e4..f39735b1595 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -256,10 +256,10 @@ void StorageMergeTree::alter( { auto & storage_ast = ast.as(); if (!new_changes.empty()) - storage_ast.settings->changes.insert(storage_ast.settings->changes.begin(), new_changes.begin(), new_changes.end()); + storage_ast.settings->changes.insert(storage_ast.settings->changes.end(), new_changes.begin(), new_changes.end()); }; - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {}); + context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, storage_modifier); setColumns(std::move(new_columns)); settings.applyChanges(new_changes); return; From 856c8ef0e8dedf6a0166673c2a1b292b4648c655 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 24 Jul 2019 18:53:41 +0300 Subject: [PATCH 103/509] test for bug in ATTACH PART --- .../0_stateless/00974_attach_active_part.reference | 5 +++++ .../0_stateless/00974_attach_active_part.sh | 14 ++++++++++++++ 2 files changed, 19 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00974_attach_active_part.reference create mode 100755 dbms/tests/queries/0_stateless/00974_attach_active_part.sh diff --git a/dbms/tests/queries/0_stateless/00974_attach_active_part.reference b/dbms/tests/queries/0_stateless/00974_attach_active_part.reference new file mode 100644 index 00000000000..3a90499810c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_attach_active_part.reference @@ -0,0 +1,5 @@ +0_1_1_0 +1_2_2_0 +2_3_3_0 +3_4_4_0 +16 \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/00974_attach_active_part.sh b/dbms/tests/queries/0_stateless/00974_attach_active_part.sh new file mode 100755 index 00000000000..a3b2505f197 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_attach_active_part.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_bug"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE attach_bug (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n"; +$CLICKHOUSE_CLIENT --query="INSERT INTO attach_bug SELECT number FROM system.numbers LIMIT 16"; +$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_bug ATTACH PART '../1_2_2_0'" 2> /dev/null; # | grep "" +$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_bug' ORDER BY name FORMAT TSV"; +$CLICKHOUSE_CLIENT --query="SElECT count() FROM attach_bug FORMAT TSV"; # will fail +$CLICKHOUSE_CLIENT --query="DROP TABLE attach_bug"; + + From 1805ab5736ec6c934af98dbda8c50a00fb6bb165 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 25 Jul 2019 13:46:07 +0300 Subject: [PATCH 104/509] attach --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 8 ++- dbms/src/Storages/MergeTree/MergeTreeData.h | 1 + .../Storages/MergeTree/MergeTreePartInfo.cpp | 9 +++ .../Storages/MergeTree/MergeTreePartInfo.h | 2 + dbms/src/Storages/StorageMergeTree.cpp | 63 +++++++++++++------ dbms/src/Storages/StorageMergeTree.h | 1 + 6 files changed, 62 insertions(+), 22 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 6a7b6d5405e..94e42c34d0f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -2336,6 +2336,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const St { MutableDataPartPtr part = std::make_shared(*this, Poco::Path(relative_path).getFileName()); part->relative_path = relative_path; + loadPartAndFixMetadata(part); + return part; +} + +void MergeTreeData::loadPartAndFixMetadata(MutableDataPartPtr part) +{ String full_part_path = part->getFullPath(); /// Earlier the list of columns was written incorrectly. Delete it and re-create. @@ -2357,8 +2363,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const St Poco::File(full_part_path + "checksums.txt.tmp").renameTo(full_part_path + "checksums.txt"); } - - return part; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 29962382749..cec3651652b 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -533,6 +533,7 @@ public: /// Check that the part is not broken and calculate the checksums for it if they are not present. MutableDataPartPtr loadPartAndFixMetadata(const String & relative_path); + void loadPartAndFixMetadata(MutableDataPartPtr part); /** Create local backup (snapshot) for parts with specified prefix. * Backup is created in directory clickhouse_dir/shadow/i/, where i - incremental number, diff --git a/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp b/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp index 732cc3436f4..45a0e1d488c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -210,4 +210,13 @@ bool DetachedPartInfo::tryParseDetachedPartName(const String & dir_name, Detache part_info->prefix = dir_name.substr(0, first_separator); return part_info->valid_name = true; } + +String DetachedPartInfo::fullDirName() const +{ + if (!valid_name) + return prefix; + if (prefix.empty()) + return getPartName(); + return prefix + "_" + fullDirName(); +} } diff --git a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h index 2a168086a1c..7d0fb446ee3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h @@ -98,6 +98,8 @@ struct DetachedPartInfo : public MergeTreePartInfo /// (directory name was not successfully parsed). bool valid_name; + String fullDirName() const; + static bool tryParseDetachedPartName(const String & dir_name, DetachedPartInfo * part_info, MergeTreeDataFormatVersion format_version); }; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index bd09588981b..2f437e6d46b 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -36,6 +36,7 @@ namespace ErrorCodes extern const int INCORRECT_FILE_NAME; extern const int CANNOT_ASSIGN_OPTIMIZE; extern const int INCOMPATIBLE_COLUMNS; + extern const int BAD_DATA_PART_NAME; } namespace ActionLocks @@ -999,25 +1000,13 @@ void StorageMergeTree::dropDetached(const ASTPtr & partition, bool part, const C throw DB::Exception("DROP DETACHED PARTITION is not implemented, use DROP DETACHED PART", ErrorCodes::NOT_IMPLEMENTED); String part_id = partition->as().value.safeGet(); - Poco::Path part_path(part_id); - const bool file_zero_depth = part_path.isFile() && part_path.depth() == 0 && part_path.getFileName() != ".."; - const bool dir_zero_depth = part_path.isDirectory() && part_path.depth() == 1 && part_path.directory(0) != ".."; - const bool zero_depth = file_zero_depth || dir_zero_depth; - if (!part_path.isRelative() || !zero_depth) - throw DB::Exception("Part name must contain exactly one path component: name of detached part", ErrorCodes::INCORRECT_FILE_NAME); - - part_id = part_path.isFile() ? part_path.getFileName() : part_path.directory(0); - Poco::Path base_dir(full_path + "detached"); - Poco::File detached_part_dir(Poco::Path(base_dir, part_id)); - if (!detached_part_dir.exists()) - throw DB::Exception("Detached part \"" + part_id + "\" not found" , ErrorCodes::INCORRECT_FILE_NAME); + validateDetachedPartName(part_id); DetachedPartInfo info; DetachedPartInfo::tryParseDetachedPartName(part_id, &info, format_version); MergeTreeDataPart detached_part(*this, part_id, info); detached_part.relative_path = "detached/" + part_id; - // TODO make sure it's ok detached_part.remove(); } @@ -1038,6 +1027,7 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par Strings parts; if (attach_part) { + validateDetachedPartName(partition_id); parts.push_back(partition_id); } else @@ -1048,6 +1038,7 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par { const String & name = it.name(); MergeTreePartInfo part_info; + /// Parts with prefix in name (e.g. attaching_1_3_3_0, delete_tmp_1_3_3_0) will be ignored if (!MergeTreePartInfo::tryParsePartName(name, &part_info, format_version) || part_info.partition_id != partition_id) { @@ -1062,16 +1053,38 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par for (const auto & source_part_name : parts) { - String source_path = source_dir + source_part_name; + MutableDataPartPtr part; + try + { + part = std::make_shared(*this, source_part_name); + part->relative_path = "detached/" + source_part_name; + part->renameTo("detached/attaching_" + source_part_name, false); - LOG_DEBUG(log, "Checking data"); - MutableDataPartPtr part = loadPartAndFixMetadata(source_path); + LOG_DEBUG(log, "Checking data in " << part->relative_path); + loadPartAndFixMetadata(part); - // TODO fix race with DROP DETACHED PARTITION - LOG_INFO(log, "Attaching part " << source_part_name << " from " << source_path); - renameTempPartAndAdd(part, &increment); + LOG_INFO(log, "Attaching part " << source_part_name << " from " << part->relative_path); + renameTempPartAndAdd(part, &increment); - LOG_INFO(log, "Finished attaching part"); + LOG_INFO(log, "Finished attaching part"); + } + catch (...) + { + tryLogCurrentException(log, String(__PRETTY_FUNCTION__) + ": cannot attach part " + source_part_name); + + if (part->relative_path == "detached/attaching_" + source_part_name) + { + try + { + part->renameTo("detached/" + source_part_name, false); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + } + + } } /// New parts with other data may appear in place of deleted parts. @@ -1150,6 +1163,16 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con } } +void StorageMergeTree::validateDetachedPartName(const String & name) const +{ + if (name.find('/') != std::string::npos || name == "." || name == "..") + throw DB::Exception("Invalid part name", ErrorCodes::INCORRECT_FILE_NAME); + + Poco::File detached_part_dir(full_path + "detached/" + name); + if (!detached_part_dir.exists()) + throw DB::Exception("Detached part \"" + name + "\" not found" , ErrorCodes::BAD_DATA_PART_NAME); +} + ActionLock StorageMergeTree::getActionLock(StorageActionBlockType action_type) { if (action_type == ActionLocks::PartsMerge) diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index fa2561e4ab2..42061894a8e 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -124,6 +124,7 @@ private: void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context); void attachPartition(const ASTPtr & partition, bool part, const Context & context); void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context); + void validateDetachedPartName(const String & name) const; friend class MergeTreeBlockOutputStream; friend class MergeTreeData; From 6ac950c6dc710fed7e1daf0a4bbdb09d3c0feb86 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 25 Jul 2019 14:09:01 +0300 Subject: [PATCH 105/509] test attach active part fails --- .../queries/0_stateless/00974_attach_active_part.reference | 3 ++- dbms/tests/queries/0_stateless/00974_attach_active_part.sh | 6 ++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00974_attach_active_part.reference b/dbms/tests/queries/0_stateless/00974_attach_active_part.reference index 3a90499810c..fc0fce0a541 100644 --- a/dbms/tests/queries/0_stateless/00974_attach_active_part.reference +++ b/dbms/tests/queries/0_stateless/00974_attach_active_part.reference @@ -1,5 +1,6 @@ +OK 0_1_1_0 1_2_2_0 2_3_3_0 3_4_4_0 -16 \ No newline at end of file +16 diff --git a/dbms/tests/queries/0_stateless/00974_attach_active_part.sh b/dbms/tests/queries/0_stateless/00974_attach_active_part.sh index a3b2505f197..32e2b21608f 100755 --- a/dbms/tests/queries/0_stateless/00974_attach_active_part.sh +++ b/dbms/tests/queries/0_stateless/00974_attach_active_part.sh @@ -6,9 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_bug"; $CLICKHOUSE_CLIENT --query="CREATE TABLE attach_bug (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n"; $CLICKHOUSE_CLIENT --query="INSERT INTO attach_bug SELECT number FROM system.numbers LIMIT 16"; -$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_bug ATTACH PART '../1_2_2_0'" 2> /dev/null; # | grep "" +$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_bug ATTACH PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK' $CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_bug' ORDER BY name FORMAT TSV"; -$CLICKHOUSE_CLIENT --query="SElECT count() FROM attach_bug FORMAT TSV"; # will fail +$CLICKHOUSE_CLIENT --query="SElECT count() FROM attach_bug FORMAT TSV"; $CLICKHOUSE_CLIENT --query="DROP TABLE attach_bug"; - - From e4212bfe593f3e28b42603c4dfeffcb2f60702ce Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 25 Jul 2019 14:57:16 +0300 Subject: [PATCH 106/509] add full part name to detached_parts --- dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp | 2 +- dbms/src/Storages/System/StorageSystemDetachedParts.cpp | 9 +++++---- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp b/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp index 45a0e1d488c..de7150e4cea 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -217,6 +217,6 @@ String DetachedPartInfo::fullDirName() const return prefix; if (prefix.empty()) return getPartName(); - return prefix + "_" + fullDirName(); + return prefix + "_" + getPartName(); } } diff --git a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp index 9f33a60b84a..9b32f1fb29b 100644 --- a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp @@ -28,7 +28,6 @@ public: protected: explicit StorageSystemDetachedParts() { - // TODO add column "directory_name" or "is_valid_name" setColumns(ColumnsDescription{{ {"database", std::make_shared()}, {"table", std::make_shared()}, @@ -37,7 +36,8 @@ protected: {"reason", std::make_shared()}, {"min_block_number", std::make_shared()}, {"max_block_number", std::make_shared()}, - {"level", std::make_shared()} + {"level", std::make_shared()}, + {"directory_name", std::make_shared()} }}); } @@ -63,12 +63,13 @@ protected: int i = 0; columns[i++]->insert(info.database); columns[i++]->insert(info.table); - columns[i++]->insert(p.partition_id); - columns[i++]->insert(p.getPartName()); + columns[i++]->insert(p.valid_name ? p.partition_id : ""); + columns[i++]->insert(p.valid_name ? p.getPartName() : ""); columns[i++]->insert(p.prefix); columns[i++]->insert(p.min_block); columns[i++]->insert(p.max_block); columns[i++]->insert(p.level); + columns[i++]->insert(p.fullDirName()); } } From 3ba26aba43e30aa92fed80fb6f842dc3eb064ef1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 25 Jul 2019 19:28:08 +0300 Subject: [PATCH 107/509] tests --- .../Storages/MergeTree/MergeTreeDataPart.cpp | 9 +++- .../Storages/MergeTree/MergeTreeDataPart.h | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 6 +-- .../00974_attach_active_part.reference | 6 --- .../0_stateless/00974_attach_active_part.sh | 12 ------ .../00974_attach_invalid_parts.reference | 17 ++++++++ .../0_stateless/00974_attach_invalid_parts.sh | 41 +++++++++++++++++++ .../0_stateless/00975_drop_detached.reference | 2 + .../0_stateless/00975_drop_detached.sh | 30 ++++++++++++++ 9 files changed, 102 insertions(+), 23 deletions(-) delete mode 100644 dbms/tests/queries/0_stateless/00974_attach_active_part.reference delete mode 100755 dbms/tests/queries/0_stateless/00974_attach_active_part.sh create mode 100644 dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference create mode 100755 dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh create mode 100644 dbms/tests/queries/0_stateless/00975_drop_detached.reference create mode 100755 dbms/tests/queries/0_stateless/00975_drop_detached.sh diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp index 7b8be970e1d..865aaf80ed1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp @@ -347,7 +347,7 @@ UInt64 MergeTreeDataPart::calculateTotalSizeOnDisk(const String & from) return res; } -void MergeTreeDataPart::remove() const +void MergeTreeDataPart::remove(bool force_recursive /*= false*/) const { if (relative_path.empty()) throw Exception("Part relative_path cannot be empty. This is bug.", ErrorCodes::LOGICAL_ERROR); @@ -398,6 +398,13 @@ void MergeTreeDataPart::remove() const return; } + if (force_recursive) + { + /// Part is not loaded (we don't know which files are there), so remove dir recursively. + to_dir.remove(true); + return; + } + try { /// Remove each expected file in directory, then remove directory itself. diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h index f41ea8af424..98af00c071a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h @@ -241,7 +241,7 @@ struct MergeTreeDataPart /// Calculate the total size of the entire directory with all the files static UInt64 calculateTotalSizeOnDisk(const String & from); - void remove() const; + void remove(bool force_recursive = false) const; /// Makes checks and move part to new directory /// Changes only relative_dir_name, you need to update other metadata (name, is_temp) explicitly diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 2f437e6d46b..0c1503347c0 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -1007,7 +1007,7 @@ void StorageMergeTree::dropDetached(const ASTPtr & partition, bool part, const C MergeTreeDataPart detached_part(*this, part_id, info); detached_part.relative_path = "detached/" + part_id; - detached_part.remove(); + detached_part.remove(true); } void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context) @@ -1070,9 +1070,9 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par } catch (...) { - tryLogCurrentException(log, String(__PRETTY_FUNCTION__) + ": cannot attach part " + source_part_name); + LOG_INFO(log, "Cannot attach part " << source_part_name << " :" << getCurrentExceptionMessage(false)); - if (part->relative_path == "detached/attaching_" + source_part_name) + if (part && part->relative_path == "detached/attaching_" + source_part_name) { try { diff --git a/dbms/tests/queries/0_stateless/00974_attach_active_part.reference b/dbms/tests/queries/0_stateless/00974_attach_active_part.reference deleted file mode 100644 index fc0fce0a541..00000000000 --- a/dbms/tests/queries/0_stateless/00974_attach_active_part.reference +++ /dev/null @@ -1,6 +0,0 @@ -OK -0_1_1_0 -1_2_2_0 -2_3_3_0 -3_4_4_0 -16 diff --git a/dbms/tests/queries/0_stateless/00974_attach_active_part.sh b/dbms/tests/queries/0_stateless/00974_attach_active_part.sh deleted file mode 100755 index 32e2b21608f..00000000000 --- a/dbms/tests/queries/0_stateless/00974_attach_active_part.sh +++ /dev/null @@ -1,12 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. $CURDIR/../shell_config.sh - -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_bug"; -$CLICKHOUSE_CLIENT --query="CREATE TABLE attach_bug (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n"; -$CLICKHOUSE_CLIENT --query="INSERT INTO attach_bug SELECT number FROM system.numbers LIMIT 16"; -$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_bug ATTACH PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK' -$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_bug' ORDER BY name FORMAT TSV"; -$CLICKHOUSE_CLIENT --query="SElECT count() FROM attach_bug FORMAT TSV"; -$CLICKHOUSE_CLIENT --query="DROP TABLE attach_bug"; diff --git a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference new file mode 100644 index 00000000000..d44f46779ca --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference @@ -0,0 +1,17 @@ +=== cannot attach active === +OK +0_1_1_0 +1_2_2_0 +2_3_3_0 +3_4_4_0 +16 120 +=== attach all valid parts === +0_5_5_0 +0_6_6_0 +1_2_2_0 +1_4_4_0 +16 120 +=== detached === +0_5_5_0 +delete_tmp_0_7_7 +attaching_0_6_6 diff --git a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh new file mode 100755 index 00000000000..89a6be183d2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh @@ -0,0 +1,41 @@ +#!/usr/bin/env bash + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +ch_dir=`${CLICKHOUSE_EXTRACT_CONFIG} -k path` +cur_db=`${CLICKHOUSE_CLIENT} --query "SELECT currentDatabase()"` + +echo '=== cannot attach active ==='; +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_active"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE attach_active (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n"; +$CLICKHOUSE_CLIENT --query="INSERT INTO attach_active SELECT number FROM system.numbers LIMIT 16"; +$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_active ATTACH PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK' +$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_active' AND database='${cur_db}' ORDER BY name FORMAT TSV"; +$CLICKHOUSE_CLIENT --query="SElECT count(), sum(n) FROM attach_active FORMAT TSV"; +$CLICKHOUSE_CLIENT --query="DROP TABLE attach_active"; + + + +echo '=== attach all valid parts ==='; +$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES"; +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_partitions"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE attach_partitions (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n"; +$CLICKHOUSE_CLIENT --query="INSERT INTO attach_partitions SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8"; +$CLICKHOUSE_CLIENT --query="INSERT INTO attach_partitions SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8"; + +$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions DETACH PARTITION 0"; +mkdir $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ # broken part +cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/ +cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/delete_tmp_0_7_7_0/ +$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions ATTACH PARTITION 0"; + +$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV"; +$CLICKHOUSE_CLIENT --query="SElECT count(), sum(n) FROM attach_partitions FORMAT TSV"; +echo '=== detached ==='; +$CLICKHOUSE_CLIENT --query="SELECT directory_name FROM system.detached_parts WHERE table='attach_partitions' AND database='${cur_db}' FORMAT TSV"; + +$CLICKHOUSE_CLIENT --query="DROP TABLE attach_partitions"; +$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES"; diff --git a/dbms/tests/queries/0_stateless/00975_drop_detached.reference b/dbms/tests/queries/0_stateless/00975_drop_detached.reference new file mode 100644 index 00000000000..40732c908ab --- /dev/null +++ b/dbms/tests/queries/0_stateless/00975_drop_detached.reference @@ -0,0 +1,2 @@ +OK +0_3_3_0 diff --git a/dbms/tests/queries/0_stateless/00975_drop_detached.sh b/dbms/tests/queries/0_stateless/00975_drop_detached.sh new file mode 100755 index 00000000000..9f831560bdc --- /dev/null +++ b/dbms/tests/queries/0_stateless/00975_drop_detached.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +ch_dir=`${CLICKHOUSE_EXTRACT_CONFIG} -k path` +cur_db=`${CLICKHOUSE_CLIENT} --query "SELECT currentDatabase()"` + +$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES"; +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS drop_detached"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE drop_detached (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n"; +$CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8"; +$CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8"; + +$CLICKHOUSE_CLIENT --query="ALTER TABLE drop_detached DETACH PARTITION 0"; +mkdir $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/ +mkdir $ch_dir/data/$cur_db/drop_detached/detached/delete_tmp_0_7_7_0/ +mkdir $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/ + +$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK' +$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '0_1_1_0'" +$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'attaching_0_6_6_0'" +$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'delete_tmp_0_7_7_0'" +$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'any_other_name'" + +$CLICKHOUSE_CLIENT --query="SElECT directory_name FROM system.detached_parts WHERE table='drop_detached' AND database='${cur_db}' FORMAT TSV"; +$CLICKHOUSE_CLIENT --query="DROP TABLE drop_detached"; +$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES"; From c5a42a69481d2ae11c379ec8a7c325a18cb27138 Mon Sep 17 00:00:00 2001 From: Sergi Vladykin Date: Sun, 28 Jul 2019 13:30:46 +0300 Subject: [PATCH 108/509] TTL: Only drop altogether the expired parts and not partially prune them. Though in regular merges eviction of expired values still must happen. --- dbms/src/DataStreams/TTLBlockInputStream.cpp | 25 ++++++++----------- dbms/src/Storages/MergeTree/MergeSelector.h | 5 +++- .../MergeTree/MergeTreeDataMergerMutator.cpp | 7 ++++-- .../MergeTree/MergeTreeDataPartTTLInfo.cpp | 8 +++--- .../MergeTree/MergeTreeDataPartTTLInfo.h | 10 +++++--- .../MergeTree/MergeTreeDataWriter.cpp | 2 +- .../Storages/MergeTree/MergeTreeSettings.h | 1 + .../Storages/MergeTree/TTLMergeSelector.cpp | 14 ++++++++--- .../src/Storages/MergeTree/TTLMergeSelector.h | 3 ++- 9 files changed, 45 insertions(+), 30 deletions(-) diff --git a/dbms/src/DataStreams/TTLBlockInputStream.cpp b/dbms/src/DataStreams/TTLBlockInputStream.cpp index 11999b894aa..0ad72876ea9 100644 --- a/dbms/src/DataStreams/TTLBlockInputStream.cpp +++ b/dbms/src/DataStreams/TTLBlockInputStream.cpp @@ -63,22 +63,19 @@ TTLBlockInputStream::TTLBlockInputStream( Block TTLBlockInputStream::readImpl() { + /// Skip all data if table ttl is expired for part + if (storage.hasTableTTL() && old_ttl_infos.table_ttl.max <= current_time) + { + rows_removed = data_part->rows_count; + return {}; + } + Block block = children.at(0)->read(); if (!block) return block; - if (storage.hasTableTTL()) - { - /// Skip all data if table ttl is expired for part - if (old_ttl_infos.table_ttl.max <= current_time) - { - rows_removed = data_part->rows_count; - return {}; - } - - if (old_ttl_infos.table_ttl.min <= current_time) - removeRowsWithExpiredTableTTL(block); - } + if (storage.hasTableTTL() && old_ttl_infos.table_ttl.min <= current_time) + removeRowsWithExpiredTableTTL(block); removeValuesWithExpiredColumnTTL(block); @@ -88,9 +85,9 @@ Block TTLBlockInputStream::readImpl() void TTLBlockInputStream::readSuffixImpl() { for (const auto & elem : new_ttl_infos.columns_ttl) - new_ttl_infos.updatePartMinTTL(elem.second.min); + new_ttl_infos.updatePartMinMaxTTL(elem.second.min, elem.second.max); - new_ttl_infos.updatePartMinTTL(new_ttl_infos.table_ttl.min); + new_ttl_infos.updatePartMinMaxTTL(new_ttl_infos.table_ttl.min, new_ttl_infos.table_ttl.max); data_part->ttl_infos = std::move(new_ttl_infos); data_part->empty_columns = std::move(empty_columns); diff --git a/dbms/src/Storages/MergeTree/MergeSelector.h b/dbms/src/Storages/MergeTree/MergeSelector.h index b01ef1d1d43..3c3cd8190ac 100644 --- a/dbms/src/Storages/MergeTree/MergeSelector.h +++ b/dbms/src/Storages/MergeTree/MergeSelector.h @@ -40,8 +40,11 @@ public: /// Opaque pointer to avoid dependencies (it is not possible to do forward declaration of typedef). const void * data; - /// Minimal time, when we need to delete some data from this part + /// Minimal time, when we need to delete some data from this part. time_t min_ttl; + + /// Maximum time, when we will need to drop this part altogether because all rows in it are expired. + time_t max_ttl; }; /// Parts are belong to partitions. Only parts within same partition could be merged. diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index e30bbdcda0b..727459291b5 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -200,8 +200,11 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( part_info.level = part->info.level; part_info.data = ∂ part_info.min_ttl = part->ttl_infos.part_min_ttl; + part_info.max_ttl = part->ttl_infos.part_max_ttl; - if (part_info.min_ttl && part_info.min_ttl <= current_time) + time_t ttl = data.settings.ttl_only_drop_parts ? part_info.max_ttl : part_info.min_ttl; + + if (ttl && ttl <= current_time) has_part_with_expired_ttl = true; partitions.back().emplace_back(part_info); @@ -228,7 +231,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( /// NOTE Could allow selection of different merge strategy. if (can_merge_with_ttl && has_part_with_expired_ttl) { - merge_selector = std::make_unique(current_time); + merge_selector = std::make_unique(current_time, data.settings.ttl_only_drop_parts); last_merge_with_ttl = current_time; } else diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp index d3fe3231e05..39665f03c84 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp @@ -12,11 +12,11 @@ void MergeTreeDataPartTTLInfos::update(const MergeTreeDataPartTTLInfos & other_i for (const auto & [name, ttl_info] : other_infos.columns_ttl) { columns_ttl[name].update(ttl_info); - updatePartMinTTL(ttl_info.min); + updatePartMinMaxTTL(ttl_info.min, ttl_info.max); } table_ttl.update(other_infos.table_ttl); - updatePartMinTTL(table_ttl.min); + updatePartMinMaxTTL(table_ttl.min, table_ttl.max); } void MergeTreeDataPartTTLInfos::read(ReadBuffer & in) @@ -37,7 +37,7 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in) String name = col["name"].getString(); columns_ttl.emplace(name, ttl_info); - updatePartMinTTL(ttl_info.min); + updatePartMinMaxTTL(ttl_info.min, ttl_info.max); } } if (json.has("table")) @@ -46,7 +46,7 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in) table_ttl.min = table["min"].getUInt(); table_ttl.max = table["max"].getUInt(); - updatePartMinTTL(table_ttl.min); + updatePartMinMaxTTL(table_ttl.min, table_ttl.max); } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h b/dbms/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h index ff181aa29a4..71a7c9f602f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h @@ -36,15 +36,19 @@ struct MergeTreeDataPartTTLInfos std::unordered_map columns_ttl; MergeTreeDataPartTTLInfo table_ttl; time_t part_min_ttl = 0; + time_t part_max_ttl = 0; void read(ReadBuffer & in); void write(WriteBuffer & out) const; void update(const MergeTreeDataPartTTLInfos & other_infos); - void updatePartMinTTL(time_t time) + void updatePartMinMaxTTL(time_t time_min, time_t time_max) { - if (time && (!part_min_ttl || time < part_min_ttl)) - part_min_ttl = time; + if (time_min && (!part_min_ttl || time_min < part_min_ttl)) + part_min_ttl = time_min; + + if (time_max && (!part_max_ttl || time_max > part_max_ttl)) + part_max_ttl = time_max; } }; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index d7a48171499..8515348a4b5 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -99,7 +99,7 @@ void updateTTL(const MergeTreeData::TTLEntry & ttl_entry, MergeTreeDataPart::TTL else throw Exception("Unexpected type of result ttl column", ErrorCodes::LOGICAL_ERROR); - ttl_infos.updatePartMinTTL(ttl_info.min); + ttl_infos.updatePartMinMaxTTL(ttl_info.min, ttl_info.max); } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index 9bd58e77f9c..457208cd0c3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -79,6 +79,7 @@ struct MergeTreeSettings : public SettingsCollection M(SettingUInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).") \ M(SettingUInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).") \ M(SettingInt64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with TTL can be repeated.") \ + M(SettingBool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.") \ M(SettingBool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)") \ M(SettingBool, enable_mixed_granularity_parts, 0, "Enable parts with adaptive and non adaptive granularity") diff --git a/dbms/src/Storages/MergeTree/TTLMergeSelector.cpp b/dbms/src/Storages/MergeTree/TTLMergeSelector.cpp index 3669bd1e2cc..0ba341fca64 100644 --- a/dbms/src/Storages/MergeTree/TTLMergeSelector.cpp +++ b/dbms/src/Storages/MergeTree/TTLMergeSelector.cpp @@ -20,9 +20,11 @@ IMergeSelector::PartsInPartition TTLMergeSelector::select( { for (auto it = partitions[i].begin(); it != partitions[i].end(); ++it) { - if (it->min_ttl && (partition_to_merge_index == -1 || it->min_ttl < partition_to_merge_min_ttl)) + time_t ttl = only_drop_parts ? it->max_ttl : it->min_ttl; + + if (ttl && (partition_to_merge_index == -1 || ttl < partition_to_merge_min_ttl)) { - partition_to_merge_min_ttl = it->min_ttl; + partition_to_merge_min_ttl = ttl; partition_to_merge_index = i; best_begin = it; } @@ -38,7 +40,9 @@ IMergeSelector::PartsInPartition TTLMergeSelector::select( while (true) { - if (!best_begin->min_ttl || best_begin->min_ttl > current_time + time_t ttl = only_drop_parts ? best_begin->max_ttl : best_begin->min_ttl; + + if (!ttl || ttl > current_time || (max_total_size_to_merge && total_size > max_total_size_to_merge)) { ++best_begin; @@ -54,7 +58,9 @@ IMergeSelector::PartsInPartition TTLMergeSelector::select( while (best_end != best_partition.end()) { - if (!best_end->min_ttl || best_end->min_ttl > current_time + time_t ttl = only_drop_parts ? best_end->max_ttl : best_end->min_ttl; + + if (!ttl || ttl > current_time || (max_total_size_to_merge && total_size > max_total_size_to_merge)) break; diff --git a/dbms/src/Storages/MergeTree/TTLMergeSelector.h b/dbms/src/Storages/MergeTree/TTLMergeSelector.h index 3c035d03a99..2f03d5b9feb 100644 --- a/dbms/src/Storages/MergeTree/TTLMergeSelector.h +++ b/dbms/src/Storages/MergeTree/TTLMergeSelector.h @@ -14,13 +14,14 @@ namespace DB class TTLMergeSelector : public IMergeSelector { public: - explicit TTLMergeSelector(time_t current_time_) : current_time(current_time_) {} + explicit TTLMergeSelector(time_t current_time_, bool only_drop_parts_) : current_time(current_time_), only_drop_parts(only_drop_parts_) {} PartsInPartition select( const Partitions & partitions, const size_t max_total_size_to_merge) override; private: time_t current_time; + bool only_drop_parts; }; } From 8f4883b0d2d84f324c530068116dabc6f5c05146 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 28 Jul 2019 15:33:40 +0300 Subject: [PATCH 109/509] Better constraint exception --- .../CheckConstraintsBlockOutputStream.cpp | 46 +++++++++++++++---- .../CheckConstraintsBlockOutputStream.h | 16 ++++--- .../Interpreters/InterpreterInsertQuery.cpp | 2 +- .../0_stateless/00952_basic_constraints.sh | 14 +++--- .../00953_constraints_operations.sh | 2 +- 5 files changed, 56 insertions(+), 24 deletions(-) diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index ec4a7bd45b8..cb9b8871a68 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -12,11 +12,23 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) for (size_t i = 0; i < expressions.size(); ++i) { auto constraint_expr = expressions[i]; - if (!checkConstraintOnBlock(block, constraint_expr)) - throw Exception{"Constraint " + constraints.constraints[i]->name + " is not satisfied, constraint expression: " + - serializeAST(*(constraints.constraints[i]->expr), true), ErrorCodes::LOGICAL_ERROR}; + auto res_column_uint8 = executeOnBlock(block, constraint_expr); + if (!memoryIsByte(res_column_uint8->getRawDataBegin<1>(), res_column_uint8->byteSize(), 0x1)) + { + auto indices_wrong = findAllWrong(res_column_uint8->getRawDataBegin<1>(), res_column_uint8->byteSize()); + std::string indices_str = "{"; + for (size_t j = 0; j < indices_wrong.size(); ++j) { + indices_str += std::to_string(indices_wrong[j]); + indices_str += (j != indices_wrong.size() - 1) ? ", " : "}"; + } + + throw Exception{"Violated constraint " + constraints.constraints[i]->name + + " in table " + table + " at indices " + indices_str + ", constraint expression: " + + serializeAST(*(constraints.constraints[i]->expr), true), ErrorCodes::LOGICAL_ERROR}; + } } output->write(block); + rows_written += block.rows(); } void CheckConstraintsBlockOutputStream::flush() @@ -34,18 +46,34 @@ void CheckConstraintsBlockOutputStream::writeSuffix() output->writeSuffix(); } -bool CheckConstraintsBlockOutputStream::checkImplMemory(const Block & block, const ExpressionActionsPtr & constraint) +const ColumnUInt8 *CheckConstraintsBlockOutputStream::executeOnBlock( + const Block & block, + const ExpressionActionsPtr & constraint) { Block res = block; + constraint->execute(res); ColumnWithTypeAndName res_column = res.safeGetByPosition(res.columns() - 1); - auto res_column_uint8 = checkAndGetColumn(res_column.column.get()); - return memoryIsByte(res_column_uint8->getRawDataBegin<1>(), res_column_uint8->byteSize(), 0x1); + return checkAndGetColumn(res_column.column.get()); } -bool CheckConstraintsBlockOutputStream::checkConstraintOnBlock(const Block & block, const ExpressionActionsPtr & constraint) +std::vector CheckConstraintsBlockOutputStream::findAllWrong(const void *data, size_t size) { - return checkImplMemory(block, constraint); -} + std::vector res; + if (size == 0) + return res; + + auto ptr = reinterpret_cast(data); + + for (size_t i = 0; i < size; ++i) + { + if (*(ptr + i) == 0x0) + { + res.push_back(i); + } + } + + return res; +} } diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h index 6ea42cf44af..ac2e7e974a1 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h @@ -17,14 +17,17 @@ class CheckConstraintsBlockOutputStream : public IBlockOutputStream { public: CheckConstraintsBlockOutputStream( + const String & table_, const BlockOutputStreamPtr & output_, const Block & header_, const ConstraintsDescription & constraints_, const Context & context_) - : output(output_), + : table(table_), + output(output_), header(header_), constraints(constraints_), - expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())) + expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())), + rows_written(0) { } Block getHeader() const override { return header; } @@ -35,14 +38,15 @@ public: void writePrefix() override; void writeSuffix() override; - bool checkImplMemory(const Block & block, const ExpressionActionsPtr & constraint); - bool checkImplBool(const Block & block, const ExpressionActionsPtr & constraint); - bool checkConstraintOnBlock(const Block & block, const ExpressionActionsPtr & constraint); - private: + const ColumnUInt8* executeOnBlock(const Block & block, const ExpressionActionsPtr & constraint); + std::vector findAllWrong(const void *data, size_t size); + + String table; BlockOutputStreamPtr output; Block header; const ConstraintsDescription constraints; const ConstraintsExpressions expressions; + size_t rows_written; }; } diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 9c0cc31cb8e..8454df97f08 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -119,7 +119,7 @@ BlockIO InterpreterInsertQuery::execute() out, query_sample_block, out->getHeader(), table->getColumns().getDefaults(), context); out = std::make_shared( - out, query_sample_block, table->getConstraints(), context); + query.table, out, query_sample_block, table->getConstraints(), context); auto out_wrapper = std::make_shared(out); out_wrapper->setProcessListElement(context.getProcessListElement()); diff --git a/dbms/tests/queries/0_stateless/00952_basic_constraints.sh b/dbms/tests/queries/0_stateless/00952_basic_constraints.sh index 1d2a46dae61..b6aa28c46bf 100755 --- a/dbms/tests/queries/0_stateless/00952_basic_constraints.sh +++ b/dbms/tests/queries/0_stateless/00952_basic_constraints.sh @@ -20,8 +20,8 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must throw and exception -EXCEPTION_TEXT="Constraint b_constraint is not satisfied" -$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (3, 4), (1, 0);" 2>&1 \ +EXCEPTION_TEXT="Violated constraint b_constraint in table test_constraints at indices {1, 3}" +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (3, 4), (1, 0), (3, 4), (6, 0);" 2>&1 \ | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" @@ -32,20 +32,20 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE test_constraints ( a UInt32, b UInt32, - CONSTRAINT b_constraint CHECK b > 10, - CONSTRAINT a_constraint CHECK a < 10 + CONSTRAINT a_constraint CHECK a < 10, + CONSTRAINT b_constraint CHECK b > 10 ) ENGINE = MergeTree ORDER BY (a);" # This one must throw an exception -EXCEPTION_TEXT="Constraint b_constraint is not satisfied" +EXCEPTION_TEXT="Violated constraint b_constraint in table test_constraints at indices {0}" $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" 2>&1 \ | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must throw an exception -EXCEPTION_TEXT="Constraint a_constraint is not satisfied" -$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (5, 16), (10, 11);" 2>&1 \ +EXCEPTION_TEXT="Violated constraint a_constraint in table test_constraints at indices {1}" +$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (5, 16), (10, 11), (9, 11), (8, 12);" 2>&1 \ | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" diff --git a/dbms/tests/queries/0_stateless/00953_constraints_operations.sh b/dbms/tests/queries/0_stateless/00953_constraints_operations.sh index f0fc5b71fbf..8a563a21e02 100755 --- a/dbms/tests/queries/0_stateless/00953_constraints_operations.sh +++ b/dbms/tests/queries/0_stateless/00953_constraints_operations.sh @@ -20,7 +20,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must throw and exception -EXCEPTION_TEXT="Constraint b_constraint is not satisfied" +EXCEPTION_TEXT="Violated constraint b_constraint in table test_constraints at indices" $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 0);" 2>&1 \ | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" From 915092515c41b5b83f0bbd9cb6d2294c5b56b0e7 Mon Sep 17 00:00:00 2001 From: Sergi Vladykin Date: Mon, 29 Jul 2019 11:20:11 +0300 Subject: [PATCH 110/509] revert the performance bug fix --- dbms/src/DataStreams/TTLBlockInputStream.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/DataStreams/TTLBlockInputStream.cpp b/dbms/src/DataStreams/TTLBlockInputStream.cpp index 0ad72876ea9..78d8d25e8bc 100644 --- a/dbms/src/DataStreams/TTLBlockInputStream.cpp +++ b/dbms/src/DataStreams/TTLBlockInputStream.cpp @@ -63,6 +63,10 @@ TTLBlockInputStream::TTLBlockInputStream( Block TTLBlockInputStream::readImpl() { + Block block = children.at(0)->read(); + if (!block) + return block; + /// Skip all data if table ttl is expired for part if (storage.hasTableTTL() && old_ttl_infos.table_ttl.max <= current_time) { @@ -70,10 +74,6 @@ Block TTLBlockInputStream::readImpl() return {}; } - Block block = children.at(0)->read(); - if (!block) - return block; - if (storage.hasTableTTL() && old_ttl_infos.table_ttl.min <= current_time) removeRowsWithExpiredTableTTL(block); From 0fc47fbbe4b70b5ffc254024798d8f9ed45b0418 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 26 Jul 2019 23:04:45 +0300 Subject: [PATCH 111/509] fixes --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 14 +++++++++- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 ++ .../Storages/MergeTree/MergeTreeDataPart.cpp | 8 ++++-- .../Storages/MergeTree/MergeTreePartInfo.cpp | 28 ++++++++----------- .../Storages/MergeTree/MergeTreePartInfo.h | 9 +++--- dbms/src/Storages/StorageMergeTree.cpp | 15 ++-------- dbms/src/Storages/StorageMergeTree.h | 1 - .../Storages/StorageReplicatedMergeTree.cpp | 2 ++ .../System/StorageSystemDetachedParts.cpp | 25 ++++++++--------- .../00974_attach_invalid_parts.reference | 4 +-- .../0_stateless/00974_attach_invalid_parts.sh | 11 +++++--- .../0_stateless/00975_drop_detached.sh | 13 +++++---- 12 files changed, 69 insertions(+), 63 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 94e42c34d0f..9d12a9ee6ea 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -87,6 +87,8 @@ namespace ErrorCodes extern const int CANNOT_MUNMAP; extern const int CANNOT_MREMAP; extern const int BAD_TTL_EXPRESSION; + extern const int INCORRECT_FILE_NAME; + extern const int BAD_DATA_PART_NAME; } @@ -2583,11 +2585,21 @@ MergeTreeData::getDetachedParts() const res.emplace_back(); auto & part = res.back(); - DetachedPartInfo::tryParseDetachedPartName(dir_name, &part, format_version); + DetachedPartInfo::tryParseDetachedPartName(dir_name, part, format_version); } return res; } +void MergeTreeData::validateDetachedPartName(const String & name) const +{ + if (name.find('/') != std::string::npos || name == "." || name == "..") + throw DB::Exception("Invalid part name", ErrorCodes::INCORRECT_FILE_NAME); + + Poco::File detached_part_dir(full_path + "detached/" + name); + if (!detached_part_dir.exists()) + throw DB::Exception("Detached part \"" + name + "\" not found" , ErrorCodes::BAD_DATA_PART_NAME); +} + MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affordable_states) const { DataParts res; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index cec3651652b..2333135d53e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -389,6 +389,8 @@ public: /// Returns all detached parts std::vector getDetachedParts() const; + void validateDetachedPartName(const String & name) const; + /// Returns Committed parts DataParts getDataParts() const; DataPartsVector getDataPartsVector() const; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp index 865aaf80ed1..24bc5cd2463 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp @@ -356,16 +356,18 @@ void MergeTreeDataPart::remove(bool force_recursive /*= false*/) const * - rename directory to temporary name; * - remove it recursive. * - * For temporary name we use "delete_tmp_" prefix. + * For temporary name we use "detached/deleting_" prefix. * - * NOTE: We cannot use "tmp_delete_" prefix, because there is a second thread, + * NOTE: We cannot use "tmp_*" prefix, because there is a second thread, * that calls "clearOldTemporaryDirectories" and removes all directories, that begin with "tmp_" and are old enough. * But when we removing data part, it can be old enough. And rename doesn't change mtime. * And a race condition can happen that will lead to "File not found" error here. + * We move directory to detached/, because if an attempt to remove directory after renaming failed for some reason + * there would be no way to remove directory from storage.full_path (except manually). */ String from = storage.full_path + relative_path; - String to = storage.full_path + "delete_tmp_" + name; + String to = storage.full_path + getRelativePathForDetachedPart("deleting_"); Poco::File from_dir{from}; Poco::File to_dir{to}; diff --git a/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp b/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp index de7150e4cea..a9e31a988b3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -188,35 +188,29 @@ String MergeTreePartInfo::getPartNameV0(DayNum left_date, DayNum right_date) con return wb.str(); } -bool DetachedPartInfo::tryParseDetachedPartName(const String & dir_name, DetachedPartInfo * part_info, +bool DetachedPartInfo::tryParseDetachedPartName(const String & dir_name, DetachedPartInfo & part_info, MergeTreeDataFormatVersion format_version) { + part_info.dir_name = dir_name; + /// First, try to parse as . - if (MergeTreePartInfo::tryParsePartName(dir_name, part_info, format_version)) - return part_info->valid_name = true; + if (MergeTreePartInfo::tryParsePartName(dir_name, &part_info, format_version)) + return part_info.valid_name = true; /// Next, as _. Use entire name as prefix if it fails. - part_info->prefix = dir_name; + part_info.prefix = dir_name; const auto first_separator = dir_name.find_first_of('_'); if (first_separator == String::npos) - return part_info->valid_name = false; + return part_info.valid_name = false; // TODO what if contains '_'? const auto part_name = dir_name.substr(first_separator + 1, dir_name.size() - first_separator - 1); - if (!MergeTreePartInfo::tryParsePartName(part_name, part_info, format_version)) - return part_info->valid_name = false; + if (!MergeTreePartInfo::tryParsePartName(part_name, &part_info, format_version)) + return part_info.valid_name = false; - part_info->prefix = dir_name.substr(0, first_separator); - return part_info->valid_name = true; + part_info.prefix = dir_name.substr(0, first_separator); + return part_info.valid_name = true; } -String DetachedPartInfo::fullDirName() const -{ - if (!valid_name) - return prefix; - if (prefix.empty()) - return getPartName(); - return prefix + "_" + getPartName(); -} } diff --git a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h index 7d0fb446ee3..25cf46ad46d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h @@ -92,15 +92,14 @@ struct MergeTreePartInfo /// addition to the above fields. struct DetachedPartInfo : public MergeTreePartInfo { + /// Suddenly, name of detached part may contain suffix (such as _tryN), which is ignored by MergeTreePartInfo::tryParsePartName(...) + String dir_name; String prefix; - /// If false, prefix contains full directory name and MergeTreePartInfo may be in invalid state - /// (directory name was not successfully parsed). + /// If false, MergeTreePartInfo is in invalid state (directory name was not successfully parsed). bool valid_name; - String fullDirName() const; - - static bool tryParseDetachedPartName(const String & dir_name, DetachedPartInfo * part_info, MergeTreeDataFormatVersion format_version); + static bool tryParseDetachedPartName(const String & dir_name, DetachedPartInfo & part_info, MergeTreeDataFormatVersion format_version); }; } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 0c1503347c0..ad4d0cd933f 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -36,7 +36,6 @@ namespace ErrorCodes extern const int INCORRECT_FILE_NAME; extern const int CANNOT_ASSIGN_OPTIMIZE; extern const int INCOMPATIBLE_COLUMNS; - extern const int BAD_DATA_PART_NAME; } namespace ActionLocks @@ -1003,7 +1002,7 @@ void StorageMergeTree::dropDetached(const ASTPtr & partition, bool part, const C validateDetachedPartName(part_id); DetachedPartInfo info; - DetachedPartInfo::tryParseDetachedPartName(part_id, &info, format_version); + DetachedPartInfo::tryParseDetachedPartName(part_id, info, format_version); MergeTreeDataPart detached_part(*this, part_id, info); detached_part.relative_path = "detached/" + part_id; @@ -1038,7 +1037,8 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par { const String & name = it.name(); MergeTreePartInfo part_info; - /// Parts with prefix in name (e.g. attaching_1_3_3_0, delete_tmp_1_3_3_0) will be ignored + /// Parts with prefix in name (e.g. attaching_1_3_3_0, deleting_1_3_3_0) will be ignored + // TODO what if name contains "_tryN" suffix? if (!MergeTreePartInfo::tryParsePartName(name, &part_info, format_version) || part_info.partition_id != partition_id) { @@ -1163,15 +1163,6 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con } } -void StorageMergeTree::validateDetachedPartName(const String & name) const -{ - if (name.find('/') != std::string::npos || name == "." || name == "..") - throw DB::Exception("Invalid part name", ErrorCodes::INCORRECT_FILE_NAME); - - Poco::File detached_part_dir(full_path + "detached/" + name); - if (!detached_part_dir.exists()) - throw DB::Exception("Detached part \"" + name + "\" not found" , ErrorCodes::BAD_DATA_PART_NAME); -} ActionLock StorageMergeTree::getActionLock(StorageActionBlockType action_type) { diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 42061894a8e..fa2561e4ab2 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -124,7 +124,6 @@ private: void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context); void attachPartition(const ASTPtr & partition, bool part, const Context & context); void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context); - void validateDetachedPartName(const String & name) const; friend class MergeTreeBlockOutputStream; friend class MergeTreeData; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 5f91c304e98..192384602eb 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3554,6 +3554,7 @@ void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool Strings parts; if (attach_part) { + validateDetachedPartName(partition_id); parts.push_back(partition_id); } else @@ -3566,6 +3567,7 @@ void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool { String name = it.name(); MergeTreePartInfo part_info; + // TODO what if name contains "_tryN" suffix? if (!MergeTreePartInfo::tryParsePartName(name, &part_info, format_version)) continue; if (part_info.partition_id != partition_id) diff --git a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp index 9b32f1fb29b..e27c7945670 100644 --- a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -31,13 +32,12 @@ protected: setColumns(ColumnsDescription{{ {"database", std::make_shared()}, {"table", std::make_shared()}, - {"partition_id", std::make_shared()}, + {"partition_id", std::make_shared(std::make_shared())}, {"name", std::make_shared()}, - {"reason", std::make_shared()}, - {"min_block_number", std::make_shared()}, - {"max_block_number", std::make_shared()}, - {"level", std::make_shared()}, - {"directory_name", std::make_shared()} + {"reason", std::make_shared(std::make_shared())}, + {"min_block_number", std::make_shared(std::make_shared())}, + {"max_block_number", std::make_shared(std::make_shared())}, + {"level", std::make_shared(std::make_shared())} }}); } @@ -63,13 +63,12 @@ protected: int i = 0; columns[i++]->insert(info.database); columns[i++]->insert(info.table); - columns[i++]->insert(p.valid_name ? p.partition_id : ""); - columns[i++]->insert(p.valid_name ? p.getPartName() : ""); - columns[i++]->insert(p.prefix); - columns[i++]->insert(p.min_block); - columns[i++]->insert(p.max_block); - columns[i++]->insert(p.level); - columns[i++]->insert(p.fullDirName()); + columns[i++]->insert(p.valid_name ? p.partition_id : Field()); + columns[i++]->insert(p.dir_name); + columns[i++]->insert(p.valid_name ? p.prefix : Field()); + columns[i++]->insert(p.valid_name ? p.min_block : Field()); + columns[i++]->insert(p.valid_name ? p.max_block : Field()); + columns[i++]->insert(p.valid_name ? p.level : Field()); } } diff --git a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference index d44f46779ca..42a04fe5666 100644 --- a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference +++ b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference @@ -13,5 +13,5 @@ OK 16 120 === detached === 0_5_5_0 -delete_tmp_0_7_7 -attaching_0_6_6 +deleting_0_7_7_0 +attaching_0_6_6_0 diff --git a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh index 89a6be183d2..4e9efa64ad1 100755 --- a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh +++ b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh @@ -27,15 +27,18 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO attach_partitions SELECT number FROM sys $CLICKHOUSE_CLIENT --query="INSERT INTO attach_partitions SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8"; $CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions DETACH PARTITION 0"; -mkdir $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ # broken part -cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/ -cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/delete_tmp_0_7_7_0/ +sudo -n mkdir $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ 2>/dev/null || \ + mkdir $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ # broken part +sudo -n cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/ 2>/dev/null || \ + cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/ +sudo -n cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/deleting_0_7_7_0/ 2>/dev/null || \ + cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/deleting_0_7_7_0/ $CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions ATTACH PARTITION 0"; $CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV"; $CLICKHOUSE_CLIENT --query="SElECT count(), sum(n) FROM attach_partitions FORMAT TSV"; echo '=== detached ==='; -$CLICKHOUSE_CLIENT --query="SELECT directory_name FROM system.detached_parts WHERE table='attach_partitions' AND database='${cur_db}' FORMAT TSV"; +$CLICKHOUSE_CLIENT --query="SELECT name FROM system.detached_parts WHERE table='attach_partitions' AND database='${cur_db}' FORMAT TSV"; $CLICKHOUSE_CLIENT --query="DROP TABLE attach_partitions"; $CLICKHOUSE_CLIENT --query="SYSTEM START MERGES"; diff --git a/dbms/tests/queries/0_stateless/00975_drop_detached.sh b/dbms/tests/queries/0_stateless/00975_drop_detached.sh index 9f831560bdc..3a5e920da75 100755 --- a/dbms/tests/queries/0_stateless/00975_drop_detached.sh +++ b/dbms/tests/queries/0_stateless/00975_drop_detached.sh @@ -15,16 +15,19 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system. $CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8"; $CLICKHOUSE_CLIENT --query="ALTER TABLE drop_detached DETACH PARTITION 0"; -mkdir $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/ -mkdir $ch_dir/data/$cur_db/drop_detached/detached/delete_tmp_0_7_7_0/ -mkdir $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/ +sudo -n mkdir $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/ 2>/dev/null || \ + mkdir $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/ +sudo -n mkdir $ch_dir/data/$cur_db/drop_detached/detached/deleting_0_7_7_0/ 2>/dev/null || \ + mkdir $ch_dir/data/$cur_db/drop_detached/detached/deleting_0_7_7_0/ +sudo -n mkdir $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/ 2>/dev/null || \ + mkdir $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/ $CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK' $CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '0_1_1_0'" $CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'attaching_0_6_6_0'" -$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'delete_tmp_0_7_7_0'" +$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'deleting_0_7_7_0'" $CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'any_other_name'" -$CLICKHOUSE_CLIENT --query="SElECT directory_name FROM system.detached_parts WHERE table='drop_detached' AND database='${cur_db}' FORMAT TSV"; +$CLICKHOUSE_CLIENT --query="SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='${cur_db}' FORMAT TSV"; $CLICKHOUSE_CLIENT --query="DROP TABLE drop_detached"; $CLICKHOUSE_CLIENT --query="SYSTEM START MERGES"; From 86831fe500c5dbf1124c7cecaf832f502700d5ba Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 29 Jul 2019 14:03:50 +0300 Subject: [PATCH 112/509] Add ability to create immutable settings --- .../performance-test/PerformanceTestInfo.cpp | 2 +- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Core/Settings.h | 2 +- dbms/src/Core/SettingsCommon.h | 78 +++++++++++++++++-- dbms/src/Storages/Kafka/KafkaSettings.cpp | 2 +- dbms/src/Storages/Kafka/KafkaSettings.h | 22 +++--- .../Storages/MergeTree/MergeTreeSettings.cpp | 2 +- .../Storages/MergeTree/MergeTreeSettings.h | 4 +- dbms/src/Storages/StorageMergeTree.cpp | 2 +- 9 files changed, 89 insertions(+), 26 deletions(-) diff --git a/dbms/programs/performance-test/PerformanceTestInfo.cpp b/dbms/programs/performance-test/PerformanceTestInfo.cpp index f016b257c5f..aa8c05f2d44 100644 --- a/dbms/programs/performance-test/PerformanceTestInfo.cpp +++ b/dbms/programs/performance-test/PerformanceTestInfo.cpp @@ -96,7 +96,7 @@ void PerformanceTestInfo::applySettings(XMLConfigurationPtr config) } extractSettings(config, "settings", config_settings, settings_to_apply); - settings.applyChanges(settings_to_apply); + settings.loadFromChanges(settings_to_apply); if (settings_contain("average_rows_speed_precision")) TestStats::avg_rows_speed_precision = diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 35213e3064f..7b955e47938 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -439,6 +439,7 @@ namespace ErrorCodes extern const int CANNOT_DELETE_TIMER = 462; extern const int CANNOT_FCNTL = 463; extern const int SETTINGS_ARE_NOT_SUPPORTED = 464; + extern const int IMMUTABLE_SETTING = 465; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 60f2599c73f..7501e4324a7 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -42,7 +42,7 @@ struct Settings : public SettingsCollection * but we are not going to do it, because settings is used everywhere as static struct fields. */ -#define LIST_OF_SETTINGS(M) \ +#define LIST_OF_SETTINGS(M, IM) \ M(SettingUInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.") \ M(SettingUInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.") \ M(SettingUInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading") \ diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index 08064096133..e9dce9c9dcc 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -17,6 +17,10 @@ class Field; class ReadBuffer; class WriteBuffer; +namespace ErrorCodes +{ + extern const int IMMUTABLE_SETTING; +} /** One setting for any type. * Stores a value within itself, as well as a flag - whether the value was changed. @@ -317,6 +321,7 @@ private: size_t offset_of_changed; StringRef name; StringRef description; + bool immutable; GetStringFunction get_string; GetFieldFunction get_field; SetStringFunction set_string; @@ -396,6 +401,7 @@ public: const_reference(const const_reference & src) = default; const StringRef & getName() const { return member->name; } const StringRef & getDescription() const { return member->description; } + bool isImmutable() const { return member->immutable; } bool isChanged() const { return member->isChanged(*collection); } Field getValue() const { return member->get_field(*collection); } String getValueAsString() const { return member->get_string(*collection); } @@ -497,6 +503,40 @@ public: void set(size_t index, const String & value) { (*this)[index].setValue(value); } void set(const String & name, const String & value) { (*this)[name].setValue(value); } + /// Updates setting's value. + void update(size_t index, const Field & value) + { + auto ref = (*this)[index]; + if (ref.isImmutable()) + throw Exception("Cannot modify immutable setting '" + ref.getName().toString() + "'", ErrorCodes::IMMUTABLE_SETTING); + ref.setValue(value); + } + + void update(const String & name, const Field & value) + { + auto ref = (*this)[name]; + if (ref.isImmutable()) + throw Exception("Cannot modify immutable setting '" + ref.getName().toString() + "'", ErrorCodes::IMMUTABLE_SETTING); + ref.setValue(value); + } + + /// Updates setting's value. Read value in text form from string (for example, from configuration file or from URL parameter). + void update(size_t index, const String & value) + { + auto ref = (*this)[index]; + if (ref.isImmutable()) + throw Exception("Cannot modify immutable setting '" + ref.getName().toString() + "'", ErrorCodes::IMMUTABLE_SETTING); + (*this)[index].setValue(value); + } + + void update(const String & name, const String & value) + { + auto ref = (*this)[name]; + if (ref.isImmutable()) + throw Exception("Cannot modify immutable setting '" + ref.getName().toString() + "'", ErrorCodes::IMMUTABLE_SETTING); + (*this)[name].setValue(value); + } + /// Returns value of a setting. Field get(size_t index) const { return (*this)[index].getValue(); } Field get(const String & name) const { return (*this)[name].getValue(); } @@ -561,17 +601,30 @@ public: } /// Applies changes to the settings. - void applyChange(const SettingChange & change) + void loadFromChange(const SettingChange & change) { set(change.name, change.value); } - void applyChanges(const SettingsChanges & changes) + void loadFromChanges(const SettingsChanges & changes) { for (const SettingChange & change : changes) - applyChange(change); + loadFromChange(change); } + /// Applies changes to the settings. + void updateFromChange(const SettingChange & change) + { + update(change.name, change.value); + } + + void updateFromChanges(const SettingsChanges & changes) + { + for (const SettingChange & change : changes) + updateFromChange(change); + } + + void copyChangesFrom(const Derived & src) { for (const auto & member : members()) @@ -615,7 +668,7 @@ public: }; #define DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS_MACRO) \ - LIST_OF_SETTINGS_MACRO(DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_) + LIST_OF_SETTINGS_MACRO(DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_, DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_) #define IMPLEMENT_SETTINGS_COLLECTION(DERIVED_CLASS_NAME, LIST_OF_SETTINGS_MACRO) \ @@ -625,9 +678,9 @@ public: using Derived = DERIVED_CLASS_NAME; \ struct Functions \ { \ - LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_) \ + LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_, IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_) \ }; \ - LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_) \ + LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_ADD_MUTABLE_MEMBER_INFO_HELPER_, IMPLEMENT_SETTINGS_COLLECTION_ADD_IMMUTABLE_MEMBER_INFO_HELPER_) \ } @@ -645,13 +698,22 @@ public: static Field NAME##_castValueWithoutApplying(const Field & value) { TYPE temp{DEFAULT}; temp.set(value); return temp.toField(); } -#define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ +#define IMPLEMENT_SETTINGS_COLLECTION_ADD_MUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ static_assert(std::is_same_v().NAME.changed), bool>); \ add({offsetof(Derived, NAME.changed), \ - StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), \ + StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), false, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \ &Functions::NAME##_serialize, &Functions::NAME##_deserialize, \ &Functions::NAME##_castValueWithoutApplying }); +#define IMPLEMENT_SETTINGS_COLLECTION_ADD_IMMUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ + static_assert(std::is_same_v().NAME.changed), bool>); \ + add({offsetof(Derived, NAME.changed), \ + StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), true, \ + &Functions::NAME##_getString, &Functions::NAME##_getField, \ + &Functions::NAME##_setString, &Functions::NAME##_setField, \ + &Functions::NAME##_serialize, &Functions::NAME##_deserialize, \ + &Functions::NAME##_castValueWithoutApplying }); + } diff --git a/dbms/src/Storages/Kafka/KafkaSettings.cpp b/dbms/src/Storages/Kafka/KafkaSettings.cpp index d08282a9794..b08d45780bb 100644 --- a/dbms/src/Storages/Kafka/KafkaSettings.cpp +++ b/dbms/src/Storages/Kafka/KafkaSettings.cpp @@ -22,7 +22,7 @@ void KafkaSettings::loadFromQuery(ASTStorage & storage_def) { try { - applyChanges(storage_def.settings->changes); + loadFromChanges(storage_def.settings->changes); } catch (Exception & e) { diff --git a/dbms/src/Storages/Kafka/KafkaSettings.h b/dbms/src/Storages/Kafka/KafkaSettings.h index f3642d66803..366ec715bf0 100644 --- a/dbms/src/Storages/Kafka/KafkaSettings.h +++ b/dbms/src/Storages/Kafka/KafkaSettings.h @@ -14,17 +14,17 @@ class ASTStorage; struct KafkaSettings : public SettingsCollection { -#define LIST_OF_KAFKA_SETTINGS(M) \ - M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.") \ - M(SettingString, kafka_topic_list, "", "A list of Kafka topics.") \ - M(SettingString, kafka_group_name, "", "A group of Kafka consumers.") \ - M(SettingString, kafka_format, "", "The message format for Kafka engine.") \ - M(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.") \ - M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine") \ - M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") \ - M(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") \ - M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block") \ - M(SettingUInt64, kafka_commit_every_batch, 0, "Commit every consumed and handled batch instead of a single commit after writing a whole block") +#define LIST_OF_KAFKA_SETTINGS(M, IM) \ + IM(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.") \ + IM(SettingString, kafka_topic_list, "", "A list of Kafka topics.") \ + IM(SettingString, kafka_group_name, "", "A group of Kafka consumers.") \ + IM(SettingString, kafka_format, "", "The message format for Kafka engine.") \ + IM(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.") \ + IM(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine") \ + IM(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") \ + IM(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") \ + IM(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block") \ + IM(SettingUInt64, kafka_commit_every_batch, 0, "Commit every consumed and handled batch instead of a single commit after writing a whole block") DECLARE_SETTINGS_COLLECTION(LIST_OF_KAFKA_SETTINGS) diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp index e3600b6ac4a..562f2d8f402 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -46,7 +46,7 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) { try { - applyChanges(storage_def.settings->changes); + loadFromChanges(storage_def.settings->changes); } catch (Exception & e) { diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index 9bd58e77f9c..c982bef324f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -24,8 +24,8 @@ class ASTStorage; struct MergeTreeSettings : public SettingsCollection { -#define LIST_OF_MERGE_TREE_SETTINGS(M) \ - M(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.") \ +#define LIST_OF_MERGE_TREE_SETTINGS(M, IM) \ + IM(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.") \ \ /** Merge settings. */ \ M(SettingUInt64, max_bytes_to_merge_at_max_space_in_pool, 150ULL * 1024 * 1024 * 1024, "Maximum in total size of parts to merge, when there are maximum free threads in background pool (or entries in replication queue).") \ diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index f39735b1595..41e2b04895a 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -261,7 +261,7 @@ void StorageMergeTree::alter( context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, storage_modifier); setColumns(std::move(new_columns)); - settings.applyChanges(new_changes); + settings.updateFromChanges(new_changes); return; } From a3ebe3153537170e344dc8766d48a2630e63146c Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Mon, 29 Jul 2019 13:05:43 +0300 Subject: [PATCH 113/509] Brace style fix --- dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index cb9b8871a68..5adf344cf0b 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -17,7 +17,8 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) { auto indices_wrong = findAllWrong(res_column_uint8->getRawDataBegin<1>(), res_column_uint8->byteSize()); std::string indices_str = "{"; - for (size_t j = 0; j < indices_wrong.size(); ++j) { + for (size_t j = 0; j < indices_wrong.size(); ++j) + { indices_str += std::to_string(indices_wrong[j]); indices_str += (j != indices_wrong.size() - 1) ? ", " : "}"; } From 2f33df1b2ef01ba19db8d465400f08a4d532e060 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 30 Jul 2019 20:24:40 +0300 Subject: [PATCH 114/509] rename all parts before attaching --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 40 ++++++++++++ dbms/src/Storages/MergeTree/MergeTreeData.h | 15 +++++ .../Storages/MergeTree/MergeTreeDataPart.cpp | 24 ++----- .../Storages/MergeTree/MergeTreeDataPart.h | 2 +- .../Storages/MergeTree/MergeTreePartInfo.cpp | 1 + dbms/src/Storages/StorageMergeTree.cpp | 65 ++++++------------- dbms/src/Storages/StorageMergeTree.h | 1 - .../Storages/StorageReplicatedMergeTree.cpp | 6 +- .../00974_attach_invalid_parts.reference | 17 +++-- .../0_stateless/00974_attach_invalid_parts.sh | 21 ++++-- .../0_stateless/00975_drop_detached.reference | 6 +- .../0_stateless/00975_drop_detached.sh | 20 +++--- 12 files changed, 130 insertions(+), 88 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 9d12a9ee6ea..4b13ebaa99f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1718,6 +1718,29 @@ MergeTreeData::AlterDataPartTransaction::~AlterDataPartTransaction() } } +void MergeTreeData::PartsTemporaryRename::addPart(const String & old_name, const String & new_name) +{ + Poco::File(base_dir + old_name).renameTo(base_dir + new_name); + old_and_new_names.push_back({old_name, new_name}); +} + +MergeTreeData::PartsTemporaryRename::~PartsTemporaryRename() +{ + for (const auto & names : old_and_new_names) + { + if (names.first.empty()) + continue; + try + { + Poco::File(base_dir + names.second).renameTo(base_dir + names.first); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +} + MergeTreeData::DataPartsVector MergeTreeData::getActivePartsToReplace( const MergeTreePartInfo & new_part_info, @@ -2600,6 +2623,23 @@ void MergeTreeData::validateDetachedPartName(const String & name) const throw DB::Exception("Detached part \"" + name + "\" not found" , ErrorCodes::BAD_DATA_PART_NAME); } +void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, const Context &) +{ + if (!part) // TODO + throw DB::Exception("DROP DETACHED PARTITION is not implemented, use DROP DETACHED PART", ErrorCodes::NOT_IMPLEMENTED); + + String part_id = partition->as().value.safeGet(); + validateDetachedPartName(part_id); + if (startsWith(part_id, "attaching_") || startsWith(part_id, "deleting_")) + throw DB::Exception("Cannot drop part " + part_id + ": " + "most likely it is used by another DROP or ATTACH query.", ErrorCodes::BAD_DATA_PART_NAME); + + PartsTemporaryRename renamed_parts(full_path + "detached/"); + renamed_parts.addPart(part_id, "deleting_" + part_id); + Poco::File(renamed_parts.base_dir + renamed_parts.old_and_new_names.front().second).remove(true); + renamed_parts.old_and_new_names.front().first.clear(); +} + MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affordable_states) const { DataParts res; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 099591a97e4..62cebf32f76 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -249,6 +249,19 @@ public: using AlterDataPartTransactionPtr = std::unique_ptr; + struct PartsTemporaryRename : private boost::noncopyable + { + PartsTemporaryRename(const String & base_dir_) : base_dir(base_dir_) {} + + /// Renames part from old_name to new_name + void addPart(const String & old_name, const String & new_name); + + /// Renames all added parts from new_name to old_name if old name is not empty + ~PartsTemporaryRename(); + + String base_dir; + std::vector> old_and_new_names; + }; /// Parameters for various modes. struct MergingParams @@ -392,6 +405,8 @@ public: void validateDetachedPartName(const String & name) const; + void dropDetached(const ASTPtr & partition, bool part, const Context & context); + /// Returns Committed parts DataParts getDataParts() const; DataPartsVector getDataPartsVector() const; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp index a0888732495..fa2847aa301 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp @@ -142,10 +142,7 @@ MergeTreeDataPart::MergeTreeDataPart(MergeTreeData & storage_, const String & na { } -MergeTreeDataPart::MergeTreeDataPart( - const MergeTreeData & storage_, - const String & name_, - const MergeTreePartInfo & info_) +MergeTreeDataPart::MergeTreeDataPart(const MergeTreeData & storage_, const String & name_, const MergeTreePartInfo & info_) : storage(storage_) , name(name_) , info(info_) @@ -350,7 +347,7 @@ UInt64 MergeTreeDataPart::calculateTotalSizeOnDisk(const String & from) return res; } -void MergeTreeDataPart::remove(bool force_recursive /*= false*/) const +void MergeTreeDataPart::remove() const { if (relative_path.empty()) throw Exception("Part relative_path cannot be empty. This is bug.", ErrorCodes::LOGICAL_ERROR); @@ -359,18 +356,18 @@ void MergeTreeDataPart::remove(bool force_recursive /*= false*/) const * - rename directory to temporary name; * - remove it recursive. * - * For temporary name we use "detached/deleting_" prefix. + * For temporary name we use "delete_tmp_" prefix. * - * NOTE: We cannot use "tmp_*" prefix, because there is a second thread, + * NOTE: We cannot use "tmp_delete_" prefix, because there is a second thread, * that calls "clearOldTemporaryDirectories" and removes all directories, that begin with "tmp_" and are old enough. * But when we removing data part, it can be old enough. And rename doesn't change mtime. * And a race condition can happen that will lead to "File not found" error here. - * We move directory to detached/, because if an attempt to remove directory after renaming failed for some reason - * there would be no way to remove directory from storage.full_path (except manually). */ + // TODO directory delete_tmp_ is never removed if server crashes before returning from this function + String from = storage.full_path + relative_path; - String to = storage.full_path + getRelativePathForDetachedPart("deleting_"); + String to = storage.full_path + "delete_tmp_" + name; Poco::File from_dir{from}; Poco::File to_dir{to}; @@ -403,13 +400,6 @@ void MergeTreeDataPart::remove(bool force_recursive /*= false*/) const return; } - if (force_recursive) - { - /// Part is not loaded (we don't know which files are there), so remove dir recursively. - to_dir.remove(true); - return; - } - try { /// Remove each expected file in directory, then remove directory itself. diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h index 98af00c071a..f41ea8af424 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h @@ -241,7 +241,7 @@ struct MergeTreeDataPart /// Calculate the total size of the entire directory with all the files static UInt64 calculateTotalSizeOnDisk(const String & from); - void remove(bool force_recursive = false) const; + void remove() const; /// Makes checks and move part to new directory /// Changes only relative_dir_name, you need to update other metadata (name, is_temp) explicitly diff --git a/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp b/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp index a9e31a988b3..449ea143e17 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -194,6 +194,7 @@ bool DetachedPartInfo::tryParseDetachedPartName(const String & dir_name, Detache part_info.dir_name = dir_name; /// First, try to parse as . + // TODO what if tryParsePartName will parse prefix as partition_id? if (MergeTreePartInfo::tryParsePartName(dir_name, &part_info, format_version)) return part_info.valid_name = true; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 98b3ae6ad6c..db5632c3fe9 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -994,6 +994,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons /// TODO: should we include PreComitted parts like in Replicated case? auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + // TODO should we throw an exception if parts_to_remove is empty? removePartsFromWorkingSet(parts_to_remove, true); if (detach) @@ -1013,22 +1014,6 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons } -void StorageMergeTree::dropDetached(const ASTPtr & partition, bool part, const Context & /*context*/) -{ - if (!part) // TODO - throw DB::Exception("DROP DETACHED PARTITION is not implemented, use DROP DETACHED PART", ErrorCodes::NOT_IMPLEMENTED); - - String part_id = partition->as().value.safeGet(); - validateDetachedPartName(part_id); - - DetachedPartInfo info; - DetachedPartInfo::tryParseDetachedPartName(part_id, info, format_version); - MergeTreeDataPart detached_part(*this, part_id, info); - detached_part.relative_path = "detached/" + part_id; - - detached_part.remove(true); -} - void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context) { // TODO: should get some locks to prevent race with 'alter … modify column' @@ -1069,42 +1054,30 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par } LOG_DEBUG(log, active_parts.size() << " of them are active"); parts = active_parts.getParts(); + + // TODO should we rename inactive parts? (see StorageReplicatedMergeTree::attachPartition) } + PartsTemporaryRename renamed_parts(full_path + source_dir); for (const auto & source_part_name : parts) + renamed_parts.addPart(source_part_name, "attaching_" + source_part_name); + + std::vector loaded_parts; + for (const auto & part_names : renamed_parts.old_and_new_names) { - MutableDataPartPtr part; - try - { - part = std::make_shared(*this, source_part_name); - part->relative_path = "detached/" + source_part_name; - part->renameTo("detached/attaching_" + source_part_name, false); - - LOG_DEBUG(log, "Checking data in " << part->relative_path); + LOG_DEBUG(log, "Checking data in " << part_names.second); + MutableDataPartPtr part = std::make_shared(*this, part_names.first); + part->relative_path = source_dir + part_names.second; loadPartAndFixMetadata(part); + loaded_parts.push_back(part); + } - LOG_INFO(log, "Attaching part " << source_part_name << " from " << part->relative_path); - renameTempPartAndAdd(part, &increment); - - LOG_INFO(log, "Finished attaching part"); - } - catch (...) - { - LOG_INFO(log, "Cannot attach part " << source_part_name << " :" << getCurrentExceptionMessage(false)); - - if (part && part->relative_path == "detached/attaching_" + source_part_name) - { - try - { - part->renameTo("detached/" + source_part_name, false); - } - catch (...) - { - tryLogCurrentException(log, __PRETTY_FUNCTION__); - } - } - - } + for (size_t i = 0; i < loaded_parts.size(); ++i) + { + LOG_INFO(log, "Attaching part " << loaded_parts[i]->name << " from " << renamed_parts.old_and_new_names[i].second); + renameTempPartAndAdd(loaded_parts[i], &increment); + renamed_parts.old_and_new_names[i].first.clear(); + LOG_INFO(log, "Finished attaching part"); } /// New parts with other data may appear in place of deleted parts. diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index fa2561e4ab2..0de9618d915 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -120,7 +120,6 @@ private: // Partition helpers void dropPartition(const ASTPtr & partition, bool detach, const Context & context); - void dropDetached(const ASTPtr & partition, bool part, const Context & context); void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context); void attachPartition(const ASTPtr & partition, bool part, const Context & context); void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 531678decc3..67577dee2b6 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3353,8 +3353,8 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part break; case PartitionCommand::DROP_DETACHED_PARTITION: - // TODO - throw DB::Exception("Not implemented yet", ErrorCodes::NOT_IMPLEMENTED); + dropDetached(command.partition, command.part, query_context); + break; case PartitionCommand::ATTACH_PARTITION: attachPartition(command.partition, command.part, query_context); @@ -3601,6 +3601,8 @@ void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool loaded_parts.push_back(loadPartAndFixMetadata(source_dir + part)); } + // TODO fix race with DROP DETACHED + ReplicatedMergeTreeBlockOutputStream output(*this, 0, 0, 0, false); /// TODO Allow to use quorum here. for (auto & part : loaded_parts) { diff --git a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference index 42a04fe5666..f30fc160dfb 100644 --- a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference +++ b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference @@ -1,17 +1,26 @@ === cannot attach active === -OK +OK1 0_1_1_0 1_2_2_0 2_3_3_0 3_4_4_0 16 120 -=== attach all valid parts === +=== check all parts before attaching === +OK2 +1_2_2_0 +1_4_4_0 +=== detached === +0_1_1_0 +0_3_3_0 +0_5_5_0 +attaching_0_6_6_0 +deleting_0_7_7_0 +=== attach === 0_5_5_0 0_6_6_0 1_2_2_0 1_4_4_0 16 120 === detached === -0_5_5_0 -deleting_0_7_7_0 attaching_0_6_6_0 +deleting_0_7_7_0 diff --git a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh index 4e9efa64ad1..a4afbe8f817 100755 --- a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh +++ b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh @@ -12,14 +12,13 @@ echo '=== cannot attach active ==='; $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_active"; $CLICKHOUSE_CLIENT --query="CREATE TABLE attach_active (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n"; $CLICKHOUSE_CLIENT --query="INSERT INTO attach_active SELECT number FROM system.numbers LIMIT 16"; -$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_active ATTACH PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK' +$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_active ATTACH PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK1' $CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_active' AND database='${cur_db}' ORDER BY name FORMAT TSV"; $CLICKHOUSE_CLIENT --query="SElECT count(), sum(n) FROM attach_active FORMAT TSV"; $CLICKHOUSE_CLIENT --query="DROP TABLE attach_active"; -echo '=== attach all valid parts ==='; $CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES"; $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_partitions"; $CLICKHOUSE_CLIENT --query="CREATE TABLE attach_partitions (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n"; @@ -27,18 +26,28 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO attach_partitions SELECT number FROM sys $CLICKHOUSE_CLIENT --query="INSERT INTO attach_partitions SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8"; $CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions DETACH PARTITION 0"; -sudo -n mkdir $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ 2>/dev/null || \ - mkdir $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ # broken part +sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ 2>/dev/null || \ + mkdir --mode=777 $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ # broken part sudo -n cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/ 2>/dev/null || \ cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/ sudo -n cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/deleting_0_7_7_0/ 2>/dev/null || \ cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/deleting_0_7_7_0/ -$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions ATTACH PARTITION 0"; +echo '=== check all parts before attaching ==='; +$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions ATTACH PARTITION 0" 2>&1 | grep "No columns in part 0_5_5_0" > /dev/null && echo 'OK2'; +$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV"; +echo '=== detached ==='; +$CLICKHOUSE_CLIENT --query="SELECT name FROM system.detached_parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV"; + +echo '=== attach ==='; +sudo -n rm -r $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ 2>/dev/null || \ + rm -r $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ +$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions ATTACH PARTITION 0"; $CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV"; $CLICKHOUSE_CLIENT --query="SElECT count(), sum(n) FROM attach_partitions FORMAT TSV"; + echo '=== detached ==='; -$CLICKHOUSE_CLIENT --query="SELECT name FROM system.detached_parts WHERE table='attach_partitions' AND database='${cur_db}' FORMAT TSV"; +$CLICKHOUSE_CLIENT --query="SELECT name FROM system.detached_parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV"; $CLICKHOUSE_CLIENT --query="DROP TABLE attach_partitions"; $CLICKHOUSE_CLIENT --query="SYSTEM START MERGES"; diff --git a/dbms/tests/queries/0_stateless/00975_drop_detached.reference b/dbms/tests/queries/0_stateless/00975_drop_detached.reference index 40732c908ab..2a355138980 100644 --- a/dbms/tests/queries/0_stateless/00975_drop_detached.reference +++ b/dbms/tests/queries/0_stateless/00975_drop_detached.reference @@ -1,2 +1,6 @@ -OK +OK1 +OK2 +OK3 0_3_3_0 +attaching_0_6_6_0 +deleting_0_7_7_0 diff --git a/dbms/tests/queries/0_stateless/00975_drop_detached.sh b/dbms/tests/queries/0_stateless/00975_drop_detached.sh index 3a5e920da75..71c0b5681fd 100755 --- a/dbms/tests/queries/0_stateless/00975_drop_detached.sh +++ b/dbms/tests/queries/0_stateless/00975_drop_detached.sh @@ -15,19 +15,19 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system. $CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8"; $CLICKHOUSE_CLIENT --query="ALTER TABLE drop_detached DETACH PARTITION 0"; -sudo -n mkdir $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/ 2>/dev/null || \ - mkdir $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/ -sudo -n mkdir $ch_dir/data/$cur_db/drop_detached/detached/deleting_0_7_7_0/ 2>/dev/null || \ - mkdir $ch_dir/data/$cur_db/drop_detached/detached/deleting_0_7_7_0/ -sudo -n mkdir $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/ 2>/dev/null || \ - mkdir $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/ +sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/ 2>/dev/null || \ + mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/ +sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/deleting_0_7_7_0/ 2>/dev/null || \ + mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/deleting_0_7_7_0/ +sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/ 2>/dev/null || \ + mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/ -$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK' +$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK1' $CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '0_1_1_0'" -$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'attaching_0_6_6_0'" -$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'deleting_0_7_7_0'" +$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'attaching_0_6_6_0'" 2>&1 | grep "Cannot drop part" > /dev/null && echo 'OK2' +$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'deleting_0_7_7_0'" 2>&1 | grep "Cannot drop part" > /dev/null && echo 'OK3' $CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'any_other_name'" -$CLICKHOUSE_CLIENT --query="SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='${cur_db}' FORMAT TSV"; +$CLICKHOUSE_CLIENT --query="SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='${cur_db}' ORDER BY name FORMAT TSV"; $CLICKHOUSE_CLIENT --query="DROP TABLE drop_detached"; $CLICKHOUSE_CLIENT --query="SYSTEM START MERGES"; From c6717e0d3f977e23aa9b778d00bcf64456e893cb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 30 Jul 2019 22:11:15 +0300 Subject: [PATCH 115/509] refactor attachPartition --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 71 +++++++++++++++++++ dbms/src/Storages/MergeTree/MergeTreeData.h | 3 + dbms/src/Storages/StorageMergeTree.cpp | 55 +------------- .../Storages/StorageReplicatedMergeTree.cpp | 68 ++---------------- 4 files changed, 83 insertions(+), 114 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 4b13ebaa99f..11ad7835b51 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -2640,6 +2640,77 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, const Cont renamed_parts.old_and_new_names.front().first.clear(); } +MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const ASTPtr & partition, bool attach_part, + const Context & context, PartsTemporaryRename & renamed_parts) +{ + String partition_id; + + if (attach_part) + partition_id = partition->as().value.safeGet(); + else + partition_id = getPartitionIDFromQuery(partition, context); + + String source_dir = "detached/"; + + /// Let's compose a list of parts that should be added. + Strings parts; + if (attach_part) + { + validateDetachedPartName(partition_id); + parts.push_back(partition_id); + } + else + { + LOG_DEBUG(log, "Looking for parts for partition " << partition_id << " in " << source_dir); + ActiveDataPartSet active_parts(format_version); + + std::set part_names; + for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it) + { + String name = it.name(); + MergeTreePartInfo part_info; + // TODO what if name contains "_tryN" suffix? + if (!MergeTreePartInfo::tryParsePartName(name, &part_info, format_version)) + continue; + if (part_info.partition_id != partition_id) + continue; + LOG_DEBUG(log, "Found part " << name); + active_parts.add(name); + part_names.insert(name); + } + LOG_DEBUG(log, active_parts.size() << " of them are active"); + parts = active_parts.getParts(); + + /// Inactive parts rename so they can not be attached in case of repeated ATTACH. + for (const auto & name : part_names) + { + // TODO maybe use PartsTemporaryRename here? + String containing_part = active_parts.getContainingPart(name); + if (!containing_part.empty() && containing_part != name) + Poco::File(full_path + source_dir + name).renameTo(full_path + source_dir + "inactive_" + name); + } + } + + /// Try to rename all parts before attaching to prevent race with DROP DETACHED and another ATTACH. + for (const auto & source_part_name : parts) + renamed_parts.addPart(source_part_name, "attaching_" + source_part_name); + + /// Synchronously check that added parts exist and are not broken. We will write checksums.txt if it does not exist. + LOG_DEBUG(log, "Checking parts"); + MutableDataPartsVector loaded_parts; + loaded_parts.reserve(parts.size()); + for (const auto & part_names : renamed_parts.old_and_new_names) + { + LOG_DEBUG(log, "Checking part " << part_names.second); + MutableDataPartPtr part = std::make_shared(*this, part_names.first); + part->relative_path = source_dir + part_names.second; + loadPartAndFixMetadata(part); + loaded_parts.push_back(part); + } + + return loaded_parts; +} + MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affordable_states) const { DataParts res; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 62cebf32f76..3592164fed5 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -407,6 +407,9 @@ public: void dropDetached(const ASTPtr & partition, bool part, const Context & context); + MutableDataPartsVector tryLoadPartsToAttach(const ASTPtr & partition, bool attach_part, + const Context & context, PartsTemporaryRename & renamed_parts); + /// Returns Committed parts DataParts getDataParts() const; DataPartsVector getDataPartsVector() const; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index db5632c3fe9..3464255e1b8 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -1018,59 +1018,8 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par { // TODO: should get some locks to prevent race with 'alter … modify column' - String partition_id; - - if (attach_part) - partition_id = partition->as().value.safeGet(); - else - partition_id = getPartitionIDFromQuery(partition, context); - - String source_dir = "detached/"; - - /// Let's make a list of parts to add. - Strings parts; - if (attach_part) - { - validateDetachedPartName(partition_id); - parts.push_back(partition_id); - } - else - { - LOG_DEBUG(log, "Looking for parts for partition " << partition_id << " in " << source_dir); - ActiveDataPartSet active_parts(format_version); - for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it) - { - const String & name = it.name(); - MergeTreePartInfo part_info; - /// Parts with prefix in name (e.g. attaching_1_3_3_0, deleting_1_3_3_0) will be ignored - // TODO what if name contains "_tryN" suffix? - if (!MergeTreePartInfo::tryParsePartName(name, &part_info, format_version) - || part_info.partition_id != partition_id) - { - continue; - } - LOG_DEBUG(log, "Found part " << name); - active_parts.add(name); - } - LOG_DEBUG(log, active_parts.size() << " of them are active"); - parts = active_parts.getParts(); - - // TODO should we rename inactive parts? (see StorageReplicatedMergeTree::attachPartition) - } - - PartsTemporaryRename renamed_parts(full_path + source_dir); - for (const auto & source_part_name : parts) - renamed_parts.addPart(source_part_name, "attaching_" + source_part_name); - - std::vector loaded_parts; - for (const auto & part_names : renamed_parts.old_and_new_names) - { - LOG_DEBUG(log, "Checking data in " << part_names.second); - MutableDataPartPtr part = std::make_shared(*this, part_names.first); - part->relative_path = source_dir + part_names.second; - loadPartAndFixMetadata(part); - loaded_parts.push_back(part); - } + PartsTemporaryRename renamed_parts(full_path + "detached/"); + MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, context, renamed_parts); for (size_t i = 0; i < loaded_parts.size(); ++i) { diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 67577dee2b6..7e192d77a33 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3545,70 +3545,16 @@ void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool assertNotReadonly(); - String partition_id; - - if (attach_part) - partition_id = partition->as().value.safeGet(); - else - partition_id = getPartitionIDFromQuery(partition, query_context); - - String source_dir = "detached/"; - - /// Let's compose a list of parts that should be added. - Strings parts; - if (attach_part) - { - validateDetachedPartName(partition_id); - parts.push_back(partition_id); - } - else - { - LOG_DEBUG(log, "Looking for parts for partition " << partition_id << " in " << source_dir); - ActiveDataPartSet active_parts(format_version); - - std::set part_names; - for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it) - { - String name = it.name(); - MergeTreePartInfo part_info; - // TODO what if name contains "_tryN" suffix? - if (!MergeTreePartInfo::tryParsePartName(name, &part_info, format_version)) - continue; - if (part_info.partition_id != partition_id) - continue; - LOG_DEBUG(log, "Found part " << name); - active_parts.add(name); - part_names.insert(name); - } - LOG_DEBUG(log, active_parts.size() << " of them are active"); - parts = active_parts.getParts(); - - /// Inactive parts rename so they can not be attached in case of repeated ATTACH. - for (const auto & name : part_names) - { - String containing_part = active_parts.getContainingPart(name); - if (!containing_part.empty() && containing_part != name) - Poco::File(full_path + source_dir + name).renameTo(full_path + source_dir + "inactive_" + name); - } - } - - /// Synchronously check that added parts exist and are not broken. We will write checksums.txt if it does not exist. - LOG_DEBUG(log, "Checking parts"); - std::vector loaded_parts; - for (const String & part : parts) - { - LOG_DEBUG(log, "Checking part " << part); - loaded_parts.push_back(loadPartAndFixMetadata(source_dir + part)); - } - - // TODO fix race with DROP DETACHED + PartsTemporaryRename renamed_parts(full_path + "detached/"); + MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); ReplicatedMergeTreeBlockOutputStream output(*this, 0, 0, 0, false); /// TODO Allow to use quorum here. - for (auto & part : loaded_parts) + for (size_t i = 0; i < loaded_parts.size(); ++i) { - String old_name = part->name; - output.writeExistingPart(part); - LOG_DEBUG(log, "Attached part " << old_name << " as " << part->name); + String old_name = loaded_parts[i]->name; + output.writeExistingPart(loaded_parts[i]); + renamed_parts.old_and_new_names[i].first.clear(); + LOG_DEBUG(log, "Attached part " << old_name << " as " << loaded_parts[i]->name); } } From bd493727b655d95bcadcce9bff7a3a4aad8cf304 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 31 Jul 2019 08:55:10 +0300 Subject: [PATCH 116/509] DOCAPI-7460: Added link to algorithm. --- docs/en/query_language/agg_functions/parametric_functions.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/query_language/agg_functions/parametric_functions.md b/docs/en/query_language/agg_functions/parametric_functions.md index da6052545dc..d27cb5d9431 100644 --- a/docs/en/query_language/agg_functions/parametric_functions.md +++ b/docs/en/query_language/agg_functions/parametric_functions.md @@ -10,6 +10,8 @@ Calculates a histogram. histogram(number_of_bins)(values) ``` +The functions uses [A Streaming Parallel Decision Tree Algorithm](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf). It calculates the borders of histogram bins automatically, and in common case the widths of bins are not equal. + **Parameters** `number_of_bins` — Number of bins for the histogram. From f0836553d449368cac474e9ed9f60d3120ea79c4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 31 Jul 2019 17:44:55 +0300 Subject: [PATCH 117/509] drop detached partition --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 94 +++++++++++++------ dbms/src/Storages/MergeTree/MergeTreeData.h | 10 +- .../Storages/MergeTree/MergeTreePartInfo.cpp | 2 +- .../Storages/MergeTree/MergeTreePartInfo.h | 3 +- dbms/src/Storages/PartitionCommands.cpp | 3 - dbms/src/Storages/StorageMergeTree.cpp | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 2 +- .../0_stateless/00974_attach_invalid_parts.sh | 8 +- .../0_stateless/00975_drop_detached.reference | 9 ++ .../0_stateless/00975_drop_detached.sh | 12 +++ 10 files changed, 103 insertions(+), 42 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 11ad7835b51..32cd3ad508e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1720,12 +1720,35 @@ MergeTreeData::AlterDataPartTransaction::~AlterDataPartTransaction() void MergeTreeData::PartsTemporaryRename::addPart(const String & old_name, const String & new_name) { - Poco::File(base_dir + old_name).renameTo(base_dir + new_name); old_and_new_names.push_back({old_name, new_name}); } +void MergeTreeData::PartsTemporaryRename::tryRenameAll() +{ + renamed = true; + for (size_t i = 0; i < old_and_new_names.size(); ++i) + { + try + { + const auto & names = old_and_new_names[i]; + if (names.first.empty() || names.second.empty()) + throw DB::Exception("Empty part name. Most likely it's a bug.", ErrorCodes::INCORRECT_FILE_NAME); + Poco::File(base_dir + names.first).renameTo(base_dir + names.second); + } + catch (...) + { + old_and_new_names.resize(i); + LOG_WARNING(storage.log, "Cannot rename parts to perform operation on them: " << getCurrentExceptionMessage(false)); + throw; + } + } +} + MergeTreeData::PartsTemporaryRename::~PartsTemporaryRename() { + // TODO what if server had crashed before this destructor was called? + if (!renamed) + return; for (const auto & names : old_and_new_names) { if (names.first.empty()) @@ -2621,46 +2644,60 @@ void MergeTreeData::validateDetachedPartName(const String & name) const Poco::File detached_part_dir(full_path + "detached/" + name); if (!detached_part_dir.exists()) throw DB::Exception("Detached part \"" + name + "\" not found" , ErrorCodes::BAD_DATA_PART_NAME); + + if (startsWith(name, "attaching_") || startsWith(name, "deleting_")) + throw DB::Exception("Cannot drop part " + name + ": " + "most likely it is used by another DROP or ATTACH query.", + ErrorCodes::BAD_DATA_PART_NAME); } -void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, const Context &) +void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, const Context & context) { - if (!part) // TODO - throw DB::Exception("DROP DETACHED PARTITION is not implemented, use DROP DETACHED PART", ErrorCodes::NOT_IMPLEMENTED); + PartsTemporaryRename renamed_parts(*this, full_path + "detached/"); - String part_id = partition->as().value.safeGet(); - validateDetachedPartName(part_id); - if (startsWith(part_id, "attaching_") || startsWith(part_id, "deleting_")) - throw DB::Exception("Cannot drop part " + part_id + ": " - "most likely it is used by another DROP or ATTACH query.", ErrorCodes::BAD_DATA_PART_NAME); + if (part) + { + String part_name = partition->as().value.safeGet(); + validateDetachedPartName(part_name); + renamed_parts.addPart(part_name, "deleting_" + part_name); + } + else + { + String partition_id = getPartitionIDFromQuery(partition, context); + DetachedPartsInfo detached_parts = getDetachedParts(); + for (const auto & part_info : detached_parts) + if (part_info.valid_name && part_info.partition_id == partition_id + && part_info.prefix != "attaching" && part_info.prefix != "deleting") + renamed_parts.addPart(part_info.dir_name, "deleting_" + part_info.dir_name); + } - PartsTemporaryRename renamed_parts(full_path + "detached/"); - renamed_parts.addPart(part_id, "deleting_" + part_id); - Poco::File(renamed_parts.base_dir + renamed_parts.old_and_new_names.front().second).remove(true); - renamed_parts.old_and_new_names.front().first.clear(); + LOG_DEBUG(log, "Will drop " << renamed_parts.old_and_new_names.size() << " detached parts."); + + renamed_parts.tryRenameAll(); + + for (auto & names : renamed_parts.old_and_new_names) + { + Poco::File(renamed_parts.base_dir + names.second).remove(true); + LOG_DEBUG(log, "Dropped detached part " << names.first); + names.first.clear(); + } } MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const ASTPtr & partition, bool attach_part, const Context & context, PartsTemporaryRename & renamed_parts) { - String partition_id; - - if (attach_part) - partition_id = partition->as().value.safeGet(); - else - partition_id = getPartitionIDFromQuery(partition, context); - String source_dir = "detached/"; /// Let's compose a list of parts that should be added. - Strings parts; if (attach_part) { - validateDetachedPartName(partition_id); - parts.push_back(partition_id); + String part_id = partition->as().value.safeGet(); + validateDetachedPartName(part_id); + renamed_parts.addPart(part_id, "attaching_" + part_id); } else { + String partition_id = getPartitionIDFromQuery(partition, context); LOG_DEBUG(log, "Looking for parts for partition " << partition_id << " in " << source_dir); ActiveDataPartSet active_parts(format_version); @@ -2670,6 +2707,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const String name = it.name(); MergeTreePartInfo part_info; // TODO what if name contains "_tryN" suffix? + /// Parts with prefix in name (e.g. attaching_1_3_3_0, deleting_1_3_3_0) will be ignored if (!MergeTreePartInfo::tryParsePartName(name, &part_info, format_version)) continue; if (part_info.partition_id != partition_id) @@ -2679,26 +2717,26 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const part_names.insert(name); } LOG_DEBUG(log, active_parts.size() << " of them are active"); - parts = active_parts.getParts(); /// Inactive parts rename so they can not be attached in case of repeated ATTACH. for (const auto & name : part_names) { - // TODO maybe use PartsTemporaryRename here? String containing_part = active_parts.getContainingPart(name); if (!containing_part.empty() && containing_part != name) + // TODO maybe use PartsTemporaryRename here? Poco::File(full_path + source_dir + name).renameTo(full_path + source_dir + "inactive_" + name); + else + renamed_parts.addPart(name, "attaching_" + name); } } /// Try to rename all parts before attaching to prevent race with DROP DETACHED and another ATTACH. - for (const auto & source_part_name : parts) - renamed_parts.addPart(source_part_name, "attaching_" + source_part_name); + renamed_parts.tryRenameAll(); /// Synchronously check that added parts exist and are not broken. We will write checksums.txt if it does not exist. LOG_DEBUG(log, "Checking parts"); MutableDataPartsVector loaded_parts; - loaded_parts.reserve(parts.size()); + loaded_parts.reserve(renamed_parts.old_and_new_names.size()); for (const auto & part_names : renamed_parts.old_and_new_names) { LOG_DEBUG(log, "Checking part " << part_names.second); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 3592164fed5..9f5d0961d27 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -251,16 +251,20 @@ public: struct PartsTemporaryRename : private boost::noncopyable { - PartsTemporaryRename(const String & base_dir_) : base_dir(base_dir_) {} + PartsTemporaryRename(const MergeTreeData & storage_, const String & base_dir_) : storage(storage_), base_dir(base_dir_) {} + + void addPart(const String & old_name, const String & new_name); /// Renames part from old_name to new_name - void addPart(const String & old_name, const String & new_name); + void tryRenameAll(); /// Renames all added parts from new_name to old_name if old name is not empty ~PartsTemporaryRename(); + const MergeTreeData & storage; String base_dir; std::vector> old_and_new_names; + bool renamed = false; }; /// Parameters for various modes. @@ -401,7 +405,7 @@ public: DataPartsVector getAllDataPartsVector(DataPartStateVector * out_states = nullptr) const; /// Returns all detached parts - std::vector getDetachedParts() const; + DetachedPartsInfo getDetachedParts() const; void validateDetachedPartName(const String & name) const; diff --git a/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp b/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp index 449ea143e17..3ee330b6d1a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -194,7 +194,7 @@ bool DetachedPartInfo::tryParseDetachedPartName(const String & dir_name, Detache part_info.dir_name = dir_name; /// First, try to parse as . - // TODO what if tryParsePartName will parse prefix as partition_id? + // TODO what if tryParsePartName will parse prefix as partition_id? It can happen if dir_name doesn't contain mutation number at the end if (MergeTreePartInfo::tryParsePartName(dir_name, &part_info, format_version)) return part_info.valid_name = true; diff --git a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h index 25cf46ad46d..9fe0fbab533 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h @@ -92,7 +92,6 @@ struct MergeTreePartInfo /// addition to the above fields. struct DetachedPartInfo : public MergeTreePartInfo { - /// Suddenly, name of detached part may contain suffix (such as _tryN), which is ignored by MergeTreePartInfo::tryParsePartName(...) String dir_name; String prefix; @@ -102,4 +101,6 @@ struct DetachedPartInfo : public MergeTreePartInfo static bool tryParseDetachedPartName(const String & dir_name, DetachedPartInfo & part_info, MergeTreeDataFormatVersion format_version); }; +using DetachedPartsInfo = std::vector; + } diff --git a/dbms/src/Storages/PartitionCommands.cpp b/dbms/src/Storages/PartitionCommands.cpp index bab3f6ced24..0537482dbc1 100644 --- a/dbms/src/Storages/PartitionCommands.cpp +++ b/dbms/src/Storages/PartitionCommands.cpp @@ -25,9 +25,6 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * } else if (command_ast->type == ASTAlterCommand::DROP_DETACHED_PARTITION) { - if (!command_ast->part) // TODO - throw DB::Exception("Not implemented yet", ErrorCodes::NOT_IMPLEMENTED); - PartitionCommand res; res.type = DROP_DETACHED_PARTITION; res.partition = command_ast->partition; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 3464255e1b8..c2ee4854c39 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -1018,7 +1018,7 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par { // TODO: should get some locks to prevent race with 'alter … modify column' - PartsTemporaryRename renamed_parts(full_path + "detached/"); + PartsTemporaryRename renamed_parts(*this, full_path + "detached/"); MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, context, renamed_parts); for (size_t i = 0; i < loaded_parts.size(); ++i) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 7e192d77a33..5109d9f7e54 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3545,7 +3545,7 @@ void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool assertNotReadonly(); - PartsTemporaryRename renamed_parts(full_path + "detached/"); + PartsTemporaryRename renamed_parts(*this, full_path + "detached/"); MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); ReplicatedMergeTreeBlockOutputStream output(*this, 0, 0, 0, false); /// TODO Allow to use quorum here. diff --git a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh index a4afbe8f817..db45cfe7f21 100755 --- a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh +++ b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh @@ -28,10 +28,10 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO attach_partitions SELECT number FROM sys $CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions DETACH PARTITION 0"; sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ 2>/dev/null || \ mkdir --mode=777 $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ # broken part -sudo -n cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/ 2>/dev/null || \ - cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/ -sudo -n cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/deleting_0_7_7_0/ 2>/dev/null || \ - cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/deleting_0_7_7_0/ +sudo -n cp -pr $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/ 2>/dev/null || \ + cp -pr $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/ +sudo -n cp -pr $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/deleting_0_7_7_0/ 2>/dev/null || \ + cp -pr $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/deleting_0_7_7_0/ echo '=== check all parts before attaching ==='; $CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions ATTACH PARTITION 0" 2>&1 | grep "No columns in part 0_5_5_0" > /dev/null && echo 'OK2'; diff --git a/dbms/tests/queries/0_stateless/00975_drop_detached.reference b/dbms/tests/queries/0_stateless/00975_drop_detached.reference index 2a355138980..414ac4b1927 100644 --- a/dbms/tests/queries/0_stateless/00975_drop_detached.reference +++ b/dbms/tests/queries/0_stateless/00975_drop_detached.reference @@ -1,6 +1,15 @@ +=== validate part name === OK1 OK2 OK3 +=== drop detached part === +0_3_3_0 +1_2_2_0 +1_4_4_0 +attaching_0_6_6_0 +deleting_0_7_7_0 +prefix_1_2_2_0_0 +=== drop detached partition === 0_3_3_0 attaching_0_6_6_0 deleting_0_7_7_0 diff --git a/dbms/tests/queries/0_stateless/00975_drop_detached.sh b/dbms/tests/queries/0_stateless/00975_drop_detached.sh index 71c0b5681fd..8da831b019a 100755 --- a/dbms/tests/queries/0_stateless/00975_drop_detached.sh +++ b/dbms/tests/queries/0_stateless/00975_drop_detached.sh @@ -15,19 +15,31 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system. $CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8"; $CLICKHOUSE_CLIENT --query="ALTER TABLE drop_detached DETACH PARTITION 0"; +$CLICKHOUSE_CLIENT --query="ALTER TABLE drop_detached DETACH PARTITION 1"; sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/ 2>/dev/null || \ mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/ sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/deleting_0_7_7_0/ 2>/dev/null || \ mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/deleting_0_7_7_0/ sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/ 2>/dev/null || \ mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/ +sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/prefix_1_2_2_0_0/ 2>/dev/null || \ + mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/prefix_1_2_2_0_0/ +#sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/prefix_1_2_2_0/ 2>/dev/null || \ +# mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/prefix_1_2_2_0/ +echo '=== validate part name ===' $CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK1' $CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '0_1_1_0'" $CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'attaching_0_6_6_0'" 2>&1 | grep "Cannot drop part" > /dev/null && echo 'OK2' $CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'deleting_0_7_7_0'" 2>&1 | grep "Cannot drop part" > /dev/null && echo 'OK3' $CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'any_other_name'" +echo '=== drop detached part ===' $CLICKHOUSE_CLIENT --query="SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='${cur_db}' ORDER BY name FORMAT TSV"; + +echo '=== drop detached partition ===' +$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PARTITION 1" +$CLICKHOUSE_CLIENT --query="SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='${cur_db}' ORDER BY name FORMAT TSV"; + $CLICKHOUSE_CLIENT --query="DROP TABLE drop_detached"; $CLICKHOUSE_CLIENT --query="SYSTEM START MERGES"; From 662364b7206ee374752889ccf17f25b7fecb49b6 Mon Sep 17 00:00:00 2001 From: Sergi Vladykin Date: Wed, 31 Jul 2019 22:36:03 +0300 Subject: [PATCH 118/509] performance bug fix: never read the children if we are not going to return any data --- dbms/src/DataStreams/TTLBlockInputStream.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/DataStreams/TTLBlockInputStream.cpp b/dbms/src/DataStreams/TTLBlockInputStream.cpp index 78d8d25e8bc..0ad72876ea9 100644 --- a/dbms/src/DataStreams/TTLBlockInputStream.cpp +++ b/dbms/src/DataStreams/TTLBlockInputStream.cpp @@ -63,10 +63,6 @@ TTLBlockInputStream::TTLBlockInputStream( Block TTLBlockInputStream::readImpl() { - Block block = children.at(0)->read(); - if (!block) - return block; - /// Skip all data if table ttl is expired for part if (storage.hasTableTTL() && old_ttl_infos.table_ttl.max <= current_time) { @@ -74,6 +70,10 @@ Block TTLBlockInputStream::readImpl() return {}; } + Block block = children.at(0)->read(); + if (!block) + return block; + if (storage.hasTableTTL() && old_ttl_infos.table_ttl.min <= current_time) removeRowsWithExpiredTableTTL(block); From 10e562cddd8668b2f564d387f9b82644990cb19d Mon Sep 17 00:00:00 2001 From: BayoNet Date: Fri, 2 Aug 2019 09:50:13 +0300 Subject: [PATCH 119/509] Link fix. --- docs/en/interfaces/formats.md | 2 +- docs/ru/interfaces/formats.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 8c9badaa02f..46a553089d9 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -173,7 +173,7 @@ Empty unquoted input values are replaced with default values for the respective [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) is enabled. -`NULL` is formatted as `\N` or `NULL` or an empty unquoted string (see settings [input_format_csv_unquoted_null_literal_as_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) and [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#settings-input_format_defaults_for_omitted_fields)). +`NULL` is formatted as `\N` or `NULL` or an empty unquoted string (see settings [input_format_csv_unquoted_null_literal_as_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) and [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). The CSV format supports the output of totals and extremes the same way as `TabSeparated`. diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 14d6408b7e7..20aa78630b7 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -165,7 +165,7 @@ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMA При парсинге, все значения могут парситься как в кавычках, так и без кавычек. Поддерживаются как двойные, так и одинарные кавычки. Строки также могут быть без кавычек. В этом случае они парсятся до символа-разделителя или перевода строки (CR или LF). В нарушение RFC, в случае парсинга строк не в кавычках, начальные и конечные пробелы и табы игнорируются. В качестве перевода строки, поддерживаются как Unix (LF), так и Windows (CR LF) и Mac OS Classic (LF CR) варианты. -`NULL` форматируется в виде `\N` или `NULL` или пустой неэкранированной строки (см. настройки [input_format_csv_unquoted_null_literal_as_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) и [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#settings-input_format_defaults_for_omitted_fields)). +`NULL` форматируется в виде `\N` или `NULL` или пустой неэкранированной строки (см. настройки [input_format_csv_unquoted_null_literal_as_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) и [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). Если установлена настройка [input_format_defaults_for_omitted_fields = 1](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) и тип столбца не `Nullable(T)`, то пустые значения без кавычек заменяются значениями по умолчанию для типа данных столбца. From cc3900817c82e2a153218961813826bdc422d746 Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Mon, 5 Aug 2019 16:00:27 +0300 Subject: [PATCH 120/509] Clickhouse-benchmark comparison mode --- dbms/programs/benchmark/Benchmark.cpp | 213 ++++++++++++++++++-------- 1 file changed, 149 insertions(+), 64 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index c69e9a54feb..3d16974db4b 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -51,16 +51,29 @@ class Benchmark : public Poco::Util::Application { public: Benchmark(unsigned concurrency_, double delay_, - const String & host_, UInt16 port_, bool secure_, const String & default_database_, + const std::vector & hosts_, const std::vector & ports_, + bool secure_, const String & default_database_, const String & user_, const String & password_, const String & stage, bool randomize_, size_t max_iterations_, double max_time_, const String & json_path_, const Settings & settings_) : concurrency(concurrency_), delay(delay_), queue(concurrency), - connections(concurrency, host_, port_, default_database_, user_, password_, "benchmark", Protocol::Compression::Enable, secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable), randomize(randomize_), max_iterations(max_iterations_), max_time(max_time_), json_path(json_path_), settings(settings_), global_context(Context::createGlobal()), pool(concurrency) { + const auto secure = secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable; + + connections.reserve(ports_.size()); + comparison_info_total.reserve(ports_.size()); + comparison_info_per_interval.reserve(ports_.size()); + + for (size_t i = 0; i < ports_.size(); ++i) + { + connections.emplace_back(std::make_shared(concurrency, hosts_[i], ports_[i], default_database_, user_, password_, "benchmark", Protocol::Compression::Enable, secure)); + comparison_info_per_interval.emplace_back(std::make_shared()); + comparison_info_total.emplace_back(std::make_shared()); + } + global_context.makeGlobalContext(); std::cerr << std::fixed << std::setprecision(3); @@ -101,18 +114,22 @@ public: } private: - using Query = std::string; + using Entry = ConnectionPool::Entry; + using EntryPtr = std::shared_ptr; + using EntryPtrs = std::vector; unsigned concurrency; double delay; + using Query = std::string; using Queries = std::vector; Queries queries; using Queue = ConcurrentBoundedQueue; Queue queue; - ConnectionPool connections; + ConnectionPoolPtrs connections; + bool randomize; size_t max_iterations; double max_time; @@ -160,15 +177,17 @@ private: } }; - Stats info_per_interval; - Stats info_total; + using MultiStats = std::vector>; + MultiStats comparison_info_per_interval; + MultiStats comparison_info_total; + + Stopwatch total_watch; Stopwatch delay_watch; std::mutex mutex; ThreadPool pool; - void readQueries() { ReadBufferFromFileDescriptor in(STDIN_FILENO); @@ -213,7 +232,7 @@ private: return false; } - if (max_time > 0 && info_total.watch.elapsedSeconds() >= max_time) + if (max_time > 0 && total_watch.elapsedSeconds() >= max_time) { std::cout << "Stopping launch of queries. Requested time limit is exhausted.\n"; return false; @@ -227,8 +246,8 @@ private: if (delay > 0 && delay_watch.elapsedSeconds() > delay) { - printNumberOfQueriesExecuted(info_total.queries); - report(info_per_interval); + printNumberOfQueriesExecuted(queries_executed); + report(comparison_info_per_interval); delay_watch.restart(); } } @@ -242,11 +261,21 @@ private: std::uniform_int_distribution distribution(0, queries.size() - 1); for (size_t i = 0; i < concurrency; ++i) - pool.schedule(std::bind(&Benchmark::thread, this, - connections.get(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings)))); + { + EntryPtrs connection_entries; + connection_entries.reserve(connections.size()); + + for (const auto & connection : connections) + connection_entries.emplace_back(std::make_shared(connection->get(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings)))); + + pool.schedule(std::bind(&Benchmark::thread, this, connection_entries)); + } InterruptListener interrupt_listener; - info_per_interval.watch.restart(); + + for (auto & connection_stats : comparison_info_per_interval) + connection_stats->watch.restart(); + delay_watch.restart(); /// Push queries into queue @@ -262,17 +291,20 @@ private: } pool.wait(); - info_total.watch.stop(); + total_watch.stop(); + + for (auto & connection_stats : comparison_info_total) + connection_stats->watch.stop(); if (!json_path.empty()) - reportJSON(info_total, json_path); + reportJSON(comparison_info_total, json_path); - printNumberOfQueriesExecuted(info_total.queries); - report(info_total); + printNumberOfQueriesExecuted(queries_executed); + report(comparison_info_total); } - void thread(ConnectionPool::Entry connection) + void thread(EntryPtrs & connection_entries) { Query query; @@ -296,8 +328,7 @@ private: if (shutdown || (max_iterations && queries_executed == max_iterations)) return; } - - execute(connection, query); + execute(connection_entries, query); ++queries_executed; } } @@ -309,20 +340,21 @@ private: } } - - void execute(ConnectionPool::Entry & connection, Query & query) + void execute(EntryPtrs & connection_entries, Query & query) { + size_t connection_index = rand() % connection_entries.size(); + Stopwatch watch; RemoteBlockInputStream stream( - *connection, + *(*connection_entries[connection_index]), query, {}, global_context, &settings, nullptr, Tables(), query_processing_stage); Progress progress; stream.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); }); stream.readPrefix(); - while (Block block = stream.read()) - ; + while (Block block = stream.read()); + stream.readSuffix(); const BlockStreamProfileInfo & info = stream.getProfileInfo(); @@ -330,33 +362,48 @@ private: double seconds = watch.elapsedSeconds(); std::lock_guard lock(mutex); - info_per_interval.add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes); - info_total.add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes); + + comparison_info_per_interval[connection_index]->add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes); + comparison_info_total[connection_index]->add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes); } - - void report(Stats & info) + void report(MultiStats & infos) { std::lock_guard lock(mutex); - /// Avoid zeros, nans or exceptions - if (0 == info.queries) - return; + std::cerr << "\n"; + size_t info_counter = 1; + for (auto & info : infos) + { + /// Avoid zeros, nans or exceptions + if (0 == info->queries) + return; - double seconds = info.watch.elapsedSeconds(); + double seconds = info->watch.elapsedSeconds(); - std::cerr - << "\n" - << "QPS: " << (info.queries / seconds) << ", " - << "RPS: " << (info.read_rows / seconds) << ", " - << "MiB/s: " << (info.read_bytes / seconds / 1048576) << ", " - << "result RPS: " << (info.result_rows / seconds) << ", " - << "result MiB/s: " << (info.result_bytes / seconds / 1048576) << "." - << "\n"; + std::cerr + << "connection: " << info_counter++ << ", " + << "QPS: " << (info->queries / seconds) << ", " + << "RPS: " << (info->read_rows / seconds) << ", " + << "MiB/s: " << (info->read_bytes / seconds / 1048576) << ", " + << "result RPS: " << (info->result_rows / seconds) << ", " + << "result MiB/s: " << (info->result_bytes / seconds / 1048576) << "." + << "\n"; + } + std::cerr << "\n\t\t"; + for (size_t i = 1; i <= infos.size(); ++i) + std::cerr << "connection " << i << "\t"; + + std::cerr << "\n"; auto print_percentile = [&](double percent) { - std::cerr << percent << "%\t" << info.sampler.quantileInterpolated(percent / 100.0) << " sec." << std::endl; + std::cerr << percent << "%\t\t"; + for (auto & info : infos) + { + std::cerr << info->sampler.quantileInterpolated(percent / 100.0) << " sec." << "\t"; + } + std::cerr << "\n"; }; for (int percent = 0; percent <= 90; percent += 10) @@ -367,10 +414,11 @@ private: print_percentile(99.9); print_percentile(99.99); - info.clear(); + for (auto & info : infos) + info->clear(); } - void reportJSON(Stats & info, const std::string & filename) + void reportJSON(MultiStats & infos, const std::string & filename) { WriteBufferFromFile json_out(filename); @@ -381,36 +429,47 @@ private: json_out << double_quote << key << ": " << value << (with_comma ? ",\n" : "\n"); }; - auto print_percentile = [&json_out, &info](auto percent, bool with_comma = true) + auto print_percentile = [&json_out](Stats & info, auto percent, bool with_comma = true) { json_out << "\"" << percent << "\"" << ": " << info.sampler.quantileInterpolated(percent / 100.0) << (with_comma ? ",\n" : "\n"); }; json_out << "{\n"; - json_out << double_quote << "statistics" << ": {\n"; + for (size_t i = 1; i <= infos.size(); ++i) + { + auto info = infos[i]; - double seconds = info.watch.elapsedSeconds(); - print_key_value("QPS", info.queries / seconds); - print_key_value("RPS", info.read_rows / seconds); - print_key_value("MiBPS", info.read_bytes / seconds); - print_key_value("RPS_result", info.result_rows / seconds); - print_key_value("MiBPS_result", info.result_bytes / seconds); - print_key_value("num_queries", info.queries.load(), false); + json_out << double_quote << "connection_" + toString(i) << ": {\n"; + json_out << double_quote << "statistics" << ": {\n"; - json_out << "},\n"; + double seconds = info->watch.elapsedSeconds(); + print_key_value("QPS", info->queries / seconds); + print_key_value("RPS", info->read_rows / seconds); + print_key_value("MiBPS", info->read_bytes / seconds); + print_key_value("RPS_result", info->result_rows / seconds); + print_key_value("MiBPS_result", info->result_bytes / seconds); + print_key_value("num_queries", info->queries.load(), false); - json_out << double_quote << "query_time_percentiles" << ": {\n"; + json_out << "},\n"; - for (int percent = 0; percent <= 90; percent += 10) - print_percentile(percent); + json_out << double_quote << "query_time_percentiles" << ": {\n"; - print_percentile(95); - print_percentile(99); - print_percentile(99.9); - print_percentile(99.99, false); + for (int percent = 0; percent <= 90; percent += 10) + print_percentile(*info, percent); - json_out << "}\n"; + print_percentile(*info, 95); + print_percentile(*info, 99); + print_percentile(*info, 99.9); + print_percentile(*info, 99.99, false); + + json_out << "}\n"; + + if (i == infos.size()) + json_out << "}\n"; + else + json_out << "},\n"; + } json_out << "}\n"; } @@ -450,7 +509,11 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("randomize,r", value()->default_value(false), "randomize order of execution") ("json", value()->default_value(""), "write final report to specified file in JSON format") ("host,h", value()->default_value("localhost"), "") + ("host1,h1", value()->default_value("localhost"), "Is used for comparison mode") + ("host2,h2", value()->default_value("localhost"), "Is used for comparison mode") ("port", value()->default_value(9000), "") + ("port1", value()->default_value(9000), "Is used for comparison mode") + ("port2", value()->default_value(9000), "Is used for comparison mode") ("secure,s", "Use TLS connection") ("user", value()->default_value("default"), "") ("password", value()->default_value(""), "") @@ -472,6 +535,28 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) return 1; } + std::vector ports; + std::vector hosts; + + if (options["port1"].as() != options["port2"].as() + || options["host1"].as() != options["host2"].as()) + { + ports.reserve(2); + hosts.reserve(2); + hosts.emplace_back(options["host1"].as()); + hosts.emplace_back(options["host2"].as()); + ports.emplace_back(options["port1"].as()); + ports.emplace_back(options["port2"].as()); + } + else + { + ports.reserve(1); + hosts.reserve(1); + hosts.emplace_back(options["host"].as()); + ports.emplace_back(options["port"].as()); + } + + print_stacktrace = options.count("stacktrace"); UseSSL use_ssl; @@ -479,8 +564,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) Benchmark benchmark( options["concurrency"].as(), options["delay"].as(), - options["host"].as(), - options["port"].as(), + hosts, + ports, options.count("secure"), options["database"].as(), options["user"].as(), From 3ea83e35d7365b7b7b973c53a22479e085a33d66 Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Mon, 5 Aug 2019 21:38:08 +0300 Subject: [PATCH 121/509] Multiple hosts and ports --- dbms/programs/benchmark/Benchmark.cpp | 52 ++++++++------------------- 1 file changed, 14 insertions(+), 38 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 3d16974db4b..d004faf7057 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -51,7 +51,7 @@ class Benchmark : public Poco::Util::Application { public: Benchmark(unsigned concurrency_, double delay_, - const std::vector & hosts_, const std::vector & ports_, + const std::vector & hosts_, const std::vector & ports_, bool secure_, const String & default_database_, const String & user_, const String & password_, const String & stage, bool randomize_, size_t max_iterations_, double max_time_, @@ -62,14 +62,18 @@ public: json_path(json_path_), settings(settings_), global_context(Context::createGlobal()), pool(concurrency) { const auto secure = secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable; + size_t connections_cnt = std::max(ports_.size(), hosts_.size()); - connections.reserve(ports_.size()); - comparison_info_total.reserve(ports_.size()); - comparison_info_per_interval.reserve(ports_.size()); + connections.reserve(connections_cnt); + comparison_info_total.reserve(connections_cnt); + comparison_info_per_interval.reserve(connections_cnt); - for (size_t i = 0; i < ports_.size(); ++i) + for (size_t i = 0; i < connections_cnt; ++i) { - connections.emplace_back(std::make_shared(concurrency, hosts_[i], ports_[i], default_database_, user_, password_, "benchmark", Protocol::Compression::Enable, secure)); + UInt16 cur_port = i >= ports_.size() ? 9000 : ports_[i]; + std::string cur_host = i >= hosts_.size() ? "localhost" : hosts_[i]; + + connections.emplace_back(std::make_shared(concurrency, cur_host, cur_port, default_database_, user_, password_, "benchmark", Protocol::Compression::Enable, secure)); comparison_info_per_interval.emplace_back(std::make_shared()); comparison_info_total.emplace_back(std::make_shared()); } @@ -452,7 +456,6 @@ private: print_key_value("num_queries", info->queries.load(), false); json_out << "},\n"; - json_out << double_quote << "query_time_percentiles" << ": {\n"; for (int percent = 0; percent <= 90; percent += 10) @@ -470,7 +473,6 @@ private: else json_out << "},\n"; } - json_out << "}\n"; } @@ -508,12 +510,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("timelimit,t", value()->default_value(0.), "stop launch of queries after specified time limit") ("randomize,r", value()->default_value(false), "randomize order of execution") ("json", value()->default_value(""), "write final report to specified file in JSON format") - ("host,h", value()->default_value("localhost"), "") - ("host1,h1", value()->default_value("localhost"), "Is used for comparison mode") - ("host2,h2", value()->default_value("localhost"), "Is used for comparison mode") - ("port", value()->default_value(9000), "") - ("port1", value()->default_value(9000), "Is used for comparison mode") - ("port2", value()->default_value(9000), "Is used for comparison mode") + ("host,h", value>()->default_value(std::vector{"localhost"}, "localhost") ,"note that more than one host can be described") + ("port,p", value>()->default_value(std::vector{9000}, "9000") ,"note that more than one port can be described") ("secure,s", "Use TLS connection") ("user", value()->default_value("default"), "") ("password", value()->default_value(""), "") @@ -535,28 +533,6 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) return 1; } - std::vector ports; - std::vector hosts; - - if (options["port1"].as() != options["port2"].as() - || options["host1"].as() != options["host2"].as()) - { - ports.reserve(2); - hosts.reserve(2); - hosts.emplace_back(options["host1"].as()); - hosts.emplace_back(options["host2"].as()); - ports.emplace_back(options["port1"].as()); - ports.emplace_back(options["port2"].as()); - } - else - { - ports.reserve(1); - hosts.reserve(1); - hosts.emplace_back(options["host"].as()); - ports.emplace_back(options["port"].as()); - } - - print_stacktrace = options.count("stacktrace"); UseSSL use_ssl; @@ -564,8 +540,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) Benchmark benchmark( options["concurrency"].as(), options["delay"].as(), - hosts, - ports, + options["host"].as>(), + options["port"].as>(), options.count("secure"), options["database"].as(), options["user"].as(), From a5009c675948b4c742d1413f194dbe5ca3ea3f03 Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Tue, 6 Aug 2019 13:54:35 +0300 Subject: [PATCH 122/509] --cumulative option added, and time statistics fixed --- dbms/programs/benchmark/Benchmark.cpp | 67 ++++++++++++--------------- 1 file changed, 30 insertions(+), 37 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index d004faf7057..63f58ae391f 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -52,13 +52,13 @@ class Benchmark : public Poco::Util::Application public: Benchmark(unsigned concurrency_, double delay_, const std::vector & hosts_, const std::vector & ports_, - bool secure_, const String & default_database_, + bool cumulative_, bool secure_, const String & default_database_, const String & user_, const String & password_, const String & stage, bool randomize_, size_t max_iterations_, double max_time_, const String & json_path_, const Settings & settings_) : - concurrency(concurrency_), delay(delay_), queue(concurrency), - randomize(randomize_), max_iterations(max_iterations_), max_time(max_time_), + concurrency(concurrency_), delay(delay_), queue(concurrency), randomize(randomize_), + cumulative(cumulative_), max_iterations(max_iterations_), max_time(max_time_), json_path(json_path_), settings(settings_), global_context(Context::createGlobal()), pool(concurrency) { const auto secure = secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable; @@ -135,6 +135,7 @@ private: ConnectionPoolPtrs connections; bool randomize; + bool cumulative; size_t max_iterations; double max_time; String json_path; @@ -149,12 +150,12 @@ private: struct Stats { - Stopwatch watch; std::atomic queries{0}; size_t read_rows = 0; size_t read_bytes = 0; size_t result_rows = 0; size_t result_bytes = 0; + double work_time = 0; using Sampler = ReservoirSampler; Sampler sampler {1 << 16}; @@ -162,6 +163,7 @@ private: void add(double seconds, size_t read_rows_inc, size_t read_bytes_inc, size_t result_rows_inc, size_t result_bytes_inc) { ++queries; + work_time += seconds; read_rows += read_rows_inc; read_bytes += read_bytes_inc; result_rows += result_rows_inc; @@ -171,8 +173,8 @@ private: void clear() { - watch.restart(); queries = 0; + work_time = 0; read_rows = 0; read_bytes = 0; result_rows = 0; @@ -251,7 +253,7 @@ private: if (delay > 0 && delay_watch.elapsedSeconds() > delay) { printNumberOfQueriesExecuted(queries_executed); - report(comparison_info_per_interval); + cumulative ? report(comparison_info_total) : report(comparison_info_per_interval); delay_watch.restart(); } } @@ -276,10 +278,6 @@ private: } InterruptListener interrupt_listener; - - for (auto & connection_stats : comparison_info_per_interval) - connection_stats->watch.restart(); - delay_watch.restart(); /// Push queries into queue @@ -297,9 +295,6 @@ private: pool.wait(); total_watch.stop(); - for (auto & connection_stats : comparison_info_total) - connection_stats->watch.stop(); - if (!json_path.empty()) reportJSON(comparison_info_total, json_path); @@ -383,15 +378,14 @@ private: if (0 == info->queries) return; - double seconds = info->watch.elapsedSeconds(); - std::cerr - << "connection: " << info_counter++ << ", " - << "QPS: " << (info->queries / seconds) << ", " - << "RPS: " << (info->read_rows / seconds) << ", " - << "MiB/s: " << (info->read_bytes / seconds / 1048576) << ", " - << "result RPS: " << (info->result_rows / seconds) << ", " - << "result MiB/s: " << (info->result_bytes / seconds / 1048576) << "." + << "connection " << info_counter++ << ", " + << "queries " << info->queries << ", " + << "QPS: " << (info->queries / info->work_time) << ", " + << "RPS: " << (info->read_rows / info->work_time) << ", " + << "MiB/s: " << (info->read_bytes / info->work_time / 1048576) << ", " + << "result RPS: " << (info->result_rows / info->work_time) << ", " + << "result MiB/s: " << (info->result_bytes / info->work_time / 1048576) << "." << "\n"; } std::cerr << "\n\t\t"; @@ -418,8 +412,9 @@ private: print_percentile(99.9); print_percentile(99.99); - for (auto & info : infos) - info->clear(); + if (!cumulative) + for (auto & info : infos) + info->clear(); } void reportJSON(MultiStats & infos, const std::string & filename) @@ -442,17 +437,16 @@ private: for (size_t i = 1; i <= infos.size(); ++i) { - auto info = infos[i]; + auto info = infos[i - 1]; json_out << double_quote << "connection_" + toString(i) << ": {\n"; json_out << double_quote << "statistics" << ": {\n"; - double seconds = info->watch.elapsedSeconds(); - print_key_value("QPS", info->queries / seconds); - print_key_value("RPS", info->read_rows / seconds); - print_key_value("MiBPS", info->read_bytes / seconds); - print_key_value("RPS_result", info->result_rows / seconds); - print_key_value("MiBPS_result", info->result_bytes / seconds); + print_key_value("QPS", info->queries / info->work_time); + print_key_value("RPS", info->read_rows / info->work_time); + print_key_value("MiBPS", info->read_bytes / info->work_time); + print_key_value("RPS_result", info->result_rows / info->work_time); + print_key_value("MiBPS_result", info->result_bytes / info->work_time); print_key_value("num_queries", info->queries.load(), false); json_out << "},\n"; @@ -467,12 +461,9 @@ private: print_percentile(*info, 99.99, false); json_out << "}\n"; - - if (i == infos.size()) - json_out << "}\n"; - else - json_out << "},\n"; + json_out << (i == infos.size() ? "}\n" : "},\n"); } + json_out << "}\n"; } @@ -510,8 +501,9 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("timelimit,t", value()->default_value(0.), "stop launch of queries after specified time limit") ("randomize,r", value()->default_value(false), "randomize order of execution") ("json", value()->default_value(""), "write final report to specified file in JSON format") - ("host,h", value>()->default_value(std::vector{"localhost"}, "localhost") ,"note that more than one host can be described") - ("port,p", value>()->default_value(std::vector{9000}, "9000") ,"note that more than one port can be described") + ("host,h", value>()->default_value(std::vector{"localhost"}, "localhost"), "note that more than one host can be described") + ("port,p", value>()->default_value(std::vector{9000}, "9000"), "note that more than one port can be described") + ("cumulative", "prints cumulative data instead of data per interval") ("secure,s", "Use TLS connection") ("user", value()->default_value("default"), "") ("password", value()->default_value(""), "") @@ -542,6 +534,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) options["delay"].as(), options["host"].as>(), options["port"].as>(), + options.count("cumulative"), options.count("secure"), options["database"].as(), options["user"].as(), From 8e535a9cb0fe66b44de1be0f5537827aad8d7767 Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Mon, 8 Jul 2019 12:41:28 +0300 Subject: [PATCH 123/509] Implement nextInBlock function --- dbms/src/Functions/nextInBlock.cpp | 159 ++++++++++++++++++ .../registerFunctionsMiscellaneous.cpp | 2 + .../0_stateless/00957_next_in_block.reference | 12 ++ .../0_stateless/00957_next_in_block.sql | 22 +++ 4 files changed, 195 insertions(+) create mode 100644 dbms/src/Functions/nextInBlock.cpp create mode 100644 dbms/tests/queries/0_stateless/00957_next_in_block.reference create mode 100644 dbms/tests/queries/0_stateless/00957_next_in_block.sql diff --git a/dbms/src/Functions/nextInBlock.cpp b/dbms/src/Functions/nextInBlock.cpp new file mode 100644 index 00000000000..e672e539f25 --- /dev/null +++ b/dbms/src/Functions/nextInBlock.cpp @@ -0,0 +1,159 @@ +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +// Implements function, giving value for column in next row +// Example: +// | c1 | +// | 10 | +// | 20 | +// SELECT c1, nextInBlock(c1, 1) as c2: +// | c1 | c2 | +// | 10 | 20 | +// | 20 | 0 | +class FunctionNextInBlock : public IFunction +{ +public: + static constexpr auto name = "nextInBlock"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + /// Get the name of the function. + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 0; } + + bool isVariadic() const override { return true; } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + size_t number_of_arguments = arguments.size(); + + if (number_of_arguments < 1 || number_of_arguments > 3) + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + toString(number_of_arguments) + + ", should be from 1 to 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + // second argument must be a positive, constant column + if (number_of_arguments == 2 && !isUnsignedInteger(arguments[1])) + throw Exception( + "Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + + " - should be positive integer", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + // check that default value has supertype with first argument + if (number_of_arguments == 3) + { + DataTypes types = {arguments[0], arguments[2]}; + try + { + return getLeastSupertype(types); + } + catch (const Exception &) + { + throw Exception( + "Illegal types of arguments (" + types[0]->getName() + ", " + types[1]->getName() + + ")" + " of function " + + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + } + + return arguments[0]; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + size_t offset_value = 1; + + if (arguments.size() > 1) + { + auto offset_column = block.getByPosition(arguments[1]); + if (!isColumnConst(*offset_column.column)) + throw Exception("Second argument of function " + getName() + " should be constant", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + Field offset_field = (*block.getByPosition(arguments[1]).column)[0]; + auto raw_value = safeGet(offset_field); + + if (raw_value == 0) + throw Exception( + "Second argument of function " + getName() + " should be positive integer, " + toString(raw_value) + " given", + ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + offset_value = raw_value; + } + + auto has_column_for_missing = arguments.size() == 3; + + DataTypes types = {block.getByPosition(arguments[0]).type}; + if (has_column_for_missing) + { + types.push_back(block.getByPosition(arguments[2]).type); + } + const DataTypePtr & result_type = getLeastSupertype(types); + + auto column = result_type->createColumn(); + column->reserve(input_rows_count); + + auto source_column = block.getByPosition(arguments[0]).column; + + for (size_t i = offset_value; i < input_rows_count; i++) + { + column->insertFrom(*source_column, i); + } + + if (has_column_for_missing) + { + auto default_values_column = block.getByPosition(arguments[2]).column; + size_t starting_pos = offset_value > input_rows_count ? 0 : input_rows_count - offset_value; + if (isColumnConst(*default_values_column)) + { + Field constant_value = (*default_values_column)[0]; + for (size_t i = starting_pos; i < input_rows_count; i++) + { + column->insert(constant_value); + } + } + else + { + for (size_t i = starting_pos; i < input_rows_count; i++) + { + column->insertFrom(*default_values_column, i); + } + } + } + else + { + for (size_t i = 0; i < std::min(offset_value, input_rows_count); i++) + { + column->insertDefault(); + } + } + + block.getByPosition(result).column = std::move(column); + } +}; + +void registerFunctionNextInBlock(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} \ No newline at end of file diff --git a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp index 6d201d65bd3..57ccfcd11c9 100644 --- a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp @@ -17,6 +17,7 @@ void registerFunctionBlockSize(FunctionFactory &); void registerFunctionBlockNumber(FunctionFactory &); void registerFunctionRowNumberInBlock(FunctionFactory &); void registerFunctionRowNumberInAllBlocks(FunctionFactory &); +void registerFunctionNextInBlock(FunctionFactory &); void registerFunctionSleep(FunctionFactory &); void registerFunctionSleepEachRow(FunctionFactory &); void registerFunctionMaterialize(FunctionFactory &); @@ -67,6 +68,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionBlockNumber(factory); registerFunctionRowNumberInBlock(factory); registerFunctionRowNumberInAllBlocks(factory); + registerFunctionNextInBlock(factory); registerFunctionSleep(factory); registerFunctionSleepEachRow(factory); registerFunctionMaterialize(factory); diff --git a/dbms/tests/queries/0_stateless/00957_next_in_block.reference b/dbms/tests/queries/0_stateless/00957_next_in_block.reference new file mode 100644 index 00000000000..860ce6dc1ba --- /dev/null +++ b/dbms/tests/queries/0_stateless/00957_next_in_block.reference @@ -0,0 +1,12 @@ +0 1 +1 0 +0 2 +1 0 +2 0 +0 0 +1 0 +0 2 +1 3 +2 4 +3 1000 +4 1000 diff --git a/dbms/tests/queries/0_stateless/00957_next_in_block.sql b/dbms/tests/queries/0_stateless/00957_next_in_block.sql new file mode 100644 index 00000000000..7cbd932cf1a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00957_next_in_block.sql @@ -0,0 +1,22 @@ +-- no arguments +select nextInBlock(); -- { serverError 42 } +-- greater than 3 arguments +select nextInBlock(1,2,3,4); -- { serverError 42 } +-- zero offset value +select nextInBlock(dummy, 0); -- { serverError 69 } +-- negative offset value +select nextInBlock(dummy, -1); -- { serverError 43 } +-- non-constant offset value +select nextInBlock(dummy, dummy); -- { serverError 43 } +-- bad default value +select nextInBlock(dummy, 1, 'hello'); -- { serverError 43 } +-- single argument test +select number, nextInBlock(number) from numbers(2); +-- filling by column's default value +select number, nextInBlock(number, 2) from numbers(3); +-- offset is greater that block - should fill everything with defaults +select number, nextInBlock(number, 5) from numbers(2); +-- substitution by constant for missing values +select number, nextInBlock(number, 2, 1000) from numbers(5); +-- substitution by expression +-- select number, nextInBlock(number, 2, number % 2) from numbers(5); \ No newline at end of file From cfec857f2c5685aafc1aeaa061497baf9ab39c53 Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Mon, 8 Jul 2019 17:53:02 +0300 Subject: [PATCH 124/509] Add trailing newline --- dbms/src/Functions/nextInBlock.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/nextInBlock.cpp b/dbms/src/Functions/nextInBlock.cpp index e672e539f25..eeb33e28146 100644 --- a/dbms/src/Functions/nextInBlock.cpp +++ b/dbms/src/Functions/nextInBlock.cpp @@ -156,4 +156,4 @@ void registerFunctionNextInBlock(FunctionFactory & factory) factory.registerFunction(); } -} \ No newline at end of file +} From e12cb8da6df1efd5b21d4e4cd1207ed8e2a55185 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Aug 2019 15:52:08 +0300 Subject: [PATCH 125/509] Add some changes --- dbms/src/Core/SettingsCommon.h | 60 +++++++----------- dbms/src/Storages/AlterCommands.cpp | 34 ++++++++++ dbms/src/Storages/AlterCommands.h | 7 +++ .../00980_merge_alter_settings.reference | 2 + .../00980_merge_alter_settings.sql | 39 ++++++++++++ ...keeper_merge_tree_alter_settings.reference | 10 +++ ...80_zookeeper_merge_tree_alter_settings.sql | 62 +++++++++++++++++++ 7 files changed, 177 insertions(+), 37 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00980_merge_alter_settings.reference create mode 100644 dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql create mode 100644 dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.reference create mode 100644 dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.sql diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index e9dce9c9dcc..618fc00aa75 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -321,7 +321,7 @@ private: size_t offset_of_changed; StringRef name; StringRef description; - bool immutable; + bool updateable; GetStringFunction get_string; GetFieldFunction get_field; SetStringFunction set_string; @@ -401,7 +401,7 @@ public: const_reference(const const_reference & src) = default; const StringRef & getName() const { return member->name; } const StringRef & getDescription() const { return member->description; } - bool isImmutable() const { return member->immutable; } + bool isUpdateable() const { return member->updateable; } bool isChanged() const { return member->isChanged(*collection); } Field getValue() const { return member->get_field(*collection); } String getValueAsString() const { return member->get_string(*collection); } @@ -421,6 +421,18 @@ public: reference(const const_reference & src) : const_reference(src) {} void setValue(const Field & value) { this->member->set_field(*const_cast(this->collection), value); } void setValue(const String & value) { this->member->set_string(*const_cast(this->collection), value); } + void updateValue(const Field & value) + { + if (!this->member->updateable) + throw Exception("Setting '" + this->member->name.toString() + "' is restricted for updates.", ErrorCodes::IMMUTABLE_SETTING); + setValue(value); + } + void updateValue(const String & value) + { + if (!this->member->updateable) + throw Exception("Setting '" + this->member->name.toString() + "' is restricted for updates.", ErrorCodes::IMMUTABLE_SETTING); + setValue(value); + } }; /// Iterator to iterating through all the settings. @@ -503,39 +515,14 @@ public: void set(size_t index, const String & value) { (*this)[index].setValue(value); } void set(const String & name, const String & value) { (*this)[name].setValue(value); } - /// Updates setting's value. - void update(size_t index, const Field & value) - { - auto ref = (*this)[index]; - if (ref.isImmutable()) - throw Exception("Cannot modify immutable setting '" + ref.getName().toString() + "'", ErrorCodes::IMMUTABLE_SETTING); - ref.setValue(value); - } + /// Updates setting's value. Checks it' mutability. + void update(size_t index, const Field & value) { (*this)[index].updateValue(value); } - void update(const String & name, const Field & value) - { - auto ref = (*this)[name]; - if (ref.isImmutable()) - throw Exception("Cannot modify immutable setting '" + ref.getName().toString() + "'", ErrorCodes::IMMUTABLE_SETTING); - ref.setValue(value); - } + void update(const String & name, const Field & value) { (*this)[name].updateValue(value); } - /// Updates setting's value. Read value in text form from string (for example, from configuration file or from URL parameter). - void update(size_t index, const String & value) - { - auto ref = (*this)[index]; - if (ref.isImmutable()) - throw Exception("Cannot modify immutable setting '" + ref.getName().toString() + "'", ErrorCodes::IMMUTABLE_SETTING); - (*this)[index].setValue(value); - } + void update(size_t index, const String & value) { (*this)[index].updateValue(value); } - void update(const String & name, const String & value) - { - auto ref = (*this)[name]; - if (ref.isImmutable()) - throw Exception("Cannot modify immutable setting '" + ref.getName().toString() + "'", ErrorCodes::IMMUTABLE_SETTING); - (*this)[name].setValue(value); - } + void update(const String & name, const String & value) { (*this)[name].updateValue(value); } /// Returns value of a setting. Field get(size_t index) const { return (*this)[index].getValue(); } @@ -600,7 +587,7 @@ public: return found_changes; } - /// Applies changes to the settings. + /// Applies changes to the settings. Doesn't check settings mutability. void loadFromChange(const SettingChange & change) { set(change.name, change.value); @@ -612,7 +599,7 @@ public: loadFromChange(change); } - /// Applies changes to the settings. + /// Applies changes to the settings, checks settings mutability void updateFromChange(const SettingChange & change) { update(change.name, change.value); @@ -701,7 +688,7 @@ public: #define IMPLEMENT_SETTINGS_COLLECTION_ADD_MUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ static_assert(std::is_same_v().NAME.changed), bool>); \ add({offsetof(Derived, NAME.changed), \ - StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), false, \ + StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), true, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \ &Functions::NAME##_serialize, &Functions::NAME##_deserialize, \ @@ -710,10 +697,9 @@ public: #define IMPLEMENT_SETTINGS_COLLECTION_ADD_IMMUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ static_assert(std::is_same_v().NAME.changed), bool>); \ add({offsetof(Derived, NAME.changed), \ - StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), true, \ + StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), false, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \ &Functions::NAME##_serialize, &Functions::NAME##_deserialize, \ &Functions::NAME##_castValueWithoutApplying }); - } diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 789b66b271a..16d6f0a0aa0 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -332,6 +332,11 @@ bool AlterCommand::isMutable() const return true; } +bool AlterCommand::isSettingsAlter() const +{ + return type == MODIFY_SETTING; +} + void AlterCommands::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast, SettingsChanges & changes) const { @@ -567,6 +572,31 @@ void AlterCommands::applyForColumnsOnly(ColumnsDescription & columns_description columns_description = std::move(out_columns_description); } + +void AlterCommands::applyForSettingsOnly(SettingsChanges & changes) const +{ + ColumnsDescription out_columns_description; + IndicesDescription indices_description; + ASTPtr out_order_by; + ASTPtr out_primary_key; + ASTPtr out_ttl_table; + SettingsChanges out_changes; + apply(out_columns_description, indices_description, out_order_by, out_primary_key, out_ttl_table, out_changes); + + if (out_columns_description.begin() != out_columns_description.end()) + throw Exception("Alter modifying columns, but only settings change applied.", ErrorCodes::LOGICAL_ERROR); + if (out_order_by) + throw Exception("Alter modifying ORDER BY expression, but only settings change applied.", ErrorCodes::LOGICAL_ERROR); + if (out_primary_key) + throw Exception("Alter modifying PRIMARY KEY expression, but only settings change applied.", ErrorCodes::LOGICAL_ERROR); + if (!indices_description.indices.empty()) + throw Exception("Alter modifying indices, but only settings change applied.", ErrorCodes::NOT_IMPLEMENTED); + if (out_ttl_table) + throw Exception("Alter modifying TTL, but only settings change applied.", ErrorCodes::NOT_IMPLEMENTED); + + changes = std::move(out_changes); +} + bool AlterCommands::isMutable() const { for (const auto & param : *this) @@ -578,4 +608,8 @@ bool AlterCommands::isMutable() const return false; } +bool AlterCommands::isSettingsAlter() const +{ + return std::all_of(begin(), end(), [](const AlterCommand & c) { return c.isSettingsAlter(); }); +} } diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index fe05d014cba..9b59514020f 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -93,6 +93,9 @@ struct AlterCommand /// Checks that not only metadata touched by that command bool isMutable() const; + + /// checks that only settings changed by alter + bool isSettingsAlter() const; }; class Context; @@ -107,8 +110,12 @@ public: /// Used for primitive table engines, where only columns metadata can be changed void applyForColumnsOnly(ColumnsDescription & columns_description) const; + /// Apply alter commands only for settings. Exception will be thrown if any other part of table structure will be modified. + void applyForSettingsOnly(SettingsChanges & changes) const; + void validate(const IStorage & table, const Context & context); bool isMutable() const; + bool isSettingsAlter() const; }; } diff --git a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.reference b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.reference new file mode 100644 index 00000000000..ad64346f90d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.reference @@ -0,0 +1,2 @@ +CREATE TABLE default.table_for_alter (`id` UInt64, `Data` String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 4096 +CREATE TABLE default.table_for_alter (`id` UInt64, `Data` String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 4096, parts_to_throw_insert = 1, parts_to_delay_insert = 1 diff --git a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql new file mode 100644 index 00000000000..52f5e4dd444 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql @@ -0,0 +1,39 @@ +DROP TABLE IF EXISTS log_for_alter; + +CREATE TABLE log_for_alter ( + id UInt64, + Data String +) ENGINE = Log(); + +ALTER TABLE log_for_alter MODIFY SETTING aaa=123; -- { serverError 468 } + +DROP TABLE IF EXISTS log_for_alter; + +DROP TABLE IF EXISTS table_for_alter; + +CREATE TABLE table_for_alter ( + id UInt64, + Data String +) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096; + +ALTER TABLE table_for_alter MODIFY SETTING index_granularity=555; -- { serverError 469 } + +SHOW CREATE TABLE table_for_alter; + +ALTER TABLE table_for_alter MODIFY SETTING parts_to_throw_insert = 1, parts_to_delay_insert = 1; + +SHOW CREATE TABLE table_for_alter; + +INSERT INTO table_for_alter VALUES (1, '1'); +INSERT INTO table_for_alter VALUES (2, '2'); -- { serverError 252 } + +DETACH TABLE table_for_alter; + +ATTACH TABLE table_for_alter; + +INSERT INTO table_for_alter VALUES (2, '2'); -- { serverError 252 } + +ALTER TABLE table_for_alter MODIFY SETTING xxx_yyy=124; -- { serverError 115 } + +DROP TABLE IF EXISTS table_for_alter; + diff --git a/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.reference b/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.reference new file mode 100644 index 00000000000..e55bfadd538 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.reference @@ -0,0 +1,10 @@ +CREATE TABLE default.replicated_table_for_alter1 (`id` UInt64, `Data` String) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/replicated_table_for_alter\', \'1\') ORDER BY id SETTINGS index_granularity = 8192 +CREATE TABLE default.replicated_table_for_alter1 (`id` UInt64, `Data` String) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/replicated_table_for_alter\', \'1\') ORDER BY id SETTINGS index_granularity = 8192 +4 +4 +4 +4 +6 +6 +CREATE TABLE default.replicated_table_for_alter1 (`id` UInt64, `Data` String) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/replicated_table_for_alter\', \'1\') ORDER BY id SETTINGS index_granularity = 8192, use_minimalistic_part_header_in_zookeeper = 1 +CREATE TABLE default.replicated_table_for_alter2 (`id` UInt64, `Data` String) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/replicated_table_for_alter\', \'2\') ORDER BY id SETTINGS index_granularity = 8192, parts_to_throw_insert = 1, parts_to_delay_insert = 1 diff --git a/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.sql b/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.sql new file mode 100644 index 00000000000..9c7a7c33329 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.sql @@ -0,0 +1,62 @@ +DROP TABLE IF EXISTS replicated_table_for_alter1; +DROP TABLE IF EXISTS replicated_table_for_alter2; + +CREATE TABLE replicated_table_for_alter1 ( + id UInt64, + Data String +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_table_for_alter', '1') ORDER BY id; + +CREATE TABLE replicated_table_for_alter2 ( + id UInt64, + Data String +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_table_for_alter', '2') ORDER BY id; + +SHOW CREATE TABLE replicated_table_for_alter1; + +ALTER TABLE replicated_table_for_alter1 MODIFY SETTING index_granularity = 4096; -- { serverError 469 } + +SHOW CREATE TABLE replicated_table_for_alter1; + +INSERT INTO replicated_table_for_alter2 VALUES (1, '1'), (2, '2'); + +SYSTEM SYNC REPLICA replicated_table_for_alter1; + +ALTER TABLE replicated_table_for_alter1 MODIFY SETTING use_minimalistic_part_header_in_zookeeper = 1; + +INSERT INTO replicated_table_for_alter1 VALUES (3, '3'), (4, '4'); + +SYSTEM SYNC REPLICA replicated_table_for_alter2; + +SELECT COUNT() FROM replicated_table_for_alter1; +SELECT COUNT() FROM replicated_table_for_alter2; + +DETACH TABLE replicated_table_for_alter2; +ATTACH TABLE replicated_table_for_alter2; + +DETACH TABLE replicated_table_for_alter1; +ATTACH TABLE replicated_table_for_alter1; + +SELECT COUNT() FROM replicated_table_for_alter1; +SELECT COUNT() FROM replicated_table_for_alter2; + +ALTER TABLE replicated_table_for_alter2 MODIFY SETTING parts_to_throw_insert = 1, parts_to_delay_insert = 1; +INSERT INTO replicated_table_for_alter2 VALUES (3, '1'), (4, '2'); -- { serverError 252 } + +INSERT INTO replicated_table_for_alter1 VALUES (5, '5'), (6, '6'); + +SYSTEM SYNC REPLICA replicated_table_for_alter2; + +SELECT COUNT() FROM replicated_table_for_alter1; +SELECT COUNT() FROM replicated_table_for_alter2; + +DETACH TABLE replicated_table_for_alter2; +ATTACH TABLE replicated_table_for_alter2; + +DETACH TABLE replicated_table_for_alter1; +ATTACH TABLE replicated_table_for_alter1; + +SHOW CREATE TABLE replicated_table_for_alter1; +SHOW CREATE TABLE replicated_table_for_alter2; + +DROP TABLE IF EXISTS replicated_table_for_alter2; +DROP TABLE IF EXISTS replicated_table_for_alter1; From 02878c233a130d0a815da597a7747ed5c4080e6b Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Tue, 6 Aug 2019 15:54:33 +0300 Subject: [PATCH 126/509] True randomness --- dbms/programs/benchmark/Benchmark.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 63f58ae391f..61acc95ad9e 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -341,7 +341,10 @@ private: void execute(EntryPtrs & connection_entries, Query & query) { - size_t connection_index = rand() % connection_entries.size(); + /// Randomly choosing connection index + pcg64 generator(randomSeed()); + std::uniform_int_distribution distribution(0, connection_entries.size() - 1); + size_t connection_index = distribution(generator); Stopwatch watch; RemoteBlockInputStream stream( From e73ea8a13120f1eecaa2362d50f9127c04e9c615 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 6 Aug 2019 16:00:56 +0300 Subject: [PATCH 127/509] Make InterpreterSelectQuery::analyzeExpressions static. --- .../Interpreters/InterpreterSelectQuery.cpp | 65 +++++++++++++------ .../src/Interpreters/InterpreterSelectQuery.h | 12 +++- 2 files changed, 55 insertions(+), 22 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 9682d0e29e4..6c486b807be 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -401,16 +401,25 @@ QueryPipeline InterpreterSelectQuery::executeWithProcessors() } InterpreterSelectQuery::AnalysisResult -InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage, bool dry_run, const FilterInfoPtr & filter_info) +InterpreterSelectQuery::analyzeExpressions( + const ASTSelectQuery & query, + const NamesAndTypesList & source_columns, + ExpressionAnalyzer & query_analyzer, + QueryProcessingStage::Enum from_stage, + QueryProcessingStage::Enum to_stage, + const Context & context, + const StoragePtr & storage, + bool dry_run, + const FilterInfoPtr & filter_info) { AnalysisResult res; /// Do I need to perform the first part of the pipeline - running on remote servers during distributed processing. res.first_stage = from_stage < QueryProcessingStage::WithMergeableState - && options.to_stage >= QueryProcessingStage::WithMergeableState; + && to_stage >= QueryProcessingStage::WithMergeableState; /// Do I need to execute the second part of the pipeline - running on the initiating server during distributed processing. res.second_stage = from_stage <= QueryProcessingStage::WithMergeableState - && options.to_stage > QueryProcessingStage::WithMergeableState; + && to_stage > QueryProcessingStage::WithMergeableState; /** First we compose a chain of actions and remember the necessary steps from it. * Regardless of from_stage and to_stage, we will compose a complete sequence of actions to perform optimization and @@ -465,8 +474,6 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage { ExpressionActionsChain chain(context); - auto & query = getSelectQuery(); - Names additional_required_columns_after_prewhere; if (storage && query.sample_size()) @@ -503,7 +510,7 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage chain.addStep(); } - if (query_analyzer->appendPrewhere(chain, !res.first_stage, additional_required_columns_after_prewhere)) + if (query_analyzer.appendPrewhere(chain, !res.first_stage, additional_required_columns_after_prewhere)) { has_prewhere = true; @@ -513,11 +520,11 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage chain.addStep(); } - res.need_aggregate = query_analyzer->hasAggregation(); + res.need_aggregate = query_analyzer.hasAggregation(); - query_analyzer->appendArrayJoin(chain, dry_run || !res.first_stage); + query_analyzer.appendArrayJoin(chain, dry_run || !res.first_stage); - if (query_analyzer->appendJoin(chain, dry_run || !res.first_stage)) + if (query_analyzer.appendJoin(chain, dry_run || !res.first_stage)) { res.before_join = chain.getLastActions(); if (!res.hasJoin()) @@ -525,7 +532,7 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage chain.addStep(); } - if (query_analyzer->appendWhere(chain, dry_run || !res.first_stage)) + if (query_analyzer.appendWhere(chain, dry_run || !res.first_stage)) { where_step_num = chain.steps.size() - 1; has_where = res.has_where = true; @@ -535,13 +542,13 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage if (res.need_aggregate) { - query_analyzer->appendGroupBy(chain, dry_run || !res.first_stage); - query_analyzer->appendAggregateFunctionsArguments(chain, dry_run || !res.first_stage); + query_analyzer.appendGroupBy(chain, dry_run || !res.first_stage); + query_analyzer.appendAggregateFunctionsArguments(chain, dry_run || !res.first_stage); res.before_aggregation = chain.getLastActions(); finalizeChain(chain); - if (query_analyzer->appendHaving(chain, dry_run || !res.second_stage)) + if (query_analyzer.appendHaving(chain, dry_run || !res.second_stage)) { res.has_having = true; res.before_having = chain.getLastActions(); @@ -550,20 +557,20 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage } /// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers. - query_analyzer->appendSelect(chain, dry_run || (res.need_aggregate ? !res.second_stage : !res.first_stage)); + query_analyzer.appendSelect(chain, dry_run || (res.need_aggregate ? !res.second_stage : !res.first_stage)); res.selected_columns = chain.getLastStep().required_output; - res.has_order_by = query_analyzer->appendOrderBy(chain, dry_run || (res.need_aggregate ? !res.second_stage : !res.first_stage)); + res.has_order_by = query_analyzer.appendOrderBy(chain, dry_run || (res.need_aggregate ? !res.second_stage : !res.first_stage)); res.before_order_and_select = chain.getLastActions(); chain.addStep(); - if (query_analyzer->appendLimitBy(chain, dry_run || !res.second_stage)) + if (query_analyzer.appendLimitBy(chain, dry_run || !res.second_stage)) { res.has_limit_by = true; res.before_limit_by = chain.getLastActions(); chain.addStep(); } - query_analyzer->appendProjectResult(chain); + query_analyzer.appendProjectResult(chain); res.final_projection = chain.getLastActions(); finalizeChain(chain); @@ -577,7 +584,7 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage if (res.has_having) res.before_having->prependProjectInput(); - res.subqueries_for_sets = query_analyzer->getSubqueriesForSets(); + res.subqueries_for_sets = query_analyzer.getSubqueriesForSets(); /// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows. if (res.prewhere_info) @@ -821,7 +828,16 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS else pipeline.streams.emplace_back(std::make_shared(source_header)); - expressions = analyzeExpressions(QueryProcessingStage::FetchColumns, true, filter_info); + expressions = analyzeExpressions( + getSelectQuery(), + source_columns, + *query_analyzer, + QueryProcessingStage::FetchColumns, + options.to_stage, + context, + storage, + true, + filter_info); if (storage && expressions.filter_info && expressions.prewhere_info) throw Exception("PREWHERE is not supported if the table is filtered by row-level security expression", ErrorCodes::ILLEGAL_PREWHERE); @@ -853,7 +869,16 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS pipeline.streams.push_back(prepared_input); } - expressions = analyzeExpressions(from_stage, false, filter_info); + expressions = analyzeExpressions( + getSelectQuery(), + source_columns, + *query_analyzer, + from_stage, + options.to_stage, + context, + storage, + false, + filter_info); if (from_stage == QueryProcessingStage::WithMergeableState && options.to_stage == QueryProcessingStage::WithMergeableState) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index f6f3c0baf19..11f8f405ecc 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -172,8 +172,16 @@ private: FilterInfoPtr filter_info; }; - AnalysisResult analyzeExpressions(QueryProcessingStage::Enum from_stage, bool dry_run, const FilterInfoPtr & filter_info); - + static AnalysisResult analyzeExpressions( + const ASTSelectQuery & query, + const NamesAndTypesList & source_columns, + ExpressionAnalyzer & query_analyzer, + QueryProcessingStage::Enum from_stage, + QueryProcessingStage::Enum to_stage, + const Context & context, + const StoragePtr & storage, + bool dry_run, + const FilterInfoPtr & filter_info); /** From which table to read. With JOIN, the "left" table is returned. */ From 84fd4906cac21b2b63ac89242fba707b4d59eefb Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Aug 2019 16:04:29 +0300 Subject: [PATCH 128/509] Changes for engines --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 24 +++++++++++++++++++ dbms/src/Storages/MergeTree/MergeTreeData.h | 8 +++++++ dbms/src/Storages/StorageMergeTree.cpp | 15 ++++++------ dbms/src/Storages/StorageMergeTree.h | 2 -- .../Storages/StorageReplicatedMergeTree.cpp | 12 +++++++++- 5 files changed, 51 insertions(+), 10 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index a99b6fbabeb..70c6c409c30 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -1583,6 +1584,29 @@ void MergeTreeData::alterDataPart( return; } +void MergeTreeData::alterSettings( + const SettingsChanges & new_changes, + const String & current_database_name, + const String & current_table_name, + const Context & context) +{ + settings.updateFromChanges(new_changes); + IDatabase::ASTModifier storage_modifier = [&] (IAST & ast) + { + if (!new_changes.empty()) + { + auto & storage_ast = ast.as(); + storage_ast.settings->changes.insert(storage_ast.settings->changes.end(), new_changes.begin(), new_changes.end()); + } + }; + context.getDatabase(current_database_name)->alterTable(context, current_table_name, getColumns(), getIndices(), storage_modifier); +} + +bool MergeTreeData::hasSetting(const String & setting_name) const +{ + return settings.findIndex(setting_name) != MergeTreeSettings::npos; +} + void MergeTreeData::removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr & data_part) { auto & empty_columns = data_part->empty_columns; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index cd95e5c7e58..d11074e1156 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -509,6 +509,14 @@ public: bool skip_sanity_checks, AlterDataPartTransactionPtr& transaction); + void alterSettings( + const SettingsChanges & new_changes, + const String & current_database_name, + const String & current_table_name, + const Context & context); + + bool hasSetting(const String & setting_name) const override; + /// Remove columns, that have been markedd as empty after zeroing values with expired ttl void removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr & data_part); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index e823136e3bb..438b9468b69 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -246,12 +246,19 @@ void StorageMergeTree::alter( if (!params.isMutable()) { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); + SettingsChanges new_changes; + if (params.isSettingsAlter()) + { + params.applyForSettingsOnly(new_changes); + alterSettings(new_changes, current_database_name, current_table_name, context); + return; + } + auto new_columns = getColumns(); auto new_indices = getIndices(); ASTPtr new_order_by_ast = order_by_ast; ASTPtr new_primary_key_ast = primary_key_ast; ASTPtr new_ttl_table_ast = ttl_table_ast; - SettingsChanges new_changes; params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes); IDatabase::ASTModifier storage_modifier = [&] (IAST & ast) { @@ -1031,12 +1038,6 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons } - -bool StorageMergeTree::hasSetting(const String & setting_name) const -{ - return settings.findIndex(setting_name) != MergeTreeSettings::npos; -} - void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context) { // TODO: should get some locks to prevent race with 'alter … modify column' diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index e2d24ceecdd..0df90604f67 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -128,8 +128,6 @@ private: friend class MergeTreeData; friend struct CurrentlyMergingPartsTagger; - bool hasSetting(const String & setting_name) const override; - protected: /** Attach the table with the appropriate name, along the appropriate path (with / at the end), diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 365f7e70143..23c49b05c1a 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3072,13 +3072,23 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p void StorageReplicatedMergeTree::alter( - const AlterCommands & params, const String & /*database_name*/, const String & /*table_name*/, + const AlterCommands & params, const String & current_database_name, const String & current_table_name, const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) { assertNotReadonly(); LOG_DEBUG(log, "Doing ALTER"); + if (params.isSettingsAlter()) + { + LOG_DEBUG(log, "ALTER settings only"); + lockStructureExclusively(table_lock_holder, query_context.getCurrentQueryId()); + SettingsChanges new_changes; + params.applyForSettingsOnly(new_changes); + alterSettings(new_changes, current_database_name, current_table_name, query_context); + return; + } + /// Alter is done by modifying the metadata nodes in ZK that are shared between all replicas /// (/columns, /metadata). We set contents of the shared nodes to the new values and wait while /// replicas asynchronously apply changes (see ReplicatedMergeTreeAlterThread.cpp) and modify From 303c4e5a58347969da67efccfc7890e33619176d Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Aug 2019 16:06:09 +0300 Subject: [PATCH 129/509] Make index_granularity_bytes immutable --- dbms/src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index c982bef324f..3012b9e1694 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -77,7 +77,7 @@ struct MergeTreeSettings : public SettingsCollection M(SettingBool, use_minimalistic_part_header_in_zookeeper, false, "Store part header (checksums and columns) in a compact format and a single part znode instead of separate znodes (/columns and /checksums). This can dramatically reduce snapshot size in ZooKeeper. Before enabling check that all replicas support new format.") \ M(SettingUInt64, finished_mutations_to_keep, 100, "How many records about mutations that are done to keep. If zero, then keep all of them.") \ M(SettingUInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).") \ - M(SettingUInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).") \ + IM(SettingUInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).") \ M(SettingInt64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with TTL can be repeated.") \ M(SettingBool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)") \ M(SettingBool, enable_mixed_granularity_parts, 0, "Enable parts with adaptive and non adaptive granularity") From 4f89f8f4dd8b05ee345abe45d9f198062c8aa5d5 Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Tue, 6 Aug 2019 16:19:06 +0300 Subject: [PATCH 130/509] random generator is now one for thread --- dbms/programs/benchmark/Benchmark.cpp | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 61acc95ad9e..923745eb204 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -307,6 +307,10 @@ private: { Query query; + /// Randomly choosing connection index + pcg64 generator(randomSeed()); + std::uniform_int_distribution distribution(0, connection_entries.size() - 1); + try { /// In these threads we do not accept INT signal. @@ -327,7 +331,7 @@ private: if (shutdown || (max_iterations && queries_executed == max_iterations)) return; } - execute(connection_entries, query); + execute(connection_entries, query, distribution(generator)); ++queries_executed; } } @@ -339,13 +343,8 @@ private: } } - void execute(EntryPtrs & connection_entries, Query & query) + void execute(EntryPtrs & connection_entries, Query & query, size_t connection_index) { - /// Randomly choosing connection index - pcg64 generator(randomSeed()); - std::uniform_int_distribution distribution(0, connection_entries.size() - 1); - size_t connection_index = distribution(generator); - Stopwatch watch; RemoteBlockInputStream stream( *(*connection_entries[connection_index]), From ca29343f546122340b934ae3b340770e35011e3d Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Aug 2019 17:09:36 +0300 Subject: [PATCH 131/509] Uniq settings --- dbms/src/Core/SettingsCommon.h | 2 +- dbms/src/Storages/Kafka/KafkaSettings.h | 20 +++++++++---------- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 12 +++++++++-- dbms/src/Storages/MergeTree/MergeTreeData.h | 1 + .../Storages/MergeTree/MergeTreeSettings.cpp | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 10 +--------- .../00980_merge_alter_settings.reference | 3 +++ .../00980_merge_alter_settings.sql | 12 +++++++++++ 8 files changed, 39 insertions(+), 23 deletions(-) diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index 618fc00aa75..c079b875b9d 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -599,7 +599,7 @@ public: loadFromChange(change); } - /// Applies changes to the settings, checks settings mutability + /// Applies changes to the settings, checks settings mutability. void updateFromChange(const SettingChange & change) { update(change.name, change.value); diff --git a/dbms/src/Storages/Kafka/KafkaSettings.h b/dbms/src/Storages/Kafka/KafkaSettings.h index 366ec715bf0..fd8d9b3dc08 100644 --- a/dbms/src/Storages/Kafka/KafkaSettings.h +++ b/dbms/src/Storages/Kafka/KafkaSettings.h @@ -15,16 +15,16 @@ struct KafkaSettings : public SettingsCollection { #define LIST_OF_KAFKA_SETTINGS(M, IM) \ - IM(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.") \ - IM(SettingString, kafka_topic_list, "", "A list of Kafka topics.") \ - IM(SettingString, kafka_group_name, "", "A group of Kafka consumers.") \ - IM(SettingString, kafka_format, "", "The message format for Kafka engine.") \ - IM(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.") \ - IM(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine") \ - IM(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") \ - IM(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") \ - IM(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block") \ - IM(SettingUInt64, kafka_commit_every_batch, 0, "Commit every consumed and handled batch instead of a single commit after writing a whole block") + M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.") \ + M(SettingString, kafka_topic_list, "", "A list of Kafka topics.") \ + M(SettingString, kafka_group_name, "", "A group of Kafka consumers.") \ + M(SettingString, kafka_format, "", "The message format for Kafka engine.") \ + M(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.") \ + M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine") \ + M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") \ + M(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") \ + M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block") \ + M(SettingUInt64, kafka_commit_every_batch, 0, "Commit every consumed and handled batch instead of a single commit after writing a whole block") DECLARE_SETTINGS_COLLECTION(LIST_OF_KAFKA_SETTINGS) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 70c6c409c30..7c65b33d830 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1595,8 +1595,16 @@ void MergeTreeData::alterSettings( { if (!new_changes.empty()) { - auto & storage_ast = ast.as(); - storage_ast.settings->changes.insert(storage_ast.settings->changes.end(), new_changes.begin(), new_changes.end()); + auto & storage_changes = ast.as().settings->changes; + /// Make storage settings unique + for (const auto & change : new_changes) + { + auto finder = [&change] (const SettingChange & c) { return c.name == change.name;}; + if (auto it = std::find_if(storage_changes.begin(), storage_changes.end(), finder); it != storage_changes.end()) + it->value = change.value; + else + storage_changes.push_back(change); + } } }; context.getDatabase(current_database_name)->alterTable(context, current_table_name, getColumns(), getIndices(), storage_modifier); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index d11074e1156..f681baa4488 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -509,6 +509,7 @@ public: bool skip_sanity_checks, AlterDataPartTransactionPtr& transaction); + /// Performs ALTER of table settings void alterSettings( const SettingsChanges & new_changes, const String & current_database_name, diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp index 562f2d8f402..9eee33554ab 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -67,7 +67,7 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) #define ADD_IF_ABSENT(NAME) \ if (std::find_if(changes.begin(), changes.end(), \ - [](const SettingChange & c) { return c.name == #NAME; }) \ + [](const SettingChange & c) { return c.name == #NAME; }) \ == changes.end()) \ changes.push_back(SettingChange{#NAME, NAME.value}); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 438b9468b69..9fe1458f5a7 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -260,16 +260,8 @@ void StorageMergeTree::alter( ASTPtr new_primary_key_ast = primary_key_ast; ASTPtr new_ttl_table_ast = ttl_table_ast; params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes); - IDatabase::ASTModifier storage_modifier = [&] (IAST & ast) - { - auto & storage_ast = ast.as(); - if (!new_changes.empty()) - storage_ast.settings->changes.insert(storage_ast.settings->changes.end(), new_changes.begin(), new_changes.end()); - }; - - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, storage_modifier); + context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {}); setColumns(std::move(new_columns)); - settings.updateFromChanges(new_changes); return; } diff --git a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.reference b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.reference index ad64346f90d..c7f912ddc79 100644 --- a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.reference +++ b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.reference @@ -1,2 +1,5 @@ CREATE TABLE default.table_for_alter (`id` UInt64, `Data` String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 4096 CREATE TABLE default.table_for_alter (`id` UInt64, `Data` String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 4096, parts_to_throw_insert = 1, parts_to_delay_insert = 1 +CREATE TABLE default.table_for_alter (`id` UInt64, `Data` String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 4096, parts_to_throw_insert = 100, parts_to_delay_insert = 100 +2 +CREATE TABLE default.table_for_alter (`id` UInt64, `Data` String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 4096, parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 30 diff --git a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql index 52f5e4dd444..cb1ba315001 100644 --- a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql +++ b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql @@ -35,5 +35,17 @@ INSERT INTO table_for_alter VALUES (2, '2'); -- { serverError 252 } ALTER TABLE table_for_alter MODIFY SETTING xxx_yyy=124; -- { serverError 115 } +ALTER TABLE table_for_alter MODIFY SETTING parts_to_throw_insert = 100, parts_to_delay_insert = 100; + +INSERT INTO table_for_alter VALUES (2, '2'); + +SHOW CREATE TABLE table_for_alter; + +SELECT COUNT() FROM table_for_alter; + +ALTER TABLE table_for_alter MODIFY SETTING check_delay_period=10, check_delay_period=20, check_delay_period=30; + +SHOW CREATE TABLE table_for_alter; + DROP TABLE IF EXISTS table_for_alter; From e62101b8e86665956e605a934e7981b3e7550a63 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Aug 2019 19:29:31 +0300 Subject: [PATCH 132/509] Better comment --- dbms/src/Core/SettingsCommon.h | 8 ++++++-- dbms/src/Storages/IStorage.h | 3 +-- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- dbms/src/Storages/MergeTree/MergeTreeData.h | 7 +++++-- dbms/src/Storages/StorageMergeTree.cpp | 5 +++-- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 5 +++-- 6 files changed, 20 insertions(+), 12 deletions(-) diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index c079b875b9d..4b3f7020a6e 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -587,24 +587,28 @@ public: return found_changes; } - /// Applies changes to the settings. Doesn't check settings mutability. + /// Applies change to the settings. Doesn't check settings mutability. void loadFromChange(const SettingChange & change) { set(change.name, change.value); } + /// Applies changes to the settings. Should be used in initial settings loading. + /// (on table creation or loading from config) void loadFromChanges(const SettingsChanges & changes) { for (const SettingChange & change : changes) loadFromChange(change); } - /// Applies changes to the settings, checks settings mutability. + /// Applies change to the settings, checks settings mutability. void updateFromChange(const SettingChange & change) { update(change.name, change.value); } + /// Applies changes to the settings. Should be used for settigns update. + /// (ALTER MODIFY SETTINGS) void updateFromChanges(const SettingsChanges & changes) { for (const SettingChange & change : changes) diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 4babb7ded3e..79760bae5a8 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -12,7 +12,6 @@ #include #include #include -#include #include #include @@ -130,7 +129,7 @@ public: /// thread-unsafe part. lockStructure must be acquired /// If |need_all| is set, then checks that all the columns of the table are in the block. void check(const Block & block, bool need_all = false) const; - /// Check storage has setting + /// Check storage has setting. Exception will be thrown if it doesn't support settings at all. virtual bool hasSetting(const String & setting_name) const; protected: /// still thread-unsafe part. diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 7c65b33d830..4f7452bd79f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1269,7 +1269,6 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c { if (!hasSetting(setting.name)) throw Exception{"Storage '" + getName() + "' doesn't have setting '" + setting.name + "'", ErrorCodes::UNKNOWN_SETTING}; - } /// Check that type conversions are possible. @@ -1588,7 +1587,8 @@ void MergeTreeData::alterSettings( const SettingsChanges & new_changes, const String & current_database_name, const String & current_table_name, - const Context & context) + const Context & context, + TableStructureWriteLockHolder & /* table_lock_holder */) { settings.updateFromChanges(new_changes); IDatabase::ASTModifier storage_modifier = [&] (IAST & ast) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index f681baa4488..78c7df3bb48 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -509,13 +509,16 @@ public: bool skip_sanity_checks, AlterDataPartTransactionPtr& transaction); - /// Performs ALTER of table settings + /// Performs ALTER of table settings (MergeTreeSettings). Lightweight operation, affects metadata only. + /// Not atomic, have to be done with alter intention lock. void alterSettings( const SettingsChanges & new_changes, const String & current_database_name, const String & current_table_name, - const Context & context); + const Context & context, + TableStructureWriteLockHolder & table_lock_holder); + /// All MergeTreeData children have settings. bool hasSetting(const String & setting_name) const override; /// Remove columns, that have been markedd as empty after zeroing values with expired ttl diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 9fe1458f5a7..bebc6619509 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -245,15 +245,16 @@ void StorageMergeTree::alter( { if (!params.isMutable()) { - lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); SettingsChanges new_changes; + /// We don't need to lock table structure exclusively to ALTER settings. if (params.isSettingsAlter()) { params.applyForSettingsOnly(new_changes); - alterSettings(new_changes, current_database_name, current_table_name, context); + alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); return; } + lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); auto new_columns = getColumns(); auto new_indices = getIndices(); ASTPtr new_order_by_ast = order_by_ast; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 23c49b05c1a..e11ea304c61 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3081,11 +3081,12 @@ void StorageReplicatedMergeTree::alter( if (params.isSettingsAlter()) { + /// We don't replicate settings ALTER. It's local operation. + /// Also we don't upgrade alter lock to table structure lock. LOG_DEBUG(log, "ALTER settings only"); - lockStructureExclusively(table_lock_holder, query_context.getCurrentQueryId()); SettingsChanges new_changes; params.applyForSettingsOnly(new_changes); - alterSettings(new_changes, current_database_name, current_table_name, query_context); + alterSettings(new_changes, current_database_name, current_table_name, query_context, table_lock_holder); return; } From bca40a860890fb1dc94dc0f0aafd6aa5591b09f4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Aug 2019 19:34:27 +0300 Subject: [PATCH 133/509] Better name --- dbms/src/Storages/StorageMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index bebc6619509..8f3897ba128 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -864,9 +864,9 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi ASTPtr ignored_order_by_ast; ASTPtr ignored_primary_key_ast; ASTPtr ignored_ttl_table_ast; - SettingsChanges ignore_settings_changes; + SettingsChanges ignored_settings_changes; - alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast, ignore_settings_changes); + alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast, ignored_settings_changes); auto columns_for_parts = new_columns.getAllPhysical(); for (const auto & part : parts) From 517730900157e84f02773165db9dbb66eea95074 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 7 Aug 2019 16:35:28 +0300 Subject: [PATCH 134/509] Add test for race condition --- .../00980_alter_settings_race.reference | 1 + .../0_stateless/00980_alter_settings_race.sh | 32 +++++++++++++++++++ .../00980_merge_alter_settings.sql | 2 +- 3 files changed, 34 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/00980_alter_settings_race.reference create mode 100755 dbms/tests/queries/0_stateless/00980_alter_settings_race.sh diff --git a/dbms/tests/queries/0_stateless/00980_alter_settings_race.reference b/dbms/tests/queries/0_stateless/00980_alter_settings_race.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00980_alter_settings_race.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00980_alter_settings_race.sh b/dbms/tests/queries/0_stateless/00980_alter_settings_race.sh new file mode 100755 index 00000000000..4a948841ed7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00980_alter_settings_race.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS table_for_concurrent_alter" + +$CLICKHOUSE_CLIENT --query="CREATE TABLE table_for_concurrent_alter (id UInt64, Data String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096;"; + +n=0 +while [ "$n" -lt 100 ]; +do + n=$(( n + 1 )) + $CLICKHOUSE_CLIENT --query="INSERT INTO table_for_concurrent_alter VALUES(1, 'Hello')" > /dev/null 2> /dev/null & + $CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE table_for_concurrent_alter FINAL" > /dev/null 2> /dev/null & +done & + + +q=0 +while [ "$q" -lt 100 ]; +do + q=$(( q + 1 )) + counter=$(( 100 + q )) + $CLICKHOUSE_CLIENT --query="ALTER TABLE table_for_concurrent_alter MODIFY SETTING parts_to_throw_insert = $counter, parts_to_delay_insert = $counter, min_merge_bytes_to_use_direct_io = $counter" > /dev/null 2> /dev/null & +done & + +sleep 4 + +# we just test race conditions, not logic +$CLICKHOUSE_CLIENT --query "SELECT 1" + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS table_for_concurrent_alter" diff --git a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql index cb1ba315001..866cf960710 100644 --- a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql +++ b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql @@ -35,7 +35,7 @@ INSERT INTO table_for_alter VALUES (2, '2'); -- { serverError 252 } ALTER TABLE table_for_alter MODIFY SETTING xxx_yyy=124; -- { serverError 115 } -ALTER TABLE table_for_alter MODIFY SETTING parts_to_throw_insert = 100, parts_to_delay_insert = 100; +ALTER TABLE table_for_alter MODIFY SETTING parts_to_throw_insert = 100, parts_to_delay_insert = 100; INSERT INTO table_for_alter VALUES (2, '2'); From 42c2833932c09864dfc2882349b796ebd001b1c5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 7 Aug 2019 16:41:36 +0300 Subject: [PATCH 135/509] Added function which calculates header for execution step (InterpreterSelectQuery::getHeaderForExecutionStep). Removed InterpreterSelectQuery:;getSampleBlock from StorageDistributed and StorageMerge. --- dbms/src/Interpreters/Context.cpp | 2 +- dbms/src/Interpreters/Context.h | 4 +- .../Interpreters/InterpreterSelectQuery.cpp | 97 ++++++++++++++++++- .../src/Interpreters/InterpreterSelectQuery.h | 10 +- dbms/src/Storages/StorageDistributed.cpp | 18 +++- dbms/src/Storages/StorageMerge.cpp | 33 +++---- dbms/src/Storages/StorageMerge.h | 2 +- 7 files changed, 132 insertions(+), 34 deletions(-) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 992593d852c..fef7b067624 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -967,7 +967,7 @@ StoragePtr Context::tryRemoveExternalTable(const String & table_name) } -StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) +StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) const { /// Slightly suboptimal. auto hash = table_expression->getTreeHash(); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 50b7ab3eba2..a44ae210c64 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -135,7 +135,7 @@ private: /// Thus, used in HTTP interface. If not specified - then some globally default format is used. TableAndCreateASTs external_tables; /// Temporary tables. StoragePtr view_source; /// Temporary StorageValues used to generate alias columns for materialized views - Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id. + mutable Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id. Context * query_context = nullptr; Context * session_context = nullptr; /// Session context or nullptr. Could be equal to this. Context * global_context = nullptr; /// Global context. Could be equal to this. @@ -245,7 +245,7 @@ public: void addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast = {}); StoragePtr tryRemoveExternalTable(const String & table_name); - StoragePtr executeTableFunction(const ASTPtr & table_expression); + StoragePtr executeTableFunction(const ASTPtr & table_expression) const; void addViewSource(const StoragePtr & storage); StoragePtr getViewSource(); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 6c486b807be..782dcddf580 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -81,6 +81,8 @@ #include #include #include +#include +#include namespace DB @@ -400,10 +402,97 @@ QueryPipeline InterpreterSelectQuery::executeWithProcessors() return query_pipeline; } + +Block InterpreterSelectQuery::getHeaderForExecutionStep( + const ASTPtr & query_ptr, + const StoragePtr & storage, + QueryProcessingStage::Enum stage, + size_t subquery_depth, + const Context & context, + const PrewhereInfoPtr & prewhere_info) +{ + SelectQueryOptions options(stage, subquery_depth); + options.only_analyze = true; + + Names required_result_column_names; + + /// TODO: remove it. + auto query = query_ptr->clone(); + + auto syntax_analyzer_result = SyntaxAnalyzer(context, options).analyze( + query, {}, required_result_column_names, storage); + + auto query_analyzer = ExpressionAnalyzer( + query, syntax_analyzer_result, context, NamesAndTypesList(), + NameSet(required_result_column_names.begin(), required_result_column_names.end()), + options.subquery_depth, !options.only_analyze); + + if (stage == QueryProcessingStage::Enum::FetchColumns) + { + auto required_columns = query_analyzer.getRequiredSourceColumns(); + auto header = storage->getSampleBlockForColumns(required_columns); + + if (prewhere_info) + { + prewhere_info->prewhere_actions->execute(header); + header = materializeBlock(header); + if (prewhere_info->remove_prewhere_column) + header.erase(prewhere_info->prewhere_column_name); + } + return header; + } + + FilterInfoPtr filter_info; + + auto & select_query = query->as(); + + auto analysis_result = analyzeExpressions( + select_query, + query_analyzer, + QueryProcessingStage::Enum::FetchColumns, + stage, + context, + storage, + true, + filter_info); + + if (stage == QueryProcessingStage::Enum::WithMergeableState) + { + if (!analysis_result.need_aggregate) + return analysis_result.before_order_and_select->getSampleBlock(); + + auto header = analysis_result.before_aggregation->getSampleBlock(); + + Names key_names; + AggregateDescriptions aggregates; + query_analyzer.getAggregateInfo(key_names, aggregates); + + Block res; + + for (auto & key : key_names) + res.insert(header.getByName(key).cloneEmpty()); + + for (auto & aggregate : aggregates) + { + size_t arguments_size = aggregate.argument_names.size(); + DataTypes argument_types(arguments_size); + for (size_t j = 0; j < arguments_size; ++j) + argument_types[j] = header.getByName(aggregate.argument_names[j]).type; + + DataTypePtr type = std::make_shared(aggregate.function, argument_types, aggregate.parameters); + + res.insert({nullptr, type, aggregate.column_name}); + } + + return res; + } + + return analysis_result.final_projection->getSampleBlock(); +} + InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpressions( const ASTSelectQuery & query, - const NamesAndTypesList & source_columns, ExpressionAnalyzer & query_analyzer, QueryProcessingStage::Enum from_stage, QueryProcessingStage::Enum to_stage, @@ -490,14 +579,14 @@ InterpreterSelectQuery::analyzeExpressions( columns_for_final.begin(), columns_for_final.end()); } - if (storage && context.hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) + if (storage && filter_info) { has_filter = true; /// XXX: aggregated copy-paste from ExpressionAnalyzer::appendSmth() if (chain.steps.empty()) { - chain.steps.emplace_back(std::make_shared(source_columns, context)); + chain.steps.emplace_back(std::make_shared(NamesAndTypesList(), context)); } ExpressionActionsChain::Step & step = chain.steps.back(); @@ -830,7 +919,6 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS expressions = analyzeExpressions( getSelectQuery(), - source_columns, *query_analyzer, QueryProcessingStage::FetchColumns, options.to_stage, @@ -871,7 +959,6 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS expressions = analyzeExpressions( getSelectQuery(), - source_columns, *query_analyzer, from_stage, options.to_stage, diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 11f8f405ecc..461daeb5ec2 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -79,6 +79,14 @@ public: ASTPtr getQuery() const { return query_ptr; } + static Block getHeaderForExecutionStep( + const ASTPtr & query, + const StoragePtr & storage, + QueryProcessingStage::Enum stage, + size_t subquery_depth, + const Context & context, + const PrewhereInfoPtr & prewhere_info); + private: InterpreterSelectQuery( const ASTPtr & query_ptr_, @@ -174,7 +182,6 @@ private: static AnalysisResult analyzeExpressions( const ASTSelectQuery & query, - const NamesAndTypesList & source_columns, ExpressionAnalyzer & query_analyzer, QueryProcessingStage::Enum from_stage, QueryProcessingStage::Enum to_stage, @@ -254,7 +261,6 @@ private: const SelectQueryOptions options; ASTPtr query_ptr; Context context; - NamesAndTypesList source_columns; SyntaxAnalyzerResultPtr syntax_analyzer_result; std::unique_ptr query_analyzer; SelectQueryInfo query_info; diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 6155dabd028..c75db697bad 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -301,8 +301,22 @@ BlockInputStreams StorageDistributed::read( const auto & modified_query_ast = rewriteSelectQuery( query_info.query, remote_database, remote_table, remote_table_function_ptr); - Block header = materializeBlock( - InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage)).getSampleBlock()); + StoragePtr tmp_storage; + + if (remote_table_function_ptr) + tmp_storage = context.getQueryContext().executeTableFunction(remote_table_function_ptr); + else + tmp_storage = context.getTable(remote_database, remote_table); + + Block header = + //InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage)).getSampleBlock()); + InterpreterSelectQuery::getHeaderForExecutionStep(query_info.query, tmp_storage, processed_stage, 0, context, query_info.prewhere_info); + + /// Create empty columns for header. + /// All columns must be empty, because otherwise (by some reason) remote query can return one excessive row. + /// So, all columns are recreated. + for (auto & col : header) + col.column = col.type->createColumn(); ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr ? ClusterProxy::SelectStreamFactory( diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 3487a1becf5..3b1a1540447 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -190,7 +190,7 @@ BlockInputStreams StorageMerge::read( modified_context.getSettingsRef().optimize_move_to_prewhere = false; /// What will be result structure depending on query processed stage in source tables? - Block header = getQueryHeader(column_names, query_info, context, processed_stage); + Block header = getQueryHeader(query_info, context, processed_stage); /** First we make list of selected tables to find out its size. * This is necessary to correctly pass the recommended number of threads to each table. @@ -407,29 +407,20 @@ void StorageMerge::alter( } Block StorageMerge::getQueryHeader( - const Names & column_names, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage) + const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage) { - switch (processed_stage) + auto storage = shared_from_this(); + auto header = InterpreterSelectQuery::getHeaderForExecutionStep(query_info.query, storage, processed_stage, 0, context, query_info.prewhere_info); + + for (auto & col : header) { - case QueryProcessingStage::FetchColumns: - { - Block header = getSampleBlockForColumns(column_names); - if (query_info.prewhere_info) - { - query_info.prewhere_info->prewhere_actions->execute(header); - header = materializeBlock(header); - if (query_info.prewhere_info->remove_prewhere_column) - header.erase(query_info.prewhere_info->prewhere_column_name); - } - return header; - } - case QueryProcessingStage::WithMergeableState: - case QueryProcessingStage::Complete: - return materializeBlock(InterpreterSelectQuery( - query_info.query, context, std::make_shared(getSampleBlockForColumns(column_names)), - SelectQueryOptions(processed_stage).analyze()).getSampleBlock()); + if (!col.column) + col.column = col.type->createColumn(); + else + col.column = col.column->convertToFullColumnIfConst(); } - throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR); + + return header; } void StorageMerge::convertingSourceStream(const Block & header, const Context & context, ASTPtr & query, diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index 6d02ad029cc..03c28fde0b4 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -79,7 +79,7 @@ protected: const String & table_name_regexp_, const Context & context_); - Block getQueryHeader(const Names & column_names, const SelectQueryInfo & query_info, + Block getQueryHeader(const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage); BlockInputStreams createSourceStreams(const SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage, From 13e4581317126661fd0d7244976cf7bda854525d Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 7 Aug 2019 18:21:45 +0300 Subject: [PATCH 136/509] More general --- dbms/src/Storages/IStorage.cpp | 51 +++++++++++++++++-- dbms/src/Storages/IStorage.h | 13 +++++ dbms/src/Storages/Kafka/StorageKafka.cpp | 17 +++++++ dbms/src/Storages/Kafka/StorageKafka.h | 10 ++++ dbms/src/Storages/MergeTree/MergeTreeData.cpp | 22 ++------ dbms/src/Storages/MergeTree/MergeTreeData.h | 4 +- 6 files changed, 93 insertions(+), 24 deletions(-) diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 0fd1994f5ce..e748b72c769 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -1,6 +1,8 @@ #include #include +#include +#include #include #include @@ -19,6 +21,7 @@ namespace ErrorCodes extern const int NOT_FOUND_COLUMN_IN_BLOCK; extern const int TYPE_MISMATCH; extern const int SETTINGS_ARE_NOT_SUPPORTED; + extern const int UNKNOWN_SETTING; } IStorage::IStorage(ColumnsDescription columns_) @@ -295,7 +298,9 @@ bool IStorage::isVirtualColumn(const String & column_name) const bool IStorage::hasSetting(const String & /* setting_name */) const { - throw Exception("Storage '" + getName() + "' doesn't support settings.", ErrorCodes::SETTINGS_ARE_NOT_SUPPORTED); + if (!supportsSettings()) + throw Exception("Storage '" + getName() + "' doesn't support settings.", ErrorCodes::SETTINGS_ARE_NOT_SUPPORTED); + return false; } TableStructureReadLockHolder IStorage::lockStructureForShare(bool will_add_new_data, const String & query_id) @@ -352,6 +357,39 @@ TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id) return result; } + +void IStorage::alterSettings( + const SettingsChanges & new_changes, + const String & current_database_name, + const String & current_table_name, + const Context & context, + TableStructureWriteLockHolder & /* table_lock_holder */) +{ + IDatabase::ASTModifier storage_modifier = [&] (IAST & ast) + { + if (!new_changes.empty()) + { + auto & storage_changes = ast.as().settings->changes; + /// Make storage settings unique + for (const auto & change : new_changes) + { + if (hasSetting(change.name)) + { + auto finder = [&change] (const SettingChange & c) { return c.name == change.name;}; + if (auto it = std::find_if(storage_changes.begin(), storage_changes.end(), finder); it != storage_changes.end()) + it->value = change.value; + else + storage_changes.push_back(change); + } + else + throw Exception{"Storage '" + getName() + "' doesn't have setting '" + change.name + "'", ErrorCodes::UNKNOWN_SETTING}; + } + } + }; + context.getDatabase(current_database_name)->alterTable(context, current_table_name, getColumns(), getIndices(), storage_modifier); +} + + void IStorage::alter( const AlterCommands & params, const String & database_name, @@ -359,12 +397,17 @@ void IStorage::alter( const Context & context, TableStructureWriteLockHolder & table_lock_holder) { - for (const auto & param : params) + if (params.isSettingsAlter()) { - if (param.isMutable()) - throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + SettingsChanges new_changes; + params.applyForSettingsOnly(new_changes); + alterSettings(new_changes, database_name, table_name, context, table_lock_holder); + return; } + if (params.isMutable()) + throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); auto new_columns = getColumns(); auto new_indices = getIndices(); diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 79760bae5a8..7000eef76f6 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -12,6 +12,7 @@ #include #include #include +#include #include #include @@ -95,6 +96,9 @@ public: /// Returns true if the storage supports deduplication of inserted data blocks. virtual bool supportsDeduplication() const { return false; } + /// Returns true if the storage supports settings. + virtual bool supportsSettings() const { return false; } + /// Optional size information of each physical column. /// Currently it's only used by the MergeTree family for query optimizations. using ColumnSizeByName = std::unordered_map; @@ -252,6 +256,15 @@ public: throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + /** ALTER table settings if possible. Otherwise throws exception. + */ + virtual void alterSettings( + const SettingsChanges & new_changes, + const String & current_database_name, + const String & current_table_name, + const Context & context, + TableStructureWriteLockHolder & table_lock_holder); + /** Perform any background work. For example, combining parts in a MergeTree type table. * Returns whether any work has been done. */ diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 20599c7e4f8..47f4fc8c610 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -40,6 +40,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int UNSUPPORTED_METHOD; } namespace @@ -388,6 +389,22 @@ bool StorageKafka::streamToViews() } +bool StorageKafka::hasSetting(const String & setting_name) const +{ + return KafkaSettings::findIndex(setting_name) != KafkaSettings::npos; +} + +void StorageKafka::alterSettings( + const SettingsChanges & /* new_changes */, + const String & /* current_database_name */, + const String & /* current_table_name */, + const Context & /* context */, + TableStructureWriteLockHolder & /* table_lock_holder */) +{ + throw Exception("Storage '" + getName() + "' doesn't support settings alter", ErrorCodes::UNSUPPORTED_METHOD); +} + + void registerStorageKafka(StorageFactory & factory) { factory.registerStorage("Kafka", [](const StorageFactory::Arguments & args) diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index f9b6609def5..38a63e8536e 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -24,6 +24,7 @@ public: std::string getName() const override { return "Kafka"; } std::string getTableName() const override { return table_name; } std::string getDatabaseName() const override { return database_name; } + bool supportsSettings() const override { return true; } void startup() override; void shutdown() override; @@ -50,6 +51,15 @@ public: const auto & getSchemaName() const { return schema_name; } const auto & skipBroken() const { return skip_broken; } + bool hasSetting(const String & setting_name) const override; + + void alterSettings( + const SettingsChanges & new_changes, + const String & current_database_name, + const String & current_table_name, + const Context & context, + TableStructureWriteLockHolder & table_lock_holder) override; + protected: StorageKafka( const std::string & table_name_, diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 4f7452bd79f..b0453b04c8a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1588,26 +1588,10 @@ void MergeTreeData::alterSettings( const String & current_database_name, const String & current_table_name, const Context & context, - TableStructureWriteLockHolder & /* table_lock_holder */) + TableStructureWriteLockHolder & table_lock_holder) { settings.updateFromChanges(new_changes); - IDatabase::ASTModifier storage_modifier = [&] (IAST & ast) - { - if (!new_changes.empty()) - { - auto & storage_changes = ast.as().settings->changes; - /// Make storage settings unique - for (const auto & change : new_changes) - { - auto finder = [&change] (const SettingChange & c) { return c.name == change.name;}; - if (auto it = std::find_if(storage_changes.begin(), storage_changes.end(), finder); it != storage_changes.end()) - it->value = change.value; - else - storage_changes.push_back(change); - } - } - }; - context.getDatabase(current_database_name)->alterTable(context, current_table_name, getColumns(), getIndices(), storage_modifier); + IStorage::alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); } bool MergeTreeData::hasSetting(const String & setting_name) const @@ -2245,7 +2229,7 @@ std::optional MergeTreeData::getMinPartDataVersion() const } -void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event *until) const +void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const { const size_t parts_count_in_total = getPartsCount(); if (parts_count_in_total >= settings.max_parts_in_total) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 78c7df3bb48..756c188c724 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -342,6 +342,8 @@ public: || merging_params.mode == MergingParams::VersionedCollapsing; } + bool supportsSettings() const override { return true; } + bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context &) const override; NameAndTypePair getColumn(const String & column_name) const override @@ -516,7 +518,7 @@ public: const String & current_database_name, const String & current_table_name, const Context & context, - TableStructureWriteLockHolder & table_lock_holder); + TableStructureWriteLockHolder & table_lock_holder) override; /// All MergeTreeData children have settings. bool hasSetting(const String & setting_name) const override; From 7409f1a3de81b5698364d9381ec113195b1cfdb3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 7 Aug 2019 18:33:10 +0300 Subject: [PATCH 137/509] More comments --- dbms/src/Core/Settings.h | 1 + dbms/src/Core/SettingsCommon.h | 3 +++ dbms/src/Storages/Kafka/KafkaSettings.h | 2 ++ dbms/src/Storages/MergeTree/MergeTreeSettings.h | 1 + 4 files changed, 7 insertions(+) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 0c5a4ceed60..54f6a14205b 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -42,6 +42,7 @@ struct Settings : public SettingsCollection * but we are not going to do it, because settings is used everywhere as static struct fields. */ +/// M (mutable) for normal settings, IM (immutable) for not updateable settings. #define LIST_OF_SETTINGS(M, IM) \ M(SettingUInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.") \ M(SettingUInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.") \ diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index 4b3f7020a6e..d4607e70904 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -321,6 +321,9 @@ private: size_t offset_of_changed; StringRef name; StringRef description; + /// Can be updated after first load for config/definition. + /// Non updatable settings can be isChanged, + /// if they were overwritten in config/definition. bool updateable; GetStringFunction get_string; GetFieldFunction get_field; diff --git a/dbms/src/Storages/Kafka/KafkaSettings.h b/dbms/src/Storages/Kafka/KafkaSettings.h index fd8d9b3dc08..e43ea7cd70e 100644 --- a/dbms/src/Storages/Kafka/KafkaSettings.h +++ b/dbms/src/Storages/Kafka/KafkaSettings.h @@ -14,6 +14,8 @@ class ASTStorage; struct KafkaSettings : public SettingsCollection { + +/// M (mutable) for normal settings, IM (immutable) for not updateable settings. #define LIST_OF_KAFKA_SETTINGS(M, IM) \ M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.") \ M(SettingString, kafka_topic_list, "", "A list of Kafka topics.") \ diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index 3012b9e1694..dc94ad5b11f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -24,6 +24,7 @@ class ASTStorage; struct MergeTreeSettings : public SettingsCollection { +/// M (mutable) for normal settings, IM (immutable) for not updateable settings. #define LIST_OF_MERGE_TREE_SETTINGS(M, IM) \ IM(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.") \ \ From 75c3ed967a9fe5fb780b0107b9887bf343d67a15 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 7 Aug 2019 19:13:28 +0300 Subject: [PATCH 138/509] Checking updatable for user settings --- dbms/src/Interpreters/Context.cpp | 11 +++++++++++ dbms/src/Interpreters/Context.h | 3 +++ dbms/src/Interpreters/InterpreterSetQuery.cpp | 4 ++-- 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 992593d852c..2484342908f 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -1126,6 +1126,17 @@ void Context::applySettingsChanges(const SettingsChanges & changes) applySettingChange(change); } +void Context::updateSettingsChanges(const SettingsChanges & changes) +{ + auto lock = getLock(); + for (const SettingChange & change : changes) + { + if (change.name == "profile") + setProfile(change.value.safeGet()); + else + settings.updateFromChange(change); + } +} void Context::checkSettingsConstraints(const SettingChange & change) { diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 50b7ab3eba2..4f97922a513 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -281,6 +281,9 @@ public: void applySettingChange(const SettingChange & change); void applySettingsChanges(const SettingsChanges & changes); + /// Update checking that each setting is updatable + void updateSettingsChanges(const SettingsChanges & changes); + /// Checks the constraints. void checkSettingsConstraints(const SettingChange & change); void checkSettingsConstraints(const SettingsChanges & changes); diff --git a/dbms/src/Interpreters/InterpreterSetQuery.cpp b/dbms/src/Interpreters/InterpreterSetQuery.cpp index f92e9638822..ae982611e60 100644 --- a/dbms/src/Interpreters/InterpreterSetQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSetQuery.cpp @@ -10,7 +10,7 @@ BlockIO InterpreterSetQuery::execute() { const auto & ast = query_ptr->as(); context.checkSettingsConstraints(ast.changes); - context.getSessionContext().applySettingsChanges(ast.changes); + context.getSessionContext().updateSettingsChanges(ast.changes); return {}; } @@ -19,7 +19,7 @@ void InterpreterSetQuery::executeForCurrentContext() { const auto & ast = query_ptr->as(); context.checkSettingsConstraints(ast.changes); - context.applySettingsChanges(ast.changes); + context.updateSettingsChanges(ast.changes); } } From c5a778934ee82d1c80db6eca7e432644ebd6362f Mon Sep 17 00:00:00 2001 From: BayoNet Date: Thu, 8 Aug 2019 11:37:08 +0300 Subject: [PATCH 139/509] DOCAPI-7984: ASOF JOIN ... ON syntax --- docs/en/query_language/select.md | 28 ++++++++++++++++++---------- 1 file changed, 18 insertions(+), 10 deletions(-) diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index b75524274e1..048284faa6a 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -547,18 +547,29 @@ ClickHouse doesn't directly support syntax with commas, so we don't recommend us Tables for `ASOF JOIN` must have an ordered sequence column. This column cannot be alone in a table, and should be one of the data types: `UInt32`, `UInt64`, `Float32`, `Float64`, `Date`, and `DateTime`. -Use the following syntax for `ASOF JOIN`: +You can use the following types of syntax: -``` -SELECT expression_list FROM table_1 ASOF JOIN table_2 USING(equi_column1, ... equi_columnN, asof_column) -``` +- `ASOF JOIN ... ON` -`ASOF JOIN` uses `equi_columnX` for joining on equality (`user_id` in our example) and `asof_column` for joining on the closest match. + ```sql + SELECT expressions_list FROM table_1 ASOF LEFT JOIN table_2 ON equi_cond AND closest_match_cond + ``` + + You can use any number of equality conditions and exactly one closest match condition. For example, `SELECT count() FROM A ASOF LEFT JOIN B ON A.a == B.b AND B.t <= A.t`. There is just `table_2.some_col <= table_1.some_col` and `table_1.some_col >= table2.some_col` types of conditions are available. You cannot apply other conditions like `>`, `!=`. + +- `ASOF JOIN ... USING` + + ```sql + SELECT expressions_list FROM table_1 ASOF JOIN table_2 USING(equi_column1, ... equi_columnN, asof_column) + ``` + + `ASOF JOIN` uses `equi_columnX` for joining on equality and `asof_column` for joining on the closest match with the `table_1.asof_column >= table2.asof_column` condition. The `asof_column` column must be the last in the `USING` clause. For example, consider the following tables: ``` - table_1 table_2 + table_1 table_2 + event | ev_time | user_id event | ev_time | user_id ----------|---------|---------- ----------|---------|---------- ... ... @@ -568,12 +579,9 @@ event_1_2 | 13:00 | 42 event_2_3 | 13:00 | 42 ... ... ``` -`ASOF JOIN` takes the timestamp of a user event from `table_1` and finds an event in `table_2` where the timestamp is closest (equal or less) to the timestamp of the event from `table_1`. Herewith the `user_id` column is used for joining on equality and the `ev_time` column is used for joining on the closest match. - In our example, `event_1_1` can be joined with `event_2_1`, `event_1_2` can be joined with `event_2_3`, but `event_2_2` cannot be joined. +`ASOF JOIN` can take the timestamp of a user event from `table_1` and find an event in `table_2` where the timestamp is closest (equal or less) to the timestamp of the event from `table_1`. Herewith the `user_id` column can be used for joining on equality and the `ev_time` column can be used for joining on the closest match. In our example, `event_1_1` can be joined with `event_2_1`, `event_1_2` can be joined with `event_2_3`, but `event_2_2` cannot be joined. -Implementation details: -- `asof_column` should be last in the `USING` clause. - `ASOF` join is not supported in the [Join](../operations/table_engines/join.md) table engine. To set the default strictness value, use the session configuration parameter [join_default_strictness](../operations/settings/settings.md#settings-join_default_strictness). From bb725eb5c42531819763bce4dcf7b08d269fe224 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Thu, 8 Aug 2019 14:57:44 +0300 Subject: [PATCH 140/509] DOCAPI-7442: Started to write. --- docs/en/operations/system_tables.md | 88 +++++++++++++++++++---------- 1 file changed, 57 insertions(+), 31 deletions(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index 14fa1ace01d..6f1ebebdff3 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -252,55 +252,81 @@ This is similar to the DUAL table found in other DBMSs. Contains information about parts of [MergeTree](table_engines/mergetree.md) tables. -Each row describes one part of the data. +Each row describes one data part. Columns: -- partition (String) – The partition name. To learn what a partition is, see the description of the [ALTER](../query_language/alter.md#query_language_queries_alter) query. +- `partition` (`String`) – The partition name. To learn what a partition is, see the description of the [ALTER](../query_language/alter.md#query_language_queries_alter) query. -Formats: -- `YYYYMM` for automatic partitioning by month. -- `any_string` when partitioning manually. + Formats: -- name (String) – Name of the data part. + - `YYYYMM` for automatic partitioning by month. + - `any_string` when partitioning manually. -- active (UInt8) – Indicates whether the part is active. If a part is active, it is used in a table; otherwise, it will be deleted. Inactive data parts remain after merging. +- `name` (`String`) – Name of the data part. -- marks (UInt64) – The number of marks. To get the approximate number of rows in a data part, multiply ``marks`` by the index granularity (usually 8192). +- `active` (`UInt8`) – Indicates whether the part is active. If a part is active, it is used in a table; otherwise, it will be deleted. Inactive data parts remain after merging. -- marks_size (UInt64) – The size of the file with marks. +- `marks` (`UInt64`) – The number of marks. To get the approximate number of rows in a data part, multiply `marks` by the index granularity (usually 8192). -- rows (UInt64) – The number of rows. +- `rows` (`UInt64`) – The number of rows. -- bytes (UInt64) – The number of bytes when compressed. +- `bytes_on_disk` (`UInt64`) – The number of bytes when compressed. -- modification_time (DateTime) – The modification time of the directory with the data part. This usually corresponds to the time of data part creation.| +- `data_compressed_bytes` (`UInt64`) – -- remove_time (DateTime) – The time when the data part became inactive. +- `data_uncompressed_bytes` (`UInt64`) – -- refcount (UInt32) – The number of places where the data part is used. A value greater than 2 indicates that the data part is used in queries or merges. +- `marks_bytes` (`UInt64`) – The size of the file with marks. -- min_date (Date) – The minimum value of the date key in the data part. +- `modification_time` (`DateTime`) – The modification time of the directory with the data part. This usually corresponds to the time of data part creation.| -- max_date (Date) – The maximum value of the date key in the data part. +- `remove_time` (`DateTime`) – The time when the data part became inactive. -- min_block_number (UInt64) – The minimum number of data parts that make up the current part after merging. +- `refcount` (`UInt32`) – The number of places where the data part is used. A value greater than 2 indicates that the data part is used in queries or merges. -- max_block_number (UInt64) – The maximum number of data parts that make up the current part after merging. +- `min_date` (`Date`) – The minimum value of the date key in the data part. -- level (UInt32) – Depth of the merge tree. If a merge was not performed, ``level=0``. +- `max_date` (`Date`) – The maximum value of the date key in the data part. -- primary_key_bytes_in_memory (UInt64) – The amount of memory (in bytes) used by primary key values. +- `min_time` (`DateTime`) – The minimum value of the date and time key in the data part. -- primary_key_bytes_in_memory_allocated (UInt64) – The amount of memory (in bytes) reserved for primary key values. +- `max_time`(`DateTime`) – The maximum value of the date and time key in the data part. -- database (String) – Name of the database. +- `partition_id` (`String`) – Id of the partition. -- table (String) – Name of the table. +- `min_block_number` (`UInt64`) – The minimum number of data parts that make up the current part after merging. -- engine (String) – Name of the table engine without parameters. +- `max_block_number` (`UInt64`) – The maximum number of data parts that make up the current part after merging. + +- `level` (`UInt32`) – Depth of the merge tree. If a merge was not performed, `level=0`. + +- `data_version` (`UInt64`) – + +- `primary_key_bytes_in_memory` (`UInt64`) – The amount of memory (in bytes) used by primary key values. + +- `primary_key_bytes_in_memory_allocated` (`UInt64`) – The amount of memory (in bytes) reserved for primary key values. + +- `is_frozen` (`UInt8`) – Flag that shows partition data backup existence. 1, the backup exists. 0, the backup doesn't exist. For more details, see [FREEZE PARTITION](../query_language/alter.md#alter_freeze-partition) + +- `database` (`String`) – Name of the database. + +- `table` (`String`) – Name of the table. + +- `engine` (`String`) – Name of the table engine without parameters. + +- `path` (`String`) – Absolute path to the folder with data part files. + +- `hash_of_all_files` (`String`) – Hash of compressed files. + +- `hash_of_uncompressed_files` (`String`) – Hash of uncompressed data. + +- `uncompressed_hash_of_compressed_files` (`String`) – Hash of the file with marks + +- `bytes` (`UInt64`) – Alias for `bytes_on_disk`. + +- `marks_size` (`UInt64`) – Alias for `marks_bytes`. -- is_frozen (UInt8) – Flag that shows partition data backup existence. 1, the backup exists. 0, the backup doesn't exist. For more details, see [FREEZE PARTITION](../query_language/alter.md#alter_freeze-partition) ## system.part_log {#system_tables-part-log} @@ -360,15 +386,15 @@ Contains information about execution of queries. For each query, you can see pro !!! note The table doesn't contain input data for `INSERT` queries. - + ClickHouse creates this table only if the [query_log](server_settings/settings.md#server_settings-query-log) server parameter is specified. This parameter sets the logging rules, such as the logging interval or the name of the table the queries will be logged in. To enable query logging, set the [log_queries](settings/settings.md#settings-log-queries) parameter to 1. For details, see the [Settings](settings/settings.md) section. The `system.query_log` table registers two kinds of queries: - + 1. Initial queries that were run directly by the client. -2. Child queries that were initiated by other queries (for distributed query execution). For these types of queries, information about the parent queries is shown in the `initial_*` columns. +2. Child queries that were initiated by other queries (for distributed query execution). For these types of queries, information about the parent queries is shown in the `initial_*` columns. Columns: @@ -380,7 +406,7 @@ Columns: - `event_date` (Date) — Event date. - `event_time` (DateTime) — Event time. - `query_start_time` (DateTime) — Start time of query processing. -- `query_duration_ms` (UInt64) — Duration of query processing. +- `query_duration_ms` (UInt64) — Duration of query processing. - `read_rows` (UInt64) — Number of read rows. - `read_bytes` (UInt64) — Number of read bytes. - `written_rows` (UInt64) — For `INSERT` queries, the number of written rows. For other queries, the column value is 0. @@ -391,7 +417,7 @@ Columns: - `query` (String) — Query string. - `exception` (String) — Exception message. - `stack_trace` (String) — Stack trace (a list of methods called before the error occurred). An empty string, if the query is completed successfully. -- `is_initial_query` (UInt8) — Kind of query. Possible values: +- `is_initial_query` (UInt8) — Kind of query. Possible values: - 1 — Query was initiated by the client. - 0 — Query was initiated by another query for distributed query execution. - `user` (String) — Name of the user who initiated the current query. @@ -413,7 +439,7 @@ Columns: - `client_version_minor` (UInt32) — Minor version of the [clickhouse-client](../interfaces/cli.md). - `client_version_patch` (UInt32) — Patch component of the [clickhouse-client](../interfaces/cli.md) version. - `http_method` (UInt8) — HTTP method that initiated the query. Possible values: - - 0 — The query was launched from the TCP interface. + - 0 — The query was launched from the TCP interface. - 1 — `GET` method was used. - 2 — `POST` method was used. - `http_user_agent` (String) — The `UserAgent` header passed in the HTTP request. From 9ac401573f5c5db6c4ba72c1baca384be9ea0dcd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 8 Aug 2019 18:18:28 +0300 Subject: [PATCH 141/509] Added InterpreterSelectQuery::getSampleBlockImpl. Disable dry_run. --- .../Interpreters/InterpreterSelectQuery.cpp | 98 +++++++++---------- .../src/Interpreters/InterpreterSelectQuery.h | 11 +-- .../InterpreterSelectWithUnionQuery.h | 1 + dbms/src/Storages/StorageDistributed.cpp | 18 +--- dbms/src/Storages/StorageMerge.cpp | 33 ++++--- dbms/src/Storages/StorageMerge.h | 2 +- 6 files changed, 72 insertions(+), 91 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 782dcddf580..9c10c317d28 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -270,7 +270,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( String database_name; String table_name; - getDatabaseAndTableNames(database_name, table_name); + getDatabaseAndTableNames(query, database_name, table_name, context); if (auto view_source = context.getViewSource()) { @@ -344,17 +344,16 @@ InterpreterSelectQuery::InterpreterSelectQuery( source_header = storage->getSampleBlockForColumns(required_columns); /// Calculate structure of the result. - { - Pipeline pipeline; - executeImpl(pipeline, nullptr, true); - result_header = pipeline.firstStream()->getHeader(); - } + result_header = getSampleBlockImpl(); + for (auto & col : result_header) + if (!col.column) + col.column = col.type->createColumn(); } -void InterpreterSelectQuery::getDatabaseAndTableNames(String & database_name, String & table_name) +void InterpreterSelectQuery::getDatabaseAndTableNames(const ASTSelectQuery & query, String & database_name, String & table_name, const Context & context) { - if (auto db_and_table = getDatabaseAndTable(getSelectQuery(), 0)) + if (auto db_and_table = getDatabaseAndTable(query, 0)) { table_name = db_and_table->table; database_name = db_and_table->database; @@ -403,60 +402,35 @@ QueryPipeline InterpreterSelectQuery::executeWithProcessors() } -Block InterpreterSelectQuery::getHeaderForExecutionStep( - const ASTPtr & query_ptr, - const StoragePtr & storage, - QueryProcessingStage::Enum stage, - size_t subquery_depth, - const Context & context, - const PrewhereInfoPtr & prewhere_info) +Block InterpreterSelectQuery::getSampleBlockImpl() { - SelectQueryOptions options(stage, subquery_depth); - options.only_analyze = true; - - Names required_result_column_names; - - /// TODO: remove it. - auto query = query_ptr->clone(); - - auto syntax_analyzer_result = SyntaxAnalyzer(context, options).analyze( - query, {}, required_result_column_names, storage); - - auto query_analyzer = ExpressionAnalyzer( - query, syntax_analyzer_result, context, NamesAndTypesList(), - NameSet(required_result_column_names.begin(), required_result_column_names.end()), - options.subquery_depth, !options.only_analyze); - - if (stage == QueryProcessingStage::Enum::FetchColumns) - { - auto required_columns = query_analyzer.getRequiredSourceColumns(); - auto header = storage->getSampleBlockForColumns(required_columns); - - if (prewhere_info) - { - prewhere_info->prewhere_actions->execute(header); - header = materializeBlock(header); - if (prewhere_info->remove_prewhere_column) - header.erase(prewhere_info->prewhere_column_name); - } - return header; - } - FilterInfoPtr filter_info; - auto & select_query = query->as(); - auto analysis_result = analyzeExpressions( - select_query, - query_analyzer, + getSelectQuery(), + *query_analyzer, QueryProcessingStage::Enum::FetchColumns, - stage, + options.to_stage, context, storage, true, filter_info); - if (stage == QueryProcessingStage::Enum::WithMergeableState) + if (options.to_stage == QueryProcessingStage::Enum::FetchColumns) + { + auto header = source_header; + + if (analysis_result.prewhere_info) + { + analysis_result.prewhere_info->prewhere_actions->execute(header); + header = materializeBlock(header); + if (analysis_result.prewhere_info->remove_prewhere_column) + header.erase(analysis_result.prewhere_info->prewhere_column_name); + } + return header; + } + + if (options.to_stage == QueryProcessingStage::Enum::WithMergeableState) { if (!analysis_result.need_aggregate) return analysis_result.before_order_and_select->getSampleBlock(); @@ -465,7 +439,7 @@ Block InterpreterSelectQuery::getHeaderForExecutionStep( Names key_names; AggregateDescriptions aggregates; - query_analyzer.getAggregateInfo(key_names, aggregates); + query_analyzer->getAggregateInfo(key_names, aggregates); Block res; @@ -524,6 +498,24 @@ InterpreterSelectQuery::analyzeExpressions( { chain.finalize(); + /// Check that actions on current step are valid. + /// Now this in needed for mutations to check in mutation is valid before execute it in background. + /// Because some functions only checking correctness of constant arguments during execution, + /// but not in getReturnType method (e.g. compare date with constant string). + if (dry_run) + { + for (auto & step : chain.steps) + { + auto step_required_columns = step.actions->getRequiredColumnsWithTypes(); + + Block sample; + for (auto & col : step_required_columns) + sample.insert({col.type->createColumn(), col.type, col.name}); + + step.actions->execute(sample); + } + } + if (has_prewhere) { const ExpressionActionsChain::Step & step = chain.steps.at(0); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 461daeb5ec2..87a4fdbb31c 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -79,14 +79,6 @@ public: ASTPtr getQuery() const { return query_ptr; } - static Block getHeaderForExecutionStep( - const ASTPtr & query, - const StoragePtr & storage, - QueryProcessingStage::Enum stage, - size_t subquery_depth, - const Context & context, - const PrewhereInfoPtr & prewhere_info); - private: InterpreterSelectQuery( const ASTPtr & query_ptr_, @@ -98,6 +90,7 @@ private: ASTSelectQuery & getSelectQuery() { return query_ptr->as(); } + Block getSampleBlockImpl(); struct Pipeline { @@ -192,7 +185,7 @@ private: /** From which table to read. With JOIN, the "left" table is returned. */ - void getDatabaseAndTableNames(String & database_name, String & table_name); + static void getDatabaseAndTableNames(const ASTSelectQuery & query, String & database_name, String & table_name, const Context & context); /// Different stages of query execution. diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h index 9f2a4a96494..aa5a763feaa 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -50,6 +50,7 @@ private: Context context; std::vector> nested_interpreters; + Blocks nested_headers; Block result_header; diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index c75db697bad..6155dabd028 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -301,22 +301,8 @@ BlockInputStreams StorageDistributed::read( const auto & modified_query_ast = rewriteSelectQuery( query_info.query, remote_database, remote_table, remote_table_function_ptr); - StoragePtr tmp_storage; - - if (remote_table_function_ptr) - tmp_storage = context.getQueryContext().executeTableFunction(remote_table_function_ptr); - else - tmp_storage = context.getTable(remote_database, remote_table); - - Block header = - //InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage)).getSampleBlock()); - InterpreterSelectQuery::getHeaderForExecutionStep(query_info.query, tmp_storage, processed_stage, 0, context, query_info.prewhere_info); - - /// Create empty columns for header. - /// All columns must be empty, because otherwise (by some reason) remote query can return one excessive row. - /// So, all columns are recreated. - for (auto & col : header) - col.column = col.type->createColumn(); + Block header = materializeBlock( + InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage)).getSampleBlock()); ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr ? ClusterProxy::SelectStreamFactory( diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 3b1a1540447..3487a1becf5 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -190,7 +190,7 @@ BlockInputStreams StorageMerge::read( modified_context.getSettingsRef().optimize_move_to_prewhere = false; /// What will be result structure depending on query processed stage in source tables? - Block header = getQueryHeader(query_info, context, processed_stage); + Block header = getQueryHeader(column_names, query_info, context, processed_stage); /** First we make list of selected tables to find out its size. * This is necessary to correctly pass the recommended number of threads to each table. @@ -407,20 +407,29 @@ void StorageMerge::alter( } Block StorageMerge::getQueryHeader( - const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage) + const Names & column_names, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage) { - auto storage = shared_from_this(); - auto header = InterpreterSelectQuery::getHeaderForExecutionStep(query_info.query, storage, processed_stage, 0, context, query_info.prewhere_info); - - for (auto & col : header) + switch (processed_stage) { - if (!col.column) - col.column = col.type->createColumn(); - else - col.column = col.column->convertToFullColumnIfConst(); + case QueryProcessingStage::FetchColumns: + { + Block header = getSampleBlockForColumns(column_names); + if (query_info.prewhere_info) + { + query_info.prewhere_info->prewhere_actions->execute(header); + header = materializeBlock(header); + if (query_info.prewhere_info->remove_prewhere_column) + header.erase(query_info.prewhere_info->prewhere_column_name); + } + return header; + } + case QueryProcessingStage::WithMergeableState: + case QueryProcessingStage::Complete: + return materializeBlock(InterpreterSelectQuery( + query_info.query, context, std::make_shared(getSampleBlockForColumns(column_names)), + SelectQueryOptions(processed_stage).analyze()).getSampleBlock()); } - - return header; + throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR); } void StorageMerge::convertingSourceStream(const Block & header, const Context & context, ASTPtr & query, diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index 03c28fde0b4..6d02ad029cc 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -79,7 +79,7 @@ protected: const String & table_name_regexp_, const Context & context_); - Block getQueryHeader(const SelectQueryInfo & query_info, + Block getQueryHeader(const Names & column_names, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage); BlockInputStreams createSourceStreams(const SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage, From 2f36d80705d5a62c8efc15c623de20250cee1537 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 8 Aug 2019 18:51:17 +0300 Subject: [PATCH 142/509] move tests with sudo to integration tests --- .../integration/test_partition/__init__.py | 0 dbms/tests/integration/test_partition/test.py | 244 ++++++++++++++++++ .../0_stateless/00428_partition.reference | 54 ---- .../queries/0_stateless/00428_partition.sh | 60 ----- .../00974_attach_invalid_parts.reference | 26 -- .../0_stateless/00974_attach_invalid_parts.sh | 53 ---- .../0_stateless/00975_drop_detached.reference | 15 -- .../0_stateless/00975_drop_detached.sh | 45 ---- 8 files changed, 244 insertions(+), 253 deletions(-) create mode 100644 dbms/tests/integration/test_partition/__init__.py create mode 100644 dbms/tests/integration/test_partition/test.py delete mode 100644 dbms/tests/queries/0_stateless/00428_partition.reference delete mode 100755 dbms/tests/queries/0_stateless/00428_partition.sh delete mode 100644 dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference delete mode 100755 dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh delete mode 100644 dbms/tests/queries/0_stateless/00975_drop_detached.reference delete mode 100755 dbms/tests/queries/0_stateless/00975_drop_detached.sh diff --git a/dbms/tests/integration/test_partition/__init__.py b/dbms/tests/integration/test_partition/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_partition/test.py b/dbms/tests/integration/test_partition/test.py new file mode 100644 index 00000000000..59c48e5d9e9 --- /dev/null +++ b/dbms/tests/integration/test_partition/test.py @@ -0,0 +1,244 @@ +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance') +q = instance.query +path_to_data = '/var/lib/clickhouse/' + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + q('CREATE DATABASE test') + + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture +def partition_table_simple(started_cluster): + q("DROP TABLE IF EXISTS test.partition") + q("CREATE TABLE test.partition (date MATERIALIZED toDate(0), x UInt64, sample_key MATERIALIZED intHash64(x)) " + "ENGINE=MergeTree PARTITION BY date SAMPLE BY sample_key ORDER BY (date,x,sample_key) " + "SETTINGS index_granularity=8192, index_granularity_bytes=0") + q("INSERT INTO test.partition ( x ) VALUES ( now() )") + q("INSERT INTO test.partition ( x ) VALUES ( now()+1 )") + + yield + + q('DROP TABLE test.partition') + + +def test_partition_simple(partition_table_simple): + q("ALTER TABLE test.partition DETACH PARTITION 197001") + q("ALTER TABLE test.partition ATTACH PARTITION 197001") + q("OPTIMIZE TABLE test.partition") + + +def exec_bash(cmd): + cmd = '/bin/bash -c "{}"'.format(cmd.replace('"', '\\"')) + return instance.exec_in_container(cmd) + + +def partition_complex_assert_columns_txt(): + path_to_parts = path_to_data + 'data/test/partition/' + parts = TSV(q("SELECT name FROM system.parts WHERE database='test' AND table='partition'")) + for part_name in parts.lines: + path_to_columns = path_to_parts + part_name + '/columns.txt' + # 2 header lines + 3 columns + assert exec_bash('cat {} | wc -l'.format(path_to_columns)) == u'5\n' + + +def partition_complex_assert_checksums(): + # Do `cd` for consistent output for reference + # Do not check increment.txt - it can be changed by other tests with FREEZE + cmd = 'cd ' + path_to_data + " && find shadow -type f -exec md5sum {} \\;" \ + " | grep partition" \ + " | sed 's!shadow/[0-9]*/data/[a-z0-9_-]*/!shadow/1/data/test/!g'" \ + " | sort" \ + " | uniq" + + checksums = "082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition/19700102_2_2_0/k.bin\n" \ + "082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition/19700201_1_1_0/v1.bin\n" \ + "13cae8e658e0ca4f75c56b1fc424e150\tshadow/1/data/test/partition/19700102_2_2_0/minmax_p.idx\n" \ + "25daad3d9e60b45043a70c4ab7d3b1c6\tshadow/1/data/test/partition/19700102_2_2_0/partition.dat\n" \ + "3726312af62aec86b64a7708d5751787\tshadow/1/data/test/partition/19700201_1_1_0/partition.dat\n" \ + "37855b06a39b79a67ea4e86e4a3299aa\tshadow/1/data/test/partition/19700102_2_2_0/checksums.txt\n" \ + "38e62ff37e1e5064e9a3f605dfe09d13\tshadow/1/data/test/partition/19700102_2_2_0/v1.bin\n" \ + "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0/k.mrk\n" \ + "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0/p.mrk\n" \ + "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0/v1.mrk\n" \ + "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0/k.mrk\n" \ + "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0/p.mrk\n" \ + "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0/v1.mrk\n" \ + "55a54008ad1ba589aa210d2629c1df41\tshadow/1/data/test/partition/19700201_1_1_0/primary.idx\n" \ + "5f087cb3e7071bf9407e095821e2af8f\tshadow/1/data/test/partition/19700201_1_1_0/checksums.txt\n" \ + "77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition/19700102_2_2_0/columns.txt\n" \ + "77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition/19700201_1_1_0/columns.txt\n" \ + "88cdc31ded355e7572d68d8cde525d3a\tshadow/1/data/test/partition/19700201_1_1_0/p.bin\n" \ + "9e688c58a5487b8eaf69c9e1005ad0bf\tshadow/1/data/test/partition/19700102_2_2_0/primary.idx\n" \ + "c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition/19700102_2_2_0/count.txt\n" \ + "c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition/19700201_1_1_0/count.txt\n" \ + "cfcb770c3ecd0990dcceb1bde129e6c6\tshadow/1/data/test/partition/19700102_2_2_0/p.bin\n" \ + "e2af3bef1fd129aea73a890ede1e7a30\tshadow/1/data/test/partition/19700201_1_1_0/k.bin\n" \ + "f2312862cc01adf34a93151377be2ddf\tshadow/1/data/test/partition/19700201_1_1_0/minmax_p.idx\n" + + assert TSV(exec_bash(cmd).replace(' ', '\t')) == TSV(checksums) + + +@pytest.fixture +def partition_table_complex(started_cluster): + q("DROP TABLE IF EXISTS test.partition") + q("CREATE TABLE test.partition (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) " + "ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0") + q("INSERT INTO test.partition (p, k) VALUES(toDate(31), 1)") + q("INSERT INTO test.partition (p, k) VALUES(toDate(1), 2)") + + yield + + q("DROP TABLE test.partition") + + +def test_partition_complex(partition_table_complex): + + partition_complex_assert_columns_txt() + + q("ALTER TABLE test.partition FREEZE") + + partition_complex_assert_checksums() + + q("ALTER TABLE test.partition DETACH PARTITION 197001") + q("ALTER TABLE test.partition ATTACH PARTITION 197001") + + partition_complex_assert_columns_txt() + + q("ALTER TABLE test.partition MODIFY COLUMN v1 Int8") + + # Check the backup hasn't changed + partition_complex_assert_checksums() + + q("OPTIMIZE TABLE test.partition") + + expected = TSV('31\t1\t2\n' + '1\t2\t3') + res = q("SELECT toUInt16(p), k, v1 FROM test.partition ORDER BY k") + assert(TSV(res) == expected) + + +@pytest.fixture +def cannot_attach_active_part_table(started_cluster): + q("DROP TABLE IF EXISTS test.attach_active") + q("CREATE TABLE test.attach_active (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n") + q("INSERT INTO test.attach_active SELECT number FROM system.numbers LIMIT 16") + + yield + + q("DROP TABLE test.attach_active") + + +def test_cannot_attach_active_part(cannot_attach_active_part_table): + error = instance.client.query_and_get_error("ALTER TABLE test.attach_active ATTACH PART '../1_2_2_0'") + print error + assert 0 <= error.find('Invalid part name') + + res = q("SElECT name FROM system.parts WHERE table='attach_active' AND database='test' ORDER BY name") + assert TSV(res) == TSV('0_1_1_0\n1_2_2_0\n2_3_3_0\n3_4_4_0') + assert TSV(q("SElECT count(), sum(n) FROM test.attach_active")) == TSV('16\t120') + + +@pytest.fixture +def attach_check_all_parts_table(started_cluster): + q("SYSTEM STOP MERGES") + q("DROP TABLE IF EXISTS test.attach_partition") + q("CREATE TABLE test.attach_partition (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n") + q("INSERT INTO test.attach_partition SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8") + q("INSERT INTO test.attach_partition SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8") + + yield + + q("DROP TABLE test.attach_partition") + q("SYSTEM START MERGES") + + +def test_attach_check_all_parts(attach_check_all_parts_table): + q("ALTER TABLE test.attach_partition DETACH PARTITION 0") + + path_to_detached = path_to_data + 'data/test/attach_partition/detached/' + exec_bash('mkdir {}'.format(path_to_detached + '0_5_5_0')) + exec_bash('cp -pr {} {}'.format(path_to_detached + '0_1_1_0', path_to_detached + 'attaching_0_6_6_0')) + exec_bash('cp -pr {} {}'.format(path_to_detached + '0_3_3_0', path_to_detached + 'deleting_0_7_7_0')) + + error = instance.client.query_and_get_error("ALTER TABLE test.attach_partition ATTACH PARTITION 0") + assert 0 <= error.find('No columns in part 0_5_5_0') + + parts = q("SElECT name FROM system.parts WHERE table='attach_partition' AND database='test' ORDER BY name") + assert TSV(parts) == TSV('1_2_2_0\n1_4_4_0') + detached = q("SELECT name FROM system.detached_parts " + "WHERE table='attach_partition' AND database='test' ORDER BY name") + assert TSV(detached) == TSV('0_1_1_0\n0_3_3_0\n0_5_5_0\nattaching_0_6_6_0\ndeleting_0_7_7_0') + + exec_bash('rm -r {}'.format(path_to_detached + '0_5_5_0')) + + q("ALTER TABLE test.attach_partition ATTACH PARTITION 0") + parts = q("SElECT name FROM system.parts WHERE table='attach_partition' AND database='test' ORDER BY name") + expected = '0_5_5_0\n0_6_6_0\n1_2_2_0\n1_4_4_0' + assert TSV(parts) == TSV(expected) + assert TSV(q("SElECT count(), sum(n) FROM test.attach_partition")) == TSV('16\t120') + + detached = q("SELECT name FROM system.detached_parts " + "WHERE table='attach_partition' AND database='test' ORDER BY name") + assert TSV(detached) == TSV('attaching_0_6_6_0\ndeleting_0_7_7_0') + + +@pytest.fixture +def drop_detached_parts_table(started_cluster): + q("SYSTEM STOP MERGES") + q("DROP TABLE IF EXISTS test.drop_detached") + q("CREATE TABLE test.drop_detached (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n") + q("INSERT INTO test.drop_detached SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8") + q("INSERT INTO test.drop_detached SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8") + + yield + + q("DROP TABLE test.drop_detached") + q("SYSTEM START MERGES") + + +def test_drop_detached_parts(drop_detached_parts_table): + s = {"allow_drop_detached_part": 1} + q("ALTER TABLE test.drop_detached DETACH PARTITION 0") + q("ALTER TABLE test.drop_detached DETACH PARTITION 1") + + path_to_detached = path_to_data + 'data/test/drop_detached/detached/' + exec_bash('mkdir {}'.format(path_to_detached + 'attaching_0_6_6_0')) + exec_bash('mkdir {}'.format(path_to_detached + 'deleting_0_7_7_0')) + exec_bash('mkdir {}'.format(path_to_detached + 'any_other_name')) + exec_bash('mkdir {}'.format(path_to_detached + 'prefix_1_2_2_0_0')) + + error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART '../1_2_2_0'", settings=s) + assert 0 <= error.find('Invalid part name') + + q("ALTER TABLE test.drop_detached DROP DETACHED PART '0_1_1_0'", settings=s) + + error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART 'attaching_0_6_6_0'", settings=s) + assert 0 <= error.find('Cannot drop part') + + error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART 'deleting_0_7_7_0'", settings=s) + assert 0 <= error.find('Cannot drop part') + + q("ALTER TABLE test.drop_detached DROP DETACHED PART 'any_other_name'", settings=s) + + detached = q("SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='test' ORDER BY name") + assert TSV(detached) == TSV('0_3_3_0\n1_2_2_0\n1_4_4_0\nattaching_0_6_6_0\ndeleting_0_7_7_0\nprefix_1_2_2_0_0') + + q("ALTER TABLE test.drop_detached DROP DETACHED PARTITION 1", settings=s) + detached = q("SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='test' ORDER BY name") + assert TSV(detached) == TSV('0_3_3_0\nattaching_0_6_6_0\ndeleting_0_7_7_0') + diff --git a/dbms/tests/queries/0_stateless/00428_partition.reference b/dbms/tests/queries/0_stateless/00428_partition.reference deleted file mode 100644 index c777fd7a5c3..00000000000 --- a/dbms/tests/queries/0_stateless/00428_partition.reference +++ /dev/null @@ -1,54 +0,0 @@ -5 -5 -082814b5aa5109160d5c0c5aff10d4df shadow/1/data/test/partition_428/19700102_2_2_0/k.bin -082814b5aa5109160d5c0c5aff10d4df shadow/1/data/test/partition_428/19700201_1_1_0/v1.bin -13cae8e658e0ca4f75c56b1fc424e150 shadow/1/data/test/partition_428/19700102_2_2_0/minmax_p.idx -25daad3d9e60b45043a70c4ab7d3b1c6 shadow/1/data/test/partition_428/19700102_2_2_0/partition.dat -3726312af62aec86b64a7708d5751787 shadow/1/data/test/partition_428/19700201_1_1_0/partition.dat -37855b06a39b79a67ea4e86e4a3299aa shadow/1/data/test/partition_428/19700102_2_2_0/checksums.txt -38e62ff37e1e5064e9a3f605dfe09d13 shadow/1/data/test/partition_428/19700102_2_2_0/v1.bin -4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/k.mrk -4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/p.mrk -4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/v1.mrk -4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/k.mrk -4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/p.mrk -4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/v1.mrk -55a54008ad1ba589aa210d2629c1df41 shadow/1/data/test/partition_428/19700201_1_1_0/primary.idx -5f087cb3e7071bf9407e095821e2af8f shadow/1/data/test/partition_428/19700201_1_1_0/checksums.txt -77d5af402ada101574f4da114f242e02 shadow/1/data/test/partition_428/19700102_2_2_0/columns.txt -77d5af402ada101574f4da114f242e02 shadow/1/data/test/partition_428/19700201_1_1_0/columns.txt -88cdc31ded355e7572d68d8cde525d3a shadow/1/data/test/partition_428/19700201_1_1_0/p.bin -9e688c58a5487b8eaf69c9e1005ad0bf shadow/1/data/test/partition_428/19700102_2_2_0/primary.idx -c4ca4238a0b923820dcc509a6f75849b shadow/1/data/test/partition_428/19700102_2_2_0/count.txt -c4ca4238a0b923820dcc509a6f75849b shadow/1/data/test/partition_428/19700201_1_1_0/count.txt -cfcb770c3ecd0990dcceb1bde129e6c6 shadow/1/data/test/partition_428/19700102_2_2_0/p.bin -e2af3bef1fd129aea73a890ede1e7a30 shadow/1/data/test/partition_428/19700201_1_1_0/k.bin -f2312862cc01adf34a93151377be2ddf shadow/1/data/test/partition_428/19700201_1_1_0/minmax_p.idx -5 -5 -082814b5aa5109160d5c0c5aff10d4df shadow/1/data/test/partition_428/19700102_2_2_0/k.bin -082814b5aa5109160d5c0c5aff10d4df shadow/1/data/test/partition_428/19700201_1_1_0/v1.bin -13cae8e658e0ca4f75c56b1fc424e150 shadow/1/data/test/partition_428/19700102_2_2_0/minmax_p.idx -25daad3d9e60b45043a70c4ab7d3b1c6 shadow/1/data/test/partition_428/19700102_2_2_0/partition.dat -3726312af62aec86b64a7708d5751787 shadow/1/data/test/partition_428/19700201_1_1_0/partition.dat -37855b06a39b79a67ea4e86e4a3299aa shadow/1/data/test/partition_428/19700102_2_2_0/checksums.txt -38e62ff37e1e5064e9a3f605dfe09d13 shadow/1/data/test/partition_428/19700102_2_2_0/v1.bin -4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/k.mrk -4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/p.mrk -4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/v1.mrk -4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/k.mrk -4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/p.mrk -4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/v1.mrk -55a54008ad1ba589aa210d2629c1df41 shadow/1/data/test/partition_428/19700201_1_1_0/primary.idx -5f087cb3e7071bf9407e095821e2af8f shadow/1/data/test/partition_428/19700201_1_1_0/checksums.txt -77d5af402ada101574f4da114f242e02 shadow/1/data/test/partition_428/19700102_2_2_0/columns.txt -77d5af402ada101574f4da114f242e02 shadow/1/data/test/partition_428/19700201_1_1_0/columns.txt -88cdc31ded355e7572d68d8cde525d3a shadow/1/data/test/partition_428/19700201_1_1_0/p.bin -9e688c58a5487b8eaf69c9e1005ad0bf shadow/1/data/test/partition_428/19700102_2_2_0/primary.idx -c4ca4238a0b923820dcc509a6f75849b shadow/1/data/test/partition_428/19700102_2_2_0/count.txt -c4ca4238a0b923820dcc509a6f75849b shadow/1/data/test/partition_428/19700201_1_1_0/count.txt -cfcb770c3ecd0990dcceb1bde129e6c6 shadow/1/data/test/partition_428/19700102_2_2_0/p.bin -e2af3bef1fd129aea73a890ede1e7a30 shadow/1/data/test/partition_428/19700201_1_1_0/k.bin -f2312862cc01adf34a93151377be2ddf shadow/1/data/test/partition_428/19700201_1_1_0/minmax_p.idx -31,1,2 -1,2,3 diff --git a/dbms/tests/queries/0_stateless/00428_partition.sh b/dbms/tests/queries/0_stateless/00428_partition.sh deleted file mode 100755 index 033d5e24c13..00000000000 --- a/dbms/tests/queries/0_stateless/00428_partition.sh +++ /dev/null @@ -1,60 +0,0 @@ -#!/usr/bin/env bash - -set -e - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. $CURDIR/../shell_config.sh - -# Not found column date in block. There are only columns: x. - -# Test 1. Complex test checking columns.txt - -chl="$CLICKHOUSE_CLIENT -q" -ch_dir=`${CLICKHOUSE_EXTRACT_CONFIG} -k path` - -$chl "DROP TABLE IF EXISTS test.partition_428" -$chl "CREATE TABLE test.partition_428 (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0" -$chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(31), 1)" -$chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(1), 2)" - -for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do - # 2 header lines + 3 columns - (sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt 2>/dev/null || \ - cat $ch_dir/data/test/partition_428/$part/columns.txt) | wc -l -done - -$chl "ALTER TABLE test.partition_428 FREEZE" - -# Do `cd` for consistent output for reference -# Do not check increment.txt - it can be changed by other tests with FREEZE -cd $ch_dir && find shadow -type f -exec md5sum {} \; | grep "partition_428" | sed 's!shadow/[0-9]*/data/[a-z0-9_-]*/!shadow/1/data/test/!g' | sort | uniq - -$chl "ALTER TABLE test.partition_428 DETACH PARTITION 197001" -$chl "ALTER TABLE test.partition_428 ATTACH PARTITION 197001" - -for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do - # 2 header lines + 3 columns - (sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt 2>/dev/null || \ - cat $ch_dir/data/test/partition_428/$part/columns.txt) | wc -l -done - -$chl "ALTER TABLE test.partition_428 MODIFY COLUMN v1 Int8" - -# Check the backup hasn't changed -cd $ch_dir && find shadow -type f -exec md5sum {} \; | grep "partition_428" | sed 's!shadow/[0-9]*/data/[a-z0-9_-]*/!shadow/1/data/test/!g' | sort | uniq - -$chl "OPTIMIZE TABLE test.partition_428" - -$chl "SELECT toUInt16(p), k, v1 FROM test.partition_428 ORDER BY k FORMAT CSV" -$chl "DROP TABLE test.partition_428" - -# Test 2. Simple test - -$chl "drop table if exists test.partition_428" -$chl "create table test.partition_428 (date MATERIALIZED toDate(0), x UInt64, sample_key MATERIALIZED intHash64(x)) ENGINE=MergeTree PARTITION BY date SAMPLE BY sample_key ORDER BY (date,x,sample_key) SETTINGS index_granularity=8192, index_granularity_bytes=0" -$chl "insert into test.partition_428 ( x ) VALUES ( now() )" -$chl "insert into test.partition_428 ( x ) VALUES ( now()+1 )" -$chl "alter table test.partition_428 detach partition 197001" -$chl "alter table test.partition_428 attach partition 197001" -$chl "optimize table test.partition_428" -$chl "drop table test.partition_428" diff --git a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference deleted file mode 100644 index f30fc160dfb..00000000000 --- a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.reference +++ /dev/null @@ -1,26 +0,0 @@ -=== cannot attach active === -OK1 -0_1_1_0 -1_2_2_0 -2_3_3_0 -3_4_4_0 -16 120 -=== check all parts before attaching === -OK2 -1_2_2_0 -1_4_4_0 -=== detached === -0_1_1_0 -0_3_3_0 -0_5_5_0 -attaching_0_6_6_0 -deleting_0_7_7_0 -=== attach === -0_5_5_0 -0_6_6_0 -1_2_2_0 -1_4_4_0 -16 120 -=== detached === -attaching_0_6_6_0 -deleting_0_7_7_0 diff --git a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh b/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh deleted file mode 100755 index db45cfe7f21..00000000000 --- a/dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/usr/bin/env bash - -set -e - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. $CURDIR/../shell_config.sh - -ch_dir=`${CLICKHOUSE_EXTRACT_CONFIG} -k path` -cur_db=`${CLICKHOUSE_CLIENT} --query "SELECT currentDatabase()"` - -echo '=== cannot attach active ==='; -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_active"; -$CLICKHOUSE_CLIENT --query="CREATE TABLE attach_active (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n"; -$CLICKHOUSE_CLIENT --query="INSERT INTO attach_active SELECT number FROM system.numbers LIMIT 16"; -$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_active ATTACH PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK1' -$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_active' AND database='${cur_db}' ORDER BY name FORMAT TSV"; -$CLICKHOUSE_CLIENT --query="SElECT count(), sum(n) FROM attach_active FORMAT TSV"; -$CLICKHOUSE_CLIENT --query="DROP TABLE attach_active"; - - - -$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES"; -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_partitions"; -$CLICKHOUSE_CLIENT --query="CREATE TABLE attach_partitions (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n"; -$CLICKHOUSE_CLIENT --query="INSERT INTO attach_partitions SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8"; -$CLICKHOUSE_CLIENT --query="INSERT INTO attach_partitions SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8"; - -$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions DETACH PARTITION 0"; -sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ 2>/dev/null || \ - mkdir --mode=777 $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ # broken part -sudo -n cp -pr $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/ 2>/dev/null || \ - cp -pr $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/ -sudo -n cp -pr $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/deleting_0_7_7_0/ 2>/dev/null || \ - cp -pr $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/deleting_0_7_7_0/ - -echo '=== check all parts before attaching ==='; -$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions ATTACH PARTITION 0" 2>&1 | grep "No columns in part 0_5_5_0" > /dev/null && echo 'OK2'; -$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV"; -echo '=== detached ==='; -$CLICKHOUSE_CLIENT --query="SELECT name FROM system.detached_parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV"; - -echo '=== attach ==='; -sudo -n rm -r $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ 2>/dev/null || \ - rm -r $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ -$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions ATTACH PARTITION 0"; -$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV"; -$CLICKHOUSE_CLIENT --query="SElECT count(), sum(n) FROM attach_partitions FORMAT TSV"; - -echo '=== detached ==='; -$CLICKHOUSE_CLIENT --query="SELECT name FROM system.detached_parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV"; - -$CLICKHOUSE_CLIENT --query="DROP TABLE attach_partitions"; -$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES"; diff --git a/dbms/tests/queries/0_stateless/00975_drop_detached.reference b/dbms/tests/queries/0_stateless/00975_drop_detached.reference deleted file mode 100644 index 414ac4b1927..00000000000 --- a/dbms/tests/queries/0_stateless/00975_drop_detached.reference +++ /dev/null @@ -1,15 +0,0 @@ -=== validate part name === -OK1 -OK2 -OK3 -=== drop detached part === -0_3_3_0 -1_2_2_0 -1_4_4_0 -attaching_0_6_6_0 -deleting_0_7_7_0 -prefix_1_2_2_0_0 -=== drop detached partition === -0_3_3_0 -attaching_0_6_6_0 -deleting_0_7_7_0 diff --git a/dbms/tests/queries/0_stateless/00975_drop_detached.sh b/dbms/tests/queries/0_stateless/00975_drop_detached.sh deleted file mode 100755 index 8da831b019a..00000000000 --- a/dbms/tests/queries/0_stateless/00975_drop_detached.sh +++ /dev/null @@ -1,45 +0,0 @@ -#!/usr/bin/env bash - -set -e - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. $CURDIR/../shell_config.sh - -ch_dir=`${CLICKHOUSE_EXTRACT_CONFIG} -k path` -cur_db=`${CLICKHOUSE_CLIENT} --query "SELECT currentDatabase()"` - -$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES"; -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS drop_detached"; -$CLICKHOUSE_CLIENT --query="CREATE TABLE drop_detached (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n"; -$CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8"; -$CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8"; - -$CLICKHOUSE_CLIENT --query="ALTER TABLE drop_detached DETACH PARTITION 0"; -$CLICKHOUSE_CLIENT --query="ALTER TABLE drop_detached DETACH PARTITION 1"; -sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/ 2>/dev/null || \ - mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/ -sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/deleting_0_7_7_0/ 2>/dev/null || \ - mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/deleting_0_7_7_0/ -sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/ 2>/dev/null || \ - mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/ -sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/prefix_1_2_2_0_0/ 2>/dev/null || \ - mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/prefix_1_2_2_0_0/ -#sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/prefix_1_2_2_0/ 2>/dev/null || \ -# mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/prefix_1_2_2_0/ - -echo '=== validate part name ===' -$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK1' -$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '0_1_1_0'" -$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'attaching_0_6_6_0'" 2>&1 | grep "Cannot drop part" > /dev/null && echo 'OK2' -$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'deleting_0_7_7_0'" 2>&1 | grep "Cannot drop part" > /dev/null && echo 'OK3' -$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'any_other_name'" - -echo '=== drop detached part ===' -$CLICKHOUSE_CLIENT --query="SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='${cur_db}' ORDER BY name FORMAT TSV"; - -echo '=== drop detached partition ===' -$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PARTITION 1" -$CLICKHOUSE_CLIENT --query="SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='${cur_db}' ORDER BY name FORMAT TSV"; - -$CLICKHOUSE_CLIENT --query="DROP TABLE drop_detached"; -$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES"; From b5eee531a9cbd9b3df2c5c373d99680e50d6b8cb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 8 Aug 2019 19:08:43 +0300 Subject: [PATCH 143/509] fix setting name --- dbms/src/Core/Settings.h | 2 +- dbms/src/Interpreters/InterpreterAlterQuery.cpp | 2 +- dbms/tests/integration/test_partition/test.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index fd11d645bd5..ffc11cef4a6 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -341,7 +341,7 @@ struct Settings : public SettingsCollection /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") \ - M(SettingBool, allow_drop_detached_part, false, "Allow ALTER TABLE ... DROP DETACHED PART ... queries") + M(SettingBool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries") DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index 074fbb7d4c2..bc419f1ff84 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -56,7 +56,7 @@ BlockIO InterpreterAlterQuery::execute() else if (auto partition_command = PartitionCommand::parse(command_ast)) { if (partition_command->type == PartitionCommand::DROP_DETACHED_PARTITION - && !context.getSettingsRef().allow_drop_detached_part) + && !context.getSettingsRef().allow_drop_detached) throw DB::Exception("Cannot execute query: DROP DETACHED PART is disabled " "(see allow_drop_detached setting)", ErrorCodes::SUPPORT_IS_DISABLED); partition_commands.emplace_back(std::move(*partition_command)); diff --git a/dbms/tests/integration/test_partition/test.py b/dbms/tests/integration/test_partition/test.py index 59c48e5d9e9..3365343b6fb 100644 --- a/dbms/tests/integration/test_partition/test.py +++ b/dbms/tests/integration/test_partition/test.py @@ -212,7 +212,7 @@ def drop_detached_parts_table(started_cluster): def test_drop_detached_parts(drop_detached_parts_table): - s = {"allow_drop_detached_part": 1} + s = {"allow_drop_detached": 1} q("ALTER TABLE test.drop_detached DETACH PARTITION 0") q("ALTER TABLE test.drop_detached DETACH PARTITION 1") From b780072a9d4fe86c884c087b88b552b641a03ac4 Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Thu, 8 Aug 2019 21:49:29 +0300 Subject: [PATCH 144/509] raw data relative_analize --- dbms/programs/benchmark/Benchmark.cpp | 192 +++++++++++++++++++++++++- 1 file changed, 187 insertions(+), 5 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 923745eb204..e97a499101b 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -67,6 +67,7 @@ public: connections.reserve(connections_cnt); comparison_info_total.reserve(connections_cnt); comparison_info_per_interval.reserve(connections_cnt); + comparison_relative.data.resize(connections_cnt); for (size_t i = 0; i < connections_cnt; ++i) { @@ -183,6 +184,178 @@ private: } }; + struct RelativeAnalysis + { + struct RelativeStats + { + size_t cnt = 0; + double sum = 0; + double squares_sum = 0; + + void add(double seconds) + { + ++cnt; + sum += seconds; + squares_sum += seconds * seconds; + } + + double avg() + { + return sum / cnt; + } + + double var() + { + return (squares_sum - (sum * sum / cnt)) / static_cast(cnt - 1); + } + }; + + std::vector confidence_level = { 80, 90, 95, 98, 99, 99.5 }; + + std::vector> students_table = { + /* inf */ { 1.282, 1.645, 1.960, 2.326, 2.576, 3.090 }, + /* 1. */ { 3.078, 6.314, 12.706, 31.821, 63.657, 318.313 }, + /* 2. */ { 1.886, 2.920, 4.303, 6.965, 9.925, 22.327 }, + /* 3. */ { 1.638, 2.353, 3.182, 4.541, 5.841, 10.215 }, + /* 4. */ { 1.533, 2.132, 2.776, 3.747, 4.604, 7.173 }, + /* 5. */ { 1.476, 2.015, 2.571, 3.365, 4.032, 5.893 }, + /* 6. */ { 1.440, 1.943, 2.447, 3.143, 3.707, 5.208 }, + /* 7. */ { 1.415, 1.895, 2.365, 2.998, 3.499, 4.782 }, + /* 8. */ { 1.397, 1.860, 2.306, 2.896, 3.355, 4.499 }, + /* 9. */ { 1.383, 1.833, 2.262, 2.821, 3.250, 4.296 }, + /* 10. */ { 1.372, 1.812, 2.228, 2.764, 3.169, 4.143 }, + /* 11. */ { 1.363, 1.796, 2.201, 2.718, 3.106, 4.024 }, + /* 12. */ { 1.356, 1.782, 2.179, 2.681, 3.055, 3.929 }, + /* 13. */ { 1.350, 1.771, 2.160, 2.650, 3.012, 3.852 }, + /* 14. */ { 1.345, 1.761, 2.145, 2.624, 2.977, 3.787 }, + /* 15. */ { 1.341, 1.753, 2.131, 2.602, 2.947, 3.733 }, + /* 16. */ { 1.337, 1.746, 2.120, 2.583, 2.921, 3.686 }, + /* 17. */ { 1.333, 1.740, 2.110, 2.567, 2.898, 3.646 }, + /* 18. */ { 1.330, 1.734, 2.101, 2.552, 2.878, 3.610 }, + /* 19. */ { 1.328, 1.729, 2.093, 2.539, 2.861, 3.579 }, + /* 20. */ { 1.325, 1.725, 2.086, 2.528, 2.845, 3.552 }, + /* 21. */ { 1.323, 1.721, 2.080, 2.518, 2.831, 3.527 }, + /* 22. */ { 1.321, 1.717, 2.074, 2.508, 2.819, 3.505 }, + /* 23. */ { 1.319, 1.714, 2.069, 2.500, 2.807, 3.485 }, + /* 24. */ { 1.318, 1.711, 2.064, 2.492, 2.797, 3.467 }, + /* 25. */ { 1.316, 1.708, 2.060, 2.485, 2.787, 3.450 }, + /* 26. */ { 1.315, 1.706, 2.056, 2.479, 2.779, 3.435 }, + /* 27. */ { 1.314, 1.703, 2.052, 2.473, 2.771, 3.421 }, + /* 28. */ { 1.313, 1.701, 2.048, 2.467, 2.763, 3.408 }, + /* 29. */ { 1.311, 1.699, 2.045, 2.462, 2.756, 3.396 }, + /* 30. */ { 1.310, 1.697, 2.042, 2.457, 2.750, 3.385 }, + /* 31. */ { 1.309, 1.696, 2.040, 2.453, 2.744, 3.375 }, + /* 32. */ { 1.309, 1.694, 2.037, 2.449, 2.738, 3.365 }, + /* 33. */ { 1.308, 1.692, 2.035, 2.445, 2.733, 3.356 }, + /* 34. */ { 1.307, 1.691, 2.032, 2.441, 2.728, 3.348 }, + /* 35. */ { 1.306, 1.690, 2.030, 2.438, 2.724, 3.340 }, + /* 36. */ { 1.306, 1.688, 2.028, 2.434, 2.719, 3.333 }, + /* 37. */ { 1.305, 1.687, 2.026, 2.431, 2.715, 3.326 }, + /* 38. */ { 1.304, 1.686, 2.024, 2.429, 2.712, 3.319 }, + /* 39. */ { 1.304, 1.685, 2.023, 2.426, 2.708, 3.313 }, + /* 40. */ { 1.303, 1.684, 2.021, 2.423, 2.704, 3.307 }, + /* 41. */ { 1.303, 1.683, 2.020, 2.421, 2.701, 3.301 }, + /* 42. */ { 1.302, 1.682, 2.018, 2.418, 2.698, 3.296 }, + /* 43. */ { 1.302, 1.681, 2.017, 2.416, 2.695, 3.291 }, + /* 44. */ { 1.301, 1.680, 2.015, 2.414, 2.692, 3.286 }, + /* 45. */ { 1.301, 1.679, 2.014, 2.412, 2.690, 3.281 }, + /* 46. */ { 1.300, 1.679, 2.013, 2.410, 2.687, 3.277 }, + /* 47. */ { 1.300, 1.678, 2.012, 2.408, 2.685, 3.273 }, + /* 48. */ { 1.299, 1.677, 2.011, 2.407, 2.682, 3.269 }, + /* 49. */ { 1.299, 1.677, 2.010, 2.405, 2.680, 3.265 }, + /* 50. */ { 1.299, 1.676, 2.009, 2.403, 2.678, 3.261 }, + /* 51. */ { 1.298, 1.675, 2.008, 2.402, 2.676, 3.258 }, + /* 52. */ { 1.298, 1.675, 2.007, 2.400, 2.674, 3.255 }, + /* 53. */ { 1.298, 1.674, 2.006, 2.399, 2.672, 3.251 }, + /* 54. */ { 1.297, 1.674, 2.005, 2.397, 2.670, 3.248 }, + /* 55. */ { 1.297, 1.673, 2.004, 2.396, 2.668, 3.245 }, + /* 56. */ { 1.297, 1.673, 2.003, 2.395, 2.667, 3.242 }, + /* 57. */ { 1.297, 1.672, 2.002, 2.394, 2.665, 3.239 }, + /* 58. */ { 1.296, 1.672, 2.002, 2.392, 2.663, 3.237 }, + /* 59. */ { 1.296, 1.671, 2.001, 2.391, 2.662, 3.234 }, + /* 60. */ { 1.296, 1.671, 2.000, 2.390, 2.660, 3.232 }, + /* 61. */ { 1.296, 1.670, 2.000, 2.389, 2.659, 3.229 }, + /* 62. */ { 1.295, 1.670, 1.999, 2.388, 2.657, 3.227 }, + /* 63. */ { 1.295, 1.669, 1.998, 2.387, 2.656, 3.225 }, + /* 64. */ { 1.295, 1.669, 1.998, 2.386, 2.655, 3.223 }, + /* 65. */ { 1.295, 1.669, 1.997, 2.385, 2.654, 3.220 }, + /* 66. */ { 1.295, 1.668, 1.997, 2.384, 2.652, 3.218 }, + /* 67. */ { 1.294, 1.668, 1.996, 2.383, 2.651, 3.216 }, + /* 68. */ { 1.294, 1.668, 1.995, 2.382, 2.650, 3.214 }, + /* 69. */ { 1.294, 1.667, 1.995, 2.382, 2.649, 3.213 }, + /* 70. */ { 1.294, 1.667, 1.994, 2.381, 2.648, 3.211 }, + /* 71. */ { 1.294, 1.667, 1.994, 2.380, 2.647, 3.209 }, + /* 72. */ { 1.293, 1.666, 1.993, 2.379, 2.646, 3.207 }, + /* 73. */ { 1.293, 1.666, 1.993, 2.379, 2.645, 3.206 }, + /* 74. */ { 1.293, 1.666, 1.993, 2.378, 2.644, 3.204 }, + /* 75. */ { 1.293, 1.665, 1.992, 2.377, 2.643, 3.202 }, + /* 76. */ { 1.293, 1.665, 1.992, 2.376, 2.642, 3.201 }, + /* 77. */ { 1.293, 1.665, 1.991, 2.376, 2.641, 3.199 }, + /* 78. */ { 1.292, 1.665, 1.991, 2.375, 2.640, 3.198 }, + /* 79. */ { 1.292, 1.664, 1.990, 2.374, 2.640, 3.197 }, + /* 80. */ { 1.292, 1.664, 1.990, 2.374, 2.639, 3.195 }, + /* 81. */ { 1.292, 1.664, 1.990, 2.373, 2.638, 3.194 }, + /* 82. */ { 1.292, 1.664, 1.989, 2.373, 2.637, 3.193 }, + /* 83. */ { 1.292, 1.663, 1.989, 2.372, 2.636, 3.191 }, + /* 84. */ { 1.292, 1.663, 1.989, 2.372, 2.636, 3.190 }, + /* 85. */ { 1.292, 1.663, 1.988, 2.371, 2.635, 3.189 }, + /* 86. */ { 1.291, 1.663, 1.988, 2.370, 2.634, 3.188 }, + /* 87. */ { 1.291, 1.663, 1.988, 2.370, 2.634, 3.187 }, + /* 88. */ { 1.291, 1.662, 1.987, 2.369, 2.633, 3.185 }, + /* 89. */ { 1.291, 1.662, 1.987, 2.369, 2.632, 3.184 }, + /* 90. */ { 1.291, 1.662, 1.987, 2.368, 2.632, 3.183 }, + /* 91. */ { 1.291, 1.662, 1.986, 2.368, 2.631, 3.182 }, + /* 92. */ { 1.291, 1.662, 1.986, 2.368, 2.630, 3.181 }, + /* 93. */ { 1.291, 1.661, 1.986, 2.367, 2.630, 3.180 }, + /* 94. */ { 1.291, 1.661, 1.986, 2.367, 2.629, 3.179 }, + /* 95. */ { 1.291, 1.661, 1.985, 2.366, 2.629, 3.178 }, + /* 96. */ { 1.290, 1.661, 1.985, 2.366, 2.628, 3.177 }, + /* 97. */ { 1.290, 1.661, 1.985, 2.365, 2.627, 3.176 }, + /* 98. */ { 1.290, 1.661, 1.984, 2.365, 2.627, 3.175 }, + /* 99. */ { 1.290, 1.660, 1.984, 2.365, 2.626, 3.175 }, + /* 100. */ { 1.290, 1.660, 1.984, 2.364, 2.626, 3.174 } + }; + + std::vector data; + + bool report(UInt8 confidence_index) + { + if (data.size() != 2) /// Works for two connections only + return true; + + size_t i = (data[0].cnt - 1) + (data[1].cnt - 1); + + double t = students_table[i > 100 ? 0 : i][confidence_index]; + + double spool = (data[0].cnt - 1) * data[0].var() + (data[1].cnt - 1) * data[1].var(); + spool = sqrt(spool / i); + + double s = spool * sqrt(1.0 / data[0].cnt + 1.0 / data[1].cnt); + + double d = data[0].avg() - data[1].avg(); + + double e = t * s; + + std::cerr << '\n'; + if (fabs(d) > e) + { + std::cerr << std::setprecision(1) << "Difference at " << confidence_level[confidence_index] << "% confidence\n" << std::setprecision(6); + std::cerr << "\t" << d << " +/- " << e << "\n"; + std::cerr << "\t" << d * 100 / data[1].avg() << " +/- " << e * 100 / data[1].avg() << "\n"; + std::cerr << "\t(Student's t, pooled s = " << spool << ")\n" << std::setprecision(3); + return false; + } + else + { + std::cerr << std::setprecision(1) << "No difference proven at " << confidence_level[confidence_index] << "% confidence\n" << std::setprecision(3); + return true; + } + } + + }; + + RelativeAnalysis comparison_relative; + using MultiStats = std::vector>; MultiStats comparison_info_per_interval; MultiStats comparison_info_total; @@ -254,6 +427,9 @@ private: { printNumberOfQueriesExecuted(queries_executed); cumulative ? report(comparison_info_total) : report(comparison_info_per_interval); + + comparison_relative.report(5); + delay_watch.restart(); } } @@ -300,6 +476,7 @@ private: printNumberOfQueriesExecuted(queries_executed); report(comparison_info_total); + comparison_relative.report(5); } @@ -366,6 +543,7 @@ private: comparison_info_per_interval[connection_index]->add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes); comparison_info_total[connection_index]->add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes); + comparison_relative.data[connection_index].add(seconds); } void report(MultiStats & infos) @@ -380,14 +558,16 @@ private: if (0 == info->queries) return; + double seconds = info->work_time / concurrency; + std::cerr << "connection " << info_counter++ << ", " << "queries " << info->queries << ", " - << "QPS: " << (info->queries / info->work_time) << ", " - << "RPS: " << (info->read_rows / info->work_time) << ", " - << "MiB/s: " << (info->read_bytes / info->work_time / 1048576) << ", " - << "result RPS: " << (info->result_rows / info->work_time) << ", " - << "result MiB/s: " << (info->result_bytes / info->work_time / 1048576) << "." + << "QPS: " << (info->queries / seconds) << ", " + << "RPS: " << (info->read_rows / seconds) << ", " + << "MiB/s: " << (info->read_bytes / seconds / 1048576) << ", " + << "result RPS: " << (info->result_rows / seconds) << ", " + << "result MiB/s: " << (info->result_bytes / seconds / 1048576) << "." << "\n"; } std::cerr << "\n\t\t"; @@ -415,8 +595,10 @@ private: print_percentile(99.99); if (!cumulative) + { for (auto & info : infos) info->clear(); + } } void reportJSON(MultiStats & infos, const std::string & filename) From d1ebfaacd6df35b2a6b55f25f6f9319e00c1f5ac Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 8 Aug 2019 22:28:25 +0300 Subject: [PATCH 145/509] update docs --- docs/en/operations/system_tables.md | 3 ++- docs/en/query_language/alter.md | 12 +++++++++++- docs/ru/operations/system_tables.md | 6 ++++++ docs/ru/query_language/alter.md | 11 ++++++++++- 4 files changed, 29 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index e63a9115270..e5eac2f1f58 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -87,13 +87,14 @@ This table contains a single String column called 'name' – the name of a datab Each database that the server knows about has a corresponding entry in the table. This system table is used for implementing the `SHOW DATABASES` query. -## system.detached_parts +## system.detached_parts {#system_tables-detached_parts} Contains information about detached parts of [MergeTree](table_engines/mergetree.md) tables. The `reason` column specifies why the part was detached. For user-detached parts, the reason is empty. Such parts can be attached with [ALTER TABLE ATTACH PARTITION|PART](../query_language/query_language/alter/#alter_attach-partition) command. For the description of other columns, see [system.parts](#system_tables-parts). +If part name is invalid, values of some columns may be `NULL`. Such parts can be deleted with [ALTER TABLE DROP DETACHED PART](../query_language/query_language/alter/#alter_drop-detached). ## system.dictionaries diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index 6e8e712ff30..2d42c4cc354 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -210,6 +210,16 @@ Read about setting the partition expression in a section [How to specify the par The query is replicated – it deletes data on all replicas. +#### DROP DETACHED PARTITION|PART {#alter_drop-detached} + +```sql +ALTER TABLE table_name DROP DETACHED PARTITION|PART partition_expr +``` + +Removes the specified part or all parts of the specified partition from `detached`. +Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). + + #### ATTACH PARTITION|PART {#alter_attach-partition} ``` sql @@ -327,7 +337,7 @@ You can specify the partition expression in `ALTER ... PARTITION` queries in dif - As a value from the `partition` column of the `system.parts` table. For example, `ALTER TABLE visits DETACH PARTITION 201901`. - As the expression from the table column. Constants and constant expressions are supported. For example, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))`. - Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`. -- In the [ALTER ATTACH PART](#alter_attach-partition) query, to specify the name of a part, use a value from the `name` column of the `system.parts` table. For example, `ALTER TABLE visits ATTACH PART 201901_1_1_0`. +- In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](../operations/system_tables.md#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. Usage of quotes when specifying the partition depends on the type of partition expression. For example, for the `String` type, you have to specify its name in quotes (`'`). For the `Date` and `Int*` types no quotes are needed. diff --git a/docs/ru/operations/system_tables.md b/docs/ru/operations/system_tables.md index eb452c8de4e..4345f83718b 100644 --- a/docs/ru/operations/system_tables.md +++ b/docs/ru/operations/system_tables.md @@ -47,6 +47,12 @@ default_expression String - выражение для значения по ум Для каждой базы данных, о которой знает сервер, будет присутствовать соответствующая запись в таблице. Эта системная таблица используется для реализации запроса `SHOW DATABASES`. +## system.detached_parts {#system_tables-detached_parts} + +Сожелржит информацию об отсоединённых кусках таблиц семейства [MergeTree](table_engines/mergetree.md). Столбец `reason` содержит причину, по которой кусок был отсоединён. Для кусов, отсоединённых пользователем, `reason` содержит пустую строку. +Такие куски могут быть присоединены с помощью [ALTER TABLE ATTACH PARTITION|PART](../query_language/query_language/alter/#alter_attach-partition). Остальные столбцы описаны в [system.parts](#system_tables-parts). +Если имя куска некорректно, значения некоторых столбцов могут быть `NULL`. Такие куски могут быть удалены с помощью [ALTER TABLE DROP DETACHED PART](../query_language/query_language/alter/#alter_drop-detached). + ## system.dictionaries Содержит информацию о внешних словарях. diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index 2367386172a..3e0030e948e 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -209,6 +209,15 @@ ALTER TABLE table_name DROP PARTITION partition_expr Запрос реплицируется — данные будут удалены на всех репликах. +#### DROP DETACHED PARTITION|PART {#alter_drop-detached} + +```sql +ALTER TABLE table_name DROP DETACHED PARTITION|PART partition_expr +``` + +Удаляет из `detached` кусок или все куски, принадлежащие партиции. +Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr). + #### ATTACH PARTITION|PART {#alter_attach-partition} ```sql @@ -328,7 +337,7 @@ ALTER TABLE users ATTACH PARTITION 201902; - Имя партиции. Посмотреть имя партиции можно в столбце `partition` системной таблицы [system.parts](../operations/system_tables.md#system_tables-parts). Например, `ALTER TABLE visits DETACH PARTITION 201901`. - Произвольное выражение из столбцов исходной таблицы. Также поддерживаются константы и константные выражения. Например, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))`. - Строковый идентификатор партиции. Идентификатор партиции используется для именования кусков партиции на файловой системе и в ZooKeeper. В запросах `ALTER` идентификатор партиции нужно указывать в секции `PARTITION ID`, в одинарных кавычках. Например, `ALTER TABLE visits DETACH PARTITION ID '201901'`. -- Для запросов [ATTACH PART](#alter_attach-partition): чтобы задать имя куска партиции, используйте значение из столбца `name` системной таблицы `system.parts`. Например, `ALTER TABLE visits ATTACH PART 201901_1_1_0`. +- Для запросов [ATTACH PART](#alter_attach-partition) и [DROP DETACHED PART](#alter_drop-detached): чтобы задать имя куска партиции, используйте строковой литерал со значением из столбца `name` системной таблицы [system.detached_parts](../operations/system_tables.md#system_tables-detached_parts). Например, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. Использование кавычек в имени партиций зависит от типа данных столбца, по которому задано партиционирование. Например, для столбца с типом `String` имя партиции необходимо указывать в кавычках (одинарных). Для типов `Date` и `Int*` кавычки указывать не нужно. From a03fcd9f121ac5ad655a4e714175266213d6666f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 8 Aug 2019 22:29:56 +0300 Subject: [PATCH 146/509] Make settings values atomic to avoid race conditions --- dbms/programs/copier/ClusterCopier.cpp | 2 +- dbms/src/Core/SettingsCommon.cpp | 26 +++++--- dbms/src/Core/SettingsCommon.h | 63 +++++++++++++++---- .../LogicalExpressionsOptimizer.cpp | 4 +- .../LogicalExpressionsOptimizer.h | 2 +- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- .../Storages/MergeTree/MergeTreeSettings.cpp | 2 +- 8 files changed, 75 insertions(+), 30 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 43158dedd71..0c383ace576 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -708,7 +708,7 @@ void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration & c auto set_default_value = [] (auto && setting, auto && default_value) { - setting = setting.changed ? setting.value : default_value; + setting = setting.changed ? setting.getValue() : default_value; }; /// Override important settings diff --git a/dbms/src/Core/SettingsCommon.cpp b/dbms/src/Core/SettingsCommon.cpp index 988a4b2d736..639e1dea3ee 100644 --- a/dbms/src/Core/SettingsCommon.cpp +++ b/dbms/src/Core/SettingsCommon.cpp @@ -34,19 +34,19 @@ namespace ErrorCodes template String SettingNumber::toString() const { - return DB::toString(value); + return DB::toString(value.load(std::memory_order_relaxed)); } template Field SettingNumber::toField() const { - return value; + return value.load(std::memory_order_relaxed); } template void SettingNumber::set(Type x) { - value = x; + value.store(x, std::memory_order_relaxed); changed = true; } @@ -136,17 +136,17 @@ template struct SettingNumber; String SettingMaxThreads::toString() const { /// Instead of the `auto` value, we output the actual value to make it easier to see. - return is_auto ? ("auto(" + DB::toString(value) + ")") : DB::toString(value); + return is_auto ? ("auto(" + DB::toString(getValue()) + ")") : DB::toString(getValue()); } Field SettingMaxThreads::toField() const { - return is_auto ? 0 : value; + return is_auto ? 0 : getValue(); } void SettingMaxThreads::set(UInt64 x) { - value = x ? x : getAutoValue(); + value.store(x ? x : getAutoValue(), std::memory_order_relaxed); is_auto = x == 0; changed = true; } @@ -169,7 +169,7 @@ void SettingMaxThreads::set(const String & x) void SettingMaxThreads::serialize(WriteBuffer & buf) const { - writeVarUInt(is_auto ? 0 : value, buf); + writeVarUInt(is_auto ? 0 : getValue(), buf); } void SettingMaxThreads::deserialize(ReadBuffer & buf) @@ -195,18 +195,21 @@ UInt64 SettingMaxThreads::getAutoValue() const template String SettingTimespan::toString() const { + std::lock_guard lock(m); return DB::toString(value.totalMicroseconds() / microseconds_per_io_unit); } template Field SettingTimespan::toField() const { + std::lock_guard lock(m); return value.totalMicroseconds() / microseconds_per_io_unit; } template void SettingTimespan::set(const Poco::Timespan & x) { + std::lock_guard lock(m); value = x; changed = true; } @@ -235,6 +238,7 @@ void SettingTimespan::set(const String & x) template void SettingTimespan::serialize(WriteBuffer & buf) const { + std::lock_guard lock(m); writeVarUInt(value.totalMicroseconds() / microseconds_per_io_unit, buf); } @@ -252,16 +256,19 @@ template struct SettingTimespan; String SettingString::toString() const { + std::lock_guard lock(m); return value; } Field SettingString::toField() const { + std::lock_guard lock(m); return value; } void SettingString::set(const String & x) { + std::lock_guard lock(m); value = x; changed = true; } @@ -273,6 +280,7 @@ void SettingString::set(const Field & x) void SettingString::serialize(WriteBuffer & buf) const { + std::lock_guard lock(m); writeBinary(value, buf); } @@ -296,7 +304,7 @@ Field SettingChar::toField() const void SettingChar::set(char x) { - value = x; + value.store(x, std::memory_order_relaxed); changed = true; } @@ -351,7 +359,7 @@ void SettingEnum::deserialize(ReadBuffer & buf) { \ using EnumType = ENUM_NAME; \ using UnderlyingType = std::underlying_type::type; \ - switch (static_cast(value)) \ + switch (static_cast(getValue())) \ { \ LIST_OF_NAMES_MACRO(IMPLEMENT_SETTING_ENUM_TO_STRING_HELPER_) \ } \ diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index d4607e70904..46bf5bca5ce 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -32,13 +33,23 @@ namespace ErrorCodes template struct SettingNumber { - Type value; +public: + /// The value is atomic, because we want to avoid race conditions on value precisely. + /// It doesn't gurantee atomicy on whole structure. It just helps to avoid the most common + /// case: when we change setting from one thread and use in another (for example in MergeTreeSettings). + /// + std::atomic value; + bool changed = false; SettingNumber(Type x = 0) : value(x) {} + SettingNumber(const SettingNumber & o) : value(o.getValue()), changed(o.changed) { } + + operator Type() const { return getValue(); } + Type getValue() const { return value.load(std::memory_order_relaxed); } - operator Type() const { return value; } SettingNumber & operator= (Type x) { set(x); return *this; } + SettingNumber & operator= (SettingNumber o) { set(o.getValue()); return *this; } /// Serialize to a test string. String toString() const; @@ -73,14 +84,23 @@ using SettingBool = SettingNumber; */ struct SettingMaxThreads { - UInt64 value; + std::atomic value; bool is_auto; bool changed = false; SettingMaxThreads(UInt64 x = 0) : value(x ? x : getAutoValue()), is_auto(x == 0) {} + SettingMaxThreads(const SettingMaxThreads & o) + : is_auto(o.is_auto) + , changed(o.changed) + { + value.store(o.value, std::memory_order_relaxed); + } + + operator UInt64() const { return getValue(); } + UInt64 getValue() const { return value.load(std::memory_order_relaxed); } - operator UInt64() const { return value; } SettingMaxThreads & operator= (UInt64 x) { set(x); return *this; } + SettingMaxThreads & operator= (const SettingMaxThreads & o) { set(o.getValue()); return *this; } String toString() const; Field toField() const; @@ -102,16 +122,20 @@ enum class SettingTimespanIO { MILLISECOND, SECOND }; template struct SettingTimespan { + mutable std::mutex m; Poco::Timespan value; bool changed = false; SettingTimespan(UInt64 x = 0) : value(x * microseconds_per_io_unit) {} + SettingTimespan(const SettingTimespan & o) : value(o.value), changed(o.changed) {} - operator Poco::Timespan() const { return value; } + operator Poco::Timespan() const { return getValue(); } + Poco::Timespan getValue() const { std::lock_guard guard(m); return value; } SettingTimespan & operator= (const Poco::Timespan & x) { set(x); return *this; } + SettingTimespan & operator= (const SettingTimespan & o) { set(o.value); return *this; } - Poco::Timespan::TimeDiff totalSeconds() const { return value.totalSeconds(); } - Poco::Timespan::TimeDiff totalMilliseconds() const { return value.totalMilliseconds(); } + Poco::Timespan::TimeDiff totalSeconds() const { return getValue().totalSeconds(); } + Poco::Timespan::TimeDiff totalMilliseconds() const { return getValue().totalMilliseconds(); } String toString() const; Field toField() const; @@ -134,13 +158,18 @@ using SettingMilliseconds = SettingTimespan; struct SettingString { + mutable std::mutex m; String value; bool changed = false; SettingString(const String & x = String{}) : value(x) {} + SettingString(const SettingString & o) : value(o.value), changed(o.changed) {} + + operator String() const { return getValue(); } + String getValue() const { std::lock_guard guard(m); return value; } - operator String() const { return value; } SettingString & operator= (const String & x) { set(x); return *this; } + SettingString & operator= (const SettingString & o) { set(o.value); return *this; } String toString() const; Field toField() const; @@ -156,13 +185,15 @@ struct SettingString struct SettingChar { public: - char value; + std::atomic value; bool changed = false; SettingChar(char x = '\0') : value(x) {} + SettingChar(const SettingChar & o) : value(o.getValue()), changed(o.changed) { } - operator char() const { return value; } + operator char() const { return getValue(); } SettingChar & operator= (char x) { set(x); return *this; } + SettingChar & operator= (const SettingChar & o) { set(o.getValue()); return *this; } String toString() const; Field toField() const; @@ -173,6 +204,8 @@ public: void serialize(WriteBuffer & buf) const; void deserialize(ReadBuffer & buf); + + char getValue() const { return value.load(std::memory_order_relaxed); } }; @@ -180,23 +213,27 @@ public: template struct SettingEnum { - EnumType value; + std::atomic value; bool changed = false; SettingEnum(EnumType x) : value(x) {} + SettingEnum(const SettingEnum & o) : value(o.getValue()), changed(o.changed) { } - operator EnumType() const { return value; } + operator EnumType() const { return getValue(); } SettingEnum & operator= (EnumType x) { set(x); return *this; } + SettingEnum & operator= (SettingEnum o) { set(o.getValue()); return *this; } String toString() const; Field toField() const { return toString(); } - void set(EnumType x) { value = x; changed = true; } + void set(EnumType x) { value.store(x, std::memory_order_relaxed); changed = true; } void set(const Field & x) { set(safeGet(x)); } void set(const String & x); void serialize(WriteBuffer & buf) const; void deserialize(ReadBuffer & buf); + + EnumType getValue() const { return value.load(std::memory_order_relaxed); } }; diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp index 1e00dd499ba..ca8fbb1c8f4 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -30,8 +30,8 @@ bool LogicalExpressionsOptimizer::OrWithExpression::operator<(const OrWithExpres return std::tie(this->or_function, this->expression) < std::tie(rhs.or_function, rhs.expression); } -LogicalExpressionsOptimizer::LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, ExtractedSettings && settings_) - : select_query(select_query_), settings(settings_) +LogicalExpressionsOptimizer::LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, UInt64 optimize_min_equality_disjunction_chain_length) + : select_query(select_query_), settings(optimize_min_equality_disjunction_chain_length) { } diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.h b/dbms/src/Interpreters/LogicalExpressionsOptimizer.h index 09c3931ce1d..fa5289b3f5f 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.h +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.h @@ -36,7 +36,7 @@ class LogicalExpressionsOptimizer final public: /// Constructor. Accepts the root of the query DAG. - LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, ExtractedSettings && settings_); + LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, UInt64 optimize_min_equality_disjunction_chain_length); /** Replace all rather long homogeneous OR-chains expr = x1 OR ... OR expr = xN * on the expressions `expr` IN (x1, ..., xN). diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 48f40d17cb3..464ef149e35 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -596,7 +596,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( InJoinSubqueriesPreprocessor(context).visit(query); /// Optimizes logical expressions. - LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform(); + LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length).perform(); } /// Creates a dictionary `aliases`: alias -> ASTPtr diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ccbcd04857d..513a713651b 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -400,8 +400,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( if (relative_sample_size == RelativeSize(0)) relative_sample_size = 1; - relative_sample_size /= settings.parallel_replicas_count.value; - relative_sample_offset += relative_sample_size * RelativeSize(settings.parallel_replica_offset.value); + relative_sample_size /= settings.parallel_replicas_count.getValue(); + relative_sample_offset += relative_sample_size * RelativeSize(settings.parallel_replica_offset.getValue()); } if (relative_sample_offset >= RelativeSize(1)) diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp index 9eee33554ab..425a0851d67 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -69,7 +69,7 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) if (std::find_if(changes.begin(), changes.end(), \ [](const SettingChange & c) { return c.name == #NAME; }) \ == changes.end()) \ - changes.push_back(SettingChange{#NAME, NAME.value}); + changes.push_back(SettingChange{#NAME, NAME.getValue()}); APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(ADD_IF_ABSENT) #undef ADD_IF_ABSENT From 50cabe4ab0b5cfc492a88bed1ddfd8ebef1dd723 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 8 Aug 2019 22:30:32 +0300 Subject: [PATCH 147/509] Comment --- dbms/src/Core/SettingsCommon.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index 46bf5bca5ce..0939ccac7ca 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -37,7 +37,6 @@ public: /// The value is atomic, because we want to avoid race conditions on value precisely. /// It doesn't gurantee atomicy on whole structure. It just helps to avoid the most common /// case: when we change setting from one thread and use in another (for example in MergeTreeSettings). - /// std::atomic value; bool changed = false; From b2cbc1a747b520ce1157f03ffaad6b60c2079aa3 Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Fri, 9 Aug 2019 13:46:57 +0300 Subject: [PATCH 148/509] Fixed some const's and added --confidence option. --- dbms/programs/benchmark/Benchmark.cpp | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index e97a499101b..1e39dc6889d 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -55,11 +55,12 @@ public: bool cumulative_, bool secure_, const String & default_database_, const String & user_, const String & password_, const String & stage, bool randomize_, size_t max_iterations_, double max_time_, - const String & json_path_, const Settings & settings_) + const String & json_path_, size_t confidence_, const Settings & settings_) : concurrency(concurrency_), delay(delay_), queue(concurrency), randomize(randomize_), cumulative(cumulative_), max_iterations(max_iterations_), max_time(max_time_), - json_path(json_path_), settings(settings_), global_context(Context::createGlobal()), pool(concurrency) + confidence(confidence_), json_path(json_path_), settings(settings_), + global_context(Context::createGlobal()), pool(concurrency) { const auto secure = secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable; size_t connections_cnt = std::max(ports_.size(), hosts_.size()); @@ -139,6 +140,7 @@ private: bool cumulative; size_t max_iterations; double max_time; + size_t confidence; String json_path; Settings settings; Context global_context; @@ -199,20 +201,20 @@ private: squares_sum += seconds * seconds; } - double avg() + double avg() const { return sum / cnt; } - double var() + double var() const { return (squares_sum - (sum * sum / cnt)) / static_cast(cnt - 1); } }; - std::vector confidence_level = { 80, 90, 95, 98, 99, 99.5 }; + const std::vector confidence_level = { 80, 90, 95, 98, 99, 99.5 }; - std::vector> students_table = { + const std::vector> students_table = { /* inf */ { 1.282, 1.645, 1.960, 2.326, 2.576, 3.090 }, /* 1. */ { 3.078, 6.314, 12.706, 31.821, 63.657, 318.313 }, /* 2. */ { 1.886, 2.920, 4.303, 6.965, 9.925, 22.327 }, @@ -318,7 +320,7 @@ private: std::vector data; - bool report(UInt8 confidence_index) + bool report(size_t confidence_index) { if (data.size() != 2) /// Works for two connections only return true; @@ -427,8 +429,7 @@ private: { printNumberOfQueriesExecuted(queries_executed); cumulative ? report(comparison_info_total) : report(comparison_info_per_interval); - - comparison_relative.report(5); + comparison_relative.report(confidence); delay_watch.restart(); } @@ -476,7 +477,7 @@ private: printNumberOfQueriesExecuted(queries_executed); report(comparison_info_total); - comparison_relative.report(5); + comparison_relative.report(confidence); } @@ -693,6 +694,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("password", value()->default_value(""), "") ("database", value()->default_value("default"), "") ("stacktrace", "print stack traces of exceptions") + ("confidence", value()->default_value(5), "set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default)") ; Settings settings; @@ -728,6 +730,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) options["iterations"].as(), options["timelimit"].as(), options["json"].as(), + options["confidence"].as(), settings); return benchmark.run(); } From 4c85e9f4814e117e155e234ce64978b804fdff31 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 9 Aug 2019 14:17:33 +0300 Subject: [PATCH 149/509] atomic drop table --- dbms/src/Databases/DatabaseOrdinary.cpp | 27 ++++++++++++- .../src/Interpreters/InterpreterDropQuery.cpp | 21 ++++++++-- .../test_atomic_drop_table/__init__.py | 0 .../config.d/zookeeper_session_timeout.xml | 6 +++ .../configs/remote_servers.xml | 14 +++++++ .../test_atomic_drop_table/test.py | 39 +++++++++++++++++++ 6 files changed, 102 insertions(+), 5 deletions(-) create mode 100644 dbms/tests/integration/test_atomic_drop_table/__init__.py create mode 100644 dbms/tests/integration/test_atomic_drop_table/configs/config.d/zookeeper_session_timeout.xml create mode 100644 dbms/tests/integration/test_atomic_drop_table/configs/remote_servers.xml create mode 100644 dbms/tests/integration/test_atomic_drop_table/test.py diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 25b3eb652b5..0bf703887ef 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -135,7 +135,25 @@ void DatabaseOrdinary::loadTables( if (endsWith(dir_it.name(), ".sql.bak")) continue; - /// There are files .sql.tmp - delete. + // There are files that we tried to delete previously + const std::string tmp_drop_ext = ".sql.tmp_drop"; + if (endsWith(dir_it.name(), ".sql.tmp_drop")) + { + const std::string table_name = dir_it.name().substr(0, dir_it.name().size() - tmp_drop_ext.size()); + if (Poco::File(data_path + '/' + table_name).exists()) + { + Poco::File(dir_it->path()).renameTo(table_name + ".sql"); + LOG_WARNING(log, "Table was not dropped previously"); + } + else + { + LOG_INFO(log, "Removing file " << dir_it->path()); + Poco::File(dir_it->path()).remove(); + } + continue; + } + + /// There are files .sql.tmp and .sql.tmp_drop - delete if (endsWith(dir_it.name(), ".sql.tmp")) { LOG_INFO(log, "Removing file " << dir_it->path()); @@ -302,6 +320,12 @@ void DatabaseOrdinary::removeTable( } catch (...) { + try + { + Poco::File(table_metadata_path + ".tmp_drop").remove(); + return; + } + catch (...) {} attachTable(table_name, res); throw; } @@ -363,7 +387,6 @@ void DatabaseOrdinary::renameTable( throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); StoragePtr table = tryGetTable(context, table_name); - if (!table) throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 91213b6100e..13911d930e1 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -90,11 +90,26 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t /// If table was already dropped by anyone, an exception will be thrown auto table_lock = database_and_table.second->lockExclusively(context.getCurrentQueryId()); - /// Delete table metadata and table itself from memory + const auto prev_metadata_name = database_and_table.first->getMetadataPath() + database_and_table.second->getTableName() + ".sql"; + const auto drop_metadata_name = database_and_table.first->getMetadataPath() + database_and_table.second->getTableName() + ".sql.tmp_drop"; + + try + { + Poco::File(prev_metadata_name).renameTo(drop_metadata_name); + std::cout << "RENAMED" << std::endl; + /// Delete table data + database_and_table.second->drop(); + } + catch (...) + { + Poco::File(drop_metadata_name).renameTo(prev_metadata_name); + std::cout << "RENAMED BACK" << std::endl; + throw; + } + + /// Delete table metadata and table itself from memory database_and_table.first->removeTable(context, database_and_table.second->getTableName()); - /// Delete table data - database_and_table.second->drop(); database_and_table.second->is_dropped = true; String database_data_path = database_and_table.first->getDataPath(); diff --git a/dbms/tests/integration/test_atomic_drop_table/__init__.py b/dbms/tests/integration/test_atomic_drop_table/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_atomic_drop_table/configs/config.d/zookeeper_session_timeout.xml b/dbms/tests/integration/test_atomic_drop_table/configs/config.d/zookeeper_session_timeout.xml new file mode 100644 index 00000000000..071725b5391 --- /dev/null +++ b/dbms/tests/integration/test_atomic_drop_table/configs/config.d/zookeeper_session_timeout.xml @@ -0,0 +1,6 @@ + + + + 3000 + + diff --git a/dbms/tests/integration/test_atomic_drop_table/configs/remote_servers.xml b/dbms/tests/integration/test_atomic_drop_table/configs/remote_servers.xml new file mode 100644 index 00000000000..538aa72d386 --- /dev/null +++ b/dbms/tests/integration/test_atomic_drop_table/configs/remote_servers.xml @@ -0,0 +1,14 @@ + + + + + true + + shard_0 + node1 + 9000 + + + + + diff --git a/dbms/tests/integration/test_atomic_drop_table/test.py b/dbms/tests/integration/test_atomic_drop_table/test.py new file mode 100644 index 00000000000..4098cce446b --- /dev/null +++ b/dbms/tests/integration/test_atomic_drop_table/test.py @@ -0,0 +1,39 @@ +import time +import pytest + +from helpers.network import PartitionManager +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + node1.query("CREATE DATABASE zktest;") + node1.query( + ''' + CREATE TABLE zktest.atomic_drop_table (n UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/zktest/tables/atomic_drop_table', 'node1') + PARTITION BY n ORDER BY n + ''' + ) + yield cluster + finally: + cluster.shutdown() + +def test_atomic_delete_with_stopped_zookeeper(start_cluster): + + node1.query("select * from zktest.atomic_drop_table") + node1.query("insert into zktest.atomic_drop_table values (8192)") + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node1) + error = node1.query_and_get_error("DROP TABLE zktest.atomic_drop_table") + assert error != "" + + time.sleep(5) + assert '8192' in node1.query("select * from zktest.atomic_drop_table") From 596a3fe1a88b98c711deb898c28642403c7454af Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 9 Aug 2019 15:54:47 +0300 Subject: [PATCH 150/509] better --- dbms/src/Interpreters/InterpreterDropQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 13911d930e1..965aa2cc997 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -96,7 +96,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t try { - Poco::File(prev_metadata_name).renameTo(drop_metadata_name); + Poco::File(prev_metadata_name).renameTo(drop_metadata_name);git std::cout << "RENAMED" << std::endl; /// Delete table data database_and_table.second->drop(); From 636793fd890a59a445dc54d4678063abbfe9f499 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 9 Aug 2019 15:56:19 +0300 Subject: [PATCH 151/509] better --- dbms/src/Interpreters/InterpreterDropQuery.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 965aa2cc997..7a26bc3c740 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -96,15 +96,13 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t try { - Poco::File(prev_metadata_name).renameTo(drop_metadata_name);git - std::cout << "RENAMED" << std::endl; + Poco::File(prev_metadata_name).renameTo(drop_metadata_name); /// Delete table data database_and_table.second->drop(); } catch (...) { Poco::File(drop_metadata_name).renameTo(prev_metadata_name); - std::cout << "RENAMED BACK" << std::endl; throw; } From e3579f215f7ef75b7ab8fd53a4b2546967608129 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 9 Aug 2019 15:58:46 +0300 Subject: [PATCH 152/509] comment to test --- dbms/tests/integration/test_atomic_drop_table/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/dbms/tests/integration/test_atomic_drop_table/test.py b/dbms/tests/integration/test_atomic_drop_table/test.py index 4098cce446b..7d845baeec6 100644 --- a/dbms/tests/integration/test_atomic_drop_table/test.py +++ b/dbms/tests/integration/test_atomic_drop_table/test.py @@ -26,13 +26,11 @@ def start_cluster(): cluster.shutdown() def test_atomic_delete_with_stopped_zookeeper(start_cluster): - - node1.query("select * from zktest.atomic_drop_table") node1.query("insert into zktest.atomic_drop_table values (8192)") with PartitionManager() as pm: pm.drop_instance_zk_connections(node1) - error = node1.query_and_get_error("DROP TABLE zktest.atomic_drop_table") + error = node1.query_and_get_error("DROP TABLE zktest.atomic_drop_table") #Table won't drop assert error != "" time.sleep(5) From 9d2c9c8c2a9076880dcaa76748207a41b58433aa Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Fri, 9 Aug 2019 16:00:47 +0300 Subject: [PATCH 153/509] Minor style and logic fixes --- dbms/programs/benchmark/Benchmark.cpp | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 1e39dc6889d..e172ebc710a 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -216,9 +216,9 @@ private: const std::vector> students_table = { /* inf */ { 1.282, 1.645, 1.960, 2.326, 2.576, 3.090 }, - /* 1. */ { 3.078, 6.314, 12.706, 31.821, 63.657, 318.313 }, - /* 2. */ { 1.886, 2.920, 4.303, 6.965, 9.925, 22.327 }, - /* 3. */ { 1.638, 2.353, 3.182, 4.541, 5.841, 10.215 }, + /* 1. */ { 3.078, 6.314, 12.706, 31.821, 63.657, 318.313}, + /* 2. */ { 1.886, 2.920, 4.303, 6.965, 9.925, 22.327 }, + /* 3. */ { 1.638, 2.353, 3.182, 4.541, 5.841, 10.215 }, /* 4. */ { 1.533, 2.132, 2.776, 3.747, 4.604, 7.173 }, /* 5. */ { 1.476, 2.015, 2.571, 3.365, 4.032, 5.893 }, /* 6. */ { 1.440, 1.943, 2.447, 3.143, 3.707, 5.208 }, @@ -353,7 +353,6 @@ private: return true; } } - }; RelativeAnalysis comparison_relative; @@ -552,9 +551,10 @@ private: std::lock_guard lock(mutex); std::cerr << "\n"; - size_t info_counter = 1; - for (auto & info : infos) + for (size_t i = 1; i <= infos.size(); ++i) { + auto & info = infos[i - 1]; + /// Avoid zeros, nans or exceptions if (0 == info->queries) return; @@ -562,7 +562,7 @@ private: double seconds = info->work_time / concurrency; std::cerr - << "connection " << info_counter++ << ", " + << "connection " << i << ", " << "queries " << info->queries << ", " << "QPS: " << (info->queries / seconds) << ", " << "RPS: " << (info->read_rows / seconds) << ", " @@ -571,6 +571,7 @@ private: << "result MiB/s: " << (info->result_bytes / seconds / 1048576) << "." << "\n"; } + std::cerr << "\n\t\t"; for (size_t i = 1; i <= infos.size(); ++i) @@ -622,7 +623,7 @@ private: for (size_t i = 1; i <= infos.size(); ++i) { - auto info = infos[i - 1]; + auto & info = infos[i - 1]; json_out << double_quote << "connection_" + toString(i) << ": {\n"; json_out << double_quote << "statistics" << ": {\n"; From 2803fcc2bab351348672b20c95c380fa5703cd9b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 9 Aug 2019 16:02:01 +0300 Subject: [PATCH 154/509] Make atomic settings --- CMakeLists.txt | 2 +- dbms/programs/client/Client.cpp | 2 +- dbms/programs/copier/ClusterCopier.cpp | 2 +- dbms/programs/server/TCPHandler.cpp | 6 +- dbms/src/Core/SettingsCommon.cpp | 135 ++++++++++++---- dbms/src/Core/SettingsCommon.h | 151 +++++++++++------- .../ClusterProxy/executeQuery.cpp | 6 +- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 2 +- dbms/src/Interpreters/ThreadStatusExt.cpp | 2 +- dbms/src/Storages/Kafka/StorageKafka.cpp | 18 +-- .../ReplicatedMergeTreeCleanupThread.cpp | 8 +- dbms/src/Storages/StorageDistributed.cpp | 2 +- dbms/src/Storages/StorageJoin.cpp | 4 +- 13 files changed, 230 insertions(+), 110 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 5e03af27cfa..c0fbdd51f0d 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -350,7 +350,7 @@ if (OS_LINUX AND NOT UNBUNDLED AND (GLIBC_COMPATIBILITY OR USE_INTERNAL_UNWIND_L endif () # Add Libc. GLIBC is actually a collection of interdependent libraries. - set (DEFAULT_LIBS "${DEFAULT_LIBS} -lrt -ldl -lpthread -lm -lc") + set (DEFAULT_LIBS "${DEFAULT_LIBS} -lrt -ldl -lpthread -lm -lc -latomic") # Note: we'd rather use Musl libc library, but it's little bit more difficult to use. diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 091a1ac063f..509e5393c34 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -323,7 +323,7 @@ private: insert_format = "Values"; /// Setting value from cmd arg overrides one from config - if (context.getSettingsRef().max_insert_block_size.changed) + if (context.getSettingsRef().max_insert_block_size.isChanged()) insert_format_max_block_size = context.getSettingsRef().max_insert_block_size; else insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size); diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 0c383ace576..a63cd54a003 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -708,7 +708,7 @@ void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration & c auto set_default_value = [] (auto && setting, auto && default_value) { - setting = setting.changed ? setting.getValue() : default_value; + setting = setting.isChanged() ? setting.getValue() : default_value; }; /// Override important settings diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 5091258acaf..c20a466e5b5 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -182,11 +182,11 @@ void TCPHandler::runImpl() /// Should we send internal logs to client? const auto client_logs_level = query_context->getSettingsRef().send_logs_level; if (client_revision >= DBMS_MIN_REVISION_WITH_SERVER_LOGS - && client_logs_level.value != LogsLevel::none) + && client_logs_level != LogsLevel::none) { state.logs_queue = std::make_shared(); state.logs_queue->max_priority = Poco::Logger::parseLevel(client_logs_level.toString()); - CurrentThread::attachInternalTextLogsQueue(state.logs_queue, client_logs_level.value); + CurrentThread::attachInternalTextLogsQueue(state.logs_queue, client_logs_level); } query_context->setExternalTablesInitializer([&connection_settings, this] (Context & context) @@ -329,7 +329,7 @@ void TCPHandler::readData(const Settings & connection_settings) const auto receive_timeout = query_context->getSettingsRef().receive_timeout.value; /// Poll interval should not be greater than receive_timeout - const size_t default_poll_interval = connection_settings.poll_interval.value * 1000000; + const size_t default_poll_interval = connection_settings.poll_interval * 1000000; size_t current_poll_interval = static_cast(receive_timeout.totalMicroseconds()); constexpr size_t min_poll_interval = 5000; // 5 ms size_t poll_interval = std::max(min_poll_interval, std::min(default_poll_interval, current_poll_interval)); diff --git a/dbms/src/Core/SettingsCommon.cpp b/dbms/src/Core/SettingsCommon.cpp index 639e1dea3ee..2d3ed4f6e14 100644 --- a/dbms/src/Core/SettingsCommon.cpp +++ b/dbms/src/Core/SettingsCommon.cpp @@ -34,20 +34,19 @@ namespace ErrorCodes template String SettingNumber::toString() const { - return DB::toString(value.load(std::memory_order_relaxed)); + return DB::toString(getValue()); } template Field SettingNumber::toField() const { - return value.load(std::memory_order_relaxed); + return getValue(); } template void SettingNumber::set(Type x) { - value.store(x, std::memory_order_relaxed); - changed = true; + data.store(Data{x, true}, std::memory_order_relaxed); } template @@ -59,6 +58,14 @@ void SettingNumber::set(const Field & x) set(applyVisitor(FieldVisitorConvertToNumber(), x)); } + +template +SettingNumber & SettingNumber::operator= (const SettingNumber & o) +{ + data.store(o.data.load(std::memory_order_relaxed), std::memory_order_relaxed); + return *this; +} + template void SettingNumber::set(const String & x) { @@ -93,9 +100,9 @@ template void SettingNumber::serialize(WriteBuffer & buf) const { if constexpr (std::is_integral_v && std::is_unsigned_v) - writeVarUInt(static_cast(value), buf); + writeVarUInt(static_cast(getValue()), buf); else if constexpr (std::is_integral_v && std::is_signed_v) - writeVarInt(static_cast(value), buf); + writeVarInt(static_cast(getValue()), buf); else { static_assert(std::is_floating_point_v); @@ -133,22 +140,28 @@ template struct SettingNumber; template struct SettingNumber; +SettingMaxThreads & SettingMaxThreads::operator= (const SettingMaxThreads & o) +{ + data.store(o.data.load(std::memory_order_relaxed), std::memory_order_relaxed); + return *this; +} + String SettingMaxThreads::toString() const { + auto d = data.load(std::memory_order_relaxed); /// Instead of the `auto` value, we output the actual value to make it easier to see. - return is_auto ? ("auto(" + DB::toString(getValue()) + ")") : DB::toString(getValue()); + return d.is_auto ? ("auto(" + DB::toString(d.value) + ")") : DB::toString(d.value); } Field SettingMaxThreads::toField() const { - return is_auto ? 0 : getValue(); + auto d = data.load(std::memory_order_relaxed); + return d.is_auto ? 0 : d.value; } void SettingMaxThreads::set(UInt64 x) { - value.store(x ? x : getAutoValue(), std::memory_order_relaxed); - is_auto = x == 0; - changed = true; + data.store({x ? x : getAutoValue(), x == 0, true}); } void SettingMaxThreads::set(const Field & x) @@ -169,7 +182,8 @@ void SettingMaxThreads::set(const String & x) void SettingMaxThreads::serialize(WriteBuffer & buf) const { - writeVarUInt(is_auto ? 0 : getValue(), buf); + auto d = data.load(std::memory_order_relaxed); + writeVarUInt(d.is_auto ? 0 : d.value, buf); } void SettingMaxThreads::deserialize(ReadBuffer & buf) @@ -181,8 +195,7 @@ void SettingMaxThreads::deserialize(ReadBuffer & buf) void SettingMaxThreads::setAuto() { - value = getAutoValue(); - is_auto = true; + data.store({getAutoValue(), true, isChanged()}); } UInt64 SettingMaxThreads::getAutoValue() const @@ -191,25 +204,54 @@ UInt64 SettingMaxThreads::getAutoValue() const return res; } +void SettingMaxThreads::setChanged(bool changed) +{ + auto d = data.load(std::memory_order_relaxed); + data.store({d.value, d.is_auto, changed}); +} + + +template +SettingTimespan & SettingTimespan::operator= (const SettingTimespan & o) +{ + std::shared_lock lock_o(o.mutex); + value = o.value; + changed = o.changed; + return *this; +} + +template +SettingTimespan::SettingTimespan(const SettingTimespan & o) +{ + std::shared_lock lock_o(o.mutex); + value = o.value; + changed = o.changed; +} + + +template +void SettingTimespan::setChanged(bool c) +{ + std::unique_lock lock(mutex); + changed = c; +} template String SettingTimespan::toString() const { - std::lock_guard lock(m); - return DB::toString(value.totalMicroseconds() / microseconds_per_io_unit); + return DB::toString(getValue().totalMicroseconds() / microseconds_per_io_unit); } template Field SettingTimespan::toField() const { - std::lock_guard lock(m); - return value.totalMicroseconds() / microseconds_per_io_unit; + return getValue().totalMicroseconds() / microseconds_per_io_unit; } template void SettingTimespan::set(const Poco::Timespan & x) { - std::lock_guard lock(m); + std::unique_lock lock(mutex); value = x; changed = true; } @@ -238,8 +280,7 @@ void SettingTimespan::set(const String & x) template void SettingTimespan::serialize(WriteBuffer & buf) const { - std::lock_guard lock(m); - writeVarUInt(value.totalMicroseconds() / microseconds_per_io_unit, buf); + writeVarUInt(getValue().totalMicroseconds() / microseconds_per_io_unit, buf); } template @@ -253,26 +294,47 @@ void SettingTimespan::deserialize(ReadBuffer & buf) template struct SettingTimespan; template struct SettingTimespan; +SettingString & SettingString::operator= (const SettingString & o) +{ + std::shared_lock lock_o(o.mutex); + value = o.value; + changed = o.changed; + return *this; +} + +SettingString::SettingString(const SettingString & o) +{ + std::shared_lock lock(o.mutex); + value = o.value; + changed = o.changed; +} + String SettingString::toString() const { - std::lock_guard lock(m); + std::shared_lock lock(mutex); return value; } Field SettingString::toField() const { - std::lock_guard lock(m); + std::shared_lock lock(mutex); return value; } void SettingString::set(const String & x) { - std::lock_guard lock(m); + std::unique_lock lock(mutex); value = x; changed = true; } +void SettingString::setChanged(bool c) +{ + std::unique_lock lock(mutex); + changed = c; +} + void SettingString::set(const Field & x) { set(safeGet(x)); @@ -280,7 +342,6 @@ void SettingString::set(const Field & x) void SettingString::serialize(WriteBuffer & buf) const { - std::lock_guard lock(m); writeBinary(value, buf); } @@ -291,10 +352,15 @@ void SettingString::deserialize(ReadBuffer & buf) set(s); } +SettingChar & SettingChar::operator= (const SettingChar & o) +{ + data.store(o.data.load(std::memory_order_relaxed), std::memory_order_relaxed); + return *this; +} String SettingChar::toString() const { - return String(1, value); + return String(1, getValue()); } Field SettingChar::toField() const @@ -304,8 +370,7 @@ Field SettingChar::toField() const void SettingChar::set(char x) { - value.store(x, std::memory_order_relaxed); - changed = true; + data.store({x, true}); } void SettingChar::set(const String & x) @@ -335,6 +400,19 @@ void SettingChar::deserialize(ReadBuffer & buf) } +template +SettingEnum & SettingEnum::operator= (const SettingEnum & o) +{ + data.store(o.data.load(std::memory_order_relaxed), std::memory_order_relaxed); + return *this; +} + +template +void SettingEnum::set(EnumType x) +{ + data.store({x, true}, std::memory_order_relaxed); +} + template void SettingEnum::serialize(WriteBuffer & buf) const { @@ -350,6 +428,7 @@ void SettingEnum::deserialize(ReadBuffer & buf) } + #define IMPLEMENT_SETTING_ENUM(ENUM_NAME, LIST_OF_NAMES_MACRO, ERROR_CODE_FOR_UNEXPECTED_NAME) \ IMPLEMENT_SETTING_ENUM_WITH_TAG(ENUM_NAME, void, LIST_OF_NAMES_MACRO, ERROR_CODE_FOR_UNEXPECTED_NAME) diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index 0939ccac7ca..83d90c28f53 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB @@ -33,22 +34,25 @@ namespace ErrorCodes template struct SettingNumber { -public: - /// The value is atomic, because we want to avoid race conditions on value precisely. - /// It doesn't gurantee atomicy on whole structure. It just helps to avoid the most common - /// case: when we change setting from one thread and use in another (for example in MergeTreeSettings). - std::atomic value; + struct Data + { + Type value; + bool changed; + }; - bool changed = false; + std::atomic data; - SettingNumber(Type x = 0) : value(x) {} - SettingNumber(const SettingNumber & o) : value(o.getValue()), changed(o.changed) { } + SettingNumber(Type x = 0) : data{{x, false}} {} + SettingNumber(const SettingNumber & o) : data{o.data.load(std::memory_order_relaxed)} {} + + bool isChanged() const { return data.load(std::memory_order_relaxed).changed; } + void setChanged(bool changed) { data.store({getValue(), changed}, std::memory_order_relaxed); } operator Type() const { return getValue(); } - Type getValue() const { return value.load(std::memory_order_relaxed); } + Type getValue() const { return data.load(std::memory_order_relaxed).value; } SettingNumber & operator= (Type x) { set(x); return *this; } - SettingNumber & operator= (SettingNumber o) { set(o.getValue()); return *this; } + SettingNumber & operator= (const SettingNumber & o); /// Serialize to a test string. String toString() const; @@ -83,23 +87,26 @@ using SettingBool = SettingNumber; */ struct SettingMaxThreads { - std::atomic value; - bool is_auto; - bool changed = false; - - SettingMaxThreads(UInt64 x = 0) : value(x ? x : getAutoValue()), is_auto(x == 0) {} - SettingMaxThreads(const SettingMaxThreads & o) - : is_auto(o.is_auto) - , changed(o.changed) + struct Data { - value.store(o.value, std::memory_order_relaxed); - } + UInt64 value; + bool is_auto; + bool changed; + }; + + std::atomic data; + + SettingMaxThreads(UInt64 x = 0) : data{{x ? x : getAutoValue(), x == 0, false}} {} + SettingMaxThreads(const SettingMaxThreads & o) : data{o.data.load(std::memory_order_relaxed)} {} + + bool isChanged() const { return data.load(std::memory_order_relaxed).changed; } + void setChanged(bool changed); operator UInt64() const { return getValue(); } - UInt64 getValue() const { return value.load(std::memory_order_relaxed); } + UInt64 getValue() const { return data.load(std::memory_order_relaxed).value; } SettingMaxThreads & operator= (UInt64 x) { set(x); return *this; } - SettingMaxThreads & operator= (const SettingMaxThreads & o) { set(o.getValue()); return *this; } + SettingMaxThreads & operator= (const SettingMaxThreads & o); String toString() const; Field toField() const; @@ -111,6 +118,7 @@ struct SettingMaxThreads void serialize(WriteBuffer & buf) const; void deserialize(ReadBuffer & buf); + bool isAuto() const { return data.load(std::memory_order_relaxed).is_auto; } void setAuto(); UInt64 getAutoValue() const; }; @@ -121,20 +129,37 @@ enum class SettingTimespanIO { MILLISECOND, SECOND }; template struct SettingTimespan { - mutable std::mutex m; + mutable std::shared_mutex mutex; Poco::Timespan value; bool changed = false; SettingTimespan(UInt64 x = 0) : value(x * microseconds_per_io_unit) {} - SettingTimespan(const SettingTimespan & o) : value(o.value), changed(o.changed) {} + SettingTimespan(const SettingTimespan & o); operator Poco::Timespan() const { return getValue(); } - Poco::Timespan getValue() const { std::lock_guard guard(m); return value; } + Poco::Timespan getValue() const { std::shared_lock lock(mutex); return value; } SettingTimespan & operator= (const Poco::Timespan & x) { set(x); return *this; } - SettingTimespan & operator= (const SettingTimespan & o) { set(o.value); return *this; } + SettingTimespan & operator= (const SettingTimespan & o); - Poco::Timespan::TimeDiff totalSeconds() const { return getValue().totalSeconds(); } - Poco::Timespan::TimeDiff totalMilliseconds() const { return getValue().totalMilliseconds(); } + Poco::Timespan::TimeDiff totalSeconds() const + { + std::shared_lock lock(mutex); + return value.totalSeconds(); + } + + Poco::Timespan::TimeDiff totalMilliseconds() const + { + std::shared_lock lock(mutex); + return value.totalMilliseconds(); + } + + bool isChanged() const + { + std::shared_lock lock(mutex); + return changed; + } + + void setChanged(bool changed); String toString() const; Field toField() const; @@ -157,18 +182,19 @@ using SettingMilliseconds = SettingTimespan; struct SettingString { - mutable std::mutex m; + mutable std::shared_mutex mutex; String value; bool changed = false; SettingString(const String & x = String{}) : value(x) {} - SettingString(const SettingString & o) : value(o.value), changed(o.changed) {} - - operator String() const { return getValue(); } - String getValue() const { std::lock_guard guard(m); return value; } + SettingString(const SettingString & o); + operator String() const { return getValue(); } + String getValue() const { std::shared_lock lock(mutex); return value; } SettingString & operator= (const String & x) { set(x); return *this; } - SettingString & operator= (const SettingString & o) { set(o.value); return *this; } + SettingString & operator= (const SettingString & o); + bool isChanged() const { std::shared_lock lock(mutex); return changed; } + void setChanged(bool changed); String toString() const; Field toField() const; @@ -184,15 +210,25 @@ struct SettingString struct SettingChar { public: - std::atomic value; - bool changed = false; + struct Data + { + char value; + bool changed; + }; - SettingChar(char x = '\0') : value(x) {} - SettingChar(const SettingChar & o) : value(o.getValue()), changed(o.changed) { } + std::atomic data; + + SettingChar(char x = '\0') : data({x, false}) {} + SettingChar(const SettingChar & o) : data{o.data.load(std::memory_order_relaxed)} {} operator char() const { return getValue(); } + char getValue() const { return data.load(std::memory_order_relaxed).value; } + SettingChar & operator= (char x) { set(x); return *this; } - SettingChar & operator= (const SettingChar & o) { set(o.getValue()); return *this; } + SettingChar & operator= (const SettingChar & o); + + bool isChanged() const { return data.load(std::memory_order_relaxed).changed; } + void setChanged(bool changed) { data.store({getValue(), changed}, std::memory_order_relaxed);} String toString() const; Field toField() const; @@ -203,8 +239,6 @@ public: void serialize(WriteBuffer & buf) const; void deserialize(ReadBuffer & buf); - - char getValue() const { return value.load(std::memory_order_relaxed); } }; @@ -212,27 +246,35 @@ public: template struct SettingEnum { - std::atomic value; - bool changed = false; + struct Data + { + EnumType value; + bool changed; + }; - SettingEnum(EnumType x) : value(x) {} - SettingEnum(const SettingEnum & o) : value(o.getValue()), changed(o.changed) { } + std::atomic data; + + SettingEnum(EnumType x) : data({x, false}) {} + SettingEnum(const SettingEnum & o) : data{o.data.load(std::memory_order_relaxed)} {} operator EnumType() const { return getValue(); } + EnumType getValue() const { return data.load(std::memory_order_relaxed).value; } + SettingEnum & operator= (EnumType x) { set(x); return *this; } - SettingEnum & operator= (SettingEnum o) { set(o.getValue()); return *this; } + SettingEnum & operator= (const SettingEnum & o); + + bool isChanged() const { return data.load(std::memory_order_relaxed).changed; } + void setChanged(bool changed) { data.store({getValue(), changed}, std::memory_order_relaxed);} String toString() const; Field toField() const { return toString(); } - void set(EnumType x) { value.store(x, std::memory_order_relaxed); changed = true; } + void set(EnumType x); void set(const Field & x) { set(safeGet(x)); } void set(const String & x); void serialize(WriteBuffer & buf) const; void deserialize(ReadBuffer & buf); - - EnumType getValue() const { return value.load(std::memory_order_relaxed); } }; @@ -344,6 +386,7 @@ private: Derived & castToDerived() { return *static_cast(this); } const Derived & castToDerived() const { return *static_cast(this); } + using IsChangedFunction = bool (*)(const Derived &); using GetStringFunction = String (*)(const Derived &); using GetFieldFunction = Field (*)(const Derived &); using SetStringFunction = void (*)(Derived &, const String &); @@ -354,7 +397,7 @@ private: struct MemberInfo { - size_t offset_of_changed; + IsChangedFunction is_changed; StringRef name; StringRef description; /// Can be updated after first load for config/definition. @@ -369,7 +412,7 @@ private: DeserializeFunction deserialize; CastValueWithoutApplyingFunction cast_value_without_applying; - bool isChanged(const Derived & collection) const { return *reinterpret_cast(reinterpret_cast(&collection) + offset_of_changed); } + bool isChanged(const Derived & collection) const { return is_changed(collection); } }; class MemberInfos @@ -729,8 +772,7 @@ public: #define IMPLEMENT_SETTINGS_COLLECTION_ADD_MUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ - static_assert(std::is_same_v().NAME.changed), bool>); \ - add({offsetof(Derived, NAME.changed), \ + add({[](const Derived & d) { return d.NAME.isChanged(); }, \ StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), true, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \ @@ -738,8 +780,7 @@ public: &Functions::NAME##_castValueWithoutApplying }); #define IMPLEMENT_SETTINGS_COLLECTION_ADD_IMMUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ - static_assert(std::is_same_v().NAME.changed), bool>); \ - add({offsetof(Derived, NAME.changed), \ + add({[](const Derived & d) { return d.NAME.isChanged(); }, \ StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), false, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \ diff --git a/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp b/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp index 1a202e064f1..e4e52eab2b7 100644 --- a/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -26,9 +26,9 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin new_settings.max_memory_usage_for_all_queries = 0; /// Set as unchanged to avoid sending to remote server. - new_settings.max_concurrent_queries_for_user.changed = false; - new_settings.max_memory_usage_for_user.changed = false; - new_settings.max_memory_usage_for_all_queries.changed = false; + new_settings.max_concurrent_queries_for_user.setChanged(false); + new_settings.max_memory_usage_for_user.setChanged(false); + new_settings.max_memory_usage_for_all_queries.setChanged(false); Context new_context(context); new_context.setSettings(new_settings); diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 464ef149e35..465e0fb786f 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -596,7 +596,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( InJoinSubqueriesPreprocessor(context).visit(query); /// Optimizes logical expressions. - LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length).perform(); + LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.getValue()).perform(); } /// Creates a dictionary `aliases`: alias -> ASTPtr diff --git a/dbms/src/Interpreters/ThreadStatusExt.cpp b/dbms/src/Interpreters/ThreadStatusExt.cpp index 8c46a3ba08f..28740417b71 100644 --- a/dbms/src/Interpreters/ThreadStatusExt.cpp +++ b/dbms/src/Interpreters/ThreadStatusExt.cpp @@ -251,7 +251,7 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) { elem.client_info = query_context->getClientInfo(); - if (query_context->getSettingsRef().log_profile_events.value != 0) + if (query_context->getSettingsRef().log_profile_events != 0) { /// NOTE: Here we are in the same thread, so we can make memcpy() elem.profile_counters = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 47f4fc8c610..2d431ed1368 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -199,7 +199,7 @@ BufferPtr StorageKafka::createBuffer() const Settings & settings = global_context.getSettingsRef(); size_t batch_size = max_block_size; if (!batch_size) - batch_size = settings.max_block_size.value; + batch_size = settings.max_block_size; size_t poll_timeout = settings.stream_poll_timeout_ms.totalMilliseconds(); return std::make_shared( @@ -350,7 +350,7 @@ bool StorageKafka::streamToViews() const Settings & settings = global_context.getSettingsRef(); size_t block_size = max_block_size; if (block_size == 0) - block_size = settings.max_block_size.value; + block_size = settings.max_block_size; // Create a stream for each consumer and join them in a union stream InterpreterInsertQuery interpreter{insert, global_context}; @@ -436,7 +436,7 @@ void registerStorageKafka(StorageFactory & factory) #define CHECK_KAFKA_STORAGE_ARGUMENT(ARG_NUM, PAR_NAME) \ /* One of the four required arguments is not specified */ \ if (args_count < ARG_NUM && ARG_NUM <= 4 && \ - !kafka_settings.PAR_NAME.changed) \ + !kafka_settings.PAR_NAME.isChanged()) \ { \ throw Exception( \ "Required parameter '" #PAR_NAME "' " \ @@ -445,7 +445,7 @@ void registerStorageKafka(StorageFactory & factory) } \ /* The same argument is given in two places */ \ if (has_settings && \ - kafka_settings.PAR_NAME.changed && \ + kafka_settings.PAR_NAME.isChanged() && \ args_count >= ARG_NUM) \ { \ throw Exception( \ @@ -469,7 +469,7 @@ void registerStorageKafka(StorageFactory & factory) #undef CHECK_KAFKA_STORAGE_ARGUMENT // Get and check broker list - String brokers = kafka_settings.kafka_broker_list.value; + String brokers = kafka_settings.kafka_broker_list; if (args_count >= 1) { const auto * ast = engine_args[0]->as(); @@ -524,7 +524,7 @@ void registerStorageKafka(StorageFactory & factory) } // Parse row delimiter (optional) - char row_delimiter = kafka_settings.kafka_row_delimiter.value; + char row_delimiter = kafka_settings.kafka_row_delimiter; if (args_count >= 5) { engine_args[4] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[4], args.local_context); @@ -571,7 +571,7 @@ void registerStorageKafka(StorageFactory & factory) } // Parse number of consumers (optional) - UInt64 num_consumers = kafka_settings.kafka_num_consumers.value; + UInt64 num_consumers = kafka_settings.kafka_num_consumers; if (args_count >= 7) { const auto * ast = engine_args[6]->as(); @@ -586,7 +586,7 @@ void registerStorageKafka(StorageFactory & factory) } // Parse max block size (optional) - UInt64 max_block_size = static_cast(kafka_settings.kafka_max_block_size.value); + UInt64 max_block_size = static_cast(kafka_settings.kafka_max_block_size); if (args_count >= 8) { const auto * ast = engine_args[7]->as(); @@ -601,7 +601,7 @@ void registerStorageKafka(StorageFactory & factory) } } - size_t skip_broken = static_cast(kafka_settings.kafka_skip_broken_messages.value); + size_t skip_broken = static_cast(kafka_settings.kafka_skip_broken_messages); if (args_count >= 9) { const auto * ast = engine_args[8]->as(); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 54da38df541..6108704b45a 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -100,8 +100,8 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs() std::sort(entries.begin(), entries.end()); String min_saved_record_log_str = entries[ - entries.size() > storage.settings.max_replicated_logs_to_keep.value - ? entries.size() - storage.settings.max_replicated_logs_to_keep.value + entries.size() > storage.settings.max_replicated_logs_to_keep + ? entries.size() - storage.settings.max_replicated_logs_to_keep : 0]; /// Replicas that were marked is_lost but are active. @@ -203,7 +203,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs() min_saved_log_pointer = std::min(min_saved_log_pointer, min_log_pointer_lost_candidate); /// We will not touch the last `min_replicated_logs_to_keep` records. - entries.erase(entries.end() - std::min(entries.size(), storage.settings.min_replicated_logs_to_keep.value), entries.end()); + entries.erase(entries.end() - std::min(entries.size(), storage.settings.min_replicated_logs_to_keep), entries.end()); /// We will not touch records that are no less than `min_saved_log_pointer`. entries.erase(std::lower_bound(entries.begin(), entries.end(), "log-" + padIndex(min_saved_log_pointer)), entries.end()); @@ -299,7 +299,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks() /// Virtual node, all nodes that are "greater" than this one will be deleted NodeWithStat block_threshold{{}, time_threshold}; - size_t current_deduplication_window = std::min(timed_blocks.size(), storage.settings.replicated_deduplication_window.value); + size_t current_deduplication_window = std::min(timed_blocks.size(), storage.settings.replicated_deduplication_window); auto first_outdated_block_fixed_threshold = timed_blocks.begin() + current_deduplication_window; auto first_outdated_block_time_threshold = std::upper_bound(timed_blocks.begin(), timed_blocks.end(), block_threshold, NodeWithStat::greaterByTime); auto first_outdated_block = std::min(first_outdated_block_fixed_threshold, first_outdated_block_time_threshold); diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 88a7c07377c..62638e1982a 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -329,7 +329,7 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const Context & c const auto & settings = context.getSettingsRef(); /// Ban an attempt to make async insert into the table belonging to DatabaseMemory - if (path.empty() && !owned_cluster && !settings.insert_distributed_sync.value) + if (path.empty() && !owned_cluster && !settings.insert_distributed_sync) { throw Exception("Storage " + getName() + " must has own data directory to enable asynchronous inserts", ErrorCodes::BAD_ARGUMENTS); diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index 3c90917b0f6..8d957a1c9cf 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -164,8 +164,8 @@ void registerStorageJoin(StorageFactory & factory) args.database_name, args.table_name, key_names, - join_use_nulls.value, - SizeLimits{max_rows_in_join.value, max_bytes_in_join.value, join_overflow_mode.value}, + join_use_nulls, + SizeLimits{max_rows_in_join, max_bytes_in_join, join_overflow_mode}, kind, strictness, args.columns, From aec386f60e8f3d281ec5e17dd3bc5c85d1f47c6a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 9 Aug 2019 16:15:23 +0300 Subject: [PATCH 155/509] better comment --- dbms/src/Databases/DatabaseOrdinary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 0bf703887ef..1e7f331c7aa 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -153,7 +153,7 @@ void DatabaseOrdinary::loadTables( continue; } - /// There are files .sql.tmp and .sql.tmp_drop - delete + /// There are files .sql.tmp - delete if (endsWith(dir_it.name(), ".sql.tmp")) { LOG_INFO(log, "Removing file " << dir_it->path()); From a1023bebff727d449af26b60c668e1676a7a793d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Aug 2019 16:37:42 +0300 Subject: [PATCH 156/509] Fix tests. --- dbms/src/Interpreters/Context.cpp | 2 +- dbms/src/Interpreters/Context.h | 4 ++-- .../Interpreters/InterpreterSelectQuery.cpp | 21 +++---------------- .../InterpreterSelectWithUnionQuery.h | 1 - .../src/Interpreters/MutationsInterpreter.cpp | 3 ++- dbms/src/Storages/StorageDistributed.cpp | 8 +++++-- 6 files changed, 14 insertions(+), 25 deletions(-) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index fef7b067624..992593d852c 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -967,7 +967,7 @@ StoragePtr Context::tryRemoveExternalTable(const String & table_name) } -StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) const +StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) { /// Slightly suboptimal. auto hash = table_expression->getTreeHash(); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index a44ae210c64..50b7ab3eba2 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -135,7 +135,7 @@ private: /// Thus, used in HTTP interface. If not specified - then some globally default format is used. TableAndCreateASTs external_tables; /// Temporary tables. StoragePtr view_source; /// Temporary StorageValues used to generate alias columns for materialized views - mutable Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id. + Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id. Context * query_context = nullptr; Context * session_context = nullptr; /// Session context or nullptr. Could be equal to this. Context * global_context = nullptr; /// Global context. Could be equal to this. @@ -245,7 +245,7 @@ public: void addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast = {}); StoragePtr tryRemoveExternalTable(const String & table_name); - StoragePtr executeTableFunction(const ASTPtr & table_expression) const; + StoragePtr executeTableFunction(const ASTPtr & table_expression); void addViewSource(const StoragePtr & storage); StoragePtr getViewSource(); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 9c10c317d28..941b3d44bb7 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -406,6 +406,9 @@ Block InterpreterSelectQuery::getSampleBlockImpl() { FilterInfoPtr filter_info; + /// Need to create sets before analyzeExpressions(). Otherwise some sets for index won't be created. + query_analyzer->makeSetsForIndex(); + auto analysis_result = analyzeExpressions( getSelectQuery(), *query_analyzer, @@ -498,24 +501,6 @@ InterpreterSelectQuery::analyzeExpressions( { chain.finalize(); - /// Check that actions on current step are valid. - /// Now this in needed for mutations to check in mutation is valid before execute it in background. - /// Because some functions only checking correctness of constant arguments during execution, - /// but not in getReturnType method (e.g. compare date with constant string). - if (dry_run) - { - for (auto & step : chain.steps) - { - auto step_required_columns = step.actions->getRequiredColumnsWithTypes(); - - Block sample; - for (auto & col : step_required_columns) - sample.insert({col.type->createColumn(), col.type, col.name}); - - step.actions->execute(sample); - } - } - if (has_prewhere) { const ExpressionActionsChain::Step & step = chain.steps.at(0); diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h index aa5a763feaa..9f2a4a96494 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -50,7 +50,6 @@ private: Context context; std::vector> nested_interpreters; - Blocks nested_headers; Block result_header; diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 69339f66712..27200249e1c 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -397,7 +397,8 @@ BlockInputStreamPtr MutationsInterpreter::addStreamsForLaterStages(BlockInputStr void MutationsInterpreter::validate() { prepare(/* dry_run = */ true); - Block first_stage_header = interpreter_select->getSampleBlock(); + /// Do not use getSampleBlock in order to check the whole pipeline. + Block first_stage_header = interpreter_select->execute().in->getHeader(); BlockInputStreamPtr in = std::make_shared(first_stage_header); addStreamsForLaterStages(in)->getHeader(); } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 2ecae5789dc..68e09c319d8 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -301,8 +301,12 @@ BlockInputStreams StorageDistributed::read( const auto & modified_query_ast = rewriteSelectQuery( query_info.query, remote_database, remote_table, remote_table_function_ptr); - Block header = materializeBlock( - InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage)).getSampleBlock()); + Block header = + InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage)).getSampleBlock(); + + for (auto & col : header) + if (isColumnConst(*col.column)) + col.column = col.type->createColumn(); ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr ? ClusterProxy::SelectStreamFactory( From 78516083acec32e3810bffe5efe0c48543cce46c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Aug 2019 16:42:58 +0300 Subject: [PATCH 157/509] Remove dry_run flag from InterpreterSelectQuery::executeImpl. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 12 ++++++------ dbms/src/Interpreters/InterpreterSelectQuery.h | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 941b3d44bb7..7c1f76f99e9 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -379,7 +379,7 @@ Block InterpreterSelectQuery::getSampleBlock() BlockIO InterpreterSelectQuery::execute() { Pipeline pipeline; - executeImpl(pipeline, input, options.only_analyze); + executeImpl(pipeline, input); executeUnion(pipeline); BlockIO res; @@ -390,14 +390,14 @@ BlockIO InterpreterSelectQuery::execute() BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams() { Pipeline pipeline; - executeImpl(pipeline, input, options.only_analyze); + executeImpl(pipeline, input); return pipeline.streams; } QueryPipeline InterpreterSelectQuery::executeWithProcessors() { QueryPipeline query_pipeline; - executeImpl(query_pipeline, input, options.only_analyze); + executeImpl(query_pipeline, input); return query_pipeline; } @@ -824,7 +824,7 @@ static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, cons template -void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, bool dry_run) +void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input) { /** Streams of data. When the query is executed in parallel, we have several data streams. * If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then @@ -848,7 +848,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS /// Turn off, if the table filter is applied. if (storage && !context.hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) { - if (!dry_run) + if (!options.only_analyze) from_stage = storage->getQueryProcessingStage(context); query_analyzer->makeSetsForIndex(); @@ -887,7 +887,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS sorting_info = optimizeReadInOrder(*merge_tree_data, query, context); } - if (dry_run) + if (options.only_analyze) { if constexpr (pipeline_with_processors) pipeline.init({std::make_shared(source_header)}); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 87a4fdbb31c..d71bd310ee6 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -136,7 +136,7 @@ private: }; template - void executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, bool dry_run); + void executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input); struct AnalysisResult { From 9bd949fff5462029601ac8e87bbd6a0c672761da Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Aug 2019 17:57:15 +0300 Subject: [PATCH 158/509] Convert streams structure to sample block in InterpreterSelectQuery. --- .../Interpreters/InterpreterSelectQuery.cpp | 28 +++++++++++++------ .../src/Interpreters/InterpreterSelectQuery.h | 4 +-- 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 7c1f76f99e9..5a0230e6798 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -380,7 +380,7 @@ BlockIO InterpreterSelectQuery::execute() { Pipeline pipeline; executeImpl(pipeline, input); - executeUnion(pipeline); + executeUnion(pipeline, getSampleBlock()); BlockIO res; res.in = pipeline.firstStream(); @@ -391,6 +391,7 @@ BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams() { Pipeline pipeline; executeImpl(pipeline, input); + unifyStreams(pipeline, getSampleBlock()); return pipeline.streams; } @@ -2221,12 +2222,12 @@ void InterpreterSelectQuery::executeDistinct(QueryPipeline & pipeline, bool befo } -void InterpreterSelectQuery::executeUnion(Pipeline & pipeline) +void InterpreterSelectQuery::executeUnion(Pipeline & pipeline, Block header) { /// If there are still several streams, then we combine them into one if (pipeline.hasMoreThanOneStream()) { - unifyStreams(pipeline); + unifyStreams(pipeline, header); pipeline.firstStream() = std::make_shared(pipeline.streams, pipeline.stream_with_non_joined_data, max_streams); pipeline.stream_with_non_joined_data = nullptr; @@ -2454,18 +2455,27 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pip } -void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline) +void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline, Block header) { - if (pipeline.hasMoreThanOneStream()) + if (pipeline.hasMoreThanOneStream() || header) { /// Unify streams in case they have different headers. - auto first_header = pipeline.streams.at(0)->getHeader(); - for (size_t i = 1; i < pipeline.streams.size(); ++i) + + size_t start = 0; + + if (!header) + { + header = pipeline.streams.at(0)->getHeader(); + start = 1; + } + + for (size_t i = start; i < pipeline.streams.size(); ++i) { auto & stream = pipeline.streams[i]; - auto header = stream->getHeader(); + auto stream_header = stream->getHeader(); auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; - if (!blocksHaveEqualStructure(first_header, header)) + + if (!blocksHaveEqualStructure(header, stream_header)) stream = std::make_shared(context, stream, first_header, mode); } } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index d71bd310ee6..7c7b657f5ce 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -206,7 +206,7 @@ private: void executeOrder(Pipeline & pipeline, SortingInfoPtr sorting_info); void executeMergeSorted(Pipeline & pipeline); void executePreLimit(Pipeline & pipeline); - void executeUnion(Pipeline & pipeline); + void executeUnion(Pipeline & pipeline, Block header = {}); void executeLimitBy(Pipeline & pipeline); void executeLimit(Pipeline & pipeline); void executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression); @@ -231,7 +231,7 @@ private: void executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, std::unordered_map & subqueries_for_sets); /// If pipeline has several streams with different headers, add ConvertingBlockInputStream to first header. - void unifyStreams(Pipeline & pipeline); + void unifyStreams(Pipeline & pipeline, Block header); enum class Modificator { From 2ce2bba496d39d239fb8498b0e9fc67e2ba05402 Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Fri, 9 Aug 2019 18:08:10 +0300 Subject: [PATCH 159/509] Minor fixes --- dbms/programs/benchmark/Benchmark.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index e172ebc710a..72ae8f2e22b 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -80,6 +80,12 @@ public: comparison_info_total.emplace_back(std::make_shared()); } + if (confidence > 5) + { + std::cerr << "Confidence can't be set to " + toString(confidence) + ". It was set to 5 instead." << '\n'; + confidence = 5; + } + global_context.makeGlobalContext(); std::cerr << std::fixed << std::setprecision(3); @@ -334,7 +340,7 @@ private: double s = spool * sqrt(1.0 / data[0].cnt + 1.0 / data[1].cnt); - double d = data[0].avg() - data[1].avg(); + double d = data[1].avg() - data[0].avg(); double e = t * s; @@ -343,7 +349,7 @@ private: { std::cerr << std::setprecision(1) << "Difference at " << confidence_level[confidence_index] << "% confidence\n" << std::setprecision(6); std::cerr << "\t" << d << " +/- " << e << "\n"; - std::cerr << "\t" << d * 100 / data[1].avg() << " +/- " << e * 100 / data[1].avg() << "\n"; + std::cerr << "\t" << d * 100 / data[0].avg() << "% +/- " << e * 100 / data[0].avg() << "%\n"; std::cerr << "\t(Student's t, pooled s = " << spool << ")\n" << std::setprecision(3); return false; } From 9caa69ce8837eaaf75857c84fe855fb610723cd4 Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Fri, 9 Aug 2019 18:18:21 +0300 Subject: [PATCH 160/509] Minor fixes --- dbms/programs/benchmark/Benchmark.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 72ae8f2e22b..43e109810ec 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -559,7 +559,7 @@ private: std::cerr << "\n"; for (size_t i = 1; i <= infos.size(); ++i) { - auto & info = infos[i - 1]; + const auto & info = infos[i - 1]; /// Avoid zeros, nans or exceptions if (0 == info->queries) @@ -587,7 +587,7 @@ private: auto print_percentile = [&](double percent) { std::cerr << percent << "%\t\t"; - for (auto & info : infos) + for (const auto & info : infos) { std::cerr << info->sampler.quantileInterpolated(percent / 100.0) << " sec." << "\t"; } @@ -629,7 +629,7 @@ private: for (size_t i = 1; i <= infos.size(); ++i) { - auto & info = infos[i - 1]; + const auto & info = infos[i - 1]; json_out << double_quote << "connection_" + toString(i) << ": {\n"; json_out << double_quote << "statistics" << ": {\n"; From 23e2d17d9dc18d2b039e7b0752eb162686179e17 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Aug 2019 18:58:07 +0300 Subject: [PATCH 161/509] Added JSONEachRowWithProgressRowOutputFormat. --- ...JSONEachRowWithProgressRowOutputStream.cpp | 47 ------------------- .../JSONEachRowWithProgressRowOutputStream.h | 27 ----------- .../Formats/Impl/JSONEachRowRowOutputFormat.h | 3 +- ...JSONEachRowWithProgressRowOutputFormat.cpp | 43 +++++++++++++++++ .../JSONEachRowWithProgressRowOutputFormat.h | 20 ++++++++ 5 files changed, 65 insertions(+), 75 deletions(-) delete mode 100644 dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.cpp delete mode 100644 dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.h create mode 100644 dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp create mode 100644 dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.h diff --git a/dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.cpp b/dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.cpp deleted file mode 100644 index 4f59e99ff6d..00000000000 --- a/dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.cpp +++ /dev/null @@ -1,47 +0,0 @@ -#include -#include -#include -#include -#include - - -namespace DB -{ - - -void JSONEachRowWithProgressRowOutputStream::writeRowStartDelimiter() -{ - writeCString("{\"row\":{", ostr); -} - - -void JSONEachRowWithProgressRowOutputStream::writeRowEndDelimiter() -{ - writeCString("}}\n", ostr); - field_number = 0; -} - - -void JSONEachRowWithProgressRowOutputStream::onProgress(const Progress & value) -{ - progress.incrementPiecewiseAtomically(value); - writeCString("{\"progress\":", ostr); - progress.writeJSON(ostr); - writeCString("}\n", ostr); -} - - -void registerOutputFormatJSONEachRowWithProgress(FormatFactory & factory) -{ - factory.registerOutputFormat("JSONEachRowWithProgress", []( - WriteBuffer & buf, - const Block & sample, - const Context &, - const FormatSettings & format_settings) - { - return std::make_shared( - std::make_shared(buf, sample, format_settings), sample); - }); -} - -} diff --git a/dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.h b/dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.h deleted file mode 100644 index e8cef8e147b..00000000000 --- a/dbms/src/Formats/JSONEachRowWithProgressRowOutputStream.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** The stream for outputting data in JSON format, by object per line - * that includes progress rows. Does not validate UTF-8. - */ -class JSONEachRowWithProgressRowOutputStream : public JSONEachRowRowOutputStream -{ -public: - using JSONEachRowRowOutputStream::JSONEachRowRowOutputStream; - - void writeRowStartDelimiter() override; - void writeRowEndDelimiter() override; - void onProgress(const Progress & value) override; - -private: - Progress progress; -}; - -} - diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h index a45f193ea39..de93d012760 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -29,8 +29,9 @@ protected: void consumeTotals(Chunk) override {} void consumeExtremes(Chunk) override {} -private: size_t field_number = 0; + +private: Names fields; FormatSettings settings; diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp new file mode 100644 index 00000000000..d3d07f1dcb0 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp @@ -0,0 +1,43 @@ +#include +#include +#include +#include + + +namespace DB +{ + + +void JSONEachRowWithProgressRowOutputFormat::writeRowStartDelimiter() +{ + writeCString("{\"row\":{", out); +} + +void JSONEachRowWithProgressRowOutputFormat::writeRowEndDelimiter() +{ + writeCString("}}\n", out); + field_number = 0; +} + +void JSONEachRowWithProgressRowOutputFormat::onProgress(const Progress & value) +{ + progress.incrementPiecewiseAtomically(value); + writeCString("{\"progress\":", out); + progress.writeJSON(out); + writeCString("}\n", out); +} + + +void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("JSONEachRowWithProgress", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, format_settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.h new file mode 100644 index 00000000000..3062d664199 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.h @@ -0,0 +1,20 @@ +#pragma once +#include + +namespace DB +{ + +class JSONEachRowWithProgressRowOutputFormat : public JSONEachRowRowOutputFormat +{ +public: + using JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat; + + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + void onProgress(const Progress & value) override; + +private: + Progress progress; +}; + +} From f4aef770ffae118ba0a6058797a2ad9231edcd9e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Aug 2019 19:01:29 +0300 Subject: [PATCH 162/509] Added JSONEachRowWithProgressRowOutputFormat. --- dbms/src/Formats/FormatFactory.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 4daa63448f2..515172bc3a9 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -202,8 +202,6 @@ void registerOutputFormatNative(FormatFactory & factory); void registerInputFormatTabSeparated(FormatFactory & factory); void registerInputFormatCSV(FormatFactory & factory); -void registerOutputFormatJSONEachRowWithProgress(FormatFactory & factory); - void registerInputFormatProcessorNative(FormatFactory & factory); void registerOutputFormatProcessorNative(FormatFactory & factory); void registerInputFormatProcessorRowBinary(FormatFactory & factory); @@ -233,6 +231,7 @@ void registerOutputFormatProcessorPrettySpace(FormatFactory & factory); void registerOutputFormatProcessorVertical(FormatFactory & factory); void registerOutputFormatProcessorJSON(FormatFactory & factory); void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); +void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); void registerOutputFormatProcessorXML(FormatFactory & factory); void registerOutputFormatProcessorODBCDriver(FormatFactory & factory); void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); @@ -249,7 +248,7 @@ FormatFactory::FormatFactory() registerInputFormatTabSeparated(*this); registerInputFormatCSV(*this); - registerOutputFormatJSONEachRowWithProgress(*this); + registerOutputFormatProcessorJSONEachRow(*this); registerInputFormatProcessorNative(*this); registerOutputFormatProcessorNative(*this); From ed11432f6885abb4a3d0f5b2e56e899791d3e306 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Aug 2019 19:10:47 +0300 Subject: [PATCH 163/509] Added JSONEachRowWithProgressRowOutputFormat. --- dbms/src/Formats/FormatFactory.cpp | 4 ++-- .../Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 515172bc3a9..6150484e6b4 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -231,7 +231,7 @@ void registerOutputFormatProcessorPrettySpace(FormatFactory & factory); void registerOutputFormatProcessorVertical(FormatFactory & factory); void registerOutputFormatProcessorJSON(FormatFactory & factory); void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); -void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); void registerOutputFormatProcessorXML(FormatFactory & factory); void registerOutputFormatProcessorODBCDriver(FormatFactory & factory); void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); @@ -248,7 +248,7 @@ FormatFactory::FormatFactory() registerInputFormatTabSeparated(*this); registerInputFormatCSV(*this); - registerOutputFormatProcessorJSONEachRow(*this); + registerOutputFormatProcessorJSONEachRowWithProgress(*this); registerInputFormatProcessorNative(*this); registerOutputFormatProcessorNative(*this); diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp index d3d07f1dcb0..6d448b00203 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp @@ -28,7 +28,7 @@ void JSONEachRowWithProgressRowOutputFormat::onProgress(const Progress & value) } -void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory) +void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory) { factory.registerOutputFormatProcessor("JSONEachRowWithProgress", []( WriteBuffer & buf, From 40cc3990372f331619946ffb05284b2dfee7e4bb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Aug 2019 20:15:01 +0300 Subject: [PATCH 164/509] Fix PushingToViewsBlockOutputStream. --- .../PushingToViewsBlockOutputStream.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index cbc63056c1f..2e96a8d3ea2 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -215,17 +215,17 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n block)); InterpreterSelectQuery select(view.query, local_context, SelectQueryOptions()); in = std::make_shared(select.execute().in); + + /// Squashing is needed here because the materialized view query can generate a lot of blocks + /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY + /// and two-level aggregation is triggered). + in = std::make_shared( + in, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); + in = std::make_shared(context, in, view.out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Position); } else in = std::make_shared(block); - /// Squashing is needed here because the materialized view query can generate a lot of blocks - /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY - /// and two-level aggregation is triggered). - in = std::make_shared( - in, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); - in = std::make_shared(context, in, view.out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Position); - in->readPrefix(); while (Block result_block = in->read()) From 67331881356ee5dd1fb158ce728e8ab42016c9f2 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Mon, 5 Aug 2019 07:09:09 +0000 Subject: [PATCH 165/509] Added gcc-9 to docker/builder container --- docker/builder/Dockerfile | 6 ++++-- docker/builder/build.sh | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/docker/builder/Dockerfile b/docker/builder/Dockerfile index 03b4e242d6d..41a558f9eb8 100644 --- a/docker/builder/Dockerfile +++ b/docker/builder/Dockerfile @@ -1,6 +1,8 @@ FROM ubuntu:18.04 RUN apt-get update -y \ + && apt-get install -y software-properties-common \ + && add-apt-repository ppa:ubuntu-toolchain-r/test \ && env DEBIAN_FRONTEND=noninteractive \ apt-get install --yes --no-install-recommends \ bash \ @@ -8,8 +10,8 @@ RUN apt-get update -y \ cmake \ curl \ expect \ - g++ \ - gcc \ + g++-9 \ + gcc-9 \ libclang-6.0-dev \ libicu-dev \ liblld-6.0-dev \ diff --git a/docker/builder/build.sh b/docker/builder/build.sh index 6a5f1359bda..57999a4b483 100755 --- a/docker/builder/build.sh +++ b/docker/builder/build.sh @@ -3,7 +3,7 @@ #ccache -s mkdir -p /server/build_docker cd /server/build_docker -cmake -G Ninja /server -DENABLE_TESTS=1 +cmake -G Ninja /server -DENABLE_TESTS=1 -DCMAKE_C_COMPILER=`which gcc-9` -DCMAKE_CXX_COMPILER=`which g++-9` # Set the number of build jobs to the half of number of virtual CPU cores (rounded up). # By default, ninja use all virtual CPU cores, that leads to very high memory consumption without much improvement in build time. From 0233f32f9b30e78d8f54c28ccdea16736a6293b1 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 11 Aug 2019 12:28:15 +0300 Subject: [PATCH 166/509] Fixed AddresSanitizer error --- .../DataStreams/CheckConstraintsBlockOutputStream.cpp | 11 +++++------ .../DataStreams/CheckConstraintsBlockOutputStream.h | 2 +- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 5adf344cf0b..4b4865f004f 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -11,8 +11,9 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) { for (size_t i = 0; i < expressions.size(); ++i) { + Block res = block; auto constraint_expr = expressions[i]; - auto res_column_uint8 = executeOnBlock(block, constraint_expr); + auto res_column_uint8 = executeOnBlock(res, constraint_expr); if (!memoryIsByte(res_column_uint8->getRawDataBegin<1>(), res_column_uint8->byteSize(), 0x1)) { auto indices_wrong = findAllWrong(res_column_uint8->getRawDataBegin<1>(), res_column_uint8->byteSize()); @@ -48,13 +49,11 @@ void CheckConstraintsBlockOutputStream::writeSuffix() } const ColumnUInt8 *CheckConstraintsBlockOutputStream::executeOnBlock( - const Block & block, + Block & block, const ExpressionActionsPtr & constraint) { - Block res = block; - - constraint->execute(res); - ColumnWithTypeAndName res_column = res.safeGetByPosition(res.columns() - 1); + constraint->execute(block); + ColumnWithTypeAndName res_column = block.safeGetByPosition(block.columns() - 1); return checkAndGetColumn(res_column.column.get()); } diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h index ac2e7e974a1..7ab6832fd28 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h @@ -39,7 +39,7 @@ public: void writeSuffix() override; private: - const ColumnUInt8* executeOnBlock(const Block & block, const ExpressionActionsPtr & constraint); + const ColumnUInt8* executeOnBlock(Block & block, const ExpressionActionsPtr & constraint); std::vector findAllWrong(const void *data, size_t size); String table; From 93a635d18a2119cc018cdbb5d73cfd12da6fbdc6 Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 11 Aug 2019 12:30:01 +0300 Subject: [PATCH 167/509] Added clang-8 to docker builder --- docker/builder/Dockerfile | 4 ++++ docker/builder/Makefile | 2 +- docker/builder/build.sh | 3 ++- 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/docker/builder/Dockerfile b/docker/builder/Dockerfile index 41a558f9eb8..5978dcd08d0 100644 --- a/docker/builder/Dockerfile +++ b/docker/builder/Dockerfile @@ -28,6 +28,10 @@ RUN apt-get update -y \ tzdata \ gperf +RUN apt install -y wget +RUN printf "deb http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main\ndeb-src http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main" >> /etc/apt/sources.list \ + && wget -O - https://apt.llvm.org/llvm-snapshot.gpg.key | apt-key add - && apt update && apt-get install -y clang-8 lldb-8 lld-8 + COPY build.sh / CMD ["/bin/bash", "/build.sh"] diff --git a/docker/builder/Makefile b/docker/builder/Makefile index 779e944b723..a9a7cddf3f2 100644 --- a/docker/builder/Makefile +++ b/docker/builder/Makefile @@ -1,6 +1,6 @@ build: image mkdir -p $(HOME)/.ccache - docker run --network=host --rm --workdir /server --volume $(realpath ../..):/server --mount=type=bind,source=$(HOME)/.ccache,destination=/ccache -e CCACHE_DIR=/ccache -it yandex/clickhouse-builder + docker run --network=host --rm --workdir /server --volume $(realpath ../..):/server --cap-add=SYS_PTRACE --mount=type=bind,source=$(HOME)/.ccache,destination=/ccache -e CCACHE_DIR=/ccache -it yandex/clickhouse-builder pull: docker pull yandex/clickhouse-builder diff --git a/docker/builder/build.sh b/docker/builder/build.sh index 57999a4b483..96468d8d820 100755 --- a/docker/builder/build.sh +++ b/docker/builder/build.sh @@ -3,7 +3,8 @@ #ccache -s mkdir -p /server/build_docker cd /server/build_docker -cmake -G Ninja /server -DENABLE_TESTS=1 -DCMAKE_C_COMPILER=`which gcc-9` -DCMAKE_CXX_COMPILER=`which g++-9` + +cmake -G Ninja /server -DCMAKE_C_COMPILER=`which clang-8` -DCMAKE_CXX_COMPILER=`which clang++-8` -DCMAKE_BUILD_TYPE=Debug # Set the number of build jobs to the half of number of virtual CPU cores (rounded up). # By default, ninja use all virtual CPU cores, that leads to very high memory consumption without much improvement in build time. From 3b9e1f9bf727764e1a4d9787b6e58313f3d381bf Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Sun, 11 Aug 2019 13:39:17 +0300 Subject: [PATCH 168/509] Fixed getIdentifierName call in AlterCommand::parse --- dbms/src/Storages/AlterCommands.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 48690df071a..13141683be8 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -190,7 +190,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ else if (command_ast->type == ASTAlterCommand::DROP_COLUMN) { command.type = AlterCommand::DROP_COLUMN; - command.column_name = *getIdentifierName(command_ast->column); + command.column_name = getIdentifierName(command_ast->column); } return command; From fd4638aa64bb4874c3eea28bec165d60b13a9265 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 12 Aug 2019 13:13:03 +0300 Subject: [PATCH 169/509] drop materialized view fixed --- dbms/src/Interpreters/InterpreterDropQuery.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 7a26bc3c740..c8cfb1605d4 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -91,8 +91,8 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t auto table_lock = database_and_table.second->lockExclusively(context.getCurrentQueryId()); - const auto prev_metadata_name = database_and_table.first->getMetadataPath() + database_and_table.second->getTableName() + ".sql"; - const auto drop_metadata_name = database_and_table.first->getMetadataPath() + database_and_table.second->getTableName() + ".sql.tmp_drop"; + const auto prev_metadata_name = database_and_table.first->getMetadataPath() + escapeForFileName(database_and_table.second->getTableName()) + ".sql"; + const auto drop_metadata_name = database_and_table.first->getMetadataPath() + escapeForFileName(database_and_table.second->getTableName()) + ".sql.tmp_drop"; try { @@ -102,7 +102,10 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t } catch (...) { - Poco::File(drop_metadata_name).renameTo(prev_metadata_name); + try + { + Poco::File(drop_metadata_name).renameTo(prev_metadata_name); + } catch (...) {} throw; } From 16bab882262e5075272b6c60dba49e2887e05933 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Mon, 12 Aug 2019 13:27:28 +0300 Subject: [PATCH 170/509] DOCAPI-7442: 7442 --- docs/en/operations/system_tables.md | 14 +++++++------- docs/en/query_language/functions/hash_functions.md | 2 +- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index 6f1ebebdff3..3abbd98961f 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -271,11 +271,11 @@ Columns: - `rows` (`UInt64`) – The number of rows. -- `bytes_on_disk` (`UInt64`) – The number of bytes when compressed. +- `bytes_on_disk` (`UInt64`) – Total size of all the data part files in bytes. -- `data_compressed_bytes` (`UInt64`) – +- `data_compressed_bytes` (`UInt64`) – Total size of compressed data in the data part. All the auxiliary files (for example, files with marks) are not included. -- `data_uncompressed_bytes` (`UInt64`) – +- `data_uncompressed_bytes` (`UInt64`) – Total size of uncompressed data in the data part. All the auxiliary files (for example, files with marks) are not included. - `marks_bytes` (`UInt64`) – The size of the file with marks. @@ -301,7 +301,7 @@ Columns: - `level` (`UInt32`) – Depth of the merge tree. If a merge was not performed, `level=0`. -- `data_version` (`UInt64`) – +- `data_version` (`UInt64`) – Block number that is used to determine which mutations should be applied to the data part (the mutations with the bigger version than `data_version`). - `primary_key_bytes_in_memory` (`UInt64`) – The amount of memory (in bytes) used by primary key values. @@ -317,11 +317,11 @@ Columns: - `path` (`String`) – Absolute path to the folder with data part files. -- `hash_of_all_files` (`String`) – Hash of compressed files. +- `hash_of_all_files` (`String`) – [sipHash128](../query_language/functions/hash_functions.md#hash_functions-siphash128) of compressed files. -- `hash_of_uncompressed_files` (`String`) – Hash of uncompressed data. +- `hash_of_uncompressed_files` (`String`) – [sipHash128](../query_language/functions/hash_functions.md#hash_functions-siphash128) of uncompressed data. -- `uncompressed_hash_of_compressed_files` (`String`) – Hash of the file with marks +- `uncompressed_hash_of_compressed_files` (`String`) – [sipHash128](../query_language/functions/hash_functions.md#hash_functions-siphash128) of the file with marks. - `bytes` (`UInt64`) – Alias for `bytes_on_disk`. diff --git a/docs/en/query_language/functions/hash_functions.md b/docs/en/query_language/functions/hash_functions.md index 296cca1e712..b384dead609 100644 --- a/docs/en/query_language/functions/hash_functions.md +++ b/docs/en/query_language/functions/hash_functions.md @@ -74,7 +74,7 @@ SELECT sipHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00 └──────────────────────┴────────┘ ``` -## sipHash128 +## sipHash128 {#hash_functions-siphash128} Calculates SipHash from a string. Accepts a String-type argument. Returns FixedString(16). From 360b1217b9c6749f4a667230001c98c7d1db5036 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Mon, 12 Aug 2019 13:31:28 +0300 Subject: [PATCH 171/509] DOCAPI-7442: Fix. --- docs/en/operations/system_tables.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index 3abbd98961f..2fd86b52c35 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -265,7 +265,7 @@ Columns: - `name` (`String`) – Name of the data part. -- `active` (`UInt8`) – Indicates whether the part is active. If a part is active, it is used in a table; otherwise, it will be deleted. Inactive data parts remain after merging. +- `active` (`UInt8`) – Flag that indicates whether the part is active. If a part is active, it is used in a table; otherwise, it will be deleted. Inactive data parts remain after merging. - `marks` (`UInt64`) – The number of marks. To get the approximate number of rows in a data part, multiply `marks` by the index granularity (usually 8192). From 18939a57765028fc65afba1bcb0464f4dd7a8ff7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Aug 2019 13:59:45 +0300 Subject: [PATCH 172/509] Fix build/ --- dbms/src/DataStreams/BlocksBlockInputStream.h | 4 ++-- dbms/src/Storages/ProxyStorage.h | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/DataStreams/BlocksBlockInputStream.h b/dbms/src/DataStreams/BlocksBlockInputStream.h index 58a63e5b85e..27407acbfae 100644 --- a/dbms/src/DataStreams/BlocksBlockInputStream.h +++ b/dbms/src/DataStreams/BlocksBlockInputStream.h @@ -23,8 +23,8 @@ class BlocksBlockInputStream : public IBlockInputStream { public: /// Acquires shared ownership of the blocks vector - BlocksBlockInputStream(const std::shared_ptr & blocks_ptr_, Block header) - : blocks(*blocks_ptr_), it((*blocks_ptr_)->begin()), end((*blocks_ptr_)->end()), header(std::move(header)) {} + BlocksBlockInputStream(const std::shared_ptr & blocks_ptr_, Block header_) + : blocks(*blocks_ptr_), it((*blocks_ptr_)->begin()), end((*blocks_ptr_)->end()), header(std::move(header_)) {} String getName() const override { return "Blocks"; } diff --git a/dbms/src/Storages/ProxyStorage.h b/dbms/src/Storages/ProxyStorage.h index 3e6926eb323..60faa907209 100644 --- a/dbms/src/Storages/ProxyStorage.h +++ b/dbms/src/Storages/ProxyStorage.h @@ -8,8 +8,8 @@ namespace DB class ProxyStorage : public IStorage { public: - ProxyStorage(StoragePtr storage, BlockInputStreams streams, QueryProcessingStage::Enum to_stage) - : storage(std::move(storage)), streams(std::move(streams)), to_stage(to_stage) {} + ProxyStorage(StoragePtr storage_, BlockInputStreams streams_, QueryProcessingStage::Enum to_stage_) + : storage(std::move(storage_)), streams(std::move(streams_)), to_stage(to_stage_) {} public: std::string getName() const override { return "ProxyStorage(" + storage->getName() + ")"; } From 545420963d49c5df63648d51e0cb0be5389e4d43 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Aug 2019 14:04:30 +0300 Subject: [PATCH 173/509] Fix build/ --- dbms/src/Parsers/ParserAlterQuery.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Parsers/ParserAlterQuery.h b/dbms/src/Parsers/ParserAlterQuery.h index 3dd7f89062b..13f5681a9da 100644 --- a/dbms/src/Parsers/ParserAlterQuery.h +++ b/dbms/src/Parsers/ParserAlterQuery.h @@ -48,7 +48,7 @@ public: bool is_live_view; bool is_live_channel; - ParserAlterCommandList(bool is_live_view = false, bool is_live_channel = false) : is_live_view(is_live_view), is_live_channel(is_live_channel) {} + ParserAlterCommandList(bool is_live_view_ = false, bool is_live_channel_ = false) : is_live_view(is_live_view_), is_live_channel(is_live_channel_) {} }; @@ -62,7 +62,7 @@ public: bool is_live_view; bool is_live_channel; - ParserAlterCommand(bool is_live_view = false, bool is_live_channel = false) : is_live_view(is_live_view), is_live_channel(is_live_channel) {} + ParserAlterCommand(bool is_live_view_ = false, bool is_live_channel_ = false) : is_live_view(is_live_view_), is_live_channel(is_live_channel_) {} }; From 43e5cbe35c5b4aef6035faac9a054c7798e72eee Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Aug 2019 14:06:28 +0300 Subject: [PATCH 174/509] Fix build/ --- dbms/src/Storages/StorageLiveView.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index df7f8c896b3..6e42b9780e1 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -103,8 +103,8 @@ StorageLiveView::StorageLiveView( const String & database_name_, Context & local_context, const ASTCreateQuery & query, - const ColumnsDescription & columns) - : IStorage(columns), table_name(table_name_), + const ColumnsDescription & columns_) + : IStorage(columns_), table_name(table_name_), database_name(database_name_), global_context(local_context.getGlobalContext()) { if (!query.select) From fafbd2b1883947ceec2850f0ded5b864419ade19 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 12 Aug 2019 15:56:10 +0300 Subject: [PATCH 175/509] fix drop with memory engine --- dbms/src/Interpreters/InterpreterDropQuery.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index c8cfb1605d4..facf377b885 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -94,18 +94,19 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t const auto prev_metadata_name = database_and_table.first->getMetadataPath() + escapeForFileName(database_and_table.second->getTableName()) + ".sql"; const auto drop_metadata_name = database_and_table.first->getMetadataPath() + escapeForFileName(database_and_table.second->getTableName()) + ".sql.tmp_drop"; + //Try to rename metadata file and delete the data try { - Poco::File(prev_metadata_name).renameTo(drop_metadata_name); + //Memory database has no metadata on disk + if (database_and_table.first->getEngineName() != "Memory") + Poco::File(prev_metadata_name).renameTo(drop_metadata_name); /// Delete table data database_and_table.second->drop(); } catch (...) { - try - { + if (database_and_table.first->getEngineName() != "Memory") Poco::File(drop_metadata_name).renameTo(prev_metadata_name); - } catch (...) {} throw; } From 1b54a52488441b753c99fe6e17d19d458e90313c Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Mon, 12 Aug 2019 17:10:29 +0300 Subject: [PATCH 176/509] Temp --- dbms/src/Functions/neighbour.cpp | 217 +++++++++++++++++++++++++++++ dbms/src/Functions/nextInBlock.cpp | 159 --------------------- 2 files changed, 217 insertions(+), 159 deletions(-) create mode 100644 dbms/src/Functions/neighbour.cpp delete mode 100644 dbms/src/Functions/nextInBlock.cpp diff --git a/dbms/src/Functions/neighbour.cpp b/dbms/src/Functions/neighbour.cpp new file mode 100644 index 00000000000..cf96282725a --- /dev/null +++ b/dbms/src/Functions/neighbour.cpp @@ -0,0 +1,217 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +// Implements function, giving value for column within range of given +// Example: +// | c1 | +// | 10 | +// | 20 | +// SELECT c1, neighbour(c1, 1) as c2: +// | c1 | c2 | +// | 10 | 20 | +// | 20 | 0 | +class FunctionNeighbour : public IFunction +{ +public: + static constexpr auto name = "neighbour"; + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + + FunctionNeighbour(const Context & context_) : context(context_) {} + + /// Get the name of the function. + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 0; } + + bool isVariadic() const override { return true; } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + size_t number_of_arguments = arguments.size(); + + if (number_of_arguments < 2 || number_of_arguments > 3) + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + toString(number_of_arguments) + + ", should be from 2 to 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + // second argument must be a positive integer + if (!isInteger(arguments[1])) + throw Exception( + "Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + + " - should be positive integer", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + // check that default value column has supertype with first argument + if (number_of_arguments == 3) + { + DataTypes types = {arguments[0], arguments[2]}; + try + { + return getLeastSupertype(types); + } + catch (const Exception &) + { + throw Exception( + "Illegal types of arguments (" + types[0]->getName() + ", " + types[1]->getName() + + ")" + " of function " + + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + } + + return arguments[0]; + } + + static void insertDefaults(const MutableColumnPtr & target, size_t row_count, ColumnPtr & default_values_column, size_t offset) + { + if (row_count == 0) { + return; + } + if (default_values_column) + { + if (isColumnConst(*default_values_column)) + { + Field constant_value = (*default_values_column)[0]; + for(size_t row = 0; row < row_count;row++) + { + target->insert(constant_value); + } + } else { + target->insertRangeFrom(*default_values_column, offset, row_count); + } + } else { + for(size_t row = 0; row <= row_count;row++) { + target->insertDefault(); + } + } + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + auto offset_structure = block.getByPosition(arguments[1]); + ColumnPtr & offset_column = offset_structure.column; + + auto is_constant_offset = isColumnConst(*offset_structure.column); + ColumnPtr default_values_column = nullptr; + if (arguments.size() == 3) + { + default_values_column = block.getByPosition(arguments[2]).column; + } + +// Field offset_field = (*block.getByPosition(arguments[1]).column)[0]; +// auto raw_value = safeGet(offset_field); + + ColumnWithTypeAndName &source_column_name_and_type = block.getByPosition(arguments[0]); + DataTypes types = {source_column_name_and_type.type}; + if (default_values_column) + { + types.push_back(block.getByPosition(arguments[2]).type); + } + const DataTypePtr & result_type = getLeastSupertype(types); + auto source_column = source_column_name_and_type.column; + + // adjust source and default values columns to resulting datatype + if (!source_column_name_and_type.type->equals(*result_type)) { + source_column = castColumn(source_column_name_and_type, result_type, context); + } + + if (default_values_column && !block.getByPosition(arguments[2]).type->equals(*result_type)) { + default_values_column = castColumn(block.getByPosition(arguments[2]), result_type, context); + } + + auto column = result_type->createColumn(); + column->reserve(input_rows_count); + + const DataTypePtr desired_type = std::make_shared(); + if (!block.getByPosition(arguments[1]).type->equals(*desired_type)) { + offset_column = castColumn(offset_structure, desired_type, context); + } + + // with constant offset - insertRangeFrom + if (is_constant_offset) + { + Int64 offset_value = offset_column->getInt(0); + + if (offset_value > 0) + { + // insert shifted value + column->insertRangeFrom(*source_column, offset_value, input_rows_count - offset_value); + // insert defaults into the end + insertDefaults(column, input_rows_count - offset_value, default_values_column, offset_value); + } else if(offset_value < 0) { + // insert defaults up to offset_value + insertDefaults(column, input_rows_count - std::abs(offset_value), default_values_column, std::abs(offset_value)); + // insert range, where possible + column->insertRangeFrom(*source_column, 0, input_rows_count - std::abs(offset_value)); + } else { + // populate column with source values + column->insertRangeFrom(*source_column, 0, input_rows_count); + } + } else { + // with dynamic offset - handle row by row + for (size_t row = 0; row < input_rows_count; row++) + { + Int64 offset_value = offset_column->getInt(row); + if (offset_value == 0) { + column->insertFrom(*source_column, row); + } else if (offset_value > 0) { + size_t real_offset = row + offset_value; + if (real_offset > input_rows_count) { + if (default_values_column) { + column->insertFrom(*default_values_column, row); + } else { + column->insertDefault(); + } + } else { + column->insertFrom(*column, real_offset); + } + } else { + // out of range + if ((size_t)std::abs(offset_value) > row) + { + if (default_values_column) { + column->insertFrom(*default_values_column, row); + } else { + column->insertDefault(); + } + } else { + column->insertFrom(*column, row - std::abs(offset_value)); + } + } + } + } + + + block.getByPosition(result).column = std::move(column); + } +private: + const Context & context; +}; + +void registerFunctionNextInBlock(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/nextInBlock.cpp b/dbms/src/Functions/nextInBlock.cpp deleted file mode 100644 index eeb33e28146..00000000000 --- a/dbms/src/Functions/nextInBlock.cpp +++ /dev/null @@ -1,159 +0,0 @@ -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ARGUMENT_OUT_OF_BOUND; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - -// Implements function, giving value for column in next row -// Example: -// | c1 | -// | 10 | -// | 20 | -// SELECT c1, nextInBlock(c1, 1) as c2: -// | c1 | c2 | -// | 10 | 20 | -// | 20 | 0 | -class FunctionNextInBlock : public IFunction -{ -public: - static constexpr auto name = "nextInBlock"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - /// Get the name of the function. - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 0; } - - bool isVariadic() const override { return true; } - - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - size_t number_of_arguments = arguments.size(); - - if (number_of_arguments < 1 || number_of_arguments > 3) - throw Exception( - "Number of arguments for function " + getName() + " doesn't match: passed " + toString(number_of_arguments) - + ", should be from 1 to 3", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - // second argument must be a positive, constant column - if (number_of_arguments == 2 && !isUnsignedInteger(arguments[1])) - throw Exception( - "Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() - + " - should be positive integer", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - // check that default value has supertype with first argument - if (number_of_arguments == 3) - { - DataTypes types = {arguments[0], arguments[2]}; - try - { - return getLeastSupertype(types); - } - catch (const Exception &) - { - throw Exception( - "Illegal types of arguments (" + types[0]->getName() + ", " + types[1]->getName() - + ")" - " of function " - + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - } - - return arguments[0]; - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - size_t offset_value = 1; - - if (arguments.size() > 1) - { - auto offset_column = block.getByPosition(arguments[1]); - if (!isColumnConst(*offset_column.column)) - throw Exception("Second argument of function " + getName() + " should be constant", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - Field offset_field = (*block.getByPosition(arguments[1]).column)[0]; - auto raw_value = safeGet(offset_field); - - if (raw_value == 0) - throw Exception( - "Second argument of function " + getName() + " should be positive integer, " + toString(raw_value) + " given", - ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - offset_value = raw_value; - } - - auto has_column_for_missing = arguments.size() == 3; - - DataTypes types = {block.getByPosition(arguments[0]).type}; - if (has_column_for_missing) - { - types.push_back(block.getByPosition(arguments[2]).type); - } - const DataTypePtr & result_type = getLeastSupertype(types); - - auto column = result_type->createColumn(); - column->reserve(input_rows_count); - - auto source_column = block.getByPosition(arguments[0]).column; - - for (size_t i = offset_value; i < input_rows_count; i++) - { - column->insertFrom(*source_column, i); - } - - if (has_column_for_missing) - { - auto default_values_column = block.getByPosition(arguments[2]).column; - size_t starting_pos = offset_value > input_rows_count ? 0 : input_rows_count - offset_value; - if (isColumnConst(*default_values_column)) - { - Field constant_value = (*default_values_column)[0]; - for (size_t i = starting_pos; i < input_rows_count; i++) - { - column->insert(constant_value); - } - } - else - { - for (size_t i = starting_pos; i < input_rows_count; i++) - { - column->insertFrom(*default_values_column, i); - } - } - } - else - { - for (size_t i = 0; i < std::min(offset_value, input_rows_count); i++) - { - column->insertDefault(); - } - } - - block.getByPosition(result).column = std::move(column); - } -}; - -void registerFunctionNextInBlock(FunctionFactory & factory) -{ - factory.registerFunction(); -} - -} From 31fdc99efc3b668b18e8ea830dfc669f743d275d Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Mon, 12 Aug 2019 18:44:28 +0300 Subject: [PATCH 177/509] In progress --- dbms/src/Functions/neighbour.cpp | 10 ++++---- .../0_stateless/00957_next_in_block.sql | 24 ++++++++----------- 2 files changed, 16 insertions(+), 18 deletions(-) diff --git a/dbms/src/Functions/neighbour.cpp b/dbms/src/Functions/neighbour.cpp index cf96282725a..3eff660c996 100644 --- a/dbms/src/Functions/neighbour.cpp +++ b/dbms/src/Functions/neighbour.cpp @@ -101,7 +101,7 @@ public: target->insertRangeFrom(*default_values_column, offset, row_count); } } else { - for(size_t row = 0; row <= row_count;row++) { + for(size_t row = 0; row < row_count;row++) { target->insertDefault(); } } @@ -156,9 +156,11 @@ public: if (offset_value > 0) { // insert shifted value - column->insertRangeFrom(*source_column, offset_value, input_rows_count - offset_value); - // insert defaults into the end - insertDefaults(column, input_rows_count - offset_value, default_values_column, offset_value); + if ((size_t)std::abs(offset_value) <= input_rows_count) { + column->insertRangeFrom(*source_column, offset_value, input_rows_count - offset_value); + // insert defaults into the end + insertDefaults(column, input_rows_count - offset_value, default_values_column, offset_value); + } } else if(offset_value < 0) { // insert defaults up to offset_value insertDefaults(column, input_rows_count - std::abs(offset_value), default_values_column, std::abs(offset_value)); diff --git a/dbms/tests/queries/0_stateless/00957_next_in_block.sql b/dbms/tests/queries/0_stateless/00957_next_in_block.sql index 7cbd932cf1a..1efda43339e 100644 --- a/dbms/tests/queries/0_stateless/00957_next_in_block.sql +++ b/dbms/tests/queries/0_stateless/00957_next_in_block.sql @@ -1,22 +1,18 @@ -- no arguments -select nextInBlock(); -- { serverError 42 } +select neighbour(); -- { serverError 42 } +-- single argument +select neighbour(1); -- { serverError 42 } -- greater than 3 arguments -select nextInBlock(1,2,3,4); -- { serverError 42 } --- zero offset value -select nextInBlock(dummy, 0); -- { serverError 69 } --- negative offset value -select nextInBlock(dummy, -1); -- { serverError 43 } --- non-constant offset value -select nextInBlock(dummy, dummy); -- { serverError 43 } +select neighbour(1,2,3,4); -- { serverError 42 } -- bad default value -select nextInBlock(dummy, 1, 'hello'); -- { serverError 43 } +select neighbour(dummy, 1, 'hello'); -- { serverError 43 } -- single argument test -select number, nextInBlock(number) from numbers(2); +select number, neighbour(number,1) from numbers(2); -- filling by column's default value -select number, nextInBlock(number, 2) from numbers(3); +select number, neighbour(number, 2) from numbers(3); -- offset is greater that block - should fill everything with defaults -select number, nextInBlock(number, 5) from numbers(2); +select number, neighbour(number, 5) from numbers(2); -- substitution by constant for missing values -select number, nextInBlock(number, 2, 1000) from numbers(5); +select number, neighbour(number, 2, 1000) from numbers(5); -- substitution by expression --- select number, nextInBlock(number, 2, number % 2) from numbers(5); \ No newline at end of file +-- select number, neighbour(number, 2, number % 2) from numbers(5); \ No newline at end of file From 8328a06eec77a7ec32cf5eb9d3f103ad3510051c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 12 Aug 2019 19:20:31 +0300 Subject: [PATCH 178/509] update renaming metadata files --- dbms/src/Interpreters/InterpreterDropQuery.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index facf377b885..c93dedada82 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -97,16 +97,20 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t //Try to rename metadata file and delete the data try { - //Memory database has no metadata on disk - if (database_and_table.first->getEngineName() != "Memory") + //There some kind of tables that have no metadata - ignore renaming + try + { Poco::File(prev_metadata_name).renameTo(drop_metadata_name); + } catch (...) {} /// Delete table data database_and_table.second->drop(); } catch (...) { - if (database_and_table.first->getEngineName() != "Memory") + try + { Poco::File(drop_metadata_name).renameTo(prev_metadata_name); + } catch (...) {} throw; } From 4be51007c57f292a92756b4adc736685726baaef Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 13 Aug 2019 00:45:59 +0300 Subject: [PATCH 179/509] empty commit --- dbms/src/Databases/DatabaseOrdinary.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 1e7f331c7aa..10bf2ad871d 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -387,6 +387,7 @@ void DatabaseOrdinary::renameTable( throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); StoragePtr table = tryGetTable(context, table_name); + if (!table) throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); From 313f72e41a65a43fd5c380a3f63be3ae1cba1952 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 13 Aug 2019 11:03:31 +0300 Subject: [PATCH 180/509] remove try catch + submodule update --- dbms/src/Interpreters/InterpreterDropQuery.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index c93dedada82..a35b9599050 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -98,19 +98,15 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t try { //There some kind of tables that have no metadata - ignore renaming - try - { + if (Poco::File(prev_metadata_name).exists()) Poco::File(prev_metadata_name).renameTo(drop_metadata_name); - } catch (...) {} /// Delete table data database_and_table.second->drop(); } catch (...) { - try - { + if (Poco::File(drop_metadata_name).exists()) Poco::File(drop_metadata_name).renameTo(prev_metadata_name); - } catch (...) {} throw; } From 8e72d4c2ec1181dd84aea79227bf7732f6594339 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Aug 2019 11:35:49 +0300 Subject: [PATCH 181/509] Tryin COW settings --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 56 +++++++++++-------- dbms/src/Storages/MergeTree/MergeTreeData.h | 18 ++++-- .../Storages/MergeTree/MergeTreeSettings.cpp | 5 ++ .../Storages/MergeTree/MergeTreeSettings.h | 11 +++- .../MergeTree/registerStorageMergeTree.cpp | 6 +- 5 files changed, 64 insertions(+), 32 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index b0453b04c8a..e6a8f2f7ba1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -104,7 +104,7 @@ MergeTreeData::MergeTreeData( const ASTPtr & sample_by_ast_, const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - const MergeTreeSettings & settings_, + MergeTreeSettingsPtr settings_, bool require_part_metadata_, bool attach, BrokenPartCallback broken_part_callback_) @@ -132,7 +132,7 @@ MergeTreeData::MergeTreeData( sampling_expr_column_name = sample_by_ast->getColumnName(); if (!primary_key_sample.has(sampling_expr_column_name) - && !attach && !settings.compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility. + && !attach && !settings->compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility. throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS); auto syntax = SyntaxAnalyzer(global_context).analyze(sample_by_ast, getColumns().getAllPhysical()); @@ -727,6 +727,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) { LOG_DEBUG(log, "Loading data parts"); + auto settings_ptr = getImmutableSettings(); Strings part_file_names; Poco::DirectoryIterator end; for (Poco::DirectoryIterator it(full_path); it != end; ++it) @@ -843,12 +844,12 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART); } - if (has_non_adaptive_parts && has_adaptive_parts && !settings.enable_mixed_granularity_parts) + if (has_non_adaptive_parts && has_adaptive_parts && !settings_ptr->enable_mixed_granularity_parts) throw Exception("Table contains parts with adaptive and non adaptive marks, but `setting enable_mixed_granularity_parts` is disabled", ErrorCodes::LOGICAL_ERROR); has_non_adaptive_index_granularity_parts = has_non_adaptive_parts; - if (suspicious_broken_parts > settings.max_suspicious_broken_parts && !skip_sanity_checks) + if (suspicious_broken_parts > settings_ptr->max_suspicious_broken_parts && !skip_sanity_checks) throw Exception("Suspiciously many (" + toString(suspicious_broken_parts) + ") broken parts to remove.", ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS); @@ -938,7 +939,7 @@ void MergeTreeData::clearOldTemporaryDirectories(ssize_t custom_directories_life time_t current_time = time(nullptr); ssize_t deadline = (custom_directories_lifetime_seconds >= 0) ? current_time - custom_directories_lifetime_seconds - : current_time - settings.temporary_directories_lifetime.totalSeconds(); + : current_time - settings_ptr->temporary_directories_lifetime.totalSeconds(); /// Delete temporary directories older than a day. Poco::DirectoryIterator end; @@ -989,7 +990,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts() if (part.unique() && /// Grab only parts that are not used by anyone (SELECTs for example). part_remove_time < now && - now - part_remove_time > settings.old_parts_lifetime.totalSeconds()) + now - part_remove_time > getSettings()->old_parts_lifetime.totalSeconds()) { parts_to_delete.emplace_back(it); } @@ -1290,7 +1291,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name if (part) part_mrk_file_extension = part->index_granularity_info.marks_file_extension; else - part_mrk_file_extension = settings.index_granularity_bytes == 0 ? getNonAdaptiveMrkExtension() : getAdaptiveMrkExtension(); + part_mrk_file_extension = mutable_settings->index_granularity_bytes == 0 ? getNonAdaptiveMrkExtension() : getAdaptiveMrkExtension(); using NameToType = std::map; NameToType new_types; @@ -1448,6 +1449,7 @@ void MergeTreeData::alterDataPart( bool skip_sanity_checks, AlterDataPartTransactionPtr & transaction) { + auto settings = getImmutableSettings(); ExpressionActionsPtr expression; const auto & part = transaction->getDataPart(); bool force_update_metadata; @@ -1463,12 +1465,12 @@ void MergeTreeData::alterDataPart( ++num_files_to_remove; if (!skip_sanity_checks - && (num_files_to_modify > settings.max_files_to_modify_in_alter_columns - || num_files_to_remove > settings.max_files_to_remove_in_alter_columns)) + && (num_files_to_modify > settings->max_files_to_modify_in_alter_columns + || num_files_to_remove > settings->max_files_to_remove_in_alter_columns)) { transaction->clear(); - const bool forbidden_because_of_modify = num_files_to_modify > settings.max_files_to_modify_in_alter_columns; + const bool forbidden_because_of_modify = num_files_to_modify > settings->max_files_to_modify_in_alter_columns; std::stringstream exception_message; exception_message @@ -1500,7 +1502,7 @@ void MergeTreeData::alterDataPart( << " If it is not an error, you could increase merge_tree/" << (forbidden_because_of_modify ? "max_files_to_modify_in_alter_columns" : "max_files_to_remove_in_alter_columns") << " parameter in configuration file (current value: " - << (forbidden_because_of_modify ? settings.max_files_to_modify_in_alter_columns : settings.max_files_to_remove_in_alter_columns) + << (forbidden_because_of_modify ? settings->max_files_to_modify_in_alter_columns : settings->max_files_to_remove_in_alter_columns) << ")"; throw Exception(exception_message.str(), ErrorCodes::TABLE_DIFFERS_TOO_MUCH); @@ -1590,13 +1592,17 @@ void MergeTreeData::alterSettings( const Context & context, TableStructureWriteLockHolder & table_lock_holder) { - settings.updateFromChanges(new_changes); - IStorage::alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); + { + MutableMergeTreeSettingsPtr settings = std::move(*mutable_settings).mutate(); + settings->updateFromChanges(new_changes); + IStorage::alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); + mutable_settings = std::move(settings); + } } bool MergeTreeData::hasSetting(const String & setting_name) const { - return settings.findIndex(setting_name) != MergeTreeSettings::npos; + return MergeTreeSettings::findIndex(setting_name) != MergeTreeSettings::npos; } void MergeTreeData::removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr & data_part) @@ -2231,26 +2237,27 @@ std::optional MergeTreeData::getMinPartDataVersion() const void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const { + auto settings_ptr = getImmutableSettings(); const size_t parts_count_in_total = getPartsCount(); - if (parts_count_in_total >= settings.max_parts_in_total) + if (parts_count_in_total >= settings_ptr->max_parts_in_total) { ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception("Too many parts (" + toString(parts_count_in_total) + ") in all partitions in total. This indicates wrong choice of partition key. The threshold can be modified with 'max_parts_in_total' setting in element in config.xml or with per-table setting.", ErrorCodes::TOO_MANY_PARTS); } const size_t parts_count_in_partition = getMaxPartsCountForPartition(); - if (parts_count_in_partition < settings.parts_to_delay_insert) + if (parts_count_in_partition < settings_ptr->parts_to_delay_insert) return; - if (parts_count_in_partition >= settings.parts_to_throw_insert) + if (parts_count_in_partition >= settings_ptr->parts_to_throw_insert) { ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception("Too many parts (" + toString(parts_count_in_partition) + "). Merges are processing significantly slower than inserts.", ErrorCodes::TOO_MANY_PARTS); } - const size_t max_k = settings.parts_to_throw_insert - settings.parts_to_delay_insert; /// always > 0 - const size_t k = 1 + parts_count_in_partition - settings.parts_to_delay_insert; /// from 1 to max_k - const double delay_milliseconds = ::pow(settings.max_delay_to_insert * 1000, static_cast(k) / max_k); + const size_t max_k = settings_ptr->parts_to_throw_insert - settings_ptr->parts_to_delay_insert; /// always > 0 + const size_t k = 1 + parts_count_in_partition - settings_ptr->parts_to_delay_insert; /// from 1 to max_k + const double delay_milliseconds = ::pow(settings_ptr->max_delay_to_insert * 1000, static_cast(k) / max_k); ProfileEvents::increment(ProfileEvents::DelayedInserts); ProfileEvents::increment(ProfileEvents::DelayedInsertsMilliseconds, delay_milliseconds); @@ -2268,8 +2275,9 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const void MergeTreeData::throwInsertIfNeeded() const { + auto settings_ptr = getImmutableSettings(); const size_t parts_count_in_total = getPartsCount(); - if (parts_count_in_total >= settings.max_parts_in_total) + if (parts_count_in_total >= settings_ptr->max_parts_in_total) { ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception("Too many parts (" + toString(parts_count_in_total) + ") in all partitions in total. This indicates wrong choice of partition key. The threshold can be modified with 'max_parts_in_total' setting in element in config.xml or with per-table setting.", ErrorCodes::TOO_MANY_PARTS); @@ -2277,7 +2285,7 @@ void MergeTreeData::throwInsertIfNeeded() const const size_t parts_count_in_partition = getMaxPartsCountForPartition(); - if (parts_count_in_partition >= settings.parts_to_throw_insert) + if (parts_count_in_partition >= settings_ptr->parts_to_throw_insert) { ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception("Too many parts (" + toString(parts_count_in_partition) + "). Merges are processing significantly slower than inserts.", ErrorCodes::TOO_MANY_PARTS); @@ -2860,7 +2868,9 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const { - if (!settings.enable_mixed_granularity_parts || settings.index_granularity_bytes == 0) + auto settings_ptr = getImmutableSettings(); + + if (!settings_ptr->enable_mixed_granularity_parts || settings_ptr->index_granularity_bytes == 0) { if (!canUseAdaptiveGranularity() && src_part->index_granularity_info.is_adaptive) return false; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 756c188c724..38e58f3a6da 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -313,7 +313,7 @@ public: const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - const MergeTreeSettings & settings_, + MergeTreeSettingsPtr settings_, bool require_part_metadata_, bool attach, BrokenPartCallback broken_part_callback_ = [](const String &){}); @@ -591,8 +591,9 @@ public: bool canUseAdaptiveGranularity() const { - return settings.index_granularity_bytes != 0 && - (settings.enable_mixed_granularity_parts || !has_non_adaptive_index_granularity_parts); + auto settings_ptr = getImmutableSettings(); + return settings_ptr->index_granularity_bytes != 0 && + (settings_ptr->enable_mixed_granularity_parts || !has_non_adaptive_index_granularity_parts); } @@ -645,8 +646,6 @@ public: String sampling_expr_column_name; Names columns_required_for_sampling; - MergeTreeSettings settings; - /// Limiting parallel sends per one table, used in DataPartsExchange std::atomic_uint current_table_sends {0}; @@ -655,7 +654,13 @@ public: bool has_non_adaptive_index_granularity_parts = false; + MergeTreeSettingsPtr getImmutableSettings() const + { + return mutable_settings; + } + protected: + friend struct MergeTreeDataPart; friend class MergeTreeDataMergerMutator; friend class ReplicatedMergeTreeAlterThread; @@ -683,6 +688,9 @@ protected: String log_name; Logger * log; + /// Settings COW pointer. Data maybe changed at any point of time. + /// If you need consistent settings, just copy pointer to your scope. + MergeTreeSettingsPtr mutable_settings; /// Work with data parts diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp index 425a0851d67..3b90bf293ba 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -75,4 +75,9 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) #undef ADD_IF_ABSENT } + +MergeTreeSettings * MergeTreeSettings::clone() const +{ + return new MergeTreeSettings(*this); +} } diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index dc94ad5b11f..54416f934a6 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -2,6 +2,7 @@ #include #include +#include namespace Poco @@ -21,9 +22,11 @@ class ASTStorage; /** Settings for the MergeTree family of engines. * Could be loaded from config or from a CREATE TABLE query (SETTINGS clause). */ -struct MergeTreeSettings : public SettingsCollection +struct MergeTreeSettings : public SettingsCollection, public COW { + friend class COW; + /// M (mutable) for normal settings, IM (immutable) for not updateable settings. #define LIST_OF_MERGE_TREE_SETTINGS(M, IM) \ IM(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.") \ @@ -93,6 +96,12 @@ struct MergeTreeSettings : public SettingsCollection /// NOTE: will rewrite the AST to add immutable settings. void loadFromQuery(ASTStorage & storage_def); + + MergeTreeSettings * clone() const; + ~MergeTreeSettings() {} }; +using MergeTreeSettingsPtr = MergeTreeSettings::Ptr; +using MutableMergeTreeSettingsPtr = MergeTreeSettings::MutablePtr; + } diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 138e7c14f9d..a8e2f137e1a 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -574,7 +574,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) ASTPtr sample_by_ast; ASTPtr ttl_table_ast; IndicesDescription indices_description; - MergeTreeSettings storage_settings = args.context.getMergeTreeSettings(); + MutableMergeTreeSettingsPtr storage_settings = MergeTreeSettings::create(args.context.getMergeTreeSettings()); if (is_extended_storage_def) { @@ -603,7 +603,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) std::dynamic_pointer_cast(index->clone())); - storage_settings.loadFromQuery(*args.storage_def); + storage_settings->loadFromQuery(*args.storage_def); } else { @@ -625,7 +625,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) const auto * ast = engine_args.back()->as(); if (ast && ast->value.getType() == Field::Types::UInt64) - storage_settings.index_granularity = safeGet(ast->value); + storage_settings->index_granularity = safeGet(ast->value); else throw Exception( "Index granularity must be a positive integer" + getMergeTreeVerboseHelp(is_extended_storage_def), From ad81c743c186cff9d2e0e5ff1019f9be44088392 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Aug 2019 13:29:31 +0300 Subject: [PATCH 182/509] Buildable code --- .../Storages/MergeTree/DataPartsExchange.cpp | 8 ++- .../MergeTree/IMergedBlockOutputStream.cpp | 7 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 54 ++++++++------- dbms/src/Storages/MergeTree/MergeTreeData.h | 36 +++++++--- .../MergeTree/MergeTreeDataMergerMutator.cpp | 32 +++++---- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 21 +++--- .../MergeTreeIndexGranularityInfo.cpp | 5 +- .../Storages/MergeTree/MergeTreeSettings.cpp | 4 +- .../Storages/MergeTree/MergeTreeSettings.h | 2 +- .../MergeTreeThreadSelectBlockInputStream.cpp | 2 +- .../ReplicatedMergeTreeCleanupThread.cpp | 30 +++++---- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 3 +- .../ReplicatedMergeTreeRestartingThread.cpp | 17 +++-- .../ReplicatedMergeTreeTableMetadata.cpp | 5 +- .../MergeTree/registerStorageMergeTree.cpp | 4 +- dbms/src/Storages/StorageMergeTree.cpp | 11 +-- dbms/src/Storages/StorageMergeTree.h | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 67 +++++++++++-------- .../src/Storages/StorageReplicatedMergeTree.h | 2 +- 19 files changed, 186 insertions(+), 126 deletions(-) diff --git a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp index f01b384d441..795cc68f1ea 100644 --- a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp @@ -54,14 +54,15 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED); String part_name = params.get("part"); + const auto data_settings = data.getCOWSettings(); /// Validation of the input that may come from malicious replica. MergeTreePartInfo::fromPartName(part_name, data.format_version); static std::atomic_uint total_sends {0}; - if ((data.settings.replicated_max_parallel_sends && total_sends >= data.settings.replicated_max_parallel_sends) - || (data.settings.replicated_max_parallel_sends_for_table && data.current_table_sends >= data.settings.replicated_max_parallel_sends_for_table)) + if ((data_settings->replicated_max_parallel_sends && total_sends >= data_settings->replicated_max_parallel_sends) + || (data_settings->replicated_max_parallel_sends_for_table && data.current_table_sends >= data_settings->replicated_max_parallel_sends_for_table)) { response.setStatus(std::to_string(HTTP_TOO_MANY_REQUESTS)); response.setReason("Too many concurrent fetches, try again later"); @@ -174,6 +175,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( { /// Validation of the input that may come from malicious replica. MergeTreePartInfo::fromPartName(part_name, data.format_version); + const auto data_settings = data.getCOWSettings(); Poco::URI uri; uri.setScheme(interserver_scheme); @@ -200,7 +202,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( timeouts, creds, DBMS_DEFAULT_BUFFER_SIZE, - data.settings.replicated_max_parallel_fetches_for_host + data_settings->replicated_max_parallel_fetches_for_host }; static const String TMP_PREFIX = "tmp_fetch_"; diff --git a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 2bd0ebb61ea..a20b03cc7b0 100644 --- a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -32,7 +32,7 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( , index_granularity(index_granularity_) , compute_granularity(index_granularity.empty()) , codec(std::move(codec_)) - , with_final_mark(storage.settings.write_final_mark && storage.canUseAdaptiveGranularity()) + , with_final_mark(storage.getCOWSettings()->write_final_mark && storage.canUseAdaptiveGranularity()) { if (blocks_are_granules_size && !index_granularity.empty()) throw Exception("Can't take information about index granularity from blocks, when non empty index_granularity array specified", ErrorCodes::LOGICAL_ERROR); @@ -133,10 +133,11 @@ void fillIndexGranularityImpl( void IMergedBlockOutputStream::fillIndexGranularity(const Block & block) { + const auto storage_settings = storage.getCOWSettings(); fillIndexGranularityImpl( block, - storage.settings.index_granularity_bytes, - storage.settings.index_granularity, + storage_settings->index_granularity_bytes, + storage_settings->index_granularity, blocks_are_granules_size, index_offset, index_granularity, diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index f8ff2ab87b1..4d47f60a2fe 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -110,7 +110,6 @@ MergeTreeData::MergeTreeData( BrokenPartCallback broken_part_callback_) : global_context(context_), merging_params(merging_params_), - settings(settings_), partition_by_ast(partition_by_ast_), sample_by_ast(sample_by_ast_), ttl_table_ast(ttl_table_ast_), @@ -119,9 +118,11 @@ MergeTreeData::MergeTreeData( full_path(full_path_), broken_part_callback(broken_part_callback_), log_name(database_name + "." + table_name), log(&Logger::get(log_name)), + guarded_settings(std::move(settings_)), data_parts_by_info(data_parts_indexes.get()), data_parts_by_state_and_info(data_parts_indexes.get()) { + const auto settings = getCOWSettings(); setPrimaryKeyIndicesAndColumns(order_by_ast_, primary_key_ast_, columns_, indices_); /// NOTE: using the same columns list as is read when performing actual merges. @@ -725,7 +726,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) { LOG_DEBUG(log, "Loading data parts"); - auto settings_ptr = getImmutableSettings(); + const auto settings = getCOWSettings(); Strings part_file_names; Poco::DirectoryIterator end; for (Poco::DirectoryIterator it(full_path); it != end; ++it) @@ -747,7 +748,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) } /// Parallel loading of data parts. - size_t num_threads = std::min(size_t(settings.max_part_loading_threads), part_file_names.size()); + size_t num_threads = std::min(size_t(settings->max_part_loading_threads), part_file_names.size()); std::mutex mutex; @@ -866,12 +867,12 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) pool.wait(); - if (has_non_adaptive_parts && has_adaptive_parts && !settings_ptr->enable_mixed_granularity_parts) + if (has_non_adaptive_parts && has_adaptive_parts && !settings->enable_mixed_granularity_parts) throw Exception("Table contains parts with adaptive and non adaptive marks, but `setting enable_mixed_granularity_parts` is disabled", ErrorCodes::LOGICAL_ERROR); has_non_adaptive_index_granularity_parts = has_non_adaptive_parts; - if (suspicious_broken_parts > settings_ptr->max_suspicious_broken_parts && !skip_sanity_checks) + if (suspicious_broken_parts > settings->max_suspicious_broken_parts && !skip_sanity_checks) throw Exception("Suspiciously many (" + toString(suspicious_broken_parts) + ") broken parts to remove.", ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS); @@ -958,10 +959,11 @@ void MergeTreeData::clearOldTemporaryDirectories(ssize_t custom_directories_life if (!lock.try_lock()) return; + const auto settings = getCOWSettings(); time_t current_time = time(nullptr); ssize_t deadline = (custom_directories_lifetime_seconds >= 0) ? current_time - custom_directories_lifetime_seconds - : current_time - settings_ptr->temporary_directories_lifetime.totalSeconds(); + : current_time - settings->temporary_directories_lifetime.totalSeconds(); /// Delete temporary directories older than a day. Poco::DirectoryIterator end; @@ -1012,7 +1014,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts() if (part.unique() && /// Grab only parts that are not used by anyone (SELECTs for example). part_remove_time < now && - now - part_remove_time > getSettings()->old_parts_lifetime.totalSeconds()) + now - part_remove_time > getCOWSettings()->old_parts_lifetime.totalSeconds()) { parts_to_delete.emplace_back(it); } @@ -1096,11 +1098,12 @@ void MergeTreeData::clearOldPartsFromFilesystem() void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_remove) { - if (parts_to_remove.size() > 1 && settings.max_part_removal_threads > 1 && parts_to_remove.size() > settings.concurrent_part_removal_threshold) + const auto settings = getCOWSettings(); + if (parts_to_remove.size() > 1 && settings->max_part_removal_threads > 1 && parts_to_remove.size() > settings->concurrent_part_removal_threshold) { /// Parallel parts removal. - size_t num_threads = std::min(size_t(settings.max_part_removal_threads), parts_to_remove.size()); + size_t num_threads = std::min(size_t(settings->max_part_removal_threads), parts_to_remove.size()); ThreadPool pool(num_threads); /// NOTE: Under heavy system load you may get "Cannot schedule a task" from ThreadPool. @@ -1332,6 +1335,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name const IndicesASTs & old_indices, const IndicesASTs & new_indices, ExpressionActionsPtr & out_expression, NameToNameMap & out_rename_map, bool & out_force_update_metadata) const { + const auto settings = getCOWSettings(); out_expression = nullptr; out_rename_map = {}; out_force_update_metadata = false; @@ -1339,7 +1343,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name if (part) part_mrk_file_extension = part->index_granularity_info.marks_file_extension; else - part_mrk_file_extension = mutable_settings->index_granularity_bytes == 0 ? getNonAdaptiveMrkExtension() : getAdaptiveMrkExtension(); + part_mrk_file_extension = settings->index_granularity_bytes == 0 ? getNonAdaptiveMrkExtension() : getAdaptiveMrkExtension(); using NameToType = std::map; NameToType new_types; @@ -1497,7 +1501,7 @@ void MergeTreeData::alterDataPart( bool skip_sanity_checks, AlterDataPartTransactionPtr & transaction) { - auto settings = getImmutableSettings(); + const auto settings = getCOWSettings(); ExpressionActionsPtr expression; const auto & part = transaction->getDataPart(); bool force_update_metadata; @@ -1641,10 +1645,10 @@ void MergeTreeData::alterSettings( TableStructureWriteLockHolder & table_lock_holder) { { - MutableMergeTreeSettingsPtr settings = std::move(*mutable_settings).mutate(); + MutableMergeTreeSettingsPtr settings = std::move(*guarded_settings.getPtr()).mutate(); settings->updateFromChanges(new_changes); IStorage::alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); - mutable_settings = std::move(settings); + guarded_settings.setPtr(std::move(settings)); } } @@ -2285,27 +2289,27 @@ std::optional MergeTreeData::getMinPartDataVersion() const void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const { - auto settings_ptr = getImmutableSettings(); + const auto settings = getCOWSettings(); const size_t parts_count_in_total = getPartsCount(); - if (parts_count_in_total >= settings_ptr->max_parts_in_total) + if (parts_count_in_total >= settings->max_parts_in_total) { ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception("Too many parts (" + toString(parts_count_in_total) + ") in all partitions in total. This indicates wrong choice of partition key. The threshold can be modified with 'max_parts_in_total' setting in element in config.xml or with per-table setting.", ErrorCodes::TOO_MANY_PARTS); } const size_t parts_count_in_partition = getMaxPartsCountForPartition(); - if (parts_count_in_partition < settings_ptr->parts_to_delay_insert) + if (parts_count_in_partition < settings->parts_to_delay_insert) return; - if (parts_count_in_partition >= settings_ptr->parts_to_throw_insert) + if (parts_count_in_partition >= settings->parts_to_throw_insert) { ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception("Too many parts (" + toString(parts_count_in_partition) + "). Merges are processing significantly slower than inserts.", ErrorCodes::TOO_MANY_PARTS); } - const size_t max_k = settings_ptr->parts_to_throw_insert - settings_ptr->parts_to_delay_insert; /// always > 0 - const size_t k = 1 + parts_count_in_partition - settings_ptr->parts_to_delay_insert; /// from 1 to max_k - const double delay_milliseconds = ::pow(settings_ptr->max_delay_to_insert * 1000, static_cast(k) / max_k); + const size_t max_k = settings->parts_to_throw_insert - settings->parts_to_delay_insert; /// always > 0 + const size_t k = 1 + parts_count_in_partition - settings->parts_to_delay_insert; /// from 1 to max_k + const double delay_milliseconds = ::pow(settings->max_delay_to_insert * 1000, static_cast(k) / max_k); ProfileEvents::increment(ProfileEvents::DelayedInserts); ProfileEvents::increment(ProfileEvents::DelayedInsertsMilliseconds, delay_milliseconds); @@ -2323,9 +2327,9 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const void MergeTreeData::throwInsertIfNeeded() const { - auto settings_ptr = getImmutableSettings(); + const auto settings = getCOWSettings(); const size_t parts_count_in_total = getPartsCount(); - if (parts_count_in_total >= settings_ptr->max_parts_in_total) + if (parts_count_in_total >= settings->max_parts_in_total) { ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception("Too many parts (" + toString(parts_count_in_total) + ") in all partitions in total. This indicates wrong choice of partition key. The threshold can be modified with 'max_parts_in_total' setting in element in config.xml or with per-table setting.", ErrorCodes::TOO_MANY_PARTS); @@ -2333,7 +2337,7 @@ void MergeTreeData::throwInsertIfNeeded() const const size_t parts_count_in_partition = getMaxPartsCountForPartition(); - if (parts_count_in_partition >= settings_ptr->parts_to_throw_insert) + if (parts_count_in_partition >= settings->parts_to_throw_insert) { ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception("Too many parts (" + toString(parts_count_in_partition) + "). Merges are processing significantly slower than inserts.", ErrorCodes::TOO_MANY_PARTS); @@ -2916,9 +2920,9 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const { - auto settings_ptr = getImmutableSettings(); + const auto settings = getCOWSettings(); - if (!settings_ptr->enable_mixed_granularity_parts || settings_ptr->index_granularity_bytes == 0) + if (!settings->enable_mixed_granularity_parts || settings->index_granularity_bytes == 0) { if (!canUseAdaptiveGranularity() && src_part->index_granularity_info.is_adaptive) return false; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index c07e2fc34f4..b94555477f9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -594,9 +594,9 @@ public: /// Has additional constraint in replicated version virtual bool canUseAdaptiveGranularity() const { - auto settings_ptr = getImmutableSettings(); - return settings_ptr->index_granularity_bytes != 0 && - (settings_ptr->enable_mixed_granularity_parts || !has_non_adaptive_index_granularity_parts); + const auto settings = getCOWSettings(); + return settings->index_granularity_bytes != 0 && + (settings->enable_mixed_granularity_parts || !has_non_adaptive_index_granularity_parts); } @@ -657,9 +657,12 @@ public: bool has_non_adaptive_index_granularity_parts = false; - MergeTreeSettingsPtr getImmutableSettings() const + /// Get copy-on-write pointer to storage settings. + /// Copy this pointer into your scope and you will + /// get consistent settings. + const MergeTreeSettingsPtr getCOWSettings() const { - return mutable_settings; + return guarded_settings.copyPtr(); } protected: @@ -691,9 +694,26 @@ protected: String log_name; Logger * log; - /// Settings COW pointer. Data maybe changed at any point of time. - /// If you need consistent settings, just copy pointer to your scope. - MergeTreeSettingsPtr mutable_settings; + /// Just hides settings pointer from direct usage + class MergeTreeSettingsGuard + { + private: + /// Settings COW pointer. Data maybe changed at any point of time. + /// If you need consistent settings, just copy pointer to your scope. + MergeTreeSettingsPtr settings_ptr; + public: + MergeTreeSettingsGuard(MergeTreeSettingsPtr settings_ptr_) + : settings_ptr(settings_ptr_) + {} + + const MergeTreeSettingsPtr copyPtr() const { return settings_ptr; } + MergeTreeSettingsPtr & getPtr() { return settings_ptr; } + void setPtr(MergeTreeSettingsPtr ptr) { settings_ptr = ptr; } + }; + + /// Storage settings. Don't use this field directly, if you + /// want readonly settings. Prefer getCOWSettings() method. + MergeTreeSettingsGuard guarded_settings; /// Work with data parts diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 3a4d4038317..fad63f35aec 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -141,15 +141,16 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); size_t free_entries = pool_size - pool_used; + const auto data_settings = data.getCOWSettings(); UInt64 max_size = 0; - if (free_entries >= data.settings.number_of_free_entries_in_pool_to_lower_max_size_of_merge) - max_size = data.settings.max_bytes_to_merge_at_max_space_in_pool; + if (free_entries >= data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge) + max_size = data_settings->max_bytes_to_merge_at_max_space_in_pool; else max_size = interpolateExponential( - data.settings.max_bytes_to_merge_at_min_space_in_pool, - data.settings.max_bytes_to_merge_at_max_space_in_pool, - static_cast(free_entries) / data.settings.number_of_free_entries_in_pool_to_lower_max_size_of_merge); + data_settings->max_bytes_to_merge_at_min_space_in_pool, + data_settings->max_bytes_to_merge_at_max_space_in_pool, + static_cast(free_entries) / data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge); return std::min(max_size, static_cast(DiskSpaceMonitor::getUnreservedFreeSpace(data.full_path) / DISK_USAGE_COEFFICIENT_TO_SELECT)); } @@ -169,6 +170,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( String * out_disable_reason) { MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector(); + const auto data_settings = data.getCOWSettings(); if (data_parts.empty()) { @@ -223,7 +225,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( merge_settings.base = 1; bool can_merge_with_ttl = - (current_time - last_merge_with_ttl > data.settings.merge_with_ttl_timeout); + (current_time - last_merge_with_ttl > data_settings->merge_with_ttl_timeout); /// NOTE Could allow selection of different merge strategy. if (can_merge_with_ttl && has_part_with_expired_ttl && !ttl_merges_blocker.isCancelled()) @@ -545,6 +547,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor Names all_column_names = data.getColumns().getNamesOfPhysical(); NamesAndTypesList all_columns = data.getColumns().getAllPhysical(); + const auto data_settings = data.getCOWSettings(); NamesAndTypesList gathering_columns, merging_columns; Names gathering_column_names, merging_column_names; @@ -617,13 +620,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor /// We count total amount of bytes in parts /// and use direct_io + aio if there is more than min_merge_bytes_to_use_direct_io bool read_with_direct_io = false; - if (data.settings.min_merge_bytes_to_use_direct_io != 0) + if (data_settings->min_merge_bytes_to_use_direct_io != 0) { size_t total_size = 0; for (const auto & part : parts) { total_size += part->bytes_on_disk; - if (total_size >= data.settings.min_merge_bytes_to_use_direct_io) + if (total_size >= data_settings->min_merge_bytes_to_use_direct_io) { LOG_DEBUG(log, "Will merge parts reading files in O_DIRECT"); read_with_direct_io = true; @@ -720,7 +723,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor merging_columns, compression_codec, merged_column_to_size, - data.settings.min_merge_bytes_to_use_direct_io, + data_settings->min_merge_bytes_to_use_direct_io, blocks_are_granules_size}; merged_stream->readPrefix(); @@ -938,6 +941,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor auto in = mutations_interpreter.execute(); NamesAndTypesList all_columns = data.getColumns().getAllPhysical(); + const auto data_settings = data.getCOWSettings(); Block in_header = in->getHeader(); @@ -995,7 +999,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor } NameSet files_to_skip = {"checksums.txt", "columns.txt"}; - auto mrk_extension = data.settings.index_granularity_bytes ? getAdaptiveMrkExtension() : getNonAdaptiveMrkExtension(); + auto mrk_extension = data_settings->index_granularity_bytes ? getAdaptiveMrkExtension() : getNonAdaptiveMrkExtension(); for (const auto & entry : in_header) { IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path) @@ -1092,9 +1096,11 @@ MergeTreeDataMergerMutator::MergeAlgorithm MergeTreeDataMergerMutator::chooseMer const MergeTreeData::DataPartsVector & parts, size_t sum_rows_upper_bound, const NamesAndTypesList & gathering_columns, bool deduplicate, bool need_remove_expired_values) const { + const auto data_settings = data.getCOWSettings(); + if (deduplicate) return MergeAlgorithm::Horizontal; - if (data.settings.enable_vertical_merge_algorithm == 0) + if (data_settings->enable_vertical_merge_algorithm == 0) return MergeAlgorithm::Horizontal; if (need_remove_expired_values) return MergeAlgorithm::Horizontal; @@ -1105,9 +1111,9 @@ MergeTreeDataMergerMutator::MergeAlgorithm MergeTreeDataMergerMutator::chooseMer data.merging_params.mode == MergeTreeData::MergingParams::Replacing || data.merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing; - bool enough_ordinary_cols = gathering_columns.size() >= data.settings.vertical_merge_algorithm_min_columns_to_activate; + bool enough_ordinary_cols = gathering_columns.size() >= data_settings->vertical_merge_algorithm_min_columns_to_activate; - bool enough_total_rows = sum_rows_upper_bound >= data.settings.vertical_merge_algorithm_min_rows_to_activate; + bool enough_total_rows = sum_rows_upper_bound >= data_settings->vertical_merge_algorithm_min_rows_to_activate; bool no_parts_overflow = parts.size() <= RowSourcePart::MAX_PARTS; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 513a713651b..314967b485d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -672,6 +672,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( size_t sum_marks = 0; size_t total_rows = 0; + const auto data_settings = data.getCOWSettings(); size_t adaptive_parts = 0; for (size_t i = 0; i < parts.size(); ++i) { @@ -688,18 +689,18 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( size_t index_granularity_bytes = 0; if (adaptive_parts > parts.size() / 2) - index_granularity_bytes = data.settings.index_granularity_bytes; + index_granularity_bytes = data_settings->index_granularity_bytes; const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( settings.merge_tree_max_rows_to_use_cache, settings.merge_tree_max_bytes_to_use_cache, - data.settings.index_granularity, + data_settings->index_granularity, index_granularity_bytes); const size_t min_marks_for_concurrent_read = roundRowsOrBytesToMarks( settings.merge_tree_min_rows_for_concurrent_read, settings.merge_tree_min_bytes_for_concurrent_read, - data.settings.index_granularity, + data_settings->index_granularity, index_granularity_bytes); if (sum_marks > max_marks_to_use_cache) @@ -830,6 +831,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO SortingInfoPtr sorting_info = query_info.sorting_info; size_t adaptive_parts = 0; std::vector sum_marks_in_parts(parts.size()); + const auto data_settings = data.getCOWSettings(); /// In case of reverse order let's split ranges to avoid reading much data. auto split_ranges = [max_block_size](const auto & ranges, size_t rows_granularity, size_t num_marks_in_part) @@ -862,7 +864,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO sum_marks += sum_marks_in_parts[i]; if (sorting_info->direction == -1) - parts[i].ranges = split_ranges(parts[i].ranges, data.settings.index_granularity, sum_marks_in_parts[i]); + parts[i].ranges = split_ranges(parts[i].ranges, data_settings->index_granularity, sum_marks_in_parts[i]); /// Let the ranges be listed from right to left so that the leftmost range can be dropped using `pop_back()`. std::reverse(parts[i].ranges.begin(), parts[i].ranges.end()); @@ -873,18 +875,18 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO size_t index_granularity_bytes = 0; if (adaptive_parts > parts.size() / 2) - index_granularity_bytes = data.settings.index_granularity_bytes; + index_granularity_bytes = data_settings->index_granularity_bytes; const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( settings.merge_tree_max_rows_to_use_cache, settings.merge_tree_max_bytes_to_use_cache, - data.settings.index_granularity, + data_settings->index_granularity, index_granularity_bytes); const size_t min_marks_for_concurrent_read = roundRowsOrBytesToMarks( settings.merge_tree_min_rows_for_concurrent_read, settings.merge_tree_min_bytes_for_concurrent_read, - data.settings.index_granularity, + data_settings->index_granularity, index_granularity_bytes); if (sum_marks > max_marks_to_use_cache) @@ -1012,6 +1014,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal const Names & virt_columns, const Settings & settings) const { + const auto data_settings = data.getCOWSettings(); size_t sum_marks = 0; size_t adaptive_parts = 0; for (size_t i = 0; i < parts.size(); ++i) @@ -1025,12 +1028,12 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal size_t index_granularity_bytes = 0; if (adaptive_parts >= parts.size() / 2) - index_granularity_bytes = data.settings.index_granularity_bytes; + index_granularity_bytes = data_settings->index_granularity_bytes; const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( settings.merge_tree_max_rows_to_use_cache, settings.merge_tree_max_bytes_to_use_cache, - data.settings.index_granularity, + data_settings->index_granularity, index_granularity_bytes); if (sum_marks > max_marks_to_use_cache) diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/dbms/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index 63b19da9e64..143af37c10d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -25,12 +25,13 @@ std::optional MergeTreeIndexGranularityInfo::getMrkExtensionFromFS( MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo( const MergeTreeData & storage) { - fixed_index_granularity = storage.settings.index_granularity; + const auto storage_settings = storage.getCOWSettings(); + fixed_index_granularity = storage_settings->index_granularity; /// Granularity is fixed if (!storage.canUseAdaptiveGranularity()) setNonAdaptive(); else - setAdaptive(storage.settings.index_granularity_bytes); + setAdaptive(storage_settings->index_granularity_bytes); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp index 3b90bf293ba..7c3c7ef02cc 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -76,8 +76,8 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) } -MergeTreeSettings * MergeTreeSettings::clone() const +MergeTreeSettings::MutablePtr MergeTreeSettings::clone() const { - return new MergeTreeSettings(*this); + return COW::create(*this); } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index 480209c740a..dff7cc2e523 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -100,7 +100,7 @@ struct MergeTreeSettings : public SettingsCollection, public /// NOTE: will rewrite the AST to add immutable settings. void loadFromQuery(ASTStorage & storage_def); - MergeTreeSettings * clone() const; + MutablePtr clone() const; ~MergeTreeSettings() {} }; diff --git a/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp index 9c34782dec8..7a09bde0998 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp @@ -31,7 +31,7 @@ MergeTreeThreadSelectBlockInputStream::MergeTreeThreadSelectBlockInputStream( /// Maybe it will make sence to add settings `max_block_size_bytes` if (max_block_size_rows && !storage.canUseAdaptiveGranularity()) { - size_t fixed_index_granularity = storage.settings.index_granularity; + size_t fixed_index_granularity = storage.getCOWSettings()->index_granularity; min_marks_to_read = (min_marks_to_read_ * fixed_index_granularity + max_block_size_rows - 1) / max_block_size_rows * max_block_size_rows / fixed_index_granularity; } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 6108704b45a..2b03ed86895 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -27,8 +27,9 @@ ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplic void ReplicatedMergeTreeCleanupThread::run() { - const auto CLEANUP_SLEEP_MS = storage.settings.cleanup_delay_period * 1000 - + std::uniform_int_distribution(0, storage.settings.cleanup_delay_period_random_add * 1000)(rng); + auto storage_settings = storage.getCOWSettings(); + const auto CLEANUP_SLEEP_MS = storage_settings->cleanup_delay_period * 1000 + + std::uniform_int_distribution(0, storage_settings->cleanup_delay_period_random_add * 1000)(rng); try { @@ -74,6 +75,7 @@ void ReplicatedMergeTreeCleanupThread::iterate() void ReplicatedMergeTreeCleanupThread::clearOldLogs() { auto zookeeper = storage.getZooKeeper(); + auto storage_settings = storage.getCOWSettings(); Coordination::Stat stat; if (!zookeeper->exists(storage.zookeeper_path + "/log", &stat)) @@ -82,7 +84,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs() int children_count = stat.numChildren; /// We will wait for 1.1 times more records to accumulate than necessary. - if (static_cast(children_count) < storage.settings.min_replicated_logs_to_keep * 1.1) + if (static_cast(children_count) < storage_settings->min_replicated_logs_to_keep * 1.1) return; Strings replicas = zookeeper->getChildren(storage.zookeeper_path + "/replicas", &stat); @@ -100,8 +102,8 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs() std::sort(entries.begin(), entries.end()); String min_saved_record_log_str = entries[ - entries.size() > storage.settings.max_replicated_logs_to_keep - ? entries.size() - storage.settings.max_replicated_logs_to_keep + entries.size() > storage_settings->max_replicated_logs_to_keep + ? entries.size() - storage_settings->max_replicated_logs_to_keep : 0]; /// Replicas that were marked is_lost but are active. @@ -203,7 +205,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs() min_saved_log_pointer = std::min(min_saved_log_pointer, min_log_pointer_lost_candidate); /// We will not touch the last `min_replicated_logs_to_keep` records. - entries.erase(entries.end() - std::min(entries.size(), storage.settings.min_replicated_logs_to_keep), entries.end()); + entries.erase(entries.end() - std::min(entries.size(), storage_settings->min_replicated_logs_to_keep), entries.end()); /// We will not touch records that are no less than `min_saved_log_pointer`. entries.erase(std::lower_bound(entries.begin(), entries.end(), "log-" + padIndex(min_saved_log_pointer)), entries.end()); @@ -285,6 +287,7 @@ struct ReplicatedMergeTreeCleanupThread::NodeWithStat void ReplicatedMergeTreeCleanupThread::clearOldBlocks() { auto zookeeper = storage.getZooKeeper(); + auto storage_settings = storage.getCOWSettings(); std::vector timed_blocks; getBlocksSortedByTime(*zookeeper, timed_blocks); @@ -294,12 +297,12 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks() /// Use ZooKeeper's first node (last according to time) timestamp as "current" time. Int64 current_time = timed_blocks.front().ctime; - Int64 time_threshold = std::max(static_cast(0), current_time - static_cast(1000 * storage.settings.replicated_deduplication_window_seconds)); + Int64 time_threshold = std::max(static_cast(0), current_time - static_cast(1000 * storage_settings->replicated_deduplication_window_seconds)); /// Virtual node, all nodes that are "greater" than this one will be deleted NodeWithStat block_threshold{{}, time_threshold}; - size_t current_deduplication_window = std::min(timed_blocks.size(), storage.settings.replicated_deduplication_window); + size_t current_deduplication_window = std::min(timed_blocks.size(), storage_settings->replicated_deduplication_window); auto first_outdated_block_fixed_threshold = timed_blocks.begin() + current_deduplication_window; auto first_outdated_block_time_threshold = std::upper_bound(timed_blocks.begin(), timed_blocks.end(), block_threshold, NodeWithStat::greaterByTime); auto first_outdated_block = std::min(first_outdated_block_fixed_threshold, first_outdated_block_time_threshold); @@ -401,10 +404,11 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper & void ReplicatedMergeTreeCleanupThread::clearOldMutations() { - if (!storage.settings.finished_mutations_to_keep) + auto storage_settings = storage.getCOWSettings(); + if (!storage_settings->finished_mutations_to_keep) return; - if (storage.queue.countFinishedMutations() <= storage.settings.finished_mutations_to_keep) + if (storage.queue.countFinishedMutations() <= storage_settings->finished_mutations_to_keep) { /// Not strictly necessary, but helps to avoid unnecessary ZooKeeper requests. /// If even this replica hasn't finished enough mutations yet, then we don't need to clean anything. @@ -431,10 +435,10 @@ void ReplicatedMergeTreeCleanupThread::clearOldMutations() /// Do not remove entries that are greater than `min_pointer` (they are not done yet). entries.erase(std::upper_bound(entries.begin(), entries.end(), padIndex(min_pointer)), entries.end()); - /// Do not remove last `storage.settings.finished_mutations_to_keep` entries. - if (entries.size() <= storage.settings.finished_mutations_to_keep) + /// Do not remove last `storage_settings->finished_mutations_to_keep` entries. + if (entries.size() <= storage_settings->finished_mutations_to_keep) return; - entries.erase(entries.end() - storage.settings.finished_mutations_to_keep, entries.end()); + entries.erase(entries.end() - storage_settings->finished_mutations_to_keep, entries.end()); if (entries.empty()) return; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 665e8c9bd5c..5d044fbf839 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -961,7 +961,8 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( * (because it may be ordered by OPTIMIZE or early with differrent settings). */ UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(); - if (max_source_parts_size != data.settings.max_bytes_to_merge_at_max_space_in_pool + const auto data_settings = data.getCOWSettings(); + if (max_source_parts_size != data_settings->max_bytes_to_merge_at_max_space_in_pool && sum_parts_size_in_bytes > max_source_parts_size) { String reason = "Not executing log entry for part " + entry.new_part_name diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index a98625336c5..6145713492f 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -44,11 +44,12 @@ ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(Storage , log(&Logger::get(log_name)) , active_node_identifier(generateActiveNodeIdentifier()) { - check_period_ms = storage.settings.zookeeper_session_expiration_check_period.totalSeconds() * 1000; + const auto storage_settings = storage.getCOWSettings(); + check_period_ms = storage_settings->zookeeper_session_expiration_check_period.totalSeconds() * 1000; /// Periodicity of checking lag of replica. - if (check_period_ms > static_cast(storage.settings.check_delay_period) * 1000) - check_period_ms = storage.settings.check_delay_period * 1000; + if (check_period_ms > static_cast(storage_settings->check_delay_period) * 1000) + check_period_ms = storage_settings->check_delay_period * 1000; task = storage.global_context.getSchedulePool().createTask(log_name, [this]{ run(); }); } @@ -121,7 +122,8 @@ void ReplicatedMergeTreeRestartingThread::run() } time_t current_time = time(nullptr); - if (current_time >= prev_time_of_check_delay + static_cast(storage.settings.check_delay_period)) + const auto storage_settings = storage.getCOWSettings(); + if (current_time >= prev_time_of_check_delay + static_cast(storage_settings->check_delay_period)) { /// Find out lag of replicas. time_t absolute_delay = 0; @@ -136,10 +138,10 @@ void ReplicatedMergeTreeRestartingThread::run() /// We give up leadership if the relative lag is greater than threshold. if (storage.is_leader - && relative_delay > static_cast(storage.settings.min_relative_delay_to_yield_leadership)) + && relative_delay > static_cast(storage_settings->min_relative_delay_to_yield_leadership)) { LOG_INFO(log, "Relative replica delay (" << relative_delay << " seconds) is bigger than threshold (" - << storage.settings.min_relative_delay_to_yield_leadership << "). Will yield leadership."); + << storage_settings->min_relative_delay_to_yield_leadership << "). Will yield leadership."); ProfileEvents::increment(ProfileEvents::ReplicaYieldLeadership); @@ -169,6 +171,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() activateReplica(); const auto & zookeeper = storage.getZooKeeper(); + const auto storage_settings = storage.getCOWSettings(); storage.cloneReplicaIfNeeded(zookeeper); @@ -181,7 +184,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() updateQuorumIfWeHavePart(); - if (storage.settings.replicated_can_become_leader) + if (storage_settings->replicated_can_become_leader) storage.enterLeaderElection(); else LOG_INFO(log, "Will not enter leader election because replicated_can_become_leader=0"); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 10f77358b7d..381d1f47412 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -27,8 +27,9 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos]; + const auto data_settings = data.getCOWSettings(); sampling_expression = formattedAST(data.sample_by_ast); - index_granularity = data.settings.index_granularity; + index_granularity = data_settings->index_granularity; merging_params_mode = static_cast(data.merging_params.mode); sign_column = data.merging_params.sign_column; @@ -48,7 +49,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr ttl_table = formattedAST(data.ttl_table_ast); skip_indices = data.getIndices().toString(); if (data.canUseAdaptiveGranularity()) - index_granularity_bytes = data.settings.index_granularity_bytes; + index_granularity_bytes = data_settings->index_granularity_bytes; else index_granularity_bytes = 0; } diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 81ead0fd20d..1e9f1a252f4 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -641,13 +641,13 @@ static StoragePtr create(const StorageFactory::Arguments & args) zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name, args.columns, indices_description, args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast, - sample_by_ast, ttl_table_ast, merging_params, storage_settings, + sample_by_ast, ttl_table_ast, merging_params, std::move(storage_settings), args.has_force_restore_data_flag); else return StorageMergeTree::create( args.data_path, args.database_name, args.table_name, args.columns, indices_description, args.attach, args.context, date_column_name, partition_by_ast, order_by_ast, - primary_key_ast, sample_by_ast, ttl_table_ast, merging_params, storage_settings, + primary_key_ast, sample_by_ast, ttl_table_ast, merging_params, std::move(storage_settings), args.has_force_restore_data_flag); } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 9ed413157a2..f6975e34dfa 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -62,7 +62,7 @@ StorageMergeTree::StorageMergeTree( const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - const MergeTreeSettings & settings_, + MergeTreeSettingsPtr settings_, bool has_force_restore_data_flag) : MergeTreeData(database_name_, table_name_, path_ + escapeForFileName(table_name_) + '/', @@ -801,14 +801,15 @@ Int64 StorageMergeTree::getCurrentMutationVersion( void StorageMergeTree::clearOldMutations() { - if (!settings.finished_mutations_to_keep) + const auto settings = getCOWSettings(); + if (!settings->finished_mutations_to_keep) return; std::vector mutations_to_delete; { std::lock_guard lock(currently_merging_mutex); - if (current_mutations_by_version.size() <= settings.finished_mutations_to_keep) + if (current_mutations_by_version.size() <= settings->finished_mutations_to_keep) return; auto begin_it = current_mutations_by_version.begin(); @@ -819,10 +820,10 @@ void StorageMergeTree::clearOldMutations() end_it = current_mutations_by_version.upper_bound(*min_version); size_t done_count = std::distance(begin_it, end_it); - if (done_count <= settings.finished_mutations_to_keep) + if (done_count <= settings->finished_mutations_to_keep) return; - size_t to_delete_count = done_count - settings.finished_mutations_to_keep; + size_t to_delete_count = done_count - settings->finished_mutations_to_keep; auto it = begin_it; for (size_t i = 0; i < to_delete_count; ++i) diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 0df90604f67..556cf7999b8 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -151,7 +151,7 @@ protected: const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - const MergeTreeSettings & settings_, + MergeTreeSettingsPtr settings_, bool has_force_restore_data_flag); }; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 0b66e98d0dc..c788c940efc 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -202,7 +202,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const ASTPtr & sample_by_ast_, const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - const MergeTreeSettings & settings_, + MergeTreeSettingsPtr settings_, bool has_force_restore_data_flag) : MergeTreeData(database_name_, table_name_, path_ + escapeForFileName(table_name_) + '/', @@ -376,7 +376,7 @@ void StorageReplicatedMergeTree::createTableIfNotExists() } -/** Verify that list of columns and table settings match those specified in ZK (/ metadata). +/** Verify that list of columns and table storage_settings match those specified in ZK (/ metadata). * If not, throw an exception. */ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bool allow_alter) @@ -633,7 +633,8 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) for (const auto & part : parts) total_rows_on_filesystem += part->rows_count; - bool insane = unexpected_parts_rows > total_rows_on_filesystem * settings.replicated_max_ratio_of_wrong_parts; + const auto storage_settings = getCOWSettings(); + bool insane = unexpected_parts_rows > total_rows_on_filesystem * storage_settings->replicated_max_ratio_of_wrong_parts; if (insane && !skip_sanity_checks) { @@ -776,12 +777,13 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: if (!has_been_already_added) { + const auto storage_settings = getCOWSettings(); String part_path = replica_path + "/parts/" + part_name; ops.emplace_back(zkutil::makeCheckRequest( zookeeper_path + "/columns", expected_columns_version)); - if (settings.use_minimalistic_part_header_in_zookeeper) + if (storage_settings->use_minimalistic_part_header_in_zookeeper) { ops.emplace_back(zkutil::makeCreateRequest( part_path, local_part_header.toString(), zkutil::CreateMode::Persistent)); @@ -858,7 +860,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd String StorageReplicatedMergeTree::getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums) const { return MinimalisticDataPartChecksums::getSerializedString(checksums, - static_cast(settings.use_minimalistic_checksums_in_zookeeper)); + getCOWSettings()->use_minimalistic_checksums_in_zookeeper); } @@ -1029,13 +1031,14 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) parts.push_back(part); } + const auto storage_settings = getCOWSettings(); if (!have_all_parts) { /// If you do not have all the necessary parts, try to take some already merged part from someone. LOG_DEBUG(log, "Don't have all parts for merge " << entry.new_part_name << "; will try to fetch it instead"); return false; } - else if (entry.create_time + settings.prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr)) + else if (entry.create_time + storage_settings->prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr)) { /// If entry is old enough, and have enough size, and part are exists in any replica, /// then prefer fetching of merged part from replica. @@ -1044,7 +1047,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) for (const auto & part : parts) sum_parts_bytes_on_disk += part->bytes_on_disk; - if (sum_parts_bytes_on_disk >= settings.prefer_fetch_merged_part_size_threshold) + if (sum_parts_bytes_on_disk >= storage_settings->prefer_fetch_merged_part_size_threshold) { String replica = findReplicaHavingPart(entry.new_part_name, true); /// NOTE excessive ZK requests for same data later, may remove. if (!replica.empty()) @@ -1154,6 +1157,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedMergeTree::LogEntry & entry) { const String & source_part_name = entry.source_parts.at(0); + const auto storage_settings = getCOWSettings(); LOG_TRACE(log, "Executing log entry to mutate part " << source_part_name << " to " << entry.new_part_name); DataPartPtr source_part = getActiveContainingPart(source_part_name); @@ -1173,8 +1177,8 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM /// TODO - some better heuristic? size_t estimated_space_for_result = MergeTreeDataMergerMutator::estimateNeededDiskSpace({source_part}); - if (entry.create_time + settings.prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr) - && estimated_space_for_result >= settings.prefer_fetch_merged_part_size_threshold) + if (entry.create_time + storage_settings->prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr) + && estimated_space_for_result >= storage_settings->prefer_fetch_merged_part_size_threshold) { /// If entry is old enough, and have enough size, and some replica has the desired part, /// then prefer fetching from replica. @@ -1268,20 +1272,21 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) { String replica = findReplicaHavingCoveringPart(entry, true); + const auto storage_settings = getCOWSettings(); static std::atomic_uint total_fetches {0}; - if (settings.replicated_max_parallel_fetches && total_fetches >= settings.replicated_max_parallel_fetches) + if (storage_settings->replicated_max_parallel_fetches && total_fetches >= storage_settings->replicated_max_parallel_fetches) { - throw Exception("Too many total fetches from replicas, maximum: " + settings.replicated_max_parallel_fetches.toString(), + throw Exception("Too many total fetches from replicas, maximum: " + storage_settings->replicated_max_parallel_fetches.toString(), ErrorCodes::TOO_MANY_FETCHES); } ++total_fetches; SCOPE_EXIT({--total_fetches;}); - if (settings.replicated_max_parallel_fetches_for_table && current_table_fetches >= settings.replicated_max_parallel_fetches_for_table) + if (storage_settings->replicated_max_parallel_fetches_for_table && current_table_fetches >= storage_settings->replicated_max_parallel_fetches_for_table) { - throw Exception("Too many fetches from replicas for table, maximum: " + settings.replicated_max_parallel_fetches_for_table.toString(), + throw Exception("Too many fetches from replicas for table, maximum: " + storage_settings->replicated_max_parallel_fetches_for_table.toString(), ErrorCodes::TOO_MANY_FETCHES); } @@ -2162,6 +2167,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() if (!is_leader) return; + const auto storage_settings = getCOWSettings(); const bool deduplicate = false; /// TODO: read deduplicate option from table config const bool force_ttl = false; @@ -2181,16 +2187,16 @@ void StorageReplicatedMergeTree::mergeSelectingTask() /// Otherwise merge queue could be filled with only large merges, /// and in the same time, many small parts could be created and won't be merged. size_t merges_and_mutations_queued = queue.countMergesAndPartMutations(); - if (merges_and_mutations_queued >= settings.max_replicated_merges_in_queue) + if (merges_and_mutations_queued >= storage_settings->max_replicated_merges_in_queue) { LOG_TRACE(log, "Number of queued merges and part mutations (" << merges_and_mutations_queued << ") is greater than max_replicated_merges_in_queue (" - << settings.max_replicated_merges_in_queue << "), so won't select new parts to merge or mutate."); + << storage_settings->max_replicated_merges_in_queue << "), so won't select new parts to merge or mutate."); } else { UInt64 max_source_parts_size_for_merge = merger_mutator.getMaxSourcePartsSizeForMerge( - settings.max_replicated_merges_in_queue, merges_and_mutations_queued); + storage_settings->max_replicated_merges_in_queue, merges_and_mutations_queued); UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); FutureMergedMutatedPart future_merged_part; @@ -2973,10 +2979,11 @@ void StorageReplicatedMergeTree::assertNotReadonly() const BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const Context & context) { + const auto storage_settings = getCOWSettings(); assertNotReadonly(); const Settings & query_settings = context.getSettingsRef(); - bool deduplicate = settings.replicated_deduplication_window != 0 && query_settings.insert_deduplicate; + bool deduplicate = storage_settings->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; return std::make_shared(*this, query_settings.insert_quorum, query_settings.insert_quorum_timeout.totalMilliseconds(), query_settings.max_partitions_per_insert_block, deduplicate); @@ -3010,6 +3017,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p }; bool force_ttl = (final && (hasTableTTL() || hasAnyColumnTTL())); + const auto storage_settings = getCOWSettings(); if (!partition && final) { @@ -3042,7 +3050,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p if (!partition) { selected = merger_mutator.selectPartsToMerge( - future_merged_part, true, settings.max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason); + future_merged_part, true, storage_settings->max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason); } else { @@ -3092,9 +3100,9 @@ void StorageReplicatedMergeTree::alter( if (params.isSettingsAlter()) { - /// We don't replicate settings ALTER. It's local operation. + /// We don't replicate storage_settings ALTER. It's local operation. /// Also we don't upgrade alter lock to table structure lock. - LOG_DEBUG(log, "ALTER settings only"); + LOG_DEBUG(log, "ALTER storage_settings only"); SettingsChanges new_changes; params.applyForSettingsOnly(new_changes); alterSettings(new_changes, current_database_name, current_table_name, query_context, table_lock_holder); @@ -3920,9 +3928,10 @@ void StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(const String & void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) { auto zookeeper = tryGetZooKeeper(); + const auto storage_settings = getCOWSettings(); res.is_leader = is_leader; - res.can_become_leader = settings.replicated_can_become_leader; + res.can_become_leader = storage_settings->replicated_can_become_leader; res.is_readonly = is_readonly; res.is_session_expired = !zookeeper || zookeeper->expired(); @@ -4112,13 +4121,14 @@ void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, t out_absolute_delay = getAbsoluteDelay(); out_relative_delay = 0; + const auto storage_settings = getCOWSettings(); /** Relative delay is the maximum difference of absolute delay from any other replica, * (if this replica lags behind any other live replica, or zero, otherwise). * Calculated only if the absolute delay is large enough. */ - if (out_absolute_delay < static_cast(settings.min_relative_delay_to_yield_leadership)) + if (out_absolute_delay < static_cast(storage_settings->min_relative_delay_to_yield_leadership)) return; auto zookeeper = getZooKeeper(); @@ -4376,7 +4386,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const /// instead. /// /// Mutations of individual parts are in fact pretty similar to merges, e.g. their assignment and execution - /// is governed by the same settings. TODO: support a single "merge-mutation" operation when the data + /// is governed by the same storage_settings. TODO: support a single "merge-mutation" operation when the data /// read from the the source parts is first mutated on the fly to some uniform mutation version and then /// merged to a resulting part. /// @@ -4939,6 +4949,7 @@ void StorageReplicatedMergeTree::getCommitPartOps( const String & block_id_path) const { const String & part_name = part->name; + const auto storage_settings = getCOWSettings(); if (!block_id_path.empty()) { @@ -4956,7 +4967,7 @@ void StorageReplicatedMergeTree::getCommitPartOps( zookeeper_path + "/columns", columns_version)); - if (settings.use_minimalistic_part_header_in_zookeeper) + if (storage_settings->use_minimalistic_part_header_in_zookeeper) { ops.emplace_back(zkutil::makeCreateRequest( replica_path + "/parts/" + part->name, @@ -4985,11 +4996,12 @@ void StorageReplicatedMergeTree::updatePartHeaderInZooKeeperAndCommit( AlterDataPartTransaction & transaction) { String part_path = replica_path + "/parts/" + transaction.getPartName(); + const auto storage_settings = getCOWSettings(); bool need_delete_columns_and_checksums_nodes = false; try { - if (settings.use_minimalistic_part_header_in_zookeeper) + if (storage_settings->use_minimalistic_part_header_in_zookeeper) { auto part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums( transaction.getNewColumns(), transaction.getNewChecksums()); @@ -5169,8 +5181,9 @@ CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, const C bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const { - return settings.index_granularity_bytes != 0 && - (settings.enable_mixed_granularity_parts || + const auto storage_settings = getCOWSettings(); + return storage_settings->index_granularity_bytes != 0 && + (storage_settings->enable_mixed_granularity_parts || (!has_non_adaptive_index_granularity_parts && !other_replicas_fixed_granularity)); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 7f632ab4cb4..a32a9f19473 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -534,7 +534,7 @@ protected: const ASTPtr & sample_by_ast_, const ASTPtr & table_ttl_ast_, const MergingParams & merging_params_, - const MergeTreeSettings & settings_, + MergeTreeSettingsPtr settings_, bool has_force_restore_data_flag); }; From 9e78781378fba7cae91239ffd02cdfcbd5314e57 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Aug 2019 13:39:06 +0300 Subject: [PATCH 183/509] Fix tests --- dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql | 4 ++-- .../0_stateless/00980_zookeeper_merge_tree_alter_settings.sql | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql index 866cf960710..973d9b95d20 100644 --- a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql +++ b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql @@ -5,7 +5,7 @@ CREATE TABLE log_for_alter ( Data String ) ENGINE = Log(); -ALTER TABLE log_for_alter MODIFY SETTING aaa=123; -- { serverError 468 } +ALTER TABLE log_for_alter MODIFY SETTING aaa=123; -- { serverError 469 } DROP TABLE IF EXISTS log_for_alter; @@ -16,7 +16,7 @@ CREATE TABLE table_for_alter ( Data String ) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096; -ALTER TABLE table_for_alter MODIFY SETTING index_granularity=555; -- { serverError 469 } +ALTER TABLE table_for_alter MODIFY SETTING index_granularity=555; -- { serverError 470 } SHOW CREATE TABLE table_for_alter; diff --git a/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.sql b/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.sql index 9c7a7c33329..24c1135c247 100644 --- a/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.sql +++ b/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.sql @@ -13,7 +13,7 @@ CREATE TABLE replicated_table_for_alter2 ( SHOW CREATE TABLE replicated_table_for_alter1; -ALTER TABLE replicated_table_for_alter1 MODIFY SETTING index_granularity = 4096; -- { serverError 469 } +ALTER TABLE replicated_table_for_alter1 MODIFY SETTING index_granularity = 4096; -- { serverError 470 } SHOW CREATE TABLE replicated_table_for_alter1; From 1b68d79c59e7037f52263a3d1ae1331037cb16f0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Aug 2019 13:56:58 +0300 Subject: [PATCH 184/509] Revert immutable settings --- dbms/src/Core/SettingsCommon.cpp | 129 +++++------------------------- dbms/src/Core/SettingsCommon.h | 133 +++++++------------------------ 2 files changed, 49 insertions(+), 213 deletions(-) diff --git a/dbms/src/Core/SettingsCommon.cpp b/dbms/src/Core/SettingsCommon.cpp index 2d3ed4f6e14..988a4b2d736 100644 --- a/dbms/src/Core/SettingsCommon.cpp +++ b/dbms/src/Core/SettingsCommon.cpp @@ -34,19 +34,20 @@ namespace ErrorCodes template String SettingNumber::toString() const { - return DB::toString(getValue()); + return DB::toString(value); } template Field SettingNumber::toField() const { - return getValue(); + return value; } template void SettingNumber::set(Type x) { - data.store(Data{x, true}, std::memory_order_relaxed); + value = x; + changed = true; } template @@ -58,14 +59,6 @@ void SettingNumber::set(const Field & x) set(applyVisitor(FieldVisitorConvertToNumber(), x)); } - -template -SettingNumber & SettingNumber::operator= (const SettingNumber & o) -{ - data.store(o.data.load(std::memory_order_relaxed), std::memory_order_relaxed); - return *this; -} - template void SettingNumber::set(const String & x) { @@ -100,9 +93,9 @@ template void SettingNumber::serialize(WriteBuffer & buf) const { if constexpr (std::is_integral_v && std::is_unsigned_v) - writeVarUInt(static_cast(getValue()), buf); + writeVarUInt(static_cast(value), buf); else if constexpr (std::is_integral_v && std::is_signed_v) - writeVarInt(static_cast(getValue()), buf); + writeVarInt(static_cast(value), buf); else { static_assert(std::is_floating_point_v); @@ -140,28 +133,22 @@ template struct SettingNumber; template struct SettingNumber; -SettingMaxThreads & SettingMaxThreads::operator= (const SettingMaxThreads & o) -{ - data.store(o.data.load(std::memory_order_relaxed), std::memory_order_relaxed); - return *this; -} - String SettingMaxThreads::toString() const { - auto d = data.load(std::memory_order_relaxed); /// Instead of the `auto` value, we output the actual value to make it easier to see. - return d.is_auto ? ("auto(" + DB::toString(d.value) + ")") : DB::toString(d.value); + return is_auto ? ("auto(" + DB::toString(value) + ")") : DB::toString(value); } Field SettingMaxThreads::toField() const { - auto d = data.load(std::memory_order_relaxed); - return d.is_auto ? 0 : d.value; + return is_auto ? 0 : value; } void SettingMaxThreads::set(UInt64 x) { - data.store({x ? x : getAutoValue(), x == 0, true}); + value = x ? x : getAutoValue(); + is_auto = x == 0; + changed = true; } void SettingMaxThreads::set(const Field & x) @@ -182,8 +169,7 @@ void SettingMaxThreads::set(const String & x) void SettingMaxThreads::serialize(WriteBuffer & buf) const { - auto d = data.load(std::memory_order_relaxed); - writeVarUInt(d.is_auto ? 0 : d.value, buf); + writeVarUInt(is_auto ? 0 : value, buf); } void SettingMaxThreads::deserialize(ReadBuffer & buf) @@ -195,7 +181,8 @@ void SettingMaxThreads::deserialize(ReadBuffer & buf) void SettingMaxThreads::setAuto() { - data.store({getAutoValue(), true, isChanged()}); + value = getAutoValue(); + is_auto = true; } UInt64 SettingMaxThreads::getAutoValue() const @@ -204,54 +191,22 @@ UInt64 SettingMaxThreads::getAutoValue() const return res; } -void SettingMaxThreads::setChanged(bool changed) -{ - auto d = data.load(std::memory_order_relaxed); - data.store({d.value, d.is_auto, changed}); -} - - -template -SettingTimespan & SettingTimespan::operator= (const SettingTimespan & o) -{ - std::shared_lock lock_o(o.mutex); - value = o.value; - changed = o.changed; - return *this; -} - -template -SettingTimespan::SettingTimespan(const SettingTimespan & o) -{ - std::shared_lock lock_o(o.mutex); - value = o.value; - changed = o.changed; -} - - -template -void SettingTimespan::setChanged(bool c) -{ - std::unique_lock lock(mutex); - changed = c; -} template String SettingTimespan::toString() const { - return DB::toString(getValue().totalMicroseconds() / microseconds_per_io_unit); + return DB::toString(value.totalMicroseconds() / microseconds_per_io_unit); } template Field SettingTimespan::toField() const { - return getValue().totalMicroseconds() / microseconds_per_io_unit; + return value.totalMicroseconds() / microseconds_per_io_unit; } template void SettingTimespan::set(const Poco::Timespan & x) { - std::unique_lock lock(mutex); value = x; changed = true; } @@ -280,7 +235,7 @@ void SettingTimespan::set(const String & x) template void SettingTimespan::serialize(WriteBuffer & buf) const { - writeVarUInt(getValue().totalMicroseconds() / microseconds_per_io_unit, buf); + writeVarUInt(value.totalMicroseconds() / microseconds_per_io_unit, buf); } template @@ -294,47 +249,23 @@ void SettingTimespan::deserialize(ReadBuffer & buf) template struct SettingTimespan; template struct SettingTimespan; -SettingString & SettingString::operator= (const SettingString & o) -{ - std::shared_lock lock_o(o.mutex); - value = o.value; - changed = o.changed; - return *this; -} - -SettingString::SettingString(const SettingString & o) -{ - std::shared_lock lock(o.mutex); - value = o.value; - changed = o.changed; -} - String SettingString::toString() const { - std::shared_lock lock(mutex); return value; } Field SettingString::toField() const { - std::shared_lock lock(mutex); return value; } void SettingString::set(const String & x) { - std::unique_lock lock(mutex); value = x; changed = true; } -void SettingString::setChanged(bool c) -{ - std::unique_lock lock(mutex); - changed = c; -} - void SettingString::set(const Field & x) { set(safeGet(x)); @@ -352,15 +283,10 @@ void SettingString::deserialize(ReadBuffer & buf) set(s); } -SettingChar & SettingChar::operator= (const SettingChar & o) -{ - data.store(o.data.load(std::memory_order_relaxed), std::memory_order_relaxed); - return *this; -} String SettingChar::toString() const { - return String(1, getValue()); + return String(1, value); } Field SettingChar::toField() const @@ -370,7 +296,8 @@ Field SettingChar::toField() const void SettingChar::set(char x) { - data.store({x, true}); + value = x; + changed = true; } void SettingChar::set(const String & x) @@ -400,19 +327,6 @@ void SettingChar::deserialize(ReadBuffer & buf) } -template -SettingEnum & SettingEnum::operator= (const SettingEnum & o) -{ - data.store(o.data.load(std::memory_order_relaxed), std::memory_order_relaxed); - return *this; -} - -template -void SettingEnum::set(EnumType x) -{ - data.store({x, true}, std::memory_order_relaxed); -} - template void SettingEnum::serialize(WriteBuffer & buf) const { @@ -428,7 +342,6 @@ void SettingEnum::deserialize(ReadBuffer & buf) } - #define IMPLEMENT_SETTING_ENUM(ENUM_NAME, LIST_OF_NAMES_MACRO, ERROR_CODE_FOR_UNEXPECTED_NAME) \ IMPLEMENT_SETTING_ENUM_WITH_TAG(ENUM_NAME, void, LIST_OF_NAMES_MACRO, ERROR_CODE_FOR_UNEXPECTED_NAME) @@ -438,7 +351,7 @@ void SettingEnum::deserialize(ReadBuffer & buf) { \ using EnumType = ENUM_NAME; \ using UnderlyingType = std::underlying_type::type; \ - switch (static_cast(getValue())) \ + switch (static_cast(value)) \ { \ LIST_OF_NAMES_MACRO(IMPLEMENT_SETTING_ENUM_TO_STRING_HELPER_) \ } \ diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index 83d90c28f53..d4607e70904 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -8,8 +8,6 @@ #include #include #include -#include -#include namespace DB @@ -34,25 +32,13 @@ namespace ErrorCodes template struct SettingNumber { - struct Data - { - Type value; - bool changed; - }; + Type value; + bool changed = false; - std::atomic data; - - SettingNumber(Type x = 0) : data{{x, false}} {} - SettingNumber(const SettingNumber & o) : data{o.data.load(std::memory_order_relaxed)} {} - - bool isChanged() const { return data.load(std::memory_order_relaxed).changed; } - void setChanged(bool changed) { data.store({getValue(), changed}, std::memory_order_relaxed); } - - operator Type() const { return getValue(); } - Type getValue() const { return data.load(std::memory_order_relaxed).value; } + SettingNumber(Type x = 0) : value(x) {} + operator Type() const { return value; } SettingNumber & operator= (Type x) { set(x); return *this; } - SettingNumber & operator= (const SettingNumber & o); /// Serialize to a test string. String toString() const; @@ -87,26 +73,14 @@ using SettingBool = SettingNumber; */ struct SettingMaxThreads { - struct Data - { - UInt64 value; - bool is_auto; - bool changed; - }; + UInt64 value; + bool is_auto; + bool changed = false; - std::atomic data; - - SettingMaxThreads(UInt64 x = 0) : data{{x ? x : getAutoValue(), x == 0, false}} {} - SettingMaxThreads(const SettingMaxThreads & o) : data{o.data.load(std::memory_order_relaxed)} {} - - bool isChanged() const { return data.load(std::memory_order_relaxed).changed; } - void setChanged(bool changed); - - operator UInt64() const { return getValue(); } - UInt64 getValue() const { return data.load(std::memory_order_relaxed).value; } + SettingMaxThreads(UInt64 x = 0) : value(x ? x : getAutoValue()), is_auto(x == 0) {} + operator UInt64() const { return value; } SettingMaxThreads & operator= (UInt64 x) { set(x); return *this; } - SettingMaxThreads & operator= (const SettingMaxThreads & o); String toString() const; Field toField() const; @@ -118,7 +92,6 @@ struct SettingMaxThreads void serialize(WriteBuffer & buf) const; void deserialize(ReadBuffer & buf); - bool isAuto() const { return data.load(std::memory_order_relaxed).is_auto; } void setAuto(); UInt64 getAutoValue() const; }; @@ -129,37 +102,16 @@ enum class SettingTimespanIO { MILLISECOND, SECOND }; template struct SettingTimespan { - mutable std::shared_mutex mutex; Poco::Timespan value; bool changed = false; SettingTimespan(UInt64 x = 0) : value(x * microseconds_per_io_unit) {} - SettingTimespan(const SettingTimespan & o); - operator Poco::Timespan() const { return getValue(); } - Poco::Timespan getValue() const { std::shared_lock lock(mutex); return value; } + operator Poco::Timespan() const { return value; } SettingTimespan & operator= (const Poco::Timespan & x) { set(x); return *this; } - SettingTimespan & operator= (const SettingTimespan & o); - Poco::Timespan::TimeDiff totalSeconds() const - { - std::shared_lock lock(mutex); - return value.totalSeconds(); - } - - Poco::Timespan::TimeDiff totalMilliseconds() const - { - std::shared_lock lock(mutex); - return value.totalMilliseconds(); - } - - bool isChanged() const - { - std::shared_lock lock(mutex); - return changed; - } - - void setChanged(bool changed); + Poco::Timespan::TimeDiff totalSeconds() const { return value.totalSeconds(); } + Poco::Timespan::TimeDiff totalMilliseconds() const { return value.totalMilliseconds(); } String toString() const; Field toField() const; @@ -182,19 +134,13 @@ using SettingMilliseconds = SettingTimespan; struct SettingString { - mutable std::shared_mutex mutex; String value; bool changed = false; SettingString(const String & x = String{}) : value(x) {} - SettingString(const SettingString & o); - operator String() const { return getValue(); } - String getValue() const { std::shared_lock lock(mutex); return value; } + operator String() const { return value; } SettingString & operator= (const String & x) { set(x); return *this; } - SettingString & operator= (const SettingString & o); - bool isChanged() const { std::shared_lock lock(mutex); return changed; } - void setChanged(bool changed); String toString() const; Field toField() const; @@ -210,25 +156,13 @@ struct SettingString struct SettingChar { public: - struct Data - { - char value; - bool changed; - }; + char value; + bool changed = false; - std::atomic data; - - SettingChar(char x = '\0') : data({x, false}) {} - SettingChar(const SettingChar & o) : data{o.data.load(std::memory_order_relaxed)} {} - - operator char() const { return getValue(); } - char getValue() const { return data.load(std::memory_order_relaxed).value; } + SettingChar(char x = '\0') : value(x) {} + operator char() const { return value; } SettingChar & operator= (char x) { set(x); return *this; } - SettingChar & operator= (const SettingChar & o); - - bool isChanged() const { return data.load(std::memory_order_relaxed).changed; } - void setChanged(bool changed) { data.store({getValue(), changed}, std::memory_order_relaxed);} String toString() const; Field toField() const; @@ -246,30 +180,18 @@ public: template struct SettingEnum { - struct Data - { - EnumType value; - bool changed; - }; + EnumType value; + bool changed = false; - std::atomic data; - - SettingEnum(EnumType x) : data({x, false}) {} - SettingEnum(const SettingEnum & o) : data{o.data.load(std::memory_order_relaxed)} {} - - operator EnumType() const { return getValue(); } - EnumType getValue() const { return data.load(std::memory_order_relaxed).value; } + SettingEnum(EnumType x) : value(x) {} + operator EnumType() const { return value; } SettingEnum & operator= (EnumType x) { set(x); return *this; } - SettingEnum & operator= (const SettingEnum & o); - - bool isChanged() const { return data.load(std::memory_order_relaxed).changed; } - void setChanged(bool changed) { data.store({getValue(), changed}, std::memory_order_relaxed);} String toString() const; Field toField() const { return toString(); } - void set(EnumType x); + void set(EnumType x) { value = x; changed = true; } void set(const Field & x) { set(safeGet(x)); } void set(const String & x); @@ -386,7 +308,6 @@ private: Derived & castToDerived() { return *static_cast(this); } const Derived & castToDerived() const { return *static_cast(this); } - using IsChangedFunction = bool (*)(const Derived &); using GetStringFunction = String (*)(const Derived &); using GetFieldFunction = Field (*)(const Derived &); using SetStringFunction = void (*)(Derived &, const String &); @@ -397,7 +318,7 @@ private: struct MemberInfo { - IsChangedFunction is_changed; + size_t offset_of_changed; StringRef name; StringRef description; /// Can be updated after first load for config/definition. @@ -412,7 +333,7 @@ private: DeserializeFunction deserialize; CastValueWithoutApplyingFunction cast_value_without_applying; - bool isChanged(const Derived & collection) const { return is_changed(collection); } + bool isChanged(const Derived & collection) const { return *reinterpret_cast(reinterpret_cast(&collection) + offset_of_changed); } }; class MemberInfos @@ -772,7 +693,8 @@ public: #define IMPLEMENT_SETTINGS_COLLECTION_ADD_MUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ - add({[](const Derived & d) { return d.NAME.isChanged(); }, \ + static_assert(std::is_same_v().NAME.changed), bool>); \ + add({offsetof(Derived, NAME.changed), \ StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), true, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \ @@ -780,7 +702,8 @@ public: &Functions::NAME##_castValueWithoutApplying }); #define IMPLEMENT_SETTINGS_COLLECTION_ADD_IMMUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ - add({[](const Derived & d) { return d.NAME.isChanged(); }, \ + static_assert(std::is_same_v().NAME.changed), bool>); \ + add({offsetof(Derived, NAME.changed), \ StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), false, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \ From 7b1039568ce987f02329e347251e779070d67380 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Aug 2019 14:24:18 +0300 Subject: [PATCH 185/509] Revert old changed --- dbms/programs/client/Client.cpp | 4 ++-- dbms/programs/copier/ClusterCopier.cpp | 2 +- dbms/src/Core/SettingsCommon.h | 13 ++++++------- dbms/src/Interpreters/ClusterProxy/executeQuery.cpp | 6 +++--- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 2 +- dbms/src/Storages/Kafka/StorageKafka.cpp | 4 ++-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 ++-- dbms/src/Storages/MergeTree/MergeTreeSettings.cpp | 2 +- 8 files changed, 18 insertions(+), 19 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index f509ebf8c1f..feffd7e6cfe 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -225,7 +225,7 @@ private: for (auto && setting : context.getSettingsRef()) { const String & name = setting.getName().toString(); - if (config().has(name) && !setting.isChanged()) + if (config().has(name) && setting.isChanged()) setting.setValue(config().getString(name)); } @@ -323,7 +323,7 @@ private: insert_format = "Values"; /// Setting value from cmd arg overrides one from config - if (context.getSettingsRef().max_insert_block_size.isChanged()) + if (context.getSettingsRef().max_insert_block_size.changed) insert_format_max_block_size = context.getSettingsRef().max_insert_block_size; else insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size); diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 5ced5677569..435d06da854 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -708,7 +708,7 @@ void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration & c auto set_default_value = [] (auto && setting, auto && default_value) { - setting = setting.isChanged() ? setting.getValue() : default_value; + setting = setting.changed ? setting.value : default_value; }; /// Override important settings diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index d4607e70904..1410dc2611f 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -308,6 +308,7 @@ private: Derived & castToDerived() { return *static_cast(this); } const Derived & castToDerived() const { return *static_cast(this); } + using IsChangedFunction = bool (*)(const Derived &); using GetStringFunction = String (*)(const Derived &); using GetFieldFunction = Field (*)(const Derived &); using SetStringFunction = void (*)(Derived &, const String &); @@ -318,11 +319,11 @@ private: struct MemberInfo { - size_t offset_of_changed; + IsChangedFunction is_changed; StringRef name; StringRef description; /// Can be updated after first load for config/definition. - /// Non updatable settings can be isChanged, + /// Non updatable settings can be `changed`, /// if they were overwritten in config/definition. bool updateable; GetStringFunction get_string; @@ -333,7 +334,7 @@ private: DeserializeFunction deserialize; CastValueWithoutApplyingFunction cast_value_without_applying; - bool isChanged(const Derived & collection) const { return *reinterpret_cast(reinterpret_cast(&collection) + offset_of_changed); } + bool isChanged(const Derived & collection) const { return is_changed(collection); } }; class MemberInfos @@ -693,8 +694,7 @@ public: #define IMPLEMENT_SETTINGS_COLLECTION_ADD_MUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ - static_assert(std::is_same_v().NAME.changed), bool>); \ - add({offsetof(Derived, NAME.changed), \ + add({[](const Derived & d) { return d.NAME.changed; }, \ StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), true, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \ @@ -702,8 +702,7 @@ public: &Functions::NAME##_castValueWithoutApplying }); #define IMPLEMENT_SETTINGS_COLLECTION_ADD_IMMUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ - static_assert(std::is_same_v().NAME.changed), bool>); \ - add({offsetof(Derived, NAME.changed), \ + add({[](const Derived & d) { return d.NAME.changed; }, \ StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), false, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \ diff --git a/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp b/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp index e4e52eab2b7..25d5a6eb0d4 100644 --- a/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -26,9 +26,9 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin new_settings.max_memory_usage_for_all_queries = 0; /// Set as unchanged to avoid sending to remote server. - new_settings.max_concurrent_queries_for_user.setChanged(false); - new_settings.max_memory_usage_for_user.setChanged(false); - new_settings.max_memory_usage_for_all_queries.setChanged(false); + new_settings.max_concurrent_queries_for_user.changed = false; + new_settings.max_memory_usage_for_user.changed = false; + new_settings.max_memory_usage_for_all_queries = false; Context new_context(context); new_context.setSettings(new_settings); diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index dfd0ec63643..282b19991b1 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -788,7 +788,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( InJoinSubqueriesPreprocessor(context).visit(query); /// Optimizes logical expressions. - LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.getValue()).perform(); + LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform(); } /// Creates a dictionary `aliases`: alias -> ASTPtr diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index afd26fde113..f5a9c2ac4b6 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -440,7 +440,7 @@ void registerStorageKafka(StorageFactory & factory) #define CHECK_KAFKA_STORAGE_ARGUMENT(ARG_NUM, PAR_NAME) \ /* One of the four required arguments is not specified */ \ if (args_count < ARG_NUM && ARG_NUM <= 4 && \ - !kafka_settings.PAR_NAME.isChanged()) \ + !kafka_settings.PAR_NAME.changed) \ { \ throw Exception( \ "Required parameter '" #PAR_NAME "' " \ @@ -449,7 +449,7 @@ void registerStorageKafka(StorageFactory & factory) } \ /* The same argument is given in two places */ \ if (has_settings && \ - kafka_settings.PAR_NAME.isChanged() && \ + kafka_settings.PAR_NAME.changed && \ args_count >= ARG_NUM) \ { \ throw Exception( \ diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 314967b485d..15a19bbc924 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -400,8 +400,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( if (relative_sample_size == RelativeSize(0)) relative_sample_size = 1; - relative_sample_size /= settings.parallel_replicas_count.getValue(); - relative_sample_offset += relative_sample_size * RelativeSize(settings.parallel_replica_offset.getValue()); + relative_sample_size /= settings.parallel_replicas_count.value; + relative_sample_offset += relative_sample_size * RelativeSize(settings.parallel_replica_offset.value); } if (relative_sample_offset >= RelativeSize(1)) diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp index 7c3c7ef02cc..5bc44eee842 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -69,7 +69,7 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) if (std::find_if(changes.begin(), changes.end(), \ [](const SettingChange & c) { return c.name == #NAME; }) \ == changes.end()) \ - changes.push_back(SettingChange{#NAME, NAME.getValue()}); + changes.push_back(SettingChange{#NAME, NAME.value}); APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(ADD_IF_ABSENT) #undef ADD_IF_ABSENT From a4d6e8d5b2f9f53487599ad510f471d4e5589492 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Aug 2019 14:28:28 +0300 Subject: [PATCH 186/509] Revert latomic --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index ac0224ac56b..d369dca7e78 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -350,7 +350,7 @@ if (OS_LINUX AND NOT UNBUNDLED AND (GLIBC_COMPATIBILITY OR USE_INTERNAL_UNWIND_L endif () # Add Libc. GLIBC is actually a collection of interdependent libraries. - set (DEFAULT_LIBS "${DEFAULT_LIBS} -lrt -ldl -lpthread -lm -lc -latomic") + set (DEFAULT_LIBS "${DEFAULT_LIBS} -lrt -ldl -lpthread -lm -lc") # Note: we'd rather use Musl libc library, but it's little bit more difficult to use. From bf5ec73582d413d9bb3dca1b32877bcad86eae86 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Aug 2019 14:40:45 +0300 Subject: [PATCH 187/509] Fix clang build --- dbms/src/Interpreters/Context.cpp | 7 ++++--- dbms/src/Storages/MergeTree/MergeTreeSettings.h | 4 +++- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 0b0cf657987..17b031dbd64 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -148,7 +148,7 @@ struct ContextShared std::unique_ptr ddl_worker; /// Process ddl commands from zk. /// Rules for selecting the compression settings, depending on the size of the part. mutable std::unique_ptr compression_codec_selector; - std::optional merge_tree_settings; /// Settings of MergeTree* engines. + MergeTreeSettingsPtr merge_tree_settings; /// Settings of MergeTree* engines. size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default) size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default) String format_schema_path; /// Path to a directory that contains schema files used by input formats. @@ -1759,8 +1759,9 @@ const MergeTreeSettings & Context::getMergeTreeSettings() const if (!shared->merge_tree_settings) { auto & config = getConfigRef(); - shared->merge_tree_settings.emplace(); - shared->merge_tree_settings->loadFromConfig("merge_tree", config); + MutableMergeTreeSettingsPtr settings_ptr = MergeTreeSettings::create(); + settings_ptr->loadFromConfig("merge_tree", config); + shared->merge_tree_settings = std::move(settings_ptr); } return *shared->merge_tree_settings; diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index dff7cc2e523..c098aac2526 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -101,7 +101,9 @@ struct MergeTreeSettings : public SettingsCollection, public void loadFromQuery(ASTStorage & storage_def); MutablePtr clone() const; - ~MergeTreeSettings() {} +private: + MergeTreeSettings() = default; + MergeTreeSettings(const MergeTreeSettings & o) = default; }; using MergeTreeSettingsPtr = MergeTreeSettings::Ptr; From 5b85373c566293aa52cd6720525f35021d57b619 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Aug 2019 14:58:59 +0300 Subject: [PATCH 188/509] Fix stupid bug --- dbms/programs/client/Client.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index feffd7e6cfe..cf72d7a87c3 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -225,7 +225,7 @@ private: for (auto && setting : context.getSettingsRef()) { const String & name = setting.getName().toString(); - if (config().has(name) && setting.isChanged()) + if (config().has(name) && !setting.isChanged()) setting.setValue(config().getString(name)); } From b00903330fe26276049929cb30079129e5d8bf85 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Aug 2019 15:03:42 +0300 Subject: [PATCH 189/509] Remove empty block --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 4d47f60a2fe..97d74d7a037 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1644,12 +1644,11 @@ void MergeTreeData::alterSettings( const Context & context, TableStructureWriteLockHolder & table_lock_holder) { - { - MutableMergeTreeSettingsPtr settings = std::move(*guarded_settings.getPtr()).mutate(); - settings->updateFromChanges(new_changes); - IStorage::alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); - guarded_settings.setPtr(std::move(settings)); - } + /// No additional locking required, because we took table_lock_holder + MutableMergeTreeSettingsPtr settings = std::move(*guarded_settings.getPtr()).mutate(); + settings->updateFromChanges(new_changes); + IStorage::alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); + guarded_settings.setPtr(std::move(settings)); } bool MergeTreeData::hasSetting(const String & setting_name) const From f988090111cf28fb8a8a300cabda0aa0d5d8ae21 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Aug 2019 15:10:58 +0300 Subject: [PATCH 190/509] Remove redundant move --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 97d74d7a037..aab384552cc 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -118,7 +118,7 @@ MergeTreeData::MergeTreeData( full_path(full_path_), broken_part_callback(broken_part_callback_), log_name(database_name + "." + table_name), log(&Logger::get(log_name)), - guarded_settings(std::move(settings_)), + guarded_settings(settings_), data_parts_by_info(data_parts_indexes.get()), data_parts_by_state_and_info(data_parts_indexes.get()) { From ea9cf3a62f42f0ed48efe7baac2be46bfaa7ae5e Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Tue, 13 Aug 2019 16:11:24 +0300 Subject: [PATCH 191/509] Done --- dbms/src/Functions/neighbour.cpp | 105 +++++++++--------- .../0_stateless/00957_neighbour.reference | 42 +++++++ .../queries/0_stateless/00957_neighbour.sql | 30 +++++ .../0_stateless/00957_next_in_block.reference | 12 -- .../0_stateless/00957_next_in_block.sql | 18 --- .../functions/other_functions.md | 43 +++++++ .../functions/other_functions.md | 42 +++++++ 7 files changed, 209 insertions(+), 83 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00957_neighbour.reference create mode 100644 dbms/tests/queries/0_stateless/00957_neighbour.sql delete mode 100644 dbms/tests/queries/0_stateless/00957_next_in_block.reference delete mode 100644 dbms/tests/queries/0_stateless/00957_next_in_block.sql diff --git a/dbms/src/Functions/neighbour.cpp b/dbms/src/Functions/neighbour.cpp index 3eff660c996..79a1feec002 100644 --- a/dbms/src/Functions/neighbour.cpp +++ b/dbms/src/Functions/neighbour.cpp @@ -119,9 +119,6 @@ public: default_values_column = block.getByPosition(arguments[2]).column; } -// Field offset_field = (*block.getByPosition(arguments[1]).column)[0]; -// auto raw_value = safeGet(offset_field); - ColumnWithTypeAndName &source_column_name_and_type = block.getByPosition(arguments[0]); DataTypes types = {source_column_name_and_type.type}; if (default_values_column) @@ -140,72 +137,74 @@ public: default_values_column = castColumn(block.getByPosition(arguments[2]), result_type, context); } - auto column = result_type->createColumn(); - column->reserve(input_rows_count); - const DataTypePtr desired_type = std::make_shared(); if (!block.getByPosition(arguments[1]).type->equals(*desired_type)) { offset_column = castColumn(offset_structure, desired_type, context); } - - // with constant offset - insertRangeFrom - if (is_constant_offset) - { - Int64 offset_value = offset_column->getInt(0); - if (offset_value > 0) - { - // insert shifted value - if ((size_t)std::abs(offset_value) <= input_rows_count) { - column->insertRangeFrom(*source_column, offset_value, input_rows_count - offset_value); - // insert defaults into the end - insertDefaults(column, input_rows_count - offset_value, default_values_column, offset_value); - } - } else if(offset_value < 0) { - // insert defaults up to offset_value - insertDefaults(column, input_rows_count - std::abs(offset_value), default_values_column, std::abs(offset_value)); - // insert range, where possible - column->insertRangeFrom(*source_column, 0, input_rows_count - std::abs(offset_value)); - } else { - // populate column with source values - column->insertRangeFrom(*source_column, 0, input_rows_count); - } + if (isColumnConst(*source_column)) { + auto column = result_type->createColumnConst(input_rows_count, (*source_column)[0]); + block.getByPosition(result).column = std::move(column); } else { - // with dynamic offset - handle row by row - for (size_t row = 0; row < input_rows_count; row++) + auto column = result_type->createColumn(); + column->reserve(input_rows_count); + // with constant offset - insertRangeFrom + if (is_constant_offset) { - Int64 offset_value = offset_column->getInt(row); - if (offset_value == 0) { - column->insertFrom(*source_column, row); - } else if (offset_value > 0) { - size_t real_offset = row + offset_value; - if (real_offset > input_rows_count) { - if (default_values_column) { - column->insertFrom(*default_values_column, row); - } else { - column->insertDefault(); - } - } else { - column->insertFrom(*column, real_offset); + Int64 offset_value = offset_column->getInt(0); + + if (offset_value > 0) + { + // insert shifted value + if ((size_t)offset_value <= input_rows_count) { + column->insertRangeFrom(*source_column, offset_value, input_rows_count - offset_value); } + size_t row_count = (size_t)offset_value > input_rows_count ? input_rows_count : offset_value; + insertDefaults(column, row_count, default_values_column, input_rows_count - row_count); + } else if (offset_value < 0) { + size_t row_count = (size_t)std::abs(offset_value) > input_rows_count ? input_rows_count : std::abs(offset_value); + // insert defaults up to offset_value + insertDefaults(column, row_count, default_values_column, 0); + column->insertRangeFrom(*source_column, 0, input_rows_count - row_count); } else { - // out of range - if ((size_t)std::abs(offset_value) > row) - { - if (default_values_column) { - column->insertFrom(*default_values_column, row); + // populate column with source values + column->insertRangeFrom(*source_column, 0, input_rows_count); + } + } else { + // with dynamic offset - handle row by row + for (size_t row = 0; row < input_rows_count; row++) + { + Int64 offset_value = offset_column->getInt(row); + if (offset_value == 0) { + column->insertFrom(*source_column, row); + } else if (offset_value > 0) { + size_t real_offset = row + offset_value; + if (real_offset > input_rows_count) { + if (default_values_column) { + column->insertFrom(*default_values_column, row); + } else { + column->insertDefault(); + } } else { - column->insertDefault(); + column->insertFrom(*column, real_offset); } } else { - column->insertFrom(*column, row - std::abs(offset_value)); + // out of range + if ((size_t)std::abs(offset_value) > row) + { + if (default_values_column) { + column->insertFrom(*default_values_column, row); + } else { + column->insertDefault(); + } + } else { + column->insertFrom(*column, row - std::abs(offset_value)); + } } } } + block.getByPosition(result).column = std::move(column); } - - - block.getByPosition(result).column = std::move(column); } private: const Context & context; diff --git a/dbms/tests/queries/0_stateless/00957_neighbour.reference b/dbms/tests/queries/0_stateless/00957_neighbour.reference new file mode 100644 index 00000000000..cd8c6310f22 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00957_neighbour.reference @@ -0,0 +1,42 @@ +Result with different type +0 1 +1 2 +2 -10 +Offset > block +0 0 +1 0 +2 0 +Abs(Offset) > block +0 0 +1 0 +2 0 +Positive offset +0 1 +1 2 +2 0 +Negative offset +0 1 +1 2 +2 0 +Positive offset with defaults +0 2 +1 3 +2 12 +3 13 +Negative offset with defaults +0 10 +1 11 +2 0 +3 1 +Positive offset with const defaults +0 1 +1 2 +2 1000 +Negative offset with const defaults +0 1000 +1 0 +2 1 +Constant column +0 1000 +1 1000 +2 1000 diff --git a/dbms/tests/queries/0_stateless/00957_neighbour.sql b/dbms/tests/queries/0_stateless/00957_neighbour.sql new file mode 100644 index 00000000000..156672155a8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00957_neighbour.sql @@ -0,0 +1,30 @@ +-- no arguments +select neighbour(); -- { serverError 42 } +-- single argument +select neighbour(1); -- { serverError 42 } +-- greater than 3 arguments +select neighbour(1,2,3,4); -- { serverError 42 } +-- bad default value +select neighbour(dummy, 1, 'hello'); -- { serverError 43 } +-- types without common supertype (UInt64 and Int8) +select number, neighbour(number, 1, -10) from numbers(3); -- { serverError 43 } +select 'Result with different type'; +select toInt32(number) as n, neighbour(n, 1, -10) from numbers(3); +select 'Offset > block'; +select number, neighbour(number, 10) from numbers(3); +select 'Abs(Offset) > block'; +select number, neighbour(number, -10) from numbers(3); +select 'Positive offset'; +select number, neighbour(number, 1) from numbers(3); +select 'Negative offset'; +select number, neighbour(number, 1) from numbers(3); +select 'Positive offset with defaults'; +select number, neighbour(number, 2, number + 10) from numbers(4); +select 'Negative offset with defaults'; +select number, neighbour(number, -2, number + 10) from numbers(4); +select 'Positive offset with const defaults'; +select number, neighbour(number, 1, 1000) from numbers(3); +select 'Negative offset with const defaults'; +select number, neighbour(number, -1, 1000) from numbers(3); +select 'Constant column'; +select number, neighbour(1000, 10) from numbers(3); \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/00957_next_in_block.reference b/dbms/tests/queries/0_stateless/00957_next_in_block.reference deleted file mode 100644 index 860ce6dc1ba..00000000000 --- a/dbms/tests/queries/0_stateless/00957_next_in_block.reference +++ /dev/null @@ -1,12 +0,0 @@ -0 1 -1 0 -0 2 -1 0 -2 0 -0 0 -1 0 -0 2 -1 3 -2 4 -3 1000 -4 1000 diff --git a/dbms/tests/queries/0_stateless/00957_next_in_block.sql b/dbms/tests/queries/0_stateless/00957_next_in_block.sql deleted file mode 100644 index 1efda43339e..00000000000 --- a/dbms/tests/queries/0_stateless/00957_next_in_block.sql +++ /dev/null @@ -1,18 +0,0 @@ --- no arguments -select neighbour(); -- { serverError 42 } --- single argument -select neighbour(1); -- { serverError 42 } --- greater than 3 arguments -select neighbour(1,2,3,4); -- { serverError 42 } --- bad default value -select neighbour(dummy, 1, 'hello'); -- { serverError 43 } --- single argument test -select number, neighbour(number,1) from numbers(2); --- filling by column's default value -select number, neighbour(number, 2) from numbers(3); --- offset is greater that block - should fill everything with defaults -select number, neighbour(number, 5) from numbers(2); --- substitution by constant for missing values -select number, neighbour(number, 2, 1000) from numbers(5); --- substitution by expression --- select number, neighbour(number, 2, number % 2) from numbers(5); \ No newline at end of file diff --git a/docs/en/query_language/functions/other_functions.md b/docs/en/query_language/functions/other_functions.md index 57fa8acfee3..05efe0fceb4 100644 --- a/docs/en/query_language/functions/other_functions.md +++ b/docs/en/query_language/functions/other_functions.md @@ -311,6 +311,49 @@ Returns the ordinal number of the row in the data block. Different data blocks a Returns the ordinal number of the row in the data block. This function only considers the affected data blocks. +## neighbour(column, offset\[, default_value\]) + +Returns value for `column`, in `offset` distance from current row. +This function is a partial implementation of [window functions](https://en.wikipedia.org/wiki/SQL_window_function) LEAD() and LAG(). + +The result of the function depends on the affected data blocks and the order of data in the block. +If you make a subquery with ORDER BY and call the function from outside the subquery, you can get the expected result. + +If `offset` value is outside block bounds, a default value for `column` returned. If `default_value` is given, then it will be used. +This function can be used to compute year-over-year metric value: + +``` sql +WITH toDate('2018-01-01') AS start_date +SELECT + toStartOfMonth(start_date + (number * 32)) AS month, + toInt32(month) % 100 AS money, + neighbour(money, -12) AS prev_year, + round(prev_year / money, 2) AS year_over_year +FROM numbers(16) +``` + +``` +┌──────month─┬─money─┬─prev_year─┬─year_over_year─┐ +│ 2018-01-01 │ 32 │ 0 │ 0 │ +│ 2018-02-01 │ 63 │ 0 │ 0 │ +│ 2018-03-01 │ 91 │ 0 │ 0 │ +│ 2018-04-01 │ 22 │ 0 │ 0 │ +│ 2018-05-01 │ 52 │ 0 │ 0 │ +│ 2018-06-01 │ 83 │ 0 │ 0 │ +│ 2018-07-01 │ 13 │ 0 │ 0 │ +│ 2018-08-01 │ 44 │ 0 │ 0 │ +│ 2018-09-01 │ 75 │ 0 │ 0 │ +│ 2018-10-01 │ 5 │ 0 │ 0 │ +│ 2018-11-01 │ 36 │ 0 │ 0 │ +│ 2018-12-01 │ 66 │ 0 │ 0 │ +│ 2019-01-01 │ 97 │ 32 │ 0.33 │ +│ 2019-02-01 │ 28 │ 63 │ 2.25 │ +│ 2019-03-01 │ 56 │ 91 │ 1.62 │ +│ 2019-04-01 │ 87 │ 22 │ 0.25 │ +└────────────┴───────┴───────────┴────────────────┘ +``` + + ## runningDifference(x) {#other_functions-runningdifference} Calculates the difference between successive row values ​​in the data block. diff --git a/docs/ru/query_language/functions/other_functions.md b/docs/ru/query_language/functions/other_functions.md index 1637c7bda93..6f5b6f1ff0d 100644 --- a/docs/ru/query_language/functions/other_functions.md +++ b/docs/ru/query_language/functions/other_functions.md @@ -288,6 +288,48 @@ SELECT ## rowNumberInAllBlocks() Возвращает порядковый номер строки в блоке данных. Функция учитывает только задействованные блоки данных. +## neighbour(column, offset\[, default_value\]) + +Функция позволяет получить доступ к значению в колонке `column`, находящемуся на смещении `offset` относительно текущей строки. +Является частичной реализацией [оконных функций](https://en.wikipedia.org/wiki/SQL_window_function) LEAD() и LAG(). + +Результат функции зависит от затронутых блоков данных и порядка данных в блоке. +Если сделать подзапрос с ORDER BY и вызывать функцию извне подзапроса, можно будет получить ожидаемый результат. + +Если значение `offset` выходит за пределы блока данных, то берётся значение по-умолчанию для колонки `column`. Если передан параметр `default_value`, то значение берётся из него. +Например, эта функция может использоваться чтобы оценить year-over-year значение показателя: + +``` sql +WITH toDate('2018-01-01') AS start_date +SELECT + toStartOfMonth(start_date + (number * 32)) AS month, + toInt32(month) % 100 AS money, + neighbour(money, -12) AS prev_year, + round(prev_year / money, 2) AS year_over_year +FROM numbers(16) +``` + +``` +┌──────month─┬─money─┬─prev_year─┬─year_over_year─┐ +│ 2018-01-01 │ 32 │ 0 │ 0 │ +│ 2018-02-01 │ 63 │ 0 │ 0 │ +│ 2018-03-01 │ 91 │ 0 │ 0 │ +│ 2018-04-01 │ 22 │ 0 │ 0 │ +│ 2018-05-01 │ 52 │ 0 │ 0 │ +│ 2018-06-01 │ 83 │ 0 │ 0 │ +│ 2018-07-01 │ 13 │ 0 │ 0 │ +│ 2018-08-01 │ 44 │ 0 │ 0 │ +│ 2018-09-01 │ 75 │ 0 │ 0 │ +│ 2018-10-01 │ 5 │ 0 │ 0 │ +│ 2018-11-01 │ 36 │ 0 │ 0 │ +│ 2018-12-01 │ 66 │ 0 │ 0 │ +│ 2019-01-01 │ 97 │ 32 │ 0.33 │ +│ 2019-02-01 │ 28 │ 63 │ 2.25 │ +│ 2019-03-01 │ 56 │ 91 │ 1.62 │ +│ 2019-04-01 │ 87 │ 22 │ 0.25 │ +└────────────┴───────┴───────────┴────────────────┘ +``` + ## runningDifference(x) Считает разницу между последовательными значениями строк в блоке данных. Возвращает 0 для первой строки и разницу с предыдущей строкой для каждой последующей строки. From 6bf3902ce5d814758381bf15689c4585346478f0 Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Tue, 13 Aug 2019 16:20:32 +0300 Subject: [PATCH 192/509] Format file --- dbms/src/Functions/neighbour.cpp | 88 ++++++++++++++++++++++---------- 1 file changed, 62 insertions(+), 26 deletions(-) diff --git a/dbms/src/Functions/neighbour.cpp b/dbms/src/Functions/neighbour.cpp index 79a1feec002..2efca01a66f 100644 --- a/dbms/src/Functions/neighbour.cpp +++ b/dbms/src/Functions/neighbour.cpp @@ -1,10 +1,10 @@ #include +#include #include #include #include #include #include -#include namespace DB { @@ -85,7 +85,8 @@ public: static void insertDefaults(const MutableColumnPtr & target, size_t row_count, ColumnPtr & default_values_column, size_t offset) { - if (row_count == 0) { + if (row_count == 0) + { return; } if (default_values_column) @@ -93,15 +94,20 @@ public: if (isColumnConst(*default_values_column)) { Field constant_value = (*default_values_column)[0]; - for(size_t row = 0; row < row_count;row++) + for (size_t row = 0; row < row_count; row++) { target->insert(constant_value); } - } else { + } + else + { target->insertRangeFrom(*default_values_column, offset, row_count); } - } else { - for(size_t row = 0; row < row_count;row++) { + } + else + { + for (size_t row = 0; row < row_count; row++) + { target->insertDefault(); } } @@ -119,7 +125,7 @@ public: default_values_column = block.getByPosition(arguments[2]).column; } - ColumnWithTypeAndName &source_column_name_and_type = block.getByPosition(arguments[0]); + ColumnWithTypeAndName & source_column_name_and_type = block.getByPosition(arguments[0]); DataTypes types = {source_column_name_and_type.type}; if (default_values_column) { @@ -129,23 +135,29 @@ public: auto source_column = source_column_name_and_type.column; // adjust source and default values columns to resulting datatype - if (!source_column_name_and_type.type->equals(*result_type)) { + if (!source_column_name_and_type.type->equals(*result_type)) + { source_column = castColumn(source_column_name_and_type, result_type, context); } - if (default_values_column && !block.getByPosition(arguments[2]).type->equals(*result_type)) { + if (default_values_column && !block.getByPosition(arguments[2]).type->equals(*result_type)) + { default_values_column = castColumn(block.getByPosition(arguments[2]), result_type, context); } const DataTypePtr desired_type = std::make_shared(); - if (!block.getByPosition(arguments[1]).type->equals(*desired_type)) { + if (!block.getByPosition(arguments[1]).type->equals(*desired_type)) + { offset_column = castColumn(offset_structure, desired_type, context); } - if (isColumnConst(*source_column)) { + if (isColumnConst(*source_column)) + { auto column = result_type->createColumnConst(input_rows_count, (*source_column)[0]); block.getByPosition(result).column = std::move(column); - } else { + } + else + { auto column = result_type->createColumn(); column->reserve(input_rows_count); // with constant offset - insertRangeFrom @@ -156,48 +168,71 @@ public: if (offset_value > 0) { // insert shifted value - if ((size_t)offset_value <= input_rows_count) { + if ((size_t)offset_value <= input_rows_count) + { column->insertRangeFrom(*source_column, offset_value, input_rows_count - offset_value); } size_t row_count = (size_t)offset_value > input_rows_count ? input_rows_count : offset_value; insertDefaults(column, row_count, default_values_column, input_rows_count - row_count); - } else if (offset_value < 0) { + } + else if (offset_value < 0) + { size_t row_count = (size_t)std::abs(offset_value) > input_rows_count ? input_rows_count : std::abs(offset_value); // insert defaults up to offset_value insertDefaults(column, row_count, default_values_column, 0); column->insertRangeFrom(*source_column, 0, input_rows_count - row_count); - } else { + } + else + { // populate column with source values column->insertRangeFrom(*source_column, 0, input_rows_count); } - } else { + } + else + { // with dynamic offset - handle row by row for (size_t row = 0; row < input_rows_count; row++) { Int64 offset_value = offset_column->getInt(row); - if (offset_value == 0) { + if (offset_value == 0) + { column->insertFrom(*source_column, row); - } else if (offset_value > 0) { + } + else if (offset_value > 0) + { size_t real_offset = row + offset_value; - if (real_offset > input_rows_count) { - if (default_values_column) { + if (real_offset > input_rows_count) + { + if (default_values_column) + { column->insertFrom(*default_values_column, row); - } else { + } + else + { column->insertDefault(); } - } else { + } + else + { column->insertFrom(*column, real_offset); } - } else { + } + else + { // out of range if ((size_t)std::abs(offset_value) > row) { - if (default_values_column) { + if (default_values_column) + { column->insertFrom(*default_values_column, row); - } else { + } + else + { column->insertDefault(); } - } else { + } + else + { column->insertFrom(*column, row - std::abs(offset_value)); } } @@ -206,6 +241,7 @@ public: block.getByPosition(result).column = std::move(column); } } + private: const Context & context; }; From a9681fa34629d34c7e53a900e9e05ca2c8c29ce0 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 13 Aug 2019 17:31:19 +0300 Subject: [PATCH 193/509] Fixed ZH docs build. --- docs/zh/data_types/{domain => domains}/ipv4.md | 0 docs/zh/data_types/{domain => domains}/ipv6.md | 0 docs/zh/data_types/{domain => domains}/overview.md | 0 3 files changed, 0 insertions(+), 0 deletions(-) rename docs/zh/data_types/{domain => domains}/ipv4.md (100%) rename docs/zh/data_types/{domain => domains}/ipv6.md (100%) rename docs/zh/data_types/{domain => domains}/overview.md (100%) diff --git a/docs/zh/data_types/domain/ipv4.md b/docs/zh/data_types/domains/ipv4.md similarity index 100% rename from docs/zh/data_types/domain/ipv4.md rename to docs/zh/data_types/domains/ipv4.md diff --git a/docs/zh/data_types/domain/ipv6.md b/docs/zh/data_types/domains/ipv6.md similarity index 100% rename from docs/zh/data_types/domain/ipv6.md rename to docs/zh/data_types/domains/ipv6.md diff --git a/docs/zh/data_types/domain/overview.md b/docs/zh/data_types/domains/overview.md similarity index 100% rename from docs/zh/data_types/domain/overview.md rename to docs/zh/data_types/domains/overview.md From 031bfc7bf331ecc24345f9f9b5b8040b2994090e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Aug 2019 17:46:18 +0300 Subject: [PATCH 194/509] Remove reference --- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index b94555477f9..8149ef9ddfe 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -707,7 +707,7 @@ protected: {} const MergeTreeSettingsPtr copyPtr() const { return settings_ptr; } - MergeTreeSettingsPtr & getPtr() { return settings_ptr; } + MergeTreeSettingsPtr getPtr() { return settings_ptr; } void setPtr(MergeTreeSettingsPtr ptr) { settings_ptr = ptr; } }; From 96f62fefcc162ceb1700e2fd91efffbc0fe964e4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Aug 2019 19:34:52 +0300 Subject: [PATCH 195/509] Add mutex for settings --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.h | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index aab384552cc..3c14b7b1084 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1644,7 +1644,7 @@ void MergeTreeData::alterSettings( const Context & context, TableStructureWriteLockHolder & table_lock_holder) { - /// No additional locking required, because we took table_lock_holder + std::unique_lock lock(settings_mutex); MutableMergeTreeSettingsPtr settings = std::move(*guarded_settings.getPtr()).mutate(); settings->updateFromChanges(new_changes); IStorage::alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 8149ef9ddfe..e1cfe7b0032 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -662,6 +662,7 @@ public: /// get consistent settings. const MergeTreeSettingsPtr getCOWSettings() const { + std::shared_lock lock(settings_mutex); return guarded_settings.copyPtr(); } @@ -801,6 +802,8 @@ protected: std::mutex grab_old_parts_mutex; /// The same for clearOldTemporaryDirectories. std::mutex clear_old_temporary_directories_mutex; + /// Mutex for settings usage + mutable std::shared_mutex settings_mutex; void setPrimaryKeyIndicesAndColumns(const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast, const ColumnsDescription & new_columns, From 986d56ba0c1d80b0fc879f22750e42a1f8b213f1 Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Wed, 14 Aug 2019 14:32:03 +0300 Subject: [PATCH 196/509] Fix casting style, work with Nullable --- dbms/src/Functions/neighbour.cpp | 30 ++++++++++++------- .../registerFunctionsMiscellaneous.cpp | 4 +-- .../queries/0_stateless/00957_neighbour.sql | 4 +++ 3 files changed, 25 insertions(+), 13 deletions(-) diff --git a/dbms/src/Functions/neighbour.cpp b/dbms/src/Functions/neighbour.cpp index 2efca01a66f..153307abd4b 100644 --- a/dbms/src/Functions/neighbour.cpp +++ b/dbms/src/Functions/neighbour.cpp @@ -54,13 +54,14 @@ public: + ", should be from 2 to 3", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - // second argument must be a positive integer + // second argument must be an integer if (!isInteger(arguments[1])) throw Exception( "Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() - + " - should be positive integer", + + " - should be an integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + // check that default value column has supertype with first argument if (number_of_arguments == 3) { @@ -116,7 +117,13 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { auto offset_structure = block.getByPosition(arguments[1]); + ColumnPtr & offset_column = offset_structure.column; + if (isColumnNullable(*offset_column)) + throw Exception( + "Illegal type " + offset_structure.type->getName() + " of second argument of function " + getName() + + " - can not be Nullable", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto is_constant_offset = isColumnConst(*offset_structure.column); ColumnPtr default_values_column = nullptr; @@ -145,6 +152,7 @@ public: default_values_column = castColumn(block.getByPosition(arguments[2]), result_type, context); } + // since we are working with both signed and unsigned - we'll try to use Int64 for handling all of them const DataTypePtr desired_type = std::make_shared(); if (!block.getByPosition(arguments[1]).type->equals(*desired_type)) { @@ -165,26 +173,26 @@ public: { Int64 offset_value = offset_column->getInt(0); + auto offset_value_casted = static_cast(std::abs(offset_value)); + size_t default_value_count = std::min(offset_value_casted, input_rows_count); if (offset_value > 0) { // insert shifted value - if ((size_t)offset_value <= input_rows_count) + if (offset_value_casted <= input_rows_count) { - column->insertRangeFrom(*source_column, offset_value, input_rows_count - offset_value); + column->insertRangeFrom(*source_column, offset_value_casted, input_rows_count - offset_value_casted); } - size_t row_count = (size_t)offset_value > input_rows_count ? input_rows_count : offset_value; - insertDefaults(column, row_count, default_values_column, input_rows_count - row_count); + insertDefaults(column, default_value_count, default_values_column, input_rows_count - default_value_count); } else if (offset_value < 0) { - size_t row_count = (size_t)std::abs(offset_value) > input_rows_count ? input_rows_count : std::abs(offset_value); // insert defaults up to offset_value - insertDefaults(column, row_count, default_values_column, 0); - column->insertRangeFrom(*source_column, 0, input_rows_count - row_count); + insertDefaults(column, default_value_count, default_values_column, 0); + column->insertRangeFrom(*source_column, 0, input_rows_count - default_value_count); } else { - // populate column with source values + // populate column with source values, when offset is equal to zero column->insertRangeFrom(*source_column, 0, input_rows_count); } } @@ -246,7 +254,7 @@ private: const Context & context; }; -void registerFunctionNextInBlock(FunctionFactory & factory) +void registerFunctionNeighbour(FunctionFactory & factory) { factory.registerFunction(); } diff --git a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp index 57ccfcd11c9..c96f5f05c7b 100644 --- a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp @@ -17,7 +17,7 @@ void registerFunctionBlockSize(FunctionFactory &); void registerFunctionBlockNumber(FunctionFactory &); void registerFunctionRowNumberInBlock(FunctionFactory &); void registerFunctionRowNumberInAllBlocks(FunctionFactory &); -void registerFunctionNextInBlock(FunctionFactory &); +void registerFunctionNeighbour(FunctionFactory &); void registerFunctionSleep(FunctionFactory &); void registerFunctionSleepEachRow(FunctionFactory &); void registerFunctionMaterialize(FunctionFactory &); @@ -68,7 +68,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionBlockNumber(factory); registerFunctionRowNumberInBlock(factory); registerFunctionRowNumberInAllBlocks(factory); - registerFunctionNextInBlock(factory); + registerFunctionNeighbour(factory); registerFunctionSleep(factory); registerFunctionSleepEachRow(factory); registerFunctionMaterialize(factory); diff --git a/dbms/tests/queries/0_stateless/00957_neighbour.sql b/dbms/tests/queries/0_stateless/00957_neighbour.sql index 156672155a8..665936fd70f 100644 --- a/dbms/tests/queries/0_stateless/00957_neighbour.sql +++ b/dbms/tests/queries/0_stateless/00957_neighbour.sql @@ -8,6 +8,10 @@ select neighbour(1,2,3,4); -- { serverError 42 } select neighbour(dummy, 1, 'hello'); -- { serverError 43 } -- types without common supertype (UInt64 and Int8) select number, neighbour(number, 1, -10) from numbers(3); -- { serverError 43 } +-- nullable offset is not allowed +select number, if(number > 1, number, null) as offset, neighbour(number, offset) from numbers(3); -- { serverError 43 } +select 'Zero offset'; +select number, neighbour(number, 0) from numbers(3); select 'Result with different type'; select toInt32(number) as n, neighbour(n, 1, -10) from numbers(3); select 'Offset > block'; From 2126196c8946126cd32322f259571c241b7f857d Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Wed, 14 Aug 2019 15:09:51 +0300 Subject: [PATCH 197/509] Nullable correct handling --- dbms/src/Functions/neighbour.cpp | 21 ++++++++++--------- .../0_stateless/00957_neighbour.reference | 8 +++++++ .../queries/0_stateless/00957_neighbour.sql | 2 ++ 3 files changed, 21 insertions(+), 10 deletions(-) diff --git a/dbms/src/Functions/neighbour.cpp b/dbms/src/Functions/neighbour.cpp index 153307abd4b..06307884f22 100644 --- a/dbms/src/Functions/neighbour.cpp +++ b/dbms/src/Functions/neighbour.cpp @@ -44,6 +44,8 @@ public: bool isDeterministicInScopeOfQuery() const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { size_t number_of_arguments = arguments.size(); @@ -57,8 +59,11 @@ public: // second argument must be an integer if (!isInteger(arguments[1])) throw Exception( - "Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() - + " - should be an integer", + "Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + " - should be an integer", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + else if (arguments[1]->isNullable()) + throw Exception( + "Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + " - can not be Nullable", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -119,11 +124,6 @@ public: auto offset_structure = block.getByPosition(arguments[1]); ColumnPtr & offset_column = offset_structure.column; - if (isColumnNullable(*offset_column)) - throw Exception( - "Illegal type " + offset_structure.type->getName() + " of second argument of function " + getName() - + " - can not be Nullable", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto is_constant_offset = isColumnConst(*offset_structure.column); ColumnPtr default_values_column = nullptr; @@ -222,13 +222,14 @@ public: } else { - column->insertFrom(*column, real_offset); + column->insertFrom(*source_column, real_offset); } } else { // out of range - if ((size_t)std::abs(offset_value) > row) + auto offset_value_casted = static_cast(std::abs(offset_value)); + if (offset_value_casted > row) { if (default_values_column) { @@ -241,7 +242,7 @@ public: } else { - column->insertFrom(*column, row - std::abs(offset_value)); + column->insertFrom(*column, row - offset_value_casted); } } } diff --git a/dbms/tests/queries/0_stateless/00957_neighbour.reference b/dbms/tests/queries/0_stateless/00957_neighbour.reference index cd8c6310f22..1983488cfc2 100644 --- a/dbms/tests/queries/0_stateless/00957_neighbour.reference +++ b/dbms/tests/queries/0_stateless/00957_neighbour.reference @@ -1,3 +1,11 @@ +Zero offset +0 0 +1 1 +2 2 +Nullable values +\N 0 \N +\N 1 2 +2 2 \N Result with different type 0 1 1 2 diff --git a/dbms/tests/queries/0_stateless/00957_neighbour.sql b/dbms/tests/queries/0_stateless/00957_neighbour.sql index 665936fd70f..753ae8493a3 100644 --- a/dbms/tests/queries/0_stateless/00957_neighbour.sql +++ b/dbms/tests/queries/0_stateless/00957_neighbour.sql @@ -12,6 +12,8 @@ select number, neighbour(number, 1, -10) from numbers(3); -- { serverError 43 } select number, if(number > 1, number, null) as offset, neighbour(number, offset) from numbers(3); -- { serverError 43 } select 'Zero offset'; select number, neighbour(number, 0) from numbers(3); +select 'Nullable values'; +select if(number > 1, number, null) as value, number as offset, neighbour(value, offset) as neighbour from numbers(3); select 'Result with different type'; select toInt32(number) as n, neighbour(n, 1, -10) from numbers(3); select 'Offset > block'; From e10f198d72837325ff83d9e6bda6c88d47e5b3b4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Aug 2019 19:30:44 +0300 Subject: [PATCH 198/509] Fix build. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 2 +- dbms/src/Interpreters/InterpreterSelectQuery.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 5a0230e6798..04ce7b03bf7 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -2476,7 +2476,7 @@ void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline, Block header) auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; if (!blocksHaveEqualStructure(header, stream_header)) - stream = std::make_shared(context, stream, first_header, mode); + stream = std::make_shared(context, stream, header, mode); } } } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 7c7b657f5ce..5b8a10d5d25 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -231,7 +231,7 @@ private: void executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, std::unordered_map & subqueries_for_sets); /// If pipeline has several streams with different headers, add ConvertingBlockInputStream to first header. - void unifyStreams(Pipeline & pipeline, Block header); + void unifyStreams(Pipeline & pipeline, Block header = {}); enum class Modificator { From de93b759c9f873a8d16681d91902a402e0654637 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Aug 2019 16:00:38 +0300 Subject: [PATCH 199/509] Return non-const column in ignoreExceptNull. --- dbms/src/Functions/ignoreExceptNull.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Functions/ignoreExceptNull.cpp b/dbms/src/Functions/ignoreExceptNull.cpp index 3ebdbea8c58..6833903d5de 100644 --- a/dbms/src/Functions/ignoreExceptNull.cpp +++ b/dbms/src/Functions/ignoreExceptNull.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB @@ -38,7 +39,8 @@ namespace DB void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override { - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(0)); + auto res = ColumnUInt8::create(input_rows_count, 0); + block.getByPosition(result).column = std::move(res); } }; From 16918c7dba2312561020b990f1c67a657ee65f2c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Aug 2019 16:07:35 +0300 Subject: [PATCH 200/509] Added more comments. --- dbms/src/Functions/ignoreExceptNull.cpp | 3 +++ dbms/src/Functions/in.cpp | 2 ++ 2 files changed, 5 insertions(+) diff --git a/dbms/src/Functions/ignoreExceptNull.cpp b/dbms/src/Functions/ignoreExceptNull.cpp index 6833903d5de..d977a87342f 100644 --- a/dbms/src/Functions/ignoreExceptNull.cpp +++ b/dbms/src/Functions/ignoreExceptNull.cpp @@ -39,6 +39,9 @@ namespace DB void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override { + /// This function is mainly used in query analysis instead of "in" functions + /// in the case when only header is needed and set for in is not calculated. + /// Because of that function must return the same column type as "in" function, which is ColumnUInt8. auto res = ColumnUInt8::create(input_rows_count, 0); block.getByPosition(result).column = std::move(res); } diff --git a/dbms/src/Functions/in.cpp b/dbms/src/Functions/in.cpp index 2911f743538..fa3a7652e9e 100644 --- a/dbms/src/Functions/in.cpp +++ b/dbms/src/Functions/in.cpp @@ -75,6 +75,8 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { + /// NOTE: after updating this code, check that FunctionIgnoreExceptNull returns the same type of column. + /// Second argument must be ColumnSet. ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column; const ColumnSet * column_set = typeid_cast(&*column_set_ptr); From 06351881049d1e8acaf20f2b7254f2046088e669 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Aug 2019 19:20:02 +0300 Subject: [PATCH 201/509] Allow constants for RemoteBlockInputStream. --- dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp | 5 ++++- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 4 ++++ dbms/src/Storages/StorageDistributed.cpp | 4 ---- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index ba0571d1863..b4285c9f701 100644 --- a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -65,7 +65,10 @@ BlockInputStreamPtr createLocalStream(const ASTPtr & query_ast, const Context & * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, * And this is not allowed, since all code is based on the assumption that in the block stream all types are the same. */ - return std::make_shared(stream); + /// Already not. + /// return std::make_shared(stream); + + return stream; } } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 04ce7b03bf7..2f51b596721 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -346,8 +346,12 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Calculate structure of the result. result_header = getSampleBlockImpl(); for (auto & col : result_header) + { if (!col.column) col.column = col.type->createColumn(); + else if (isColumnConst(*col.column) && !col.column->empty()) + col.column = col.column->cloneEmpty(); + } } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 68e09c319d8..3fb9e5fd9c6 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -304,10 +304,6 @@ BlockInputStreams StorageDistributed::read( Block header = InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage)).getSampleBlock(); - for (auto & col : header) - if (isColumnConst(*col.column)) - col.column = col.type->createColumn(); - ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr ? ClusterProxy::SelectStreamFactory( header, processed_stage, remote_table_function_ptr, context.getExternalTables()) From b5c9be75c4b4784656965da2acc29e479b396a4c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Aug 2019 21:00:41 +0300 Subject: [PATCH 202/509] Materialize constant keys in header. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 2f51b596721..8e01d9b5209 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -452,7 +452,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() Block res; for (auto & key : key_names) - res.insert(header.getByName(key).cloneEmpty()); + res.insert({nullptr, header.getByName(key).type, key}); for (auto & aggregate : aggregates) { From 8f55fdbd1578ae7c9458936c96d0985a26c7311f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 13 Aug 2019 13:02:58 +0300 Subject: [PATCH 203/509] Wrap const columns in RemoteBlockInputStream. --- dbms/src/DataStreams/RemoteBlockInputStream.cpp | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index eb576075f80..9e9d47f9516 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -164,7 +164,18 @@ static Block adaptBlockStructure(const Block & block, const Block & header, cons res.info = block.info; for (const auto & elem : header) - res.insert({ castColumn(block.getByName(elem.name), elem.type, context), elem.type, elem.name }); + { + ColumnPtr column; + + if (elem.column && isColumnConst(*elem.column)) + /// TODO: check that column from block contains the same value. + /// TODO: serialize const columns. + column = elem.column->cloneResized(block.rows()); + else + column = castColumn(block.getByName(elem.name), elem.type, context); + + res.insert({column, elem.type, elem.name}); + } return res; } From 5131b6af4b0512ea696e7f876e4bb5f7e5fa815f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 14 Aug 2019 18:11:20 +0300 Subject: [PATCH 204/509] Fix header for joins. Make function randConstant not constant. --- dbms/src/Functions/randConstant.cpp | 2 +- dbms/src/Interpreters/ExpressionActions.cpp | 46 +++++++++++++++++++- dbms/src/Interpreters/ExpressionActions.h | 7 ++- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 12 ++--- dbms/src/Interpreters/ExpressionAnalyzer.h | 2 +- dbms/src/Interpreters/Join.h | 3 ++ dbms/src/Interpreters/QueryAliasesVisitor.h | 1 - dbms/src/Interpreters/QueryNormalizer.cpp | 14 +++++- dbms/src/Interpreters/QueryNormalizer.h | 1 + 9 files changed, 76 insertions(+), 12 deletions(-) diff --git a/dbms/src/Functions/randConstant.cpp b/dbms/src/Functions/randConstant.cpp index 91df7a1dfb8..490bcb6f239 100644 --- a/dbms/src/Functions/randConstant.cpp +++ b/dbms/src/Functions/randConstant.cpp @@ -45,7 +45,7 @@ public: value = vec_to[0]; } - block.getByPosition(result).column = DataTypeNumber().createColumnConst(input_rows_count, toField(value)); + block.getByPosition(result).column = ColumnVector::create(input_rows_count, value); } }; diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 0bb59713868..34ce178cff0 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace ProfileEvents @@ -159,14 +160,19 @@ ExpressionAction ExpressionAction::arrayJoin(const NameSet & array_joined_column } ExpressionAction ExpressionAction::ordinaryJoin( + const ASTTableJoin & join_params, std::shared_ptr join_, const Names & join_key_names_left, + const Names & join_key_names_right, const NamesAndTypesList & columns_added_by_join_) { ExpressionAction a; a.type = JOIN; a.join = std::move(join_); + a.join_kind = join_params.kind; + // a.join_strictness = join_params.strictness; a.join_key_names_left = join_key_names_left; + a.join_key_names_right = join_key_names_right; a.columns_added_by_join = columns_added_by_join_; return a; } @@ -252,10 +258,46 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) case JOIN: { - /// TODO join_use_nulls setting + bool is_null_used_as_default = settings.join_use_nulls; + bool right_or_full_join = join_kind == ASTTableJoin::Kind::Right || join_kind == ASTTableJoin::Kind::Full; + bool left_or_full_join = join_kind == ASTTableJoin::Kind::Left || join_kind == ASTTableJoin::Kind::Full; +// bool inner_or_right_join = join_kind == ASTTableJoin::Kind::Inner || join_kind == ASTTableJoin::Kind::Right; +// bool all_join = join_strictness == ASTTableJoin::Strictness::All; + + for (auto & col : sample_block) + { + /// Materialize column. + if (col.column) + col.column = nullptr; + + bool make_nullable = is_null_used_as_default && right_or_full_join; + + if (make_nullable && !col.type->isNullable()) + col.type = std::make_shared(col.type); + } for (const auto & col : columns_added_by_join) - sample_block.insert(ColumnWithTypeAndName(nullptr, col.type, col.name)); + { + auto res_type = col.type; + + bool make_nullable = is_null_used_as_default && left_or_full_join; + + if (!make_nullable) // && (all_join || !inner_or_right_join)) + { + auto it = std::find(join_key_names_right.begin(), join_key_names_right.end(), col.name); + if (it != join_key_names_right.end()) + { + auto pos = it - join_key_names_right.begin(); + const auto & left_key_name = join_key_names_left[pos]; + make_nullable = sample_block.getByName(left_key_name).type->isNullable(); + } + } + + if (make_nullable && !res_type->isNullable()) + res_type = std::make_shared(res_type); + + sample_block.insert(ColumnWithTypeAndName(nullptr, res_type, col.name)); + } break; } diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index f280e723dd5..984ecf124be 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -10,6 +10,7 @@ #include "config_core.h" #include #include +#include namespace DB @@ -104,7 +105,10 @@ public: /// For JOIN std::shared_ptr join; + ASTTableJoin::Kind join_kind; + // ASTTableJoin::Strictness join_strictness; Names join_key_names_left; + Names join_key_names_right; NamesAndTypesList columns_added_by_join; /// For PROJECT. @@ -121,7 +125,8 @@ public: static ExpressionAction project(const Names & projected_columns_); static ExpressionAction addAliases(const NamesWithAliases & aliased_columns_); static ExpressionAction arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left, const Context & context); - static ExpressionAction ordinaryJoin(std::shared_ptr join_, const Names & join_key_names_left, + static ExpressionAction ordinaryJoin(const ASTTableJoin & join_params, std::shared_ptr join_, + const Names & join_key_names_left, const Names & join_key_names_right, const NamesAndTypesList & columns_added_by_join_); /// Which columns necessary to perform this action. diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index fdc8226a42a..ef61795bf79 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -160,7 +160,7 @@ void ExpressionAnalyzer::analyzeAggregation() for (const auto & key_ast : analyzedJoin().key_asts_left) getRootActions(key_ast, true, temp_actions); - addJoinAction(temp_actions, true); + addJoinAction(table_join, temp_actions, true); } } @@ -504,14 +504,16 @@ bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool on return true; } -void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, bool only_types) const +void ExpressionAnalyzer::addJoinAction(const ASTTableJoin & join_params, ExpressionActionsPtr & actions, bool only_types) const { if (only_types) - actions->add(ExpressionAction::ordinaryJoin(nullptr, analyzedJoin().key_names_left, columns_added_by_join)); + actions->add(ExpressionAction::ordinaryJoin(join_params, nullptr, + analyzedJoin().key_names_left, analyzedJoin().key_names_right, columns_added_by_join)); else for (auto & subquery_for_set : subqueries_for_sets) if (subquery_for_set.second.join) - actions->add(ExpressionAction::ordinaryJoin(subquery_for_set.second.join, analyzedJoin().key_names_left, + actions->add(ExpressionAction::ordinaryJoin(join_params, subquery_for_set.second.join, + analyzedJoin().key_names_left, analyzedJoin().key_names_right, columns_added_by_join)); } @@ -632,7 +634,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty subquery_for_set.joined_block_actions = joined_block_actions; } - addJoinAction(step.actions, false); + addJoinAction(join_params, step.actions, false); return true; } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 644d10da1be..8fa18f16045 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -207,7 +207,7 @@ private: void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const; - void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const; + void addJoinAction(const ASTTableJoin & join_params, ExpressionActionsPtr & actions, bool only_types) const; /// If ast is ASTSelectQuery with JOIN, add actions for JOIN key columns. void getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions); diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index f57755fad91..11a20e34b67 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -126,6 +126,8 @@ public: bool empty() { return type == Type::EMPTY; } + bool isNullUsedAsDefault() const { return use_nulls; } + /** Set information about structure of right hand of JOIN (joined data). * You must call this method before subsequent calls to insertFromBlock. */ @@ -168,6 +170,7 @@ public: size_t getTotalByteCount() const; ASTTableJoin::Kind getKind() const { return kind; } + ASTTableJoin::Strictness getStrictness() const { return strictness; } AsofRowRefs::Type getAsofType() const { return *asof_type; } bool anyTakeLastRow() const { return any_take_last_row; } diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.h b/dbms/src/Interpreters/QueryAliasesVisitor.h index c4e297965c3..5a510a1fd86 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.h +++ b/dbms/src/Interpreters/QueryAliasesVisitor.h @@ -2,7 +2,6 @@ #include #include - namespace DB { diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index cea801c7c2f..4d720e1fbde 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -102,8 +102,20 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) /// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column"). auto it_alias = data.aliases.find(node.name); - if (IdentifierSemantic::canBeAlias(node) && it_alias != data.aliases.end() && current_alias != node.name) + if (it_alias != data.aliases.end() && current_alias != node.name) { + if (!IdentifierSemantic::canBeAlias(node)) + { + if (node.alias.empty()) + { + node.name.swap(node.alias); + node.restoreCompoundName(); + node.name.swap(node.alias); + } + + return; + } + auto & alias_node = it_alias->second; /// Let's replace it with the corresponding tree node. diff --git a/dbms/src/Interpreters/QueryNormalizer.h b/dbms/src/Interpreters/QueryNormalizer.h index 6d6fea86e44..587a10668a4 100644 --- a/dbms/src/Interpreters/QueryNormalizer.h +++ b/dbms/src/Interpreters/QueryNormalizer.h @@ -5,6 +5,7 @@ #include #include #include +#include "DatabaseAndTableWithAlias.h" namespace DB { From b323f0890770ebd4b0c5dff9df1153a4a30a3ccb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 14 Aug 2019 18:23:57 +0300 Subject: [PATCH 205/509] Fix build. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index ce00f954432..6cc0b2931f1 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -415,7 +415,8 @@ Block InterpreterSelectQuery::getSampleBlockImpl() FilterInfoPtr filter_info; /// Need to create sets before analyzeExpressions(). Otherwise some sets for index won't be created. - query_analyzer->makeSetsForIndex(); + query_analyzer->makeSetsForIndex(getSelectQuery().where()); + query_analyzer->makeSetsForIndex(getSelectQuery().prewhere()); auto analysis_result = analyzeExpressions( getSelectQuery(), From ab1c4139deaad29f00104580df09f2334bb4efaa Mon Sep 17 00:00:00 2001 From: Gleb Novikov Date: Wed, 14 Aug 2019 22:51:03 +0300 Subject: [PATCH 206/509] Added ReplicatedMergeTree support and test for constraints, also added VIOLATED_CONSTRAINT error --- dbms/programs/server/config.xml | 5 ++- dbms/src/Common/ErrorCodes.cpp | 1 + .../CheckConstraintsBlockOutputStream.cpp | 2 +- .../CheckConstraintsBlockOutputStream.h | 2 +- .../ReplicatedMergeTreeTableMetadata.cpp | 6 +++ .../Storages/StorageReplicatedMergeTree.cpp | 4 ++ ...onstraints_replication_zookeeper.reference | 0 ...0988_constraints_replication_zookeeper.sql | 43 +++++++++++++++++++ 8 files changed, 60 insertions(+), 3 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00988_constraints_replication_zookeeper.reference create mode 100644 dbms/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index c09913cbd87..188a98779e9 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -217,7 +217,10 @@ See https://clickhouse.yandex/docs/en/table_engines/replication/ --> - + + + testkeeper + - - testkeeper - + diff --git a/docs/en/operations/table_engines/tinylog.md b/docs/en/operations/table_engines/tinylog.md index 563912d92f1..fe4913118e9 100644 --- a/docs/en/operations/table_engines/tinylog.md +++ b/docs/en/operations/table_engines/tinylog.md @@ -1,22 +1,12 @@ # TinyLog -Engine belongs to the family of log engines. See the common properties of log engines and their differences in the [Log Engine Family](log_family.md) article. +Engine belongs to the family of log engines. See [Log Engine Family](log_family.md) for common properties of log engines and for their differences. -The simplest table engine, which stores data on a disk. -Each column is stored in a separate compressed file. -When writing, data is appended to the end of files. +The typical way using this table engine is write-once method: firstly write the data one time, then read it as many times as needed. For example, you can use `TinyLog`-type tables for intermediary data that is processed in small batches. -Concurrent data access is not restricted in any way: +Queries are executed in a single stream. In other words, this engine is intended for relatively small tables (recommended up to about 1,000,000 rows). It makes sense to use this table engine if you have many small tables, since it is simpler than the [Log](log.md) engine (fewer files need to be opened). -- If you are simultaneously reading from a table and writing to it in a different query, the read operation will complete with an error. -- If you are writing to a table in multiple queries simultaneously, the data will be broken. +The situation when you have a large number of small tables guarantees poor productivity, but may already be used when working with another DBMS, and you may find it easier to switch to using `TinyLog`-type tables. -The typical way to use this table is write-once: first just write the data one time, then read it as many times as needed. -Queries are executed in a single stream. In other words, this engine is intended for relatively small tables (recommended up to 1,000,000 rows). -It makes sense to use this table engine if you have many small tables, since it is simpler than the Log engine (fewer files need to be opened). -The situation when you have a large number of small tables guarantees poor productivity, but may already be used when working with another DBMS, and you may find it easier to switch to using TinyLog types of tables. -**Indexes are not supported.** - -In Yandex.Metrica, TinyLog tables are used for intermediary data that is processed in small batches. [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/tinylog/) From 08021fe125fbad24625dd5412004fc39772fc34c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 16 Aug 2019 12:38:12 +0300 Subject: [PATCH 225/509] Add alwaysReturnsConstant for IFunctionBase. --- dbms/src/Interpreters/ExpressionActions.cpp | 21 ++++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 4d8da6b28b4..487959bd063 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -204,12 +204,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) } size_t result_position = sample_block.columns(); - - ColumnPtr const_col; - if (function_base->alwaysReturnsConstant()) - const_col = result_type->createColumnConstWithDefaultValue(0); - - sample_block.insert({const_col, result_type, result_name}); + sample_block.insert({nullptr, result_type, result_name}); function = function_base->prepare(sample_block, arguments, result_position); if (auto * prepared_function = dynamic_cast(function.get())) @@ -963,7 +958,7 @@ void ExpressionActions::finalize(const Names & output_columns) /** If the function is a constant expression, then replace the action by adding a column-constant - result. * That is, we perform constant folding. */ - if (action.type == ExpressionAction::APPLY_FUNCTION && action.is_suitable_for_constant_folding && sample_block.has(out)) + if (action.type == ExpressionAction::APPLY_FUNCTION && sample_block.has(out)) { auto & result = sample_block.getByName(out); if (result.column) @@ -1089,6 +1084,18 @@ void ExpressionActions::finalize(const Names & output_columns) actions.swap(new_actions); + for (auto & action : new_actions) + { + if (action.type == ExpressionAction::APPLY_FUNCTION + && action.function_base->alwaysReturnsConstant() + && sample_block.has(action.result_name)) + { + auto & result = sample_block.getByName(action.result_name); + if (!result.column) + result.column = result.type->createColumnConstWithDefaultValue(0); + } + } + /* std::cerr << "\n"; for (const auto & action : actions) std::cerr << action.toString() << "\n"; From 0eb1a931b434abae192a6dfebf18e3d11ea6e2b3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 16 Aug 2019 13:29:33 +0300 Subject: [PATCH 226/509] Add alwaysReturnsConstant for IFunctionBase. --- dbms/src/Interpreters/ExpressionActions.cpp | 34 ++++++++++----------- dbms/src/Interpreters/ExpressionActions.h | 5 +-- 2 files changed, 19 insertions(+), 20 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 487959bd063..dc312b30ef1 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -178,7 +178,7 @@ ExpressionAction ExpressionAction::ordinaryJoin( } -void ExpressionAction::prepare(Block & sample_block, const Settings & settings) +void ExpressionAction::prepare(Block & sample_block, const Settings & settings, NameSet & names_not_for_constant_folding) { // std::cerr << "preparing: " << toString() << std::endl; @@ -193,6 +193,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); bool all_const = true; + bool all_suitable_for_constant_folding = true; ColumnNumbers arguments(argument_names.size()); for (size_t i = 0; i < argument_names.size(); ++i) @@ -201,10 +202,18 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) ColumnPtr col = sample_block.safeGetByPosition(arguments[i]).column; if (!col || !isColumnConst(*col)) all_const = false; + + if (names_not_for_constant_folding.count(argument_names[i])) + all_suitable_for_constant_folding = false; } size_t result_position = sample_block.columns(); - sample_block.insert({nullptr, result_type, result_name}); + + ColumnPtr const_col; + if (function_base->alwaysReturnsConstant()) + const_col = result_type->createColumnConstWithDefaultValue(1); + + sample_block.insert({const_col, result_type, result_name}); function = function_base->prepare(sample_block, arguments, result_position); if (auto * prepared_function = dynamic_cast(function.get())) @@ -217,7 +226,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) /// If all arguments are constants, and function is suitable to be executed in 'prepare' stage - execute function. /// But if we compile expressions compiled version of this function maybe placed in cache, /// so we don't want to unfold non deterministic functions - if (all_const && function_base->isSuitableForConstantFolding() && (!compile_expressions || function_base->isDeterministic())) + if (all_const && (!compile_expressions || function_base->isDeterministic())) { function->execute(sample_block, arguments, result_position, sample_block.rows(), true); @@ -236,7 +245,8 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) if (col.column->empty()) col.column = col.column->cloneResized(1); - is_suitable_for_constant_folding = true; + if (!all_suitable_for_constant_folding || !function_base->isSuitableForConstantFolding()) + names_not_for_constant_folding.insert(result_name); } } @@ -729,7 +739,7 @@ void ExpressionActions::addImpl(ExpressionAction action, Names & new_names) for (const auto & name_with_alias : action.projection) new_names.emplace_back(name_with_alias.second); - action.prepare(sample_block, settings); + action.prepare(sample_block, settings, names_not_for_constant_folding); actions.push_back(action); } @@ -961,7 +971,7 @@ void ExpressionActions::finalize(const Names & output_columns) if (action.type == ExpressionAction::APPLY_FUNCTION && sample_block.has(out)) { auto & result = sample_block.getByName(out); - if (result.column) + if (result.column && names_not_for_constant_folding.count(result.name) == 0) { action.type = ExpressionAction::ADD_COLUMN; action.result_type = result.type; @@ -1084,18 +1094,6 @@ void ExpressionActions::finalize(const Names & output_columns) actions.swap(new_actions); - for (auto & action : new_actions) - { - if (action.type == ExpressionAction::APPLY_FUNCTION - && action.function_base->alwaysReturnsConstant() - && sample_block.has(action.result_name)) - { - auto & result = sample_block.getByName(action.result_name); - if (!result.column) - result.column = result.type->createColumnConstWithDefaultValue(0); - } - } - /* std::cerr << "\n"; for (const auto & action : actions) std::cerr << action.toString() << "\n"; diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index 48162188b6c..8b959235cc5 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -97,7 +97,6 @@ public: PreparedFunctionPtr function; Names argument_names; bool is_function_compiled = false; - bool is_suitable_for_constant_folding = false; /// For ARRAY_JOIN NameSet array_joined_columns; @@ -145,7 +144,7 @@ public: private: friend class ExpressionActions; - void prepare(Block & sample_block, const Settings & settings); + void prepare(Block & sample_block, const Settings & settings, NameSet & names_not_for_constant_folding); void execute(Block & block, bool dry_run) const; void executeOnTotals(Block & block) const; }; @@ -269,6 +268,8 @@ private: Actions actions; /// The example of result (output) block. Block sample_block; + /// Columns which can't be used for constant folding. + NameSet names_not_for_constant_folding; Settings settings; #if USE_EMBEDDED_COMPILER From 670c8a3e50223bc1c23c027d5ab1a6f9092920d1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 16 Aug 2019 13:41:17 +0300 Subject: [PATCH 227/509] Add alwaysReturnsConstant for IFunctionBase. --- dbms/src/Interpreters/ExpressionActions.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index dc312b30ef1..6636fed488c 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -208,12 +208,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings, } size_t result_position = sample_block.columns(); - - ColumnPtr const_col; - if (function_base->alwaysReturnsConstant()) - const_col = result_type->createColumnConstWithDefaultValue(1); - - sample_block.insert({const_col, result_type, result_name}); + sample_block.insert({nullptr, result_type, result_name}); function = function_base->prepare(sample_block, arguments, result_position); if (auto * prepared_function = dynamic_cast(function.get())) @@ -250,6 +245,13 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings, } } + auto & res = sample_block.getByPosition(result_position); + if (!res.column && function_base->alwaysReturnsConstant()) + { + res.column = result_type->createColumnConstWithDefaultValue(1); + names_not_for_constant_folding.insert(result_name); + } + break; } From 688cc4976a028a013dbba8800767b5240061ee1c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 16 Aug 2019 14:03:23 +0300 Subject: [PATCH 228/509] Moved NOTICE to dbms/src --- NOTICE | 41 ----------------------------------------- dbms/src/NOTICE | 41 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 41 insertions(+), 41 deletions(-) delete mode 100644 NOTICE create mode 100644 dbms/src/NOTICE diff --git a/NOTICE b/NOTICE deleted file mode 100644 index 59c6923e66a..00000000000 --- a/NOTICE +++ /dev/null @@ -1,41 +0,0 @@ --- -The following notice shall be applied to the files listed below. - -Some modifications Copyright (c) 2018 BlackBerry Limited - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at -http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - -dbms/src/Common/ErrorCodes.cpp -dbms/src/Common/UInt128.h -dbms/src/Core/Block.h -dbms/src/Core/Defines.h -dbms/src/Core/Settings.h -dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp -dbms/src/DataStreams/PushingToViewsBlockOutputStream.h -dbms/src/DataStreams/copyData.cpp -dbms/src/Databases/DatabasesCommon.cpp -dbms/src/IO/WriteBufferValidUTF8.cpp -dbms/src/Interpreters/InterpreterAlterQuery.cpp -dbms/src/Interpreters/InterpreterCreateQuery.cpp -dbms/src/Interpreters/InterpreterFactory.cpp -dbms/src/Parsers/ASTAlterQuery.cpp -dbms/src/Parsers/ASTAlterQuery.h -dbms/src/Parsers/ASTCreateQuery.cpp -dbms/src/Parsers/ASTCreateQuery.h -dbms/src/Parsers/ParserAlterQuery.cpp -dbms/src/Parsers/ParserAlterQuery.h -dbms/src/Parsers/ParserCreateQuery.cpp -dbms/src/Parsers/ParserCreateQuery.h -dbms/src/Parsers/ParserQueryWithOutput.cpp -dbms/src/Storages/IStorage.h -dbms/src/Storages/StorageFactory.cpp -dbms/src/Storages/registerStorages.cpp --- diff --git a/dbms/src/NOTICE b/dbms/src/NOTICE new file mode 100644 index 00000000000..d0d3efe3f8e --- /dev/null +++ b/dbms/src/NOTICE @@ -0,0 +1,41 @@ +-- +The following notice shall be applied to the files listed below. + +Some modifications Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + +Common/ErrorCodes.cpp +Common/UInt128.h +Core/Block.h +Core/Defines.h +Core/Settings.h +DataStreams/PushingToViewsBlockOutputStream.cpp +DataStreams/PushingToViewsBlockOutputStream.h +DataStreams/copyData.cpp +Databases/DatabasesCommon.cpp +IO/WriteBufferValidUTF8.cpp +Interpreters/InterpreterAlterQuery.cpp +Interpreters/InterpreterCreateQuery.cpp +Interpreters/InterpreterFactory.cpp +Parsers/ASTAlterQuery.cpp +Parsers/ASTAlterQuery.h +Parsers/ASTCreateQuery.cpp +Parsers/ASTCreateQuery.h +Parsers/ParserAlterQuery.cpp +Parsers/ParserAlterQuery.h +Parsers/ParserCreateQuery.cpp +Parsers/ParserCreateQuery.h +Parsers/ParserQueryWithOutput.cpp +Storages/IStorage.h +Storages/StorageFactory.cpp +Storages/registerStorages.cpp +-- From 29f4f83c39e1915491ac8d9ee5d9dec1b77808e9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 16 Aug 2019 15:54:50 +0300 Subject: [PATCH 229/509] Fix build. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 2 +- dbms/src/Interpreters/InterpreterSelectQuery.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 7d0ddaef9a8..37e895331ab 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -479,7 +479,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpressions( const ASTSelectQuery & query, - ExpressionAnalyzer & query_analyzer, + SelectQueryExpressionAnalyzer & query_analyzer, QueryProcessingStage::Enum from_stage, QueryProcessingStage::Enum to_stage, const Context & context, diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 466f806f8ff..fc3e46dd345 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -175,7 +175,7 @@ private: static AnalysisResult analyzeExpressions( const ASTSelectQuery & query, - ExpressionAnalyzer & query_analyzer, + SelectQueryExpressionAnalyzer & query_analyzer, QueryProcessingStage::Enum from_stage, QueryProcessingStage::Enum to_stage, const Context & context, From ebdc0eb3389d647a7a985bc78fd0f428a82d48cd Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 16 Aug 2019 17:56:42 +0300 Subject: [PATCH 230/509] Update 00991_system_parts_race_condition.sh --- .../queries/0_stateless/00991_system_parts_race_condition.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/tests/queries/0_stateless/00991_system_parts_race_condition.sh b/dbms/tests/queries/0_stateless/00991_system_parts_race_condition.sh index 66c1af8f447..28dbcbdd938 100755 --- a/dbms/tests/queries/0_stateless/00991_system_parts_race_condition.sh +++ b/dbms/tests/queries/0_stateless/00991_system_parts_race_condition.sh @@ -66,6 +66,7 @@ timeout $TIMEOUT bash -c thread3 2> /dev/null & timeout $TIMEOUT bash -c thread4 2> /dev/null & timeout $TIMEOUT bash -c thread5 2> /dev/null & + wait $CLICKHOUSE_CLIENT -q "DROP TABLE alter_table" From e9070ede0a0162b21984be5ef4e2d0e4dc24e50b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 16 Aug 2019 19:28:25 +0300 Subject: [PATCH 231/509] Try to rewrite live view test. --- .../00991_live_view_watch_http.python | 87 +++++++++++++++++++ .../00991_live_view_watch_http.reference | 0 .../0_stateless/00991_live_view_watch_http.sh | 6 ++ 3 files changed, 93 insertions(+) create mode 100755 dbms/tests/queries/0_stateless/00991_live_view_watch_http.python create mode 100644 dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference create mode 100755 dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python new file mode 100755 index 00000000000..d2e55cd1ea1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python @@ -0,0 +1,87 @@ +#!/usr/bin/env python + +import subprocess +import threading +import Queue as queue +import os +import sys + + +CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') +CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') +CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') + + +print(CLICKHOUSE_CLIENT) +print(CLICKHOUSE_CURL) +print(CLICKHOUSE_URL) + + +def send_query(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + print(' '.join(cmd)) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, close_fds=True).stdout + + +def send_http_query(query): + cmd = list(['curl', '--max-time', '20']) + cmd += ['-sS', CLICKHOUSE_URL, '-d', query] + return subprocess.Popen(cmd, stdout=subprocess.PIPE, close_fds=True, bufsize=1).stdout + + +def read_lines_and_push_to_queue(pipe, queue): + print('--------') + sys.stdout.flush() + s = '' + while True: + char = pipe.read(1) + if char == '': + break + + print('> ' + char) + if char == '\n': + queue.put(s) + s = '' + else: + s += char + sys.stdout.flush() + #queue.put(line) + + if s: + queue.put(s) + print(None) + queue.put(None) + + +def test(): + print(1) + send_query('DROP TABLE IF EXISTS test.lv').read() + print(2) + send_query('DROP TABLE IF EXISTS test.mt').read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() + send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + + q = queue.Queue() + pipe = send_http_query('WATCH test.lv') + sys.stdout.flush() + thread = threading.Thread(target=read_lines_and_push_to_queue, args=(pipe, q)) + thread.start() + print(3) + sys.stdout.flush() + assert (q.get() == '0\t1') + print(3.1) + sys.stdout.flush() + send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + print(4) + assert (q.get() == '6\t2') + print(4.1) + send_query('DROP TABLE test.lv').read() + print(5) + assert (q.get() is None) + print(5.2) + send_query('DROP TABLE test.mt').read() + print(6) + thread.join() + +test() diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh new file mode 100755 index 00000000000..88cce77f595 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +python $CURDIR/00991_live_view_watch_http.python From ab4018e98e816415aaa8cb2c7825c7e38e121a19 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 16 Aug 2019 19:56:34 +0300 Subject: [PATCH 232/509] Fix test. --- dbms/src/Interpreters/ExpressionActions.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 6636fed488c..d1808442ab2 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -221,7 +221,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings, /// If all arguments are constants, and function is suitable to be executed in 'prepare' stage - execute function. /// But if we compile expressions compiled version of this function maybe placed in cache, /// so we don't want to unfold non deterministic functions - if (all_const && (!compile_expressions || function_base->isDeterministic())) + if (all_const && function_base->isSuitableForConstantFolding() && (!compile_expressions || function_base->isDeterministic())) { function->execute(sample_block, arguments, result_position, sample_block.rows(), true); @@ -240,13 +240,13 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings, if (col.column->empty()) col.column = col.column->cloneResized(1); - if (!all_suitable_for_constant_folding || !function_base->isSuitableForConstantFolding()) + if (!all_suitable_for_constant_folding) names_not_for_constant_folding.insert(result_name); } } auto & res = sample_block.getByPosition(result_position); - if (!res.column && function_base->alwaysReturnsConstant()) + if (!res.column && function_base->alwaysReturnsConstant() && function_base->isSuitableForConstantFolding()) { res.column = result_type->createColumnConstWithDefaultValue(1); names_not_for_constant_folding.insert(result_name); From 5c10bff9c2089d78234fa31fe6ceb6624e09e547 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 16 Aug 2019 21:52:29 +0300 Subject: [PATCH 233/509] Try to rewrite live view test. --- .../00991_live_view_watch_http.python | 83 ++++++++----------- 1 file changed, 34 insertions(+), 49 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python index d2e55cd1ea1..2f46aa11046 100755 --- a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python @@ -12,76 +12,61 @@ CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') -print(CLICKHOUSE_CLIENT) -print(CLICKHOUSE_CURL) -print(CLICKHOUSE_URL) - - def send_query(query): cmd = list(CLICKHOUSE_CLIENT.split()) cmd += ['--query', query] - print(' '.join(cmd)) - return subprocess.Popen(cmd, stdout=subprocess.PIPE, close_fds=True).stdout + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT) def send_http_query(query): - cmd = list(['curl', '--max-time', '20']) - cmd += ['-sS', CLICKHOUSE_URL, '-d', query] - return subprocess.Popen(cmd, stdout=subprocess.PIPE, close_fds=True, bufsize=1).stdout + cmd = list(CLICKHOUSE_CURL.split()) # list(['curl', '-sSN', '--max-time', '10']) + cmd += ['-sSN', CLICKHOUSE_URL, '-d', query] + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT) -def read_lines_and_push_to_queue(pipe, queue): - print('--------') +def read_lines_and_push_to_queue(p, queue): + pipe = p.stdout sys.stdout.flush() - s = '' - while True: - char = pipe.read(1) - if char == '': - break - print('> ' + char) - if char == '\n': - queue.put(s) - s = '' - else: - s += char + for line in iter(pipe.readline, ''): + line = line.strip() + print(line) sys.stdout.flush() - #queue.put(line) + queue.put(line) - if s: - queue.put(s) - print(None) queue.put(None) + p.wait() def test(): - print(1) - send_query('DROP TABLE IF EXISTS test.lv').read() - print(2) - send_query('DROP TABLE IF EXISTS test.mt').read() - send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() - send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + send_query('DROP TABLE IF EXISTS test.lv').stdout.read() + send_query('DROP TABLE IF EXISTS test.mt').stdout.read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').stdout.read() + send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').stdout.read() q = queue.Queue() pipe = send_http_query('WATCH test.lv') - sys.stdout.flush() thread = threading.Thread(target=read_lines_and_push_to_queue, args=(pipe, q)) thread.start() - print(3) - sys.stdout.flush() - assert (q.get() == '0\t1') - print(3.1) - sys.stdout.flush() - send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() - print(4) - assert (q.get() == '6\t2') - print(4.1) - send_query('DROP TABLE test.lv').read() - print(5) - assert (q.get() is None) - print(5.2) - send_query('DROP TABLE test.mt').read() - print(6) + + line = q.get() + print(line) + assert (line == '0\t1') + + p = send_query('INSERT INTO test.mt VALUES (1),(2),(3)') + p.stdout.read() + p.wait() + line = q.get() + print(line) + assert (line == '6\t2') + + p = send_query('DROP TABLE if exitst test.lv') + p.stdout.read() + p.wait() + p = send_query('DROP TABLE if exitst test.mt') + p.stdout.read() + p.wait() thread.join() test() From b85c50e7c324c06284a83ccf2c60110740a77967 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 18 Aug 2019 17:55:20 +0300 Subject: [PATCH 234/509] Fix tests. --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index a7f17eea66b..9836c9eff37 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -443,7 +443,7 @@ bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, b initChain(chain, sourceColumns()); ExpressionActionsChain::Step & step = chain.steps.back(); - auto & join_params = ast_join->as(); + auto & join_params = ast_join->table_join->as(); getRootActions(left_keys_list, only_types, step.actions); addJoinAction(join_params, step.actions, subquery_for_set.join); From 99e85ff7530cca14d881d54321809cc52969243e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 19 Aug 2019 11:25:07 +0300 Subject: [PATCH 235/509] Fix build --- dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 8c04dc372c8..8cbb0819d20 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -871,7 +871,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO return streams; /// Let's split ranges to avoid reading much data. - auto split_ranges = [rows_granularity = data.settings.index_granularity, max_block_size](const auto & ranges, int direction) + auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size](const auto & ranges, int direction) { MarkRanges new_ranges; const size_t max_marks_in_range = (max_block_size + rows_granularity - 1) / rows_granularity; From 4c45ce3038808aef468edc95fa9f8bb16f21c4bd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 12:47:08 +0300 Subject: [PATCH 236/509] Apply patches to Live View tests. --- .../queries/0_stateless/helpers/client.py | 35 +++++++++++++------ .../queries/0_stateless/helpers/uexpect.py | 3 +- 2 files changed, 26 insertions(+), 12 deletions(-) diff --git a/dbms/tests/queries/0_stateless/helpers/client.py b/dbms/tests/queries/0_stateless/helpers/client.py index de4da794805..f3938d3bf63 100644 --- a/dbms/tests/queries/0_stateless/helpers/client.py +++ b/dbms/tests/queries/0_stateless/helpers/client.py @@ -1,5 +1,6 @@ import os import sys +import time CURDIR = os.path.dirname(os.path.realpath(__file__)) @@ -10,14 +11,26 @@ import uexpect prompt = ':\) ' end_of_block = r'.*\r\n.*\r\n' -def client(command=None, name='', log=None): - client = uexpect.spawn(['/bin/bash','--noediting']) - if command is None: - command = os.environ.get('CLICKHOUSE_BINARY', 'clickhouse') + '-client' - client.command = command - client.eol('\r') - client.logger(log, prefix=name) - client.timeout(20) - client.expect('[#\$] ', timeout=2) - client.send(command) - return client +class client(object): + def __init__(self, command=None, name='', log=None): + self.client = uexpect.spawn(['/bin/bash','--noediting']) + if command is None: + command = os.environ.get('CLICKHOUSE_BINARY', 'clickhouse') + '-client' + self.client.command = command + self.client.eol('\r') + self.client.logger(log, prefix=name) + self.client.timeout(20) + self.client.expect('[#\$] ', timeout=2) + self.client.send(command) + + def __enter__(self): + return self.client.__enter__() + + def __exit__(self, type, value, traceback): + self.client.reader['kill_event'].set() + # send Ctrl-C + self.client.send('\x03', eol='') + time.sleep(0.3) + self.client.send('quit', eol='\r') + self.client.send('\x03', eol='') + return self.client.__exit__(type, value, traceback) diff --git a/dbms/tests/queries/0_stateless/helpers/uexpect.py b/dbms/tests/queries/0_stateless/helpers/uexpect.py index d65190e2b29..930a45a099e 100644 --- a/dbms/tests/queries/0_stateless/helpers/uexpect.py +++ b/dbms/tests/queries/0_stateless/helpers/uexpect.py @@ -189,6 +189,7 @@ def spawn(command): queue = Queue() reader_kill_event = Event() thread = Thread(target=reader, args=(process, master, queue, reader_kill_event)) + thread.daemon = True thread.start() return IO(process, master, queue, reader={'thread':thread, 'kill_event':reader_kill_event}) @@ -199,6 +200,6 @@ def reader(process, out, queue, kill_event): data = os.read(out, 65536) queue.put(data) except OSError, e: - if e.errno == 5 and kill_event.is_set(): + if kill_event.is_set(): break raise From a33c59d83b618450ba31df29fb3aeee31810c476 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 14:08:10 +0300 Subject: [PATCH 237/509] Update Live View test. --- .../00991_live_view_watch_http.python | 31 +++++++------------ .../00991_live_view_watch_http.reference | 4 +++ 2 files changed, 15 insertions(+), 20 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python index 2f46aa11046..938547ca0cb 100755 --- a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python @@ -16,19 +16,16 @@ def send_query(query): cmd = list(CLICKHOUSE_CLIENT.split()) cmd += ['--query', query] # print(cmd) - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout def send_http_query(query): cmd = list(CLICKHOUSE_CURL.split()) # list(['curl', '-sSN', '--max-time', '10']) cmd += ['-sSN', CLICKHOUSE_URL, '-d', query] - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout -def read_lines_and_push_to_queue(p, queue): - pipe = p.stdout - sys.stdout.flush() - +def read_lines_and_push_to_queue(pipe, queue): for line in iter(pipe.readline, ''): line = line.strip() print(line) @@ -36,14 +33,13 @@ def read_lines_and_push_to_queue(p, queue): queue.put(line) queue.put(None) - p.wait() def test(): - send_query('DROP TABLE IF EXISTS test.lv').stdout.read() - send_query('DROP TABLE IF EXISTS test.mt').stdout.read() - send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').stdout.read() - send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').stdout.read() + send_query('DROP TABLE IF EXISTS test.lv').read() + send_query('DROP TABLE IF EXISTS test.mt').read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() + send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() q = queue.Queue() pipe = send_http_query('WATCH test.lv') @@ -54,19 +50,14 @@ def test(): print(line) assert (line == '0\t1') - p = send_query('INSERT INTO test.mt VALUES (1),(2),(3)') - p.stdout.read() - p.wait() + send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() line = q.get() print(line) assert (line == '6\t2') - p = send_query('DROP TABLE if exitst test.lv') - p.stdout.read() - p.wait() - p = send_query('DROP TABLE if exitst test.mt') - p.stdout.read() - p.wait() + send_query('DROP TABLE if exists test.lv').read() + send_query('DROP TABLE if exists test.lv').read() + thread.join() test() diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference index e69de29bb2d..489457d751b 100644 --- a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference @@ -0,0 +1,4 @@ +0 1 +0 1 +6 2 +6 2 From 69015f292e3af3824868839a31510b48c84efd9a Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 19 Aug 2019 14:18:39 +0300 Subject: [PATCH 238/509] Fix wrong merge --- dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 7c4b3fd1fcb..1d064b9b0c7 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1032,7 +1032,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor NameSet files_to_skip = {"checksums.txt", "columns.txt"}; auto mrk_extension = data_settings->index_granularity_bytes ? getAdaptiveMrkExtension() : getNonAdaptiveMrkExtension(); - for (const auto & entry : in_header) + for (const auto & entry : updated_header) { IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path) { From f2a8c18c1ff98779a21451209d43a326d22b1f53 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 19 Aug 2019 14:18:58 +0300 Subject: [PATCH 239/509] Simplify lock --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 6f6c43facf8..29aafc1aa45 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1645,7 +1645,7 @@ void MergeTreeData::alterSettings( const Context & context, TableStructureWriteLockHolder & table_lock_holder) { - std::unique_lock lock(settings_mutex); + std::lock_guard lock(settings_mutex); MutableMergeTreeSettingsPtr settings = std::move(*guarded_settings.getPtr()).mutate(); settings->updateFromChanges(new_changes); IStorage::alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); From 61432b83d86028170ec2725a65b70de51c508668 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 15:27:51 +0300 Subject: [PATCH 240/509] Added table function numbers_mt. --- .../src/TableFunctions/TableFunctionNumbers.cpp | 17 +++++++++-------- dbms/src/TableFunctions/TableFunctionNumbers.h | 5 +++-- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/dbms/src/TableFunctions/TableFunctionNumbers.cpp b/dbms/src/TableFunctions/TableFunctionNumbers.cpp index 94c618bd9b1..771ff8d5019 100644 --- a/dbms/src/TableFunctions/TableFunctionNumbers.cpp +++ b/dbms/src/TableFunctions/TableFunctionNumbers.cpp @@ -16,34 +16,35 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } - -StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +template +StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { if (const auto * function = ast_function->as()) { auto arguments = function->arguments->children; if (arguments.size() != 1 && arguments.size() != 2) - throw Exception("Table function 'numbers' requires 'length' or 'offset, length'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception("Table function '" + getName() + "' requires 'length' or 'offset, length'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); UInt64 offset = arguments.size() == 2 ? evaluateArgument(context, arguments[0]) : 0; UInt64 length = arguments.size() == 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]); - auto res = StorageSystemNumbers::create(table_name, false, length, offset); + auto res = StorageSystemNumbers::create(table_name, multithreaded, length, offset); res->startup(); return res; } - throw Exception("Table function 'numbers' requires 'limit' or 'offset, limit'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception("Table function '" + getName() + "' requires 'limit' or 'offset, limit'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } void registerTableFunctionNumbers(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction>(); + factory.registerFunction>(); } - -UInt64 TableFunctionNumbers::evaluateArgument(const Context & context, ASTPtr & argument) const +template +UInt64 TableFunctionNumbers::evaluateArgument(const Context & context, ASTPtr & argument) const { return evaluateConstantExpressionOrIdentifierAsLiteral(argument, context)->as().value.safeGet(); } diff --git a/dbms/src/TableFunctions/TableFunctionNumbers.h b/dbms/src/TableFunctions/TableFunctionNumbers.h index 733b4508f51..e5ab38ccad8 100644 --- a/dbms/src/TableFunctions/TableFunctionNumbers.h +++ b/dbms/src/TableFunctions/TableFunctionNumbers.h @@ -7,14 +7,15 @@ namespace DB { -/* numbers(limit) +/* numbers(limit), numbers_mt(limit) * - the same as SELECT number FROM system.numbers LIMIT limit. * Used for testing purposes, as a simple example of table function. */ +template class TableFunctionNumbers : public ITableFunction { public: - static constexpr auto name = "numbers"; + static constexpr auto name = multithreaded ? "numbers_mt" : "numbers"; std::string getName() const override { return name; } private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; From 619ee4489ec4387e06cddfa9a8e37375df329572 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 16:26:37 +0300 Subject: [PATCH 241/509] Added 00991_live_view_watch_event_live. --- .../00991_live_view_watch_event_live.python | 81 +++++++++++++++++++ ...00991_live_view_watch_event_live.reference | 8 ++ .../00991_live_view_watch_event_live.sh | 6 ++ 3 files changed, 95 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python create mode 100644 dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference create mode 100755 dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python new file mode 100644 index 00000000000..6f2e55f72e6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python @@ -0,0 +1,81 @@ +#!/usr/bin/env python + +import subprocess +import threading +import Queue as queue +import os +import sys +import signal + + +CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') +CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') +CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') + + +def send_query(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + + +def send_query_in_process_group(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) + + +def read_lines_and_push_to_queue(pipe, queue): + try: + for line in iter(pipe.readline, ''): + line = line.strip() + print(line) + sys.stdout.flush() + queue.put(line) + except KeyboardInterrupt: + pass + + queue.put(None) + + +def test(): + send_query('DROP TABLE IF EXISTS test.lv').read() + send_query('DROP TABLE IF EXISTS test.mt').read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() + send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + + q = queue.Queue() + p = send_query_in_process_group('WATCH test.lv') + thread = threading.Thread(target=read_lines_and_push_to_queue, args=(p.stdout, q)) + thread.start() + + line = q.get() + print(line) + assert (line == '0\t1') + + send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + line = q.get() + print(line) + assert (line == '6\t2') + + send_query('INSERT INTO test.mt VALUES (4),(5),(6)').read() + line = q.get() + print(line) + assert (line == '21\t3') + + # Send Ctrl+C to client. + os.killpg(os.getpgid(p.pid), signal.SIGINT) + # This insert shouldn't affect lv. + send_query('INSERT INTO test.mt VALUES (7),(8),(9)').read() + line = q.get() + print(line) + assert (line is None) + + send_query('DROP TABLE if exists test.lv').read() + send_query('DROP TABLE if exists test.lv').read() + + thread.join() + +test() diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference new file mode 100644 index 00000000000..e48dcf35d3e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference @@ -0,0 +1,8 @@ +['/home/nik-kochetov/dev/ClickHouse/build-tsan/dbms/programs/clickhouse', 'client', '--send_logs_level=warning', '--database=default', '--query', 'WATCH test.lv'] +0 1 +0 1 +6 2 +6 2 +21 3 +21 3 +None diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh new file mode 100755 index 00000000000..10e4e98b2e3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +python $CURDIR/00991_live_view_watch_event_live.python From e188ae40ceea110463a4d58f644e96d35b71be3d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 16:35:00 +0300 Subject: [PATCH 242/509] Added 00991_temporary_live_view_watch_live. --- ...0991_temporary_live_view_watch_live.python | 81 +++++++++++++++++++ ...1_temporary_live_view_watch_live.reference | 8 ++ .../00991_temporary_live_view_watch_live.sh | 6 ++ 3 files changed, 95 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python create mode 100644 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference create mode 100755 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python new file mode 100644 index 00000000000..341a9f04c56 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python @@ -0,0 +1,81 @@ +#!/usr/bin/env python + +import subprocess +import threading +import Queue as queue +import os +import sys +import signal + + +CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') +CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') +CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') + + +def send_query(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + + +def send_query_in_process_group(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) + + +def read_lines_and_push_to_queue(pipe, queue): + try: + for line in iter(pipe.readline, ''): + line = line.strip() + print(line) + sys.stdout.flush() + queue.put(line) + except KeyboardInterrupt: + pass + + queue.put(None) + + +def test(): + send_query('DROP TABLE IF EXISTS test.lv').read() + send_query('DROP TABLE IF EXISTS test.mt').read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() + send_query('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + + q = queue.Queue() + p = send_query_in_process_group('WATCH test.lv') + thread = threading.Thread(target=read_lines_and_push_to_queue, args=(p.stdout, q)) + thread.start() + + line = q.get() + print(line) + assert (line == '0\t1') + + send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + line = q.get() + print(line) + assert (line == '6\t2') + + send_query('INSERT INTO test.mt VALUES (4),(5),(6)').read() + line = q.get() + print(line) + assert (line == '21\t3') + + # Send Ctrl+C to client. + os.killpg(os.getpgid(p.pid), signal.SIGINT) + # This insert shouldn't affect lv. + send_query('INSERT INTO test.mt VALUES (7),(8),(9)').read() + line = q.get() + print(line) + assert (line is None) + + send_query('DROP TABLE if exists test.lv').read() + send_query('DROP TABLE if exists test.lv').read() + + thread.join() + +test() diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference new file mode 100644 index 00000000000..e48dcf35d3e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference @@ -0,0 +1,8 @@ +['/home/nik-kochetov/dev/ClickHouse/build-tsan/dbms/programs/clickhouse', 'client', '--send_logs_level=warning', '--database=default', '--query', 'WATCH test.lv'] +0 1 +0 1 +6 2 +6 2 +21 3 +21 3 +None diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh new file mode 100755 index 00000000000..4d01d1c3a8e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +python $CURDIR/00991_temporary_live_view_watch_live.python From a466040201d23e228c5c8f3b7f8d163ee73f3122 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 17:39:35 +0300 Subject: [PATCH 243/509] Added 00991_temporary_live_view_watch_events_heartbeat. --- .../00991_live_view_watch_event_live.python | 2 +- ...00991_live_view_watch_event_live.reference | 1 - ...ry_live_view_watch_events_heartbeat.python | 83 +++++++++++++++++++ ...live_view_watch_events_heartbeat.reference | 0 ...porary_live_view_watch_events_heartbeat.sh | 6 ++ ...0991_temporary_live_view_watch_live.python | 2 +- ...1_temporary_live_view_watch_live.reference | 1 - 7 files changed, 91 insertions(+), 4 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python create mode 100644 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.reference create mode 100755 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python index 6f2e55f72e6..782671cdfaf 100644 --- a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python @@ -23,7 +23,7 @@ def send_query(query): def send_query_in_process_group(query): cmd = list(CLICKHOUSE_CLIENT.split()) cmd += ['--query', query] - print(cmd) + # print(cmd) return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference index e48dcf35d3e..1e94cdade41 100644 --- a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference @@ -1,4 +1,3 @@ -['/home/nik-kochetov/dev/ClickHouse/build-tsan/dbms/programs/clickhouse', 'client', '--send_logs_level=warning', '--database=default', '--query', 'WATCH test.lv'] 0 1 0 1 6 2 diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python new file mode 100644 index 00000000000..70063adc6e3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python @@ -0,0 +1,83 @@ +#!/usr/bin/env python + +import subprocess +import threading +import Queue as queue +import os +import sys +import signal + + +CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') +CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') +CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') + + +def send_query(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + + +def send_query_in_process_group(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query, '--live_view_heartbeat_interval=1', '--progress'] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) + + +def read_lines_and_push_to_queue(pipe, queue): + try: + for line in iter(pipe.readline, ''): + line = line.strip() + # print(line) + sys.stdout.flush() + queue.put(line) + except KeyboardInterrupt: + pass + + queue.put(None) + + +def test(): + send_query('DROP TABLE IF EXISTS test.lv').read() + send_query('DROP TABLE IF EXISTS test.mt').read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() + send_query('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + + q = queue.Queue() + p = send_query_in_process_group('WATCH test.lv') + thread = threading.Thread(target=read_lines_and_push_to_queue, args=(p.stdout, q)) + thread.start() + + line = q.get() + # print(line) + assert (line.endswith('0\t1')) + assert ('Progress: 0.00 rows' in line) + + send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + line = q.get() + assert (line.endswith('6\t2')) + assert ('Progress: 1.00 rows' in line) + + # send_query('INSERT INTO test.mt VALUES (4),(5),(6)').read() + # line = q.get() + # print(line) + # assert (line.endswith('6\t2')) + # assert ('Progress: 1.00 rows' in line) + + # Send Ctrl+C to client. + os.killpg(os.getpgid(p.pid), signal.SIGINT) + # This insert shouldn't affect lv. + send_query('INSERT INTO test.mt VALUES (7),(8),(9)').read() + line = q.get() + # print(line) + # assert (line is None) + + send_query('DROP TABLE if exists test.lv').read() + send_query('DROP TABLE if exists test.lv').read() + + thread.join() + +test() diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.reference b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh new file mode 100755 index 00000000000..f7aa13d52b3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +python $CURDIR/00991_temporary_live_view_watch_events_heartbeat.python diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python index 341a9f04c56..d290018a02c 100644 --- a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python @@ -23,7 +23,7 @@ def send_query(query): def send_query_in_process_group(query): cmd = list(CLICKHOUSE_CLIENT.split()) cmd += ['--query', query] - print(cmd) + # print(cmd) return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference index e48dcf35d3e..1e94cdade41 100644 --- a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference @@ -1,4 +1,3 @@ -['/home/nik-kochetov/dev/ClickHouse/build-tsan/dbms/programs/clickhouse', 'client', '--send_logs_level=warning', '--database=default', '--query', 'WATCH test.lv'] 0 1 0 1 6 2 From a4e2fcdd4933b33416a17c15afacf7330f81bc53 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 18:41:42 +0300 Subject: [PATCH 244/509] Remove recursiveMaterializeConstants. --- dbms/src/Columns/ColumnConst.cpp | 32 ------------------- dbms/src/Columns/ColumnConst.h | 3 -- .../ConvertingBlockInputStream.cpp | 4 +-- 3 files changed, 2 insertions(+), 37 deletions(-) diff --git a/dbms/src/Columns/ColumnConst.cpp b/dbms/src/Columns/ColumnConst.cpp index 1d186f5f70e..91036499871 100644 --- a/dbms/src/Columns/ColumnConst.cpp +++ b/dbms/src/Columns/ColumnConst.cpp @@ -1,8 +1,6 @@ #include #include -#include -#include #include #include @@ -104,34 +102,4 @@ void ColumnConst::getPermutation(bool /*reverse*/, size_t /*limit*/, int /*nan_d res[i] = i; } - -ColumnPtr recursiveMaterializeConstants(const ColumnPtr & column) -{ - if (!column) - return column; - - if (const auto * column_array = typeid_cast(column.get())) - { - auto & data = column_array->getDataPtr(); - auto data_no_const = recursiveMaterializeConstants(data); - if (data.get() == data_no_const.get()) - return column; - - return ColumnArray::create(data_no_const, column_array->getOffsetsPtr()); - } - - if (const auto * column_tuple = typeid_cast(column.get())) - { - auto columns = column_tuple->getColumns(); - for (auto & element : columns) - element = recursiveMaterializeConstants(element); - return ColumnTuple::create(columns); - } - - if (const auto * column_const = typeid_cast(column.get())) - return column_const->convertToFullColumn(); - - return column; -} - } diff --git a/dbms/src/Columns/ColumnConst.h b/dbms/src/Columns/ColumnConst.h index dc7a748cfbe..6b320f12f28 100644 --- a/dbms/src/Columns/ColumnConst.h +++ b/dbms/src/Columns/ColumnConst.h @@ -222,7 +222,4 @@ public: T getValue() const { return getField().safeGet>(); } }; -/// Materialize constants recursively for all nested columns. -ColumnPtr recursiveMaterializeConstants(const ColumnPtr & column); - } diff --git a/dbms/src/DataStreams/ConvertingBlockInputStream.cpp b/dbms/src/DataStreams/ConvertingBlockInputStream.cpp index ae4d2e592e0..0ab250ac7ee 100644 --- a/dbms/src/DataStreams/ConvertingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ConvertingBlockInputStream.cpp @@ -103,8 +103,8 @@ Block ConvertingBlockInputStream::readImpl() ColumnPtr converted = castColumnWithDiagnostic(src_elem, res_elem, context); - if (!isColumnConst(*res_elem.column)) - converted = recursiveMaterializeConstants(converted); + if (isColumnConst(*src_elem.column) && !isColumnConst(*res_elem.column)) + converted = converted->convertToFullColumnIfConst(); res_elem.column = std::move(converted); } From d4363768e60e5e2e2fb77aa27d9ced5fbe1b88c2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 20:48:19 +0300 Subject: [PATCH 245/509] Added IFunctonBase::getResultIfAlwaysReturnsConstantAndHasArguments --- dbms/src/Functions/IFunction.h | 12 +++- .../Functions/defaultValueOfArgumentType.cpp | 6 ++ dbms/src/Functions/getSizeOfEnumType.cpp | 9 ++- dbms/src/Functions/ignore.cpp | 5 +- dbms/src/Functions/toColumnTypeName.cpp | 5 ++ dbms/src/Functions/toTypeName.cpp | 71 +++++++++++++------ .../ClusterProxy/SelectStreamFactory.cpp | 5 +- dbms/src/Interpreters/ExpressionActions.cpp | 9 ++- 8 files changed, 92 insertions(+), 30 deletions(-) diff --git a/dbms/src/Functions/IFunction.h b/dbms/src/Functions/IFunction.h index e5c24b98cec..ce15891f30a 100644 --- a/dbms/src/Functions/IFunction.h +++ b/dbms/src/Functions/IFunction.h @@ -159,7 +159,12 @@ public: */ virtual bool isSuitableForConstantFolding() const { return true; } - virtual bool alwaysReturnsConstant() const { return false; } + /** Some functions like ignore(...) or toTypeName(...) always return constant result which doesn't depend on arguments. + * In this case we can calculate result and assume that it's constant in stream header. + * There is no need to implement function if it has zero arguments. + * Must return ColumnConst with single row or nullptr. + */ + virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & /*block*/, const ColumnNumbers & /*arguments*/) const { return nullptr; } /** Function is called "injective" if it returns different result for different values of arguments. * Example: hex, negate, tuple... @@ -458,7 +463,10 @@ public: } bool isSuitableForConstantFolding() const override { return function->isSuitableForConstantFolding(); } - bool alwaysReturnsConstant() const override { return function->alwaysReturnsConstant(); } + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments_) const override + { + return function->getResultIfAlwaysReturnsConstantAndHasArguments(block, arguments_); + } bool isInjective(const Block & sample_block) override { return function->isInjective(sample_block); } diff --git a/dbms/src/Functions/defaultValueOfArgumentType.cpp b/dbms/src/Functions/defaultValueOfArgumentType.cpp index 452f25b51d8..db1a1533163 100644 --- a/dbms/src/Functions/defaultValueOfArgumentType.cpp +++ b/dbms/src/Functions/defaultValueOfArgumentType.cpp @@ -37,6 +37,12 @@ public: const IDataType & type = *block.getByPosition(arguments[0]).type; block.getByPosition(result).column = type.createColumnConst(input_rows_count, type.getDefault()); } + + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments) const override + { + const IDataType & type = *block.getByPosition(arguments[0]).type; + return type.createColumnConst(1, type.getDefault()); + } }; diff --git a/dbms/src/Functions/getSizeOfEnumType.cpp b/dbms/src/Functions/getSizeOfEnumType.cpp index 11a22ecddfb..9b598ccca3c 100644 --- a/dbms/src/Functions/getSizeOfEnumType.cpp +++ b/dbms/src/Functions/getSizeOfEnumType.cpp @@ -49,11 +49,16 @@ public: } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = getResultIfAlwaysReturnsConstantAndHasArguments(block, arguments)->cloneResized(input_rows_count); + } + + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments) const override { if (auto type8 = checkAndGetDataType(block.getByPosition(arguments[0]).type.get())) - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, type8->getValues().size()); + return DataTypeUInt8().createColumnConst(1, type8->getValues().size()); else if (auto type16 = checkAndGetDataType(block.getByPosition(arguments[0]).type.get())) - block.getByPosition(result).column = DataTypeUInt16().createColumnConst(input_rows_count, type16->getValues().size()); + return DataTypeUInt16().createColumnConst(1, type16->getValues().size()); else throw Exception("The argument for function " + getName() + " must be Enum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } diff --git a/dbms/src/Functions/ignore.cpp b/dbms/src/Functions/ignore.cpp index c3ed34ac994..592de576659 100644 --- a/dbms/src/Functions/ignore.cpp +++ b/dbms/src/Functions/ignore.cpp @@ -43,7 +43,10 @@ public: block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, 0u); } - bool alwaysReturnsConstant() const override { return true; } + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block &, const ColumnNumbers &) const override + { + return DataTypeUInt8().createColumnConst(1, 0u); + } }; diff --git a/dbms/src/Functions/toColumnTypeName.cpp b/dbms/src/Functions/toColumnTypeName.cpp index 35d332d082f..023a9350547 100644 --- a/dbms/src/Functions/toColumnTypeName.cpp +++ b/dbms/src/Functions/toColumnTypeName.cpp @@ -38,6 +38,11 @@ public: block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, block.getByPosition(arguments[0]).column->getName()); } + + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments) const override + { + return DataTypeString().createColumnConst(1, block.getByPosition(arguments[0]).type->createColumn()->getName()); + } }; diff --git a/dbms/src/Functions/toTypeName.cpp b/dbms/src/Functions/toTypeName.cpp index a631c7f4044..22d7d7eb023 100644 --- a/dbms/src/Functions/toTypeName.cpp +++ b/dbms/src/Functions/toTypeName.cpp @@ -9,33 +9,16 @@ namespace DB /** toTypeName(x) - get the type name * Returns name of IDataType instance (name of data type). */ -class FunctionToTypeName : public IFunction +class PreparedFunctionToTypeName : public PreparedFunctionImpl { public: static constexpr auto name = "toTypeName"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } + String getName() const override { return name; } +protected: bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - /// Execute the function on the block. void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { @@ -45,9 +28,55 @@ public: }; +class BaseFunctionToTypeName : public IFunctionBase +{ +public: + BaseFunctionToTypeName(DataTypes argument_types_, DataTypePtr return_type_) + : argument_types(std::move(argument_types_)), return_type(std::move(return_type_)) {} + + static constexpr auto name = "toTypeName"; + String getName() const override { return name; } + + const DataTypes & getArgumentTypes() const override { return argument_types; } + const DataTypePtr & getReturnType() const override { return return_type; } + + PreparedFunctionPtr prepare(const Block &, const ColumnNumbers &, size_t) const override + { + return std::make_shared(); + } + + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block &, const ColumnNumbers &) const override + { + return DataTypeString().createColumnConst(1, argument_types.at(0)->getName()); + } + +private: + DataTypes argument_types; + DataTypePtr return_type; +}; + + +class FunctionToTypeNameBuilder : public FunctionBuilderImpl +{ + static constexpr auto name = "toTypeName"; + String getName() const override { return name; } + static FunctionBuilderPtr create(const Context &) { return std::make_shared(); } + + size_t getNumberOfArguments() const override { return 1; } + +protected: + DataTypePtr getReturnTypeImpl(const DataTypes &) const override { return std::make_shared(); } + + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + return std::make_shared(arguments, return_type); + } +}; + + void registerFunctionToTypeName(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 15cefe3c27c..905827205b4 100644 --- a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -68,7 +68,10 @@ BlockInputStreamPtr createLocalStream(const ASTPtr & query_ast, const Context & * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, * And this is not allowed, since all code is based on the assumption that in the block stream all types are the same. */ - /// Already not. + + /* Now we don't need to materialize constants, because RemoteBlockInputStream will ignore constant and take it from header. + * So, streams from different threads will always have the same header. + */ /// return std::make_shared(stream); return stream; diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index d1808442ab2..75545abc210 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -246,10 +246,13 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings, } auto & res = sample_block.getByPosition(result_position); - if (!res.column && function_base->alwaysReturnsConstant() && function_base->isSuitableForConstantFolding()) + if (!res.column && function_base->isSuitableForConstantFolding()) { - res.column = result_type->createColumnConstWithDefaultValue(1); - names_not_for_constant_folding.insert(result_name); + if (auto col = function_base->getResultIfAlwaysReturnsConstantAndHasArguments(sample_block, arguments)) + { + res.column = std::move(col); + names_not_for_constant_folding.insert(result_name); + } } break; From 3277f9e3deb862cee88d3c5210c86b9a61545ce1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 20:56:00 +0300 Subject: [PATCH 246/509] Add patch to Live View tests. --- ...nt_live.sh => 00991_live_view_watch_event_live.sh.disabled} | 0 ...ew_watch_http.sh => 00991_live_view_watch_http.sh.disabled} | 0 ...991_temporary_live_view_watch_events_heartbeat.sh.disabled} | 0 ...ive.sh => 00991_temporary_live_view_watch_live.sh.disabled} | 0 dbms/tests/queries/0_stateless/helpers/uexpect.py | 3 ++- 5 files changed, 2 insertions(+), 1 deletion(-) rename dbms/tests/queries/0_stateless/{00991_live_view_watch_event_live.sh => 00991_live_view_watch_event_live.sh.disabled} (100%) rename dbms/tests/queries/0_stateless/{00991_live_view_watch_http.sh => 00991_live_view_watch_http.sh.disabled} (100%) rename dbms/tests/queries/0_stateless/{00991_temporary_live_view_watch_events_heartbeat.sh => 00991_temporary_live_view_watch_events_heartbeat.sh.disabled} (100%) rename dbms/tests/queries/0_stateless/{00991_temporary_live_view_watch_live.sh => 00991_temporary_live_view_watch_live.sh.disabled} (100%) diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled similarity index 100% rename from dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh rename to dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled similarity index 100% rename from dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh rename to dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled similarity index 100% rename from dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh rename to dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled similarity index 100% rename from dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh rename to dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled diff --git a/dbms/tests/queries/0_stateless/helpers/uexpect.py b/dbms/tests/queries/0_stateless/helpers/uexpect.py index 930a45a099e..f71b32a53e1 100644 --- a/dbms/tests/queries/0_stateless/helpers/uexpect.py +++ b/dbms/tests/queries/0_stateless/helpers/uexpect.py @@ -101,6 +101,7 @@ class IO(object): def close(self, force=True): self.reader['kill_event'].set() + os.system('pkill -TERM -P %d' % self.process.pid) if force: self.process.kill() else: @@ -199,7 +200,7 @@ def reader(process, out, queue, kill_event): try: data = os.read(out, 65536) queue.put(data) - except OSError, e: + except: if kill_event.is_set(): break raise From 04c17c56c2a3fc71083df08c84c0610dcbe59398 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 21:10:40 +0300 Subject: [PATCH 247/509] Review fixes. --- dbms/src/Interpreters/ExpressionActions.cpp | 6 +++- dbms/src/Interpreters/ExpressionActions.h | 1 - .../Interpreters/InterpreterSelectQuery.cpp | 32 ++++++++----------- .../src/Interpreters/InterpreterSelectQuery.h | 4 +-- dbms/src/Interpreters/QueryAliasesVisitor.h | 1 + 5 files changed, 21 insertions(+), 23 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 75545abc210..c7b510abcf0 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -170,7 +170,6 @@ ExpressionAction ExpressionAction::ordinaryJoin( a.type = JOIN; a.join = std::move(join_); a.join_kind = join_params.kind; - // a.join_strictness = join_params.strictness; a.join_key_names_left = join_key_names_left; a.join_key_names_right = join_key_names_right; a.columns_added_by_join = columns_added_by_join_; @@ -245,6 +244,9 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings, } } + /// Some functions like ignore() or getTypeName() always return constant result even if arguments are not constant. + /// We can't do constant folding, but can specify in sample block that function result is constant to avoid + /// unnecessary materialization. auto & res = sample_block.getByPosition(result_position); if (!res.column && function_base->isSuitableForConstantFolding()) { @@ -282,6 +284,8 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings, for (auto & col : sample_block) { /// Materialize column. + /// Column is not empty if it is constant, but after Join all constants will be materialized. + /// So, we need remove constants from header. if (col.column) col.column = nullptr; diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index 8b959235cc5..62d50131b45 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -106,7 +106,6 @@ public: /// For JOIN std::shared_ptr join; ASTTableJoin::Kind join_kind; - // ASTTableJoin::Strictness join_strictness; Names join_key_names_left; Names join_key_names_right; NamesAndTypesList columns_added_by_join; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 37e895331ab..91f5168b548 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -2127,7 +2127,7 @@ void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline) /// If there are several streams, then we merge them into one if (pipeline.hasMoreThanOneStream()) { - unifyStreams(pipeline); + unifyStreams(pipeline, pipeline.streams.at(0)->getHeader()); /** MergingSortedBlockInputStream reads the sources sequentially. * To make the data on the remote servers prepared in parallel, we wrap it in AsynchronousBlockInputStream. @@ -2466,26 +2466,20 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pip void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline, Block header) { - if (pipeline.hasMoreThanOneStream() || header) + /// Unify streams in case they have different headers. + + /// TODO: remove previos addition of _dummy column. + if (header.columns() > 1 && header.has("_dummy")) + header.erase("_dummy"); + + for (size_t i = 0; i < pipeline.streams.size(); ++i) { - /// Unify streams in case they have different headers. + auto & stream = pipeline.streams[i]; + auto stream_header = stream->getHeader(); + auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; - if (!header) - header = pipeline.streams.at(0)->getHeader(); - - /// TODO: remove _dummy column - if (header.columns() > 1 && header.has("_dummy")) - header.erase("_dummy"); - - for (size_t i = 0; i < pipeline.streams.size(); ++i) - { - auto & stream = pipeline.streams[i]; - auto stream_header = stream->getHeader(); - auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; - - if (!blocksHaveEqualStructure(header, stream_header)) - stream = std::make_shared(context, stream, header, mode); - } + if (!blocksHaveEqualStructure(header, stream_header)) + stream = std::make_shared(context, stream, header, mode); } } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index fc3e46dd345..4271e9320a7 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -230,8 +230,8 @@ private: void executeExtremes(QueryPipeline & pipeline); void executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, std::unordered_map & subqueries_for_sets); - /// If pipeline has several streams with different headers, add ConvertingBlockInputStream to first header. - void unifyStreams(Pipeline & pipeline, Block header = {}); + /// Add ConvertingBlockInputStream to specified header. + void unifyStreams(Pipeline & pipeline, Block header); enum class Modificator { diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.h b/dbms/src/Interpreters/QueryAliasesVisitor.h index 5a510a1fd86..c4e297965c3 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.h +++ b/dbms/src/Interpreters/QueryAliasesVisitor.h @@ -2,6 +2,7 @@ #include #include + namespace DB { From d2e5789f82693bbddc371b5f3861322895f2fb56 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Aug 2019 21:12:22 +0300 Subject: [PATCH 248/509] Fixed strange code in mutations that may lead to memory corruption --- dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 665e8c9bd5c..4dcd43add30 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1281,9 +1281,9 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep } if (!finished.empty()) + { zookeeper->set(replica_path + "/mutation_pointer", finished.back()->znode_name); - { std::lock_guard lock(state_mutex); mutation_pointer = finished.back()->znode_name; From 8535ba9e4705aafd33cacf9690f8f363e1abc49b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 21:23:37 +0300 Subject: [PATCH 249/509] Review fixes. --- .../Interpreters/InterpreterSelectQuery.cpp | 19 +++++++++++-------- .../src/Interpreters/InterpreterSelectQuery.h | 2 +- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 91f5168b548..27be35ad57f 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1192,7 +1192,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if constexpr (pipeline_with_processors) pipeline.resize(1); else - executeUnion(pipeline); + executeUnion(pipeline, {}); } /** If there was more than one stream, @@ -1791,7 +1791,7 @@ void InterpreterSelectQuery::executeMergeAggregated(Pipeline & pipeline, bool ov if (!settings.distributed_aggregation_memory_efficient) { /// We union several sources into one, parallelizing the work. - executeUnion(pipeline); + executeUnion(pipeline, {}); /// Now merge the aggregated blocks pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params, final, settings.max_threads); @@ -1893,7 +1893,7 @@ void InterpreterSelectQuery::executeHaving(QueryPipeline & pipeline, const Expre void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final) { - executeUnion(pipeline); + executeUnion(pipeline, {}); const Settings & settings = context.getSettingsRef(); @@ -1922,7 +1922,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(QueryPipeline & pipeline, bo void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificator modificator) { - executeUnion(pipeline); + executeUnion(pipeline, {}); Names key_names; AggregateDescriptions aggregates; @@ -2067,7 +2067,7 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr so }); /// If there are several streams, we merge them into one - executeUnion(pipeline); + executeUnion(pipeline, {}); /// Merge the sorted blocks. pipeline.firstStream() = std::make_shared( @@ -2127,7 +2127,7 @@ void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline) /// If there are several streams, then we merge them into one if (pipeline.hasMoreThanOneStream()) { - unifyStreams(pipeline, pipeline.streams.at(0)->getHeader()); + unifyStreams(pipeline, pipeline.firstStream()->getHeader()); /** MergingSortedBlockInputStream reads the sources sequentially. * To make the data on the remote servers prepared in parallel, we wrap it in AsynchronousBlockInputStream. @@ -2236,7 +2236,10 @@ void InterpreterSelectQuery::executeUnion(Pipeline & pipeline, Block header) /// If there are still several streams, then we combine them into one if (pipeline.hasMoreThanOneStream()) { - unifyStreams(pipeline, header); + if (!header) + header = pipeline.firstStream()->getHeader(); + + unifyStreams(pipeline, std::move(header)); pipeline.firstStream() = std::make_shared(pipeline.streams, pipeline.stream_with_non_joined_data, max_streams); pipeline.stream_with_non_joined_data = nullptr; @@ -2446,7 +2449,7 @@ void InterpreterSelectQuery::executeExtremes(QueryPipeline & pipeline) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline, SubqueriesForSets & subqueries_for_sets) { - executeUnion(pipeline); + executeUnion(pipeline, {}); pipeline.firstStream() = std::make_shared( pipeline.firstStream(), subqueries_for_sets, context); } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 4271e9320a7..3a441445c9b 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -206,7 +206,7 @@ private: void executeOrder(Pipeline & pipeline, SortingInfoPtr sorting_info); void executeMergeSorted(Pipeline & pipeline); void executePreLimit(Pipeline & pipeline); - void executeUnion(Pipeline & pipeline, Block header = {}); + void executeUnion(Pipeline & pipeline, Block header); /// If header is not empty, convert streams structure to it. void executeLimitBy(Pipeline & pipeline); void executeLimit(Pipeline & pipeline); void executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression); From 56a9c3920fd6d10b8889c9052af1993353f5375c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 21:28:48 +0300 Subject: [PATCH 250/509] Fix build. --- dbms/src/Functions/toTypeName.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Functions/toTypeName.cpp b/dbms/src/Functions/toTypeName.cpp index 22d7d7eb023..0dd26ea734c 100644 --- a/dbms/src/Functions/toTypeName.cpp +++ b/dbms/src/Functions/toTypeName.cpp @@ -58,6 +58,7 @@ private: class FunctionToTypeNameBuilder : public FunctionBuilderImpl { +public: static constexpr auto name = "toTypeName"; String getName() const override { return name; } static FunctionBuilderPtr create(const Context &) { return std::make_shared(); } From d359ff3d4af6be93ea965b79d46b893f7ccec264 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 21:34:12 +0300 Subject: [PATCH 251/509] Fix build. --- dbms/src/Functions/toTypeName.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dbms/src/Functions/toTypeName.cpp b/dbms/src/Functions/toTypeName.cpp index 0dd26ea734c..36a2f396aab 100644 --- a/dbms/src/Functions/toTypeName.cpp +++ b/dbms/src/Functions/toTypeName.cpp @@ -70,7 +70,12 @@ protected: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override { - return std::make_shared(arguments, return_type); + DataTypes types; + types.reserve(arguments.size()); + for (auto & elem : arguments) + types.emplace_back(elem.type); + + return std::make_shared(types, return_type); } }; From 19a303f4986482c380cff3c87cd2ee80dd7254c2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Aug 2019 21:55:42 +0300 Subject: [PATCH 252/509] Update perf tests with hash functions. --- .../performance/general_purpose_hashes.xml | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/dbms/tests/performance/general_purpose_hashes.xml b/dbms/tests/performance/general_purpose_hashes.xml index c2462f8710b..14da22e6c97 100644 --- a/dbms/tests/performance/general_purpose_hashes.xml +++ b/dbms/tests/performance/general_purpose_hashes.xml @@ -1,18 +1,18 @@ - once - - - 10000 + + 3 + 10000 - 5000 - 20000 + 5 + 60000 + - + @@ -45,11 +45,11 @@ table - numbers - numbers_mt + numbers(1000000) + numbers_mt(10000000) - SELECT ignore({gp_hash_func}({string})) FROM system.{table} + SELECT count() from {table} where not ignore({gp_hash_func}({string})) From 22a11343e82f035c874e1353feb046975e64b121 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Aug 2019 22:02:20 +0300 Subject: [PATCH 253/509] Attempt to fix race condition --- dbms/src/Interpreters/InterpreterAlterQuery.cpp | 3 ++- dbms/src/Interpreters/MutationsInterpreter.cpp | 4 ++-- dbms/src/Interpreters/MutationsInterpreter.h | 5 ++--- .../src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 8 ++++---- dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 4 ++-- dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 4 ++-- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 8 files changed, 17 insertions(+), 17 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index 8751ff067b1..9a033978bdb 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -62,7 +62,8 @@ BlockIO InterpreterAlterQuery::execute() if (!mutation_commands.empty()) { - MutationsInterpreter(table, mutation_commands, context).validate(); + auto table_lock_holder = table->lockStructureForShare(false /* because mutation is executed asyncronously */, context.getCurrentQueryId()); + MutationsInterpreter(table, mutation_commands, context).validate(table_lock_holder); table->mutate(mutation_commands, context); } diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 160d0bc8023..f58194256e2 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -458,7 +458,7 @@ BlockInputStreamPtr MutationsInterpreter::addStreamsForLaterStages(const std::ve return in; } -void MutationsInterpreter::validate() +void MutationsInterpreter::validate(TableStructureReadLockHolder &) { prepare(/* dry_run = */ true); Block first_stage_header = interpreter_select->getSampleBlock(); @@ -466,7 +466,7 @@ void MutationsInterpreter::validate() addStreamsForLaterStages(stages, in)->getHeader(); } -BlockInputStreamPtr MutationsInterpreter::execute() +BlockInputStreamPtr MutationsInterpreter::execute(TableStructureReadLockHolder &) { prepare(/* dry_run = */ false); BlockInputStreamPtr in = interpreter_select->execute().in; diff --git a/dbms/src/Interpreters/MutationsInterpreter.h b/dbms/src/Interpreters/MutationsInterpreter.h index 3fa8961e8f7..4b9a295b3d1 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.h +++ b/dbms/src/Interpreters/MutationsInterpreter.h @@ -25,13 +25,13 @@ public: { } - void validate(); + void validate(TableStructureReadLockHolder & table_lock_holder); /// Return false if the data isn't going to be changed by mutations. bool isStorageTouchedByMutations() const; /// The resulting stream will return blocks containing only changed columns and columns, that we need to recalculate indices. - BlockInputStreamPtr execute(); + BlockInputStreamPtr execute(TableStructureReadLockHolder & table_lock_holder); /// Only changed columns. const Block & getUpdatedHeader() const; @@ -44,7 +44,6 @@ private: std::unique_ptr prepareInterpreterSelect(std::vector & prepared_stages, bool dry_run); BlockInputStreamPtr addStreamsForLaterStages(const std::vector & prepared_stages, BlockInputStreamPtr in) const; -private: StoragePtr storage; std::vector commands; const Context & context; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 74193fa7156..9c9dbe2dfb6 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -521,7 +521,7 @@ public: /// parts should be sorted. MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( - const FutureMergedMutatedPart & future_part, MergeList::Entry & merge_entry, + const FutureMergedMutatedPart & future_part, MergeList::Entry & merge_entry, TableStructureReadLockHolder &, time_t time_of_merge, DiskSpaceMonitor::Reservation * disk_reservation, bool deduplicate, bool force_ttl) { static const String TMP_PREFIX = "tmp_merge_"; @@ -883,7 +883,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor const FutureMergedMutatedPart & future_part, const std::vector & commands, MergeListEntry & merge_entry, - const Context & context) + const Context & context, + TableStructureReadLockHolder & table_lock_holder) { auto check_not_cancelled = [&]() { @@ -918,7 +919,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor command.partition, context_for_reading); }); - MutationsInterpreter mutations_interpreter(storage_from_source_part, commands_for_part, context_for_reading); if (!mutations_interpreter.isStorageTouchedByMutations()) @@ -949,7 +949,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor Poco::File(new_part_tmp_path).createDirectories(); - auto in = mutations_interpreter.execute(); + auto in = mutations_interpreter.execute(table_lock_holder); const auto & updated_header = mutations_interpreter.getUpdatedHeader(); NamesAndTypesList all_columns = data.getColumns().getAllPhysical(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 43e3cd15910..914bbc8d06c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -94,14 +94,14 @@ public: */ MergeTreeData::MutableDataPartPtr mergePartsToTemporaryPart( const FutureMergedMutatedPart & future_part, - MergeListEntry & merge_entry, time_t time_of_merge, + MergeListEntry & merge_entry, TableStructureReadLockHolder & table_lock_holder, time_t time_of_merge, DiskSpaceMonitor::Reservation * disk_reservation, bool deduplication, bool force_ttl); /// Mutate a single data part with the specified commands. Will create and return a temporary part. MergeTreeData::MutableDataPartPtr mutatePartToTemporaryPart( const FutureMergedMutatedPart & future_part, const std::vector & commands, - MergeListEntry & merge_entry, const Context & context); + MergeListEntry & merge_entry, const Context & context, TableStructureReadLockHolder & table_lock_holder); MergeTreeData::DataPartPtr renameMergedTemporaryPart( MergeTreeData::MutableDataPartPtr & new_data_part, diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 4dcd43add30..28a6d017f22 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1655,7 +1655,7 @@ bool ReplicatedMergeTreeMergePredicate::operator()( std::optional ReplicatedMergeTreeMergePredicate::getDesiredMutationVersion(const MergeTreeData::DataPartPtr & part) const { /// Assigning mutations is easier than assigning merges because mutations appear in the same order as - /// the order of their version numbers (see StorageReplicatedMergeTree::mutate()). + /// the order of their version numbers (see StorageReplicatedMergeTree::mutate). /// This means that if we have loaded the mutation with version number X then all mutations with /// the version numbers less than X are also loaded and if there is no merge or mutation assigned to /// the part (checked by querying queue.virtual_parts), we can confidently assign a mutation to diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index d062bb197ca..3107ee4419c 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -596,7 +596,7 @@ bool StorageMergeTree::merge( bool force_ttl = (final && (hasTableTTL() || hasAnyColumnTTL())); new_part = merger_mutator.mergePartsToTemporaryPart( - future_part, *merge_entry, time(nullptr), + future_part, *merge_entry, table_lock_holder, time(nullptr), merging_tagger->reserved_space.get(), deduplicate, force_ttl); merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); removeEmptyColumnsFromPart(new_part); @@ -714,7 +714,7 @@ bool StorageMergeTree::tryMutatePart() try { - new_part = merger_mutator.mutatePartToTemporaryPart(future_part, commands, *merge_entry, global_context); + new_part = merger_mutator.mutatePartToTemporaryPart(future_part, commands, *merge_entry, global_context, table_lock_holder); renameTempPartAndReplace(new_part); tagger->is_successful = true; write_part_log({}); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 8b32cc32704..7153b5731db 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -1088,7 +1088,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) try { part = merger_mutator.mergePartsToTemporaryPart( - future_merged_part, *merge_entry, entry.create_time, reserved_space.get(), entry.deduplicate, entry.force_ttl); + future_merged_part, *merge_entry, table_lock, entry.create_time, reserved_space.get(), entry.deduplicate, entry.force_ttl); merger_mutator.renameMergedTemporaryPart(part, parts, &transaction); removeEmptyColumnsFromPart(part); @@ -1217,7 +1217,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM try { - new_part = merger_mutator.mutatePartToTemporaryPart(future_mutated_part, commands, *merge_entry, global_context); + new_part = merger_mutator.mutatePartToTemporaryPart(future_mutated_part, commands, *merge_entry, global_context, table_lock); renameTempPartAndReplace(new_part, nullptr, &transaction); try From 4809586ed6b18224a7c33269d96fd02f216addf9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Aug 2019 22:07:13 +0300 Subject: [PATCH 254/509] Updated instruction --- dbms/tests/instructions/sanitizers.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/instructions/sanitizers.md b/dbms/tests/instructions/sanitizers.md index 3ec8d30ae09..cfa465fcbda 100644 --- a/dbms/tests/instructions/sanitizers.md +++ b/dbms/tests/instructions/sanitizers.md @@ -40,7 +40,7 @@ ninja ## Start ClickHouse and run tests ``` -sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1,suppressions=../dbms/tests/tsan_suppressions.txt' ./clickhouse-tsan server --config /etc/clickhouse-server/config.xml +sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1' ./clickhouse-tsan server --config /etc/clickhouse-server/config.xml ``` From 5575be034dfe7dc90f5d52568a22ed053451b2d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Aug 2019 22:32:02 +0300 Subject: [PATCH 255/509] Whitespaces --- dbms/src/Storages/StorageMergeTree.cpp | 7 ++----- dbms/src/Storages/System/StorageSystemPartsBase.cpp | 1 - 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 3107ee4419c..11d378ecfa0 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -208,7 +208,6 @@ std::vector StorageMergeTree::prepar size_t thread_pool_size = std::min(parts.size(), settings_.max_alter_threads); ThreadPool thread_pool(thread_pool_size); - for (const auto & part : parts) { transactions.push_back(std::make_unique(part)); @@ -217,8 +216,7 @@ std::vector StorageMergeTree::prepar [this, & transaction = transactions.back(), & columns_for_parts, & new_indices = new_indices.indices] { this->alterDataPart(columns_for_parts, new_indices, false, transaction); - } - ); + }); } thread_pool.wait(); @@ -227,8 +225,7 @@ std::vector StorageMergeTree::prepar [](const MergeTreeData::AlterDataPartTransactionPtr & transaction) { return !transaction->isValid(); - } - ); + }); transactions.erase(erase_pos, transactions.end()); return transactions; diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.cpp b/dbms/src/Storages/System/StorageSystemPartsBase.cpp index 7a242e74e75..c78bd7f2430 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsBase.cpp @@ -152,7 +152,6 @@ StoragesInfo StoragesInfoStream::next() while (next_row < rows) { - info.database = (*database_column)[next_row].get(); info.table = (*table_column)[next_row].get(); From 5d17e0e0e0c69f239b11b612e147c7f2dad3913e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Aug 2019 22:51:53 +0300 Subject: [PATCH 256/509] Updated test --- .../0_stateless/00992_system_parts_race_condition_zookeeper.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh b/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh index 83c74b3363a..99c9e8774fe 100755 --- a/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh @@ -71,3 +71,4 @@ timeout $TIMEOUT bash -c thread5 2> /dev/null & wait $CLICKHOUSE_CLIENT -q "DROP TABLE alter_table" +$CLICKHOUSE_CLIENT -q "DROP TABLE alter_table2" From 48867580e67b2bc5663b63bf93a07fb4d6989f5e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Aug 2019 03:10:32 +0300 Subject: [PATCH 257/509] Added another test that produce segfault under ASan easily --- ..._system_parts_race_condition_drop.referece | 0 .../00993_system_parts_race_condition_drop.sh | 100 ++++++++++++++++++ 2 files changed, 100 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.referece create mode 100755 dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.sh diff --git a/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.referece b/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.referece new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.sh b/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.sh new file mode 100755 index 00000000000..064d425ec1e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.sh @@ -0,0 +1,100 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -e + +function thread1() +{ + while true; do + $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; + done +} + +function thread2() +{ + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table_$REPLICA ADD COLUMN h String; ALTER TABLE alter_table_$REPLICA MODIFY COLUMN h UInt64; ALTER TABLE alter_table_$REPLICA DROP COLUMN h;"; + done +} + +function thread3() +{ + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table_$REPLICA SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(100000)"; + done +} + +function thread4() +{ + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table_$REPLICA FINAL"; + sleep 0.$RANDOM; + done +} + +function thread5() +{ + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table_$REPLICA DELETE WHERE rand() % 2 = 1"; + sleep 0.$RANDOM; + done +} + +function thread6() +{ + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS alter_table_$REPLICA; + CREATE TABLE alter_table_$REPLICA (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/alter_table', 'r_$REPLICA') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1;"; + sleep 0.$RANDOM; + done +} + + +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; +export -f thread5; +export -f thread6; + +TIMEOUT=30 + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & + +wait + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS alter_table" From 99aa5cce3895d6aa30990da54ca2b1f46e448782 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Aug 2019 03:48:08 +0300 Subject: [PATCH 258/509] Fixed idiotic error in system.parts --- dbms/src/Storages/System/StorageSystemPartsBase.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.cpp b/dbms/src/Storages/System/StorageSystemPartsBase.cpp index 7a242e74e75..bced500a072 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsBase.cpp @@ -148,10 +148,9 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const StoragesInfo StoragesInfoStream::next() { - StoragesInfo info; - while (next_row < rows) { + StoragesInfo info; info.database = (*database_column)[next_row].get(); info.table = (*table_column)[next_row].get(); @@ -198,10 +197,10 @@ StoragesInfo StoragesInfoStream::next() if (!info.data) throw Exception("Unknown engine " + info.engine, ErrorCodes::LOGICAL_ERROR); - break; + return info; } - return info; + return {}; } BlockInputStreams StorageSystemPartsBase::read( From ebbcd269815f361205dd768675cfc9cf9c5713ae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Aug 2019 03:53:12 +0300 Subject: [PATCH 259/509] Improved test --- .../0_stateless/00993_system_parts_race_condition_drop.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.sh b/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.sh index 064d425ec1e..20f848155bc 100755 --- a/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.sh +++ b/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.sh @@ -97,4 +97,4 @@ timeout $TIMEOUT bash -c thread6 2> /dev/null & wait -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS alter_table" +for i in {0..9}; do $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS alter_table_$i"; done From a5747272695f8293dc453f05aca0c48f47c027d4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Aug 2019 04:24:53 +0300 Subject: [PATCH 260/509] Attempt to fix deadlock --- dbms/src/Interpreters/InterpreterOptimizeQuery.cpp | 1 - dbms/src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp b/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp index 7326bb62924..69d601ca462 100644 --- a/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -23,7 +23,6 @@ BlockIO InterpreterOptimizeQuery::execute() return executeDDLQueryOnCluster(query_ptr, context, {ast.database}); StoragePtr table = context.getTable(ast.database, ast.table); - auto table_lock = table->lockStructureForShare(true, context.getCurrentQueryId()); table->optimize(query_ptr, ast.partition, ast.final, ast.deduplicate, context); return {}; } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 7153b5731db..ed73a76505b 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3084,9 +3084,10 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p } } - /// TODO: Bad setting name for such purpose if (query_context.getSettingsRef().replication_alter_partitions_sync != 0) { + /// NOTE Table lock must not be held while waiting. Some combination of R-W-R locks from different threads will yield to deadlock. + /// TODO Check all other "wait" places. for (auto & merge_entry : merge_entries) waitForAllReplicasToProcessLogEntry(merge_entry); } From b7cb55a06ec8ddd3038f681938f666aa481f75f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Aug 2019 04:46:48 +0300 Subject: [PATCH 261/509] Added a comment --- dbms/src/Storages/StorageReplicatedMergeTree.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 1d967021e11..c9ec99e9e6a 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -470,10 +470,15 @@ private: /** Wait until all replicas, including this, execute the specified action from the log. * If replicas are added at the same time, it can not wait the added replica . + * + * NOTE: This method must be called without table lock held. + * Because it effectively waits for other thread that usually has to also acquire a lock to proceed and this yields deadlock. + * TODO: There are wrong usages of this method that are not fixed yet. */ void waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry); /** Wait until the specified replica executes the specified action from the log. + * NOTE: See comment about locks above. */ void waitForReplicaToProcessLogEntry(const String & replica_name, const ReplicatedMergeTreeLogEntryData & entry); From 0e99eb82f43c115acef59ffed58f1ae4b9f73b6f Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 20 Aug 2019 10:23:47 +0300 Subject: [PATCH 262/509] DOCAPI-7129: nested json insert. EN review. RU translation. (#6419) * Update formats.md * Update settings.md * Link fix. * DOCAPI-7129: RU translation. * DOCAPI-7129: Fixed build of ZH docs. * Update docs/en/interfaces/formats.md Co-Authored-By: Ivan Blinkov * Update docs/en/operations/settings/settings.md Co-Authored-By: Ivan Blinkov * Update docs/ru/interfaces/formats.md Co-Authored-By: Ivan Blinkov * Update docs/ru/interfaces/formats.md Co-Authored-By: Ivan Blinkov * Update docs/ru/interfaces/formats.md Co-Authored-By: Ivan Blinkov * Update docs/ru/operations/settings/settings.md Co-Authored-By: Ivan Blinkov * Update docs/ru/operations/settings/settings.md Co-Authored-By: Ivan Blinkov * DOCAPI-7129: Clarifications and fixes. --- docs/en/interfaces/formats.md | 13 ++--- docs/en/operations/settings/settings.md | 10 ++-- docs/ru/interfaces/formats.md | 71 ++++++++++++++++++++++--- docs/ru/operations/settings/settings.md | 44 ++++++++++++++- 4 files changed, 119 insertions(+), 19 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 46a553089d9..c5dbbce674d 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1,9 +1,6 @@ # Formats for input and output data {#formats} -ClickHouse can accept and return data in various formats. A format supported -for input can be used to parse the data provided to `INSERT`s, to perform -`SELECT`s from a file-backed table such as File, URL or HDFS, or to read an -external dictionary. A format supported for output can be used to arrange the +ClickHouse can accept and return data in various formats. A format supported for input can be used to parse the data provided to `INSERT`s, to perform `SELECT`s from a file-backed table such as File, URL or HDFS, or to read an external dictionary. A format supported for output can be used to arrange the results of a `SELECT`, and to perform `INSERT`s into a file-backed table. The supported formats are: @@ -388,7 +385,7 @@ Unlike the [JSON](#json) format, there is no substitution of invalid UTF-8 seque ### Usage of Nested Structures {#jsoneachrow-nested} -If you have a table with the [Nested](../data_types/nested_data_structures/nested.md) data type columns, you can insert JSON data having the same structure. Enable this functionality with the [input_format_import_nested_json](../operations/settings/settings.md#settings-input_format_import_nested_json) setting. +If you have a table with [Nested](../data_types/nested_data_structures/nested.md) data type columns, you can insert JSON data with the same structure. Enable this feature with the [input_format_import_nested_json](../operations/settings/settings.md#settings-input_format_import_nested_json) setting. For example, consider the following table: @@ -396,13 +393,13 @@ For example, consider the following table: CREATE TABLE json_each_row_nested (n Nested (s String, i Int32) ) ENGINE = Memory ``` -As you can find in the `Nested` data type description, ClickHouse treats each component of the nested structure as a separate column, `n.s` and `n.i` for our table. So you can insert the data the following way: +As you can see in the `Nested` data type description, ClickHouse treats each component of the nested structure as a separate column (`n.s` and `n.i` for our table). You can insert data in the following way: ```sql INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n.s": ["abc", "def"], "n.i": [1, 23]} ``` -To insert data as hierarchical JSON object set [input_format_import_nested_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). +To insert data as a hierarchical JSON object, set [input_format_import_nested_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). ```json { @@ -413,7 +410,7 @@ To insert data as hierarchical JSON object set [input_format_import_nested_json= } ``` -Without this setting ClickHouse throws the exception. +Without this setting, ClickHouse throws an exception. ```sql SELECT name, value FROM system.settings WHERE name = 'input_format_import_nested_json' diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9ff1971ab88..9e298583b38 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -238,7 +238,7 @@ Default value: 0. ## input_format_import_nested_json {#settings-input_format_import_nested_json} -Enables or disables inserting of JSON data with nested objects. +Enables or disables the insertion of JSON data with nested objects. Supported formats: @@ -275,7 +275,7 @@ Default value: 1. ## date_time_input_format {#settings-date_time_input_format} -Enables or disables extended parsing of date and time formatted strings. +Allows to choose a parser of text representation of date and time. The setting doesn't apply to [date and time functions](../../query_language/functions/date_time_functions.md). @@ -283,11 +283,13 @@ Possible values: - `'best_effort'` — Enables extended parsing. - ClickHouse can parse the basic format `YYYY-MM-DD HH:MM:SS` and all the [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) date and time formats. For example, `'2018-06-08T01:02:03.000Z'`. + ClickHouse can parse the basic `YYYY-MM-DD HH:MM:SS` format and all [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) date and time formats. For example, `'2018-06-08T01:02:03.000Z'`. - `'basic'` — Use basic parser. - ClickHouse can parse only the basic format. + ClickHouse can parse only the basic `YYYY-MM-DD HH:MM:SS` format. For example, `'2019-08-20 10:18:56'`. + +Default value: `'basic'`. **See Also** diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 20aa78630b7..20c919665d5 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -377,9 +377,68 @@ CREATE TABLE IF NOT EXISTS example_table В отличие от формата [JSON](#json), для `JSONEachRow` ClickHouse не заменяет невалидные UTF-8 последовательности. Значения экранируются так же, как и для формата `JSON`. -!!! Примечание " Примечание" +!!! note "Примечание" В строках может выводиться произвольный набор байт. Используйте формат `JSONEachRow`, если вы уверены, что данные в таблице могут быть представлены в формате JSON без потери информации. +### Использование вложенных структур {#jsoneachrow-nested} + +Если у вас есть таблица со столбцами типа [Nested](../data_types/nested_data_structures/nested.md), то в неё можно вставить данные из JSON-документа с такой же структурой. Функциональность включается настройкой [input_format_import_nested_json](../operations/settings/settings.md#settings-input_format_import_nested_json). + +Например, рассмотрим следующую таблицу: + +```sql +CREATE TABLE json_each_row_nested (n Nested (s String, i Int32) ) ENGINE = Memory +``` + +Из описания типа данных `Nested` видно, что ClickHouse трактует каждый компонент вложенной структуры как отдельный столбец (для нашей таблицы `n.s` и `n.i`). Можно вставить данные следующим образом: + +```sql +INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n.s": ["abc", "def"], "n.i": [1, 23]} +``` + +Чтобы вставить данные как иерархический объект JSON, установите [input_format_import_nested_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). + +```json +{ + "n": { + "s": ["abc", "def"], + "i": [1, 23] + } +} +``` + +Без этой настройки ClickHouse сгенерирует исключение. + +```sql +SELECT name, value FROM system.settings WHERE name = 'input_format_import_nested_json' +``` + +```text +┌─name────────────────────────────┬─value─┐ +│ input_format_import_nested_json │ 0 │ +└─────────────────────────────────┴───────┘ +``` + +```sql +INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n": {"s": ["abc", "def"], "i": [1, 23]}} +``` + +```text +Code: 117. DB::Exception: Unknown field found while parsing JSONEachRow format: n: (at row 1) +``` + +```sql +SET input_format_import_nested_json=1 +INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n": {"s": ["abc", "def"], "i": [1, 23]}} +SELECT * FROM json_each_row_nested +``` + +```text +┌─n.s───────────┬─n.i────┐ +│ ['abc','def'] │ [1,23] │ +└───────────────┴────────┘ +``` + ## Native {#native} Самый эффективный формат. Данные пишутся и читаются блоками в бинарном виде. Для каждого блока пишется количество строк, количество столбцов, имена и типы столбцов, а затем кусочки столбцов этого блока, один за другим. То есть, этот формат является "столбцовым" - не преобразует столбцы в строки. Именно этот формат используется в родном интерфейсе - при межсерверном взаимодействии, при использовании клиента командной строки, при работе клиентов, написанных на C++. @@ -399,7 +458,7 @@ CREATE TABLE IF NOT EXISTS example_table [NULL](../query_language/syntax.md) выводится как `ᴺᵁᴸᴸ`. -``` sql +```sql SELECT * FROM t_null ``` ``` @@ -410,7 +469,7 @@ SELECT * FROM t_null В форматах `Pretty*` строки выводятся без экранирования. Ниже приведен пример для формата [PrettyCompact](#prettycompact): -``` sql +```sql SELECT 'String with \'quotes\' and \t character' AS Escaping_test ``` @@ -425,7 +484,7 @@ SELECT 'String with \'quotes\' and \t character' AS Escaping_test Формат `Pretty` поддерживает вывод тотальных значений (при использовании WITH TOTALS) и экстремальных значений (при настройке extremes выставленной в 1). В этих случаях, после основных данных выводятся тотальные значения, и экстремальные значения, в отдельных табличках. Пример (показан для формата [PrettyCompact](#prettycompact)): -``` sql +```sql SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT PrettyCompact ``` @@ -516,7 +575,7 @@ Array представлены как длина в формате varint (unsig Пример: -``` sql +```sql SELECT * FROM t_null FORMAT Vertical ``` ``` @@ -528,7 +587,7 @@ y: ᴺᵁᴸᴸ В формате `Vertical` строки выводятся без экранирования. Например: -``` sql +```sql SELECT 'string with \'quotes\' and \t with some special \n characters' AS test FORMAT Vertical ``` diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index c3d17ddfe35..07c1e5ae977 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -233,7 +233,26 @@ Ok. - 0 — выключена. - 1 — включена. -Значение по умолчанию: 0. +Значение по умолчанию — 0. + +## input_format_import_nested_json {#settings-input_format_import_nested_json} + +Включает или отключает вставку данных JSON с вложенными объектами. + +Поддерживаемые форматы: + +- [JSONEachRow](../../interfaces/formats.md#jsoneachrow) + +Возможные значения: + +- 0 — выключена. +- 1 — включена. + +Значение по умолчанию — 0. + +**Смотрите также** + +- [Использование вложенных структур](../../interfaces/formats.md#jsoneachrow-nested) with the `JSONEachRow` format. ## input_format_with_names_use_header {#settings-input_format_with_names_use_header} @@ -253,6 +272,29 @@ Ok. Значение по умолчанию: 1. +## date_time_input_format {#settings-date_time_input_format} + +Выбор парсера для текстового представления дат и времени при обработке входного формата. + +Настройка не применяется к [функциям для работы с датой и временем](../../query_language/functions/date_time_functions.md). + +Возможные значения: + +- `'best_effort'` — включает расширенный парсинг. + + ClickHouse может парсить базовый формат `YYYY-MM-DD HH:MM:SS` и все форматы [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601). Например, `'2018-06-08T01:02:03.000Z'`. + +- `'basic'` — используется базовый парсер. + + ClickHouse может парсить только базовый формат `YYYY-MM-DD HH:MM:SS`. Например, `'2019-08-20 10:18:56'`. + +Значение по умолчанию — `'basic'`. + +**Смотрите также** + +- [Тип данных DateTime.](../../data_types/datetime.md) +- [Функции для работы с датой и временем.](../../query_language/functions/date_time_functions.md) + ## join_default_strictness {#settings-join_default_strictness} Устанавливает строгость по умолчанию для [JOIN](../../query_language/select.md#select-join). From 87e13c494ec9761129dee694c4e59e8c1f9b6c96 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 20 Aug 2019 10:46:31 +0300 Subject: [PATCH 263/509] DOCAPI-7414: groupArrayMovingSum, groupArrayMovingAvg docs. EN review, RU translation. (#6463) * Update reference.md * DOCAPI-7414: RU translation * DOCAPI-7414: Fix. * Update docs/ru/query_language/agg_functions/reference.md Co-Authored-By: Ivan Blinkov * Update docs/ru/query_language/agg_functions/reference.md Co-Authored-By: Ivan Blinkov * Update docs/ru/query_language/agg_functions/reference.md Co-Authored-By: Ivan Blinkov * Update docs/ru/query_language/agg_functions/reference.md Co-Authored-By: Ivan Blinkov --- .../query_language/agg_functions/reference.md | 8 +- .../query_language/agg_functions/reference.md | 150 +++++++++++++++++- 2 files changed, 153 insertions(+), 5 deletions(-) diff --git a/docs/en/query_language/agg_functions/reference.md b/docs/en/query_language/agg_functions/reference.md index 3e3e1e2cc01..5f3007a60cd 100644 --- a/docs/en/query_language/agg_functions/reference.md +++ b/docs/en/query_language/agg_functions/reference.md @@ -681,11 +681,11 @@ groupArrayMovingSum(numbers_for_summing) groupArrayMovingSum(window_size)(numbers_for_summing) ``` -The function can take the window size as a parameter. If it not specified, the function takes the window size equal to the number of rows in the column. +The function can take the window size as a parameter. If left unspecified, the function takes the window size equal to the number of rows in the column. **Parameters** -- `numbers_for_summing` — [Expression](../syntax.md#syntax-expressions) resulting with a value in a numeric data type. +- `numbers_for_summing` — [Expression](../syntax.md#syntax-expressions) resulting in a numeric data type value. - `window_size` — Size of the calculation window. **Returned values** @@ -750,11 +750,11 @@ groupArrayMovingAvg(numbers_for_summing) groupArrayMovingAvg(window_size)(numbers_for_summing) ``` -The function can take the window size as a parameter. If it not specified, the function takes the window size equal to the number of rows in the column. +The function can take the window size as a parameter. If left unspecified, the function takes the window size equal to the number of rows in the column. **Parameters** -- `numbers_for_summing` — [Expression](../syntax.md#syntax-expressions) resulting with a value in a numeric data type. +- `numbers_for_summing` — [Expression](../syntax.md#syntax-expressions) resulting in a numeric data type value. - `window_size` — Size of the calculation window. **Returned values** diff --git a/docs/ru/query_language/agg_functions/reference.md b/docs/ru/query_language/agg_functions/reference.md index 12308169f9a..b948b17abee 100644 --- a/docs/ru/query_language/agg_functions/reference.md +++ b/docs/ru/query_language/agg_functions/reference.md @@ -87,7 +87,7 @@ anyHeavy(column) **Аргументы** -- `column` – Имя столбца. +- `column` — имя столбца. **Пример** @@ -681,6 +681,154 @@ uniqExact(x[, ...]) - Значение по умолчанию для подстановки на пустые позиции. - Длина результирующего массива. Например, если вы хотите получать массивы одинакового размера для всех агрегатных ключей. При использовании этого параметра значение по умолчанию задавать обязательно. +## groupArrayMovingSum {#agg_function-grouparraymovingsum} + +Вычисляет скользящую сумму входных значений. + +``` +groupArrayMovingSum(numbers_for_summing) +groupArrayMovingSum(window_size)(numbers_for_summing) +``` + +Функция может принимать размер окна в качестве параметра. Если окно не указано, то функция использует размер окна, равный количеству строк в столбце. + +**Параметры** + +- `numbers_for_summing` — [выражение](../syntax.md#syntax-expressions), возвращающее значение числового типа. +- `window_size` — размер окна. + +**Возвращаемые значения** + +- Массив того же размера и типа, что и входные данные. + +**Пример** + +Таблица с исходными данными: + +```sql +CREATE TABLE t +( + `int` UInt8, + `float` Float32, + `dec` Decimal32(2) +) +ENGINE = TinyLog +``` + +```text +┌─int─┬─float─┬──dec─┐ +│ 1 │ 1.1 │ 1.10 │ +│ 2 │ 2.2 │ 2.20 │ +│ 4 │ 4.4 │ 4.40 │ +│ 7 │ 7.77 │ 7.77 │ +└─────┴───────┴──────┘ +``` + +Запросы: + +```sql +SELECT + groupArrayMovingSum(int) AS I, + groupArrayMovingSum(float) AS F, + groupArrayMovingSum(dec) AS D +FROM t +``` + +```text +┌─I──────────┬─F───────────────────────────────┬─D──────────────────────┐ +│ [1,3,7,14] │ [1.1,3.3000002,7.7000003,15.47] │ [1.10,3.30,7.70,15.47] │ +└────────────┴─────────────────────────────────┴────────────────────────┘ +``` + +```sql +SELECT + groupArrayMovingSum(2)(int) AS I, + groupArrayMovingSum(2)(float) AS F, + groupArrayMovingSum(2)(dec) AS D +FROM t +``` + +```text +┌─I──────────┬─F───────────────────────────────┬─D──────────────────────┐ +│ [1,3,6,11] │ [1.1,3.3000002,6.6000004,12.17] │ [1.10,3.30,6.60,12.17] │ +└────────────┴─────────────────────────────────┴────────────────────────┘ +``` + +## groupArrayMovingAvg {#agg_function-grouparraymovingavg} + +Вычисляет скользящее среднее для входных значений. + +``` +groupArrayMovingAvg(numbers_for_summing) +groupArrayMovingAvg(window_size)(numbers_for_summing) +``` + +Функция может принимать размер окна в качестве параметра. Если окно не указано, то функция использует размер окна, равный количеству строк в столбце. + +**Параметры** + +- `numbers_for_summing` — [выражение](../syntax.md#syntax-expressions), возвращающее значение числового типа. +- `window_size` — размер окна. + +**Возвращаемые значения** + +- Массив того же размера и типа, что и входные данные. + +Функция использует [округление к меньшему по модулю](https://ru.wikipedia.org/wiki/Округление#Методы). Оно усекает десятичные разряды, незначимые для результирующего типа данных. + +**Пример** + +Таблица с исходными данными: + +```sql +CREATE TABLE t +( + `int` UInt8, + `float` Float32, + `dec` Decimal32(2) +) +ENGINE = TinyLog +``` + +```text +┌─int─┬─float─┬──dec─┐ +│ 1 │ 1.1 │ 1.10 │ +│ 2 │ 2.2 │ 2.20 │ +│ 4 │ 4.4 │ 4.40 │ +│ 7 │ 7.77 │ 7.77 │ +└─────┴───────┴──────┘ +``` + +Запросы: + +```sql +SELECT + groupArrayMovingAvg(int) AS I, + groupArrayMovingAvg(float) AS F, + groupArrayMovingAvg(dec) AS D +FROM t +``` + +```text +┌─I─────────┬─F───────────────────────────────────┬─D─────────────────────┐ +│ [0,0,1,3] │ [0.275,0.82500005,1.9250001,3.8675] │ [0.27,0.82,1.92,3.86] │ +└───────────┴─────────────────────────────────────┴───────────────────────┘ +``` + +```sql +SELECT + groupArrayMovingAvg(2)(int) AS I, + groupArrayMovingAvg(2)(float) AS F, + groupArrayMovingAvg(2)(dec) AS D +FROM t +``` + +```text +┌─I─────────┬─F────────────────────────────────┬─D─────────────────────┐ +│ [0,1,3,5] │ [0.55,1.6500001,3.3000002,6.085] │ [0.55,1.65,3.30,6.08] │ +└───────────┴──────────────────────────────────┴───────────────────────┘ +``` + ## groupUniqArray(x), groupUniqArray(max_size)(x) Составляет массив из различных значений аргумента. Расход оперативной памяти такой же, как у функции `uniqExact`. From bfd0505d99e065934df58bfb465d7d8220aa5a8c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Aug 2019 11:36:10 +0300 Subject: [PATCH 264/509] Fix tests. --- dbms/src/Functions/toTypeName.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/src/Functions/toTypeName.cpp b/dbms/src/Functions/toTypeName.cpp index 36a2f396aab..84ab32c3643 100644 --- a/dbms/src/Functions/toTypeName.cpp +++ b/dbms/src/Functions/toTypeName.cpp @@ -77,6 +77,9 @@ protected: return std::make_shared(types, return_type); } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } }; From 1cebcd3e73bc5a3d6d3c5692fb4126a9d02dc262 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Aug 2019 12:15:08 +0300 Subject: [PATCH 265/509] Fixed test. --- .../0_stateless/00834_hints_for_type_function_typos.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh b/dbms/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh index 92478119fe0..9d31234c877 100755 --- a/dbms/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh +++ b/dbms/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh @@ -21,8 +21,8 @@ $CLICKHOUSE_CLIENT -q "select multisearchallposicionutf7casesensitive('abc');" 2 $CLICKHOUSE_CLIENT -q "select multiSearchAllposicionutf7sensitive('abc');" 2>&1 | grep "Maybe you meant: \['multiSearchAllPositionsCaseInsensitive','multiSearchAnyCaseInsensitive'\]." &>/dev/null; $CLICKHOUSE_CLIENT -q "select multiSearchAllPosicionSensitiveUTF8('abc');" 2>&1 | grep "Maybe you meant: \['multiSearchAnyCaseInsensitiveUTF8','multiSearchAllPositionsCaseInsensitiveUTF8'\]." &>/dev/null; -$CLICKHOUSE_CLIENT -q "select * FROM numberss(10);" 2>&1 | grep "Maybe you meant: \['numbers'\]." &>/dev/null -$CLICKHOUSE_CLIENT -q "select * FROM anothernumbers(10);" 2>&1 | grep -v "Maybe you meant: \['numbers'\]." &>/dev/null +$CLICKHOUSE_CLIENT -q "select * FROM numberss(10);" 2>&1 | grep "Maybe you meant: \['numbers'\,'numbers_mt'\]." &>/dev/null +$CLICKHOUSE_CLIENT -q "select * FROM anothernumbers(10);" 2>&1 | grep -v "Maybe you meant: \['numbers'\,'numbers_mt'\]." &>/dev/null $CLICKHOUSE_CLIENT -q "select * FROM mynumbers(10);" 2>&1 | grep "Maybe you meant: \['numbers'\]." &>/dev/null $CLICKHOUSE_CLIENT -q "CREATE TABLE stored_aggregates (d Date, Uniq AggregateFunction(uniq, UInt64)) ENGINE = MergeTre(d, d, 8192);" 2>&1 | grep "Maybe you meant: \['MergeTree'\]." &>/dev/null From 7f31b8bf9c9cdba81abaaaebe91a510fae6a251c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Aug 2019 13:28:20 +0300 Subject: [PATCH 266/509] Added NumbersMtBlockInputStream. --- .../Storages/System/StorageSystemNumbers.cpp | 71 ++++++++++++++++++- .../Storages/System/StorageSystemNumbers.h | 5 +- .../TableFunctions/TableFunctionNumbers.cpp | 2 +- 3 files changed, 74 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemNumbers.cpp b/dbms/src/Storages/System/StorageSystemNumbers.cpp index c909a338453..b08e074af60 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.cpp +++ b/dbms/src/Storages/System/StorageSystemNumbers.cpp @@ -43,8 +43,63 @@ private: }; -StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multithreaded_, std::optional limit_, UInt64 offset_) - : name(name_), multithreaded(multithreaded_), limit(limit_), offset(offset_) +struct NumbersMtState +{ + std::atomic counter; + explicit NumbersMtState(UInt64 offset) : counter(offset) {} +}; + +using NumbersMtStatePtr = std::shared_ptr; + +class NumbersMtBlockInputStream : public IBlockInputStream +{ +public: + NumbersMtBlockInputStream(NumbersMtStatePtr state_, UInt64 block_size_, UInt64 max_counter_) + : state(std::move(state_)), counter(state->counter), block_size(block_size_), max_counter(max_counter_) {} + + String getName() const override { return "NumbersMt"; } + + Block getHeader() const override + { + return { ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "number") }; + } + +protected: + Block readImpl() override + { + if (block_size == 0) + return {}; + + UInt64 curr = counter.fetch_add(block_size, std::memory_order_acquire); + + if (curr >= max_counter) + return {}; + + if (curr + block_size > max_counter) + block_size = max_counter - curr; + + auto column = ColumnUInt64::create(block_size); + ColumnUInt64::Container & vec = column->getData(); + + UInt64 * pos = vec.data(); + UInt64 * end = &vec[block_size]; + while (pos < end) + *pos++ = curr++; + + return { ColumnWithTypeAndName(std::move(column), std::make_shared(), "number") }; + } + +private: + NumbersMtStatePtr state; + std::atomic & counter; + + UInt64 block_size; + UInt64 max_counter; +}; + + +StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multithreaded_, std::optional limit_, UInt64 offset_, bool even_distribution_) + : name(name_), multithreaded(multithreaded_), even_distribution(even_distribution_), limit(limit_), offset(offset_) { setColumns(ColumnsDescription({{"number", std::make_shared()}})); } @@ -69,6 +124,18 @@ BlockInputStreams StorageSystemNumbers::read( num_streams = 1; BlockInputStreams res(num_streams); + + if (num_streams > 1 && !even_distribution && *limit) + { + auto state = std::make_shared(offset); + UInt64 max_counter = offset + *limit; + + for (size_t i = 0; i < num_streams; ++i) + res[i] = std::make_shared(state, max_block_size, max_counter); + + return res; + } + for (size_t i = 0; i < num_streams; ++i) { res[i] = std::make_shared(max_block_size, offset + i * max_block_size, num_streams * max_block_size); diff --git a/dbms/src/Storages/System/StorageSystemNumbers.h b/dbms/src/Storages/System/StorageSystemNumbers.h index 452ec5a9ef5..6ecc6785320 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.h +++ b/dbms/src/Storages/System/StorageSystemNumbers.h @@ -38,11 +38,14 @@ public: private: const std::string name; bool multithreaded; + bool bool even_distribution; std::optional limit; UInt64 offset; protected: - StorageSystemNumbers(const std::string & name_, bool multithreaded_, std::optional limit_ = std::nullopt, UInt64 offset_ = 0); + /// If even_distribution is true, numbers are distributed evenly between streams. + /// Otherwise, streams concurrently increment atomic. + StorageSystemNumbers(const std::string & name_, bool multithreaded_, std::optional limit_ = std::nullopt, UInt64 offset_ = 0, bool even_distribution_ = true); }; } diff --git a/dbms/src/TableFunctions/TableFunctionNumbers.cpp b/dbms/src/TableFunctions/TableFunctionNumbers.cpp index 771ff8d5019..14947a40812 100644 --- a/dbms/src/TableFunctions/TableFunctionNumbers.cpp +++ b/dbms/src/TableFunctions/TableFunctionNumbers.cpp @@ -30,7 +30,7 @@ StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_f UInt64 offset = arguments.size() == 2 ? evaluateArgument(context, arguments[0]) : 0; UInt64 length = arguments.size() == 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]); - auto res = StorageSystemNumbers::create(table_name, multithreaded, length, offset); + auto res = StorageSystemNumbers::create(table_name, multithreaded, length, offset, false); res->startup(); return res; } From 0c0e5010dc18805ce67816caa8527f0121ca1f5c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Aug 2019 13:29:04 +0300 Subject: [PATCH 267/509] Added NumbersMtBlockInputStream. --- dbms/src/Storages/System/StorageSystemNumbers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/System/StorageSystemNumbers.h b/dbms/src/Storages/System/StorageSystemNumbers.h index 6ecc6785320..75e717c0f60 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.h +++ b/dbms/src/Storages/System/StorageSystemNumbers.h @@ -38,7 +38,7 @@ public: private: const std::string name; bool multithreaded; - bool bool even_distribution; + bool even_distribution; std::optional limit; UInt64 offset; From a502424c3378572e3cf0a2e498d762d938a5bd93 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Tue, 20 Aug 2019 14:17:57 +0300 Subject: [PATCH 268/509] Implement support for insertion into Kafka tables (#6012) * Add write-callback on each row for RowOutputStream * Fix build of new rdkafka library * Poll messages if Kafka outgoing queue is full * Add test * Add test producer-consumer * Truncate delimiter from last row in message --- dbms/src/Core/Settings.h | 2 +- .../BlockOutputStreamFromRowOutputStream.cpp | 45 ------ .../BlockOutputStreamFromRowOutputStream.h | 38 ----- dbms/src/Formats/FormatFactory.cpp | 14 +- dbms/src/Formats/FormatFactory.h | 12 +- dbms/src/Formats/NativeFormat.cpp | 1 + dbms/src/Formats/NullFormat.cpp | 1 + .../Formats/tests/block_row_transforms.cpp | 3 +- .../Formats/tests/tab_separated_streams.cpp | 3 +- dbms/src/IO/MemoryReadWriteBuffer.h | 2 +- dbms/src/IO/WriteBuffer.h | 2 +- .../Processors/Formats/IRowOutputFormat.cpp | 6 +- .../src/Processors/Formats/IRowOutputFormat.h | 13 +- .../Formats/Impl/BinaryRowOutputFormat.cpp | 10 +- .../Formats/Impl/BinaryRowOutputFormat.h | 3 +- .../Formats/Impl/CSVRowOutputFormat.cpp | 7 +- .../Formats/Impl/CSVRowOutputFormat.h | 3 +- .../Impl/JSONCompactRowOutputFormat.cpp | 7 +- .../Formats/Impl/JSONCompactRowOutputFormat.h | 2 +- .../Impl/JSONEachRowRowOutputFormat.cpp | 7 +- .../Formats/Impl/JSONEachRowRowOutputFormat.h | 3 +- .../Formats/Impl/JSONRowOutputFormat.cpp | 8 +- .../Formats/Impl/JSONRowOutputFormat.h | 3 +- .../Formats/Impl/MySQLOutputFormat.cpp | 10 +- .../Formats/Impl/MySQLOutputFormat.h | 1 - .../Processors/Formats/Impl/NativeFormat.cpp | 1 + .../Processors/Formats/Impl/NullFormat.cpp | 1 + .../Impl/ODBCDriver2BlockOutputFormat.cpp | 2 +- .../Impl/ODBCDriverBlockOutputFormat.cpp | 1 + .../Formats/Impl/ParquetBlockOutputFormat.cpp | 7 +- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 2 + .../Impl/PrettyCompactBlockOutputFormat.cpp | 2 + .../Impl/PrettySpaceBlockOutputFormat.cpp | 2 + .../Formats/Impl/ProtobufRowOutputFormat.cpp | 12 +- .../Formats/Impl/ProtobufRowOutputFormat.h | 1 + .../Formats/Impl/TSKVRowOutputFormat.cpp | 7 +- .../Formats/Impl/TSKVRowOutputFormat.h | 3 +- .../Impl/TabSeparatedRawRowOutputFormat.h | 13 +- .../Impl/TabSeparatedRowOutputFormat.cpp | 22 ++- .../Impl/TabSeparatedRowOutputFormat.h | 9 +- .../Formats/Impl/ValuesRowOutputFormat.cpp | 7 +- .../Formats/Impl/ValuesRowOutputFormat.h | 3 +- .../Formats/Impl/VerticalRowOutputFormat.cpp | 7 +- .../Formats/Impl/VerticalRowOutputFormat.h | 3 +- .../Formats/Impl/XMLRowOutputFormat.cpp | 7 +- .../Formats/Impl/XMLRowOutputFormat.h | 3 +- .../Storages/Kafka/KafkaBlockInputStream.cpp | 13 +- .../Storages/Kafka/KafkaBlockInputStream.h | 2 +- .../Storages/Kafka/KafkaBlockOutputStream.cpp | 54 +++++++ .../Storages/Kafka/KafkaBlockOutputStream.h | 31 ++++ .../Kafka/ReadBufferFromKafkaConsumer.h | 2 +- dbms/src/Storages/Kafka/StorageKafka.cpp | 140 ++++++++++-------- dbms/src/Storages/Kafka/StorageKafka.h | 27 ++-- .../Kafka/WriteBufferToKafkaProducer.cpp | 90 +++++++++++ .../Kafka/WriteBufferToKafkaProducer.h | 46 ++++++ .../integration/test_storage_kafka/test.py | 115 +++++++++++++- .../libcommon/include/ext/shared_ptr_helper.h | 2 +- utils/kafka/consume.py | 1 + 58 files changed, 590 insertions(+), 254 deletions(-) delete mode 100644 dbms/src/Formats/BlockOutputStreamFromRowOutputStream.cpp delete mode 100644 dbms/src/Formats/BlockOutputStreamFromRowOutputStream.h create mode 100644 dbms/src/Storages/Kafka/KafkaBlockOutputStream.cpp create mode 100644 dbms/src/Storages/Kafka/KafkaBlockOutputStream.h create mode 100644 dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp create mode 100644 dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.h diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index c4f374fbc70..86e8746e854 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -212,7 +212,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, insert_distributed_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.") \ M(SettingInt64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite.") \ M(SettingMilliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.") \ - M(SettingMilliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from streaming storages.") \ + M(SettingMilliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.") \ M(SettingString, format_schema, "", "Schema identifier (used by schema-based formats)") \ M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.") \ M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.") \ diff --git a/dbms/src/Formats/BlockOutputStreamFromRowOutputStream.cpp b/dbms/src/Formats/BlockOutputStreamFromRowOutputStream.cpp deleted file mode 100644 index c4fabfa83d8..00000000000 --- a/dbms/src/Formats/BlockOutputStreamFromRowOutputStream.cpp +++ /dev/null @@ -1,45 +0,0 @@ -#include -#include - - -namespace DB -{ - -BlockOutputStreamFromRowOutputStream::BlockOutputStreamFromRowOutputStream(RowOutputStreamPtr row_output_, const Block & header_) - : row_output(row_output_), header(header_) {} - - -void BlockOutputStreamFromRowOutputStream::write(const Block & block) -{ - size_t rows = block.rows(); - for (size_t i = 0; i < rows; ++i) - { - if (!first_row) - row_output->writeRowBetweenDelimiter(); - first_row = false; - row_output->write(block, i); - } -} - - -void BlockOutputStreamFromRowOutputStream::setRowsBeforeLimit(size_t rows_before_limit) -{ - row_output->setRowsBeforeLimit(rows_before_limit); -} - -void BlockOutputStreamFromRowOutputStream::setTotals(const Block & totals) -{ - row_output->setTotals(totals); -} - -void BlockOutputStreamFromRowOutputStream::setExtremes(const Block & extremes) -{ - row_output->setExtremes(extremes); -} - -void BlockOutputStreamFromRowOutputStream::onProgress(const Progress & progress) -{ - row_output->onProgress(progress); -} - -} diff --git a/dbms/src/Formats/BlockOutputStreamFromRowOutputStream.h b/dbms/src/Formats/BlockOutputStreamFromRowOutputStream.h deleted file mode 100644 index ada924bb5b4..00000000000 --- a/dbms/src/Formats/BlockOutputStreamFromRowOutputStream.h +++ /dev/null @@ -1,38 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** Transforms a stream to write data by rows to a stream to write data by blocks. - * For example, to write a text dump. - */ -class BlockOutputStreamFromRowOutputStream : public IBlockOutputStream -{ -public: - BlockOutputStreamFromRowOutputStream(RowOutputStreamPtr row_output_, const Block & header_); - - Block getHeader() const override { return header; } - void write(const Block & block) override; - void writePrefix() override { row_output->writePrefix(); } - void writeSuffix() override { row_output->writeSuffix(); } - - void flush() override { row_output->flush(); } - - void setRowsBeforeLimit(size_t rows_before_limit) override; - void setTotals(const Block & totals) override; - void setExtremes(const Block & extremes) override; - void onProgress(const Progress & progress) override; - - String getContentType() const override { return row_output->getContentType(); } - -private: - RowOutputStreamPtr row_output; - Block header; - bool first_row = true; -}; - -} diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 7fa4b59cf66..f587b2fd5ae 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -100,7 +100,8 @@ BlockInputStreamPtr FormatFactory::getInput( } -BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer & buf, const Block & sample, const Context & context) const +BlockOutputStreamPtr FormatFactory::getOutput( + const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback) const { if (name == "PrettyCompactMonoBlock") { @@ -124,14 +125,14 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer & const Settings & settings = context.getSettingsRef(); FormatSettings format_settings = getOutputFormatSetting(settings); - /** Materialization is needed, because formats can use the functions `IDataType`, + /** Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. */ return std::make_shared( - output_getter(buf, sample, context, format_settings), sample); + output_getter(buf, sample, context, callback, format_settings), sample); } - auto format = getOutputFormat(name, buf, sample, context); + auto format = getOutputFormat(name, buf, sample, context, callback); return std::make_shared(std::make_shared(format), sample); } @@ -165,7 +166,8 @@ InputFormatPtr FormatFactory::getInputFormat( } -OutputFormatPtr FormatFactory::getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample, const Context & context) const +OutputFormatPtr FormatFactory::getOutputFormat( + const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback) const { const auto & output_getter = getCreators(name).output_processor_creator; if (!output_getter) @@ -177,7 +179,7 @@ OutputFormatPtr FormatFactory::getOutputFormat(const String & name, WriteBuffer /** TODO: Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. */ - return output_getter(buf, sample, context, format_settings); + return output_getter(buf, sample, context, callback, format_settings); } diff --git a/dbms/src/Formats/FormatFactory.h b/dbms/src/Formats/FormatFactory.h index 83728c96cd2..576faa9174e 100644 --- a/dbms/src/Formats/FormatFactory.h +++ b/dbms/src/Formats/FormatFactory.h @@ -41,6 +41,10 @@ public: /// It's initial purpose was to extract payload for virtual columns from Kafka Consumer ReadBuffer. using ReadCallback = std::function; + /// This callback allows to perform some additional actions after writing a single row. + /// It's initial purpose was to flush Kafka message for each row. + using WriteCallback = std::function; + private: using InputCreator = std::function; using InputProcessorCreator = std::function; struct Creators @@ -91,7 +97,7 @@ public: ReadCallback callback = {}) const; BlockOutputStreamPtr getOutput(const String & name, WriteBuffer & buf, - const Block & sample, const Context & context) const; + const Block & sample, const Context & context, WriteCallback callback = {}) const; InputFormatPtr getInputFormat( const String & name, @@ -102,8 +108,8 @@ public: UInt64 rows_portion_size = 0, ReadCallback callback = {}) const; - OutputFormatPtr getOutputFormat(const String & name, WriteBuffer & buf, - const Block & sample, const Context & context) const; + OutputFormatPtr getOutputFormat( + const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback = {}) const; /// Register format by its name. void registerInputFormat(const String & name, InputCreator input_creator); diff --git a/dbms/src/Formats/NativeFormat.cpp b/dbms/src/Formats/NativeFormat.cpp index f324879608b..f9063e0eced 100644 --- a/dbms/src/Formats/NativeFormat.cpp +++ b/dbms/src/Formats/NativeFormat.cpp @@ -27,6 +27,7 @@ void registerOutputFormatNative(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback, const FormatSettings &) { return std::make_shared(buf, 0, sample); diff --git a/dbms/src/Formats/NullFormat.cpp b/dbms/src/Formats/NullFormat.cpp index 0012b4c54b2..d5376b70fe1 100644 --- a/dbms/src/Formats/NullFormat.cpp +++ b/dbms/src/Formats/NullFormat.cpp @@ -11,6 +11,7 @@ void registerOutputFormatNull(FormatFactory & factory) WriteBuffer &, const Block & sample, const Context &, + FormatFactory::WriteCallback, const FormatSettings &) { return std::make_shared(sample); diff --git a/dbms/src/Formats/tests/block_row_transforms.cpp b/dbms/src/Formats/tests/block_row_transforms.cpp index b2fe876fb2a..9edc520d85f 100644 --- a/dbms/src/Formats/tests/block_row_transforms.cpp +++ b/dbms/src/Formats/tests/block_row_transforms.cpp @@ -14,7 +14,6 @@ #include #include -#include #include #include @@ -47,7 +46,7 @@ try RowInputStreamPtr row_input = std::make_shared(in_buf, sample, false, false, format_settings); BlockInputStreamFromRowInputStream block_input(row_input, sample, DEFAULT_INSERT_BLOCK_SIZE, 0, []{}, format_settings); - BlockOutputStreamPtr block_output = std::make_shared(std::make_shared(out_buf, sample, false, false, format_settings)); + BlockOutputStreamPtr block_output = std::make_shared(std::make_shared(out_buf, sample, false, false, []{}, format_settings)); copyData(block_input, *block_output); } diff --git a/dbms/src/Formats/tests/tab_separated_streams.cpp b/dbms/src/Formats/tests/tab_separated_streams.cpp index bb5e657229d..2c44ccf2b43 100644 --- a/dbms/src/Formats/tests/tab_separated_streams.cpp +++ b/dbms/src/Formats/tests/tab_separated_streams.cpp @@ -11,7 +11,6 @@ #include #include -#include #include #include @@ -44,7 +43,7 @@ try BlockInputStreamFromRowInputStream block_input(row_input, sample, DEFAULT_INSERT_BLOCK_SIZE, 0, []{}, format_settings); BlockOutputStreamPtr block_output = std::make_shared( - std::make_shared(out_buf, sample, false, false, format_settings)); + std::make_shared(out_buf, sample, false, false, [] {}, format_settings)); copyData(block_input, *block_output); return 0; diff --git a/dbms/src/IO/MemoryReadWriteBuffer.h b/dbms/src/IO/MemoryReadWriteBuffer.h index 36750cb21b0..f9c11084f62 100644 --- a/dbms/src/IO/MemoryReadWriteBuffer.h +++ b/dbms/src/IO/MemoryReadWriteBuffer.h @@ -11,7 +11,7 @@ namespace DB { -/// Stores data in memory chunks, size of cunks are exponentially increasing during write +/// Stores data in memory chunks, size of chunks are exponentially increasing during write /// Written data could be reread after write class MemoryWriteBuffer : public WriteBuffer, public IReadableWriteBuffer, boost::noncopyable, private Allocator { diff --git a/dbms/src/IO/WriteBuffer.h b/dbms/src/IO/WriteBuffer.h index e6a73ea90e3..18c6c8c6640 100644 --- a/dbms/src/IO/WriteBuffer.h +++ b/dbms/src/IO/WriteBuffer.h @@ -35,7 +35,7 @@ public: */ inline void next() { - if (!offset()) + if (!offset() && available()) return; bytes += offset(); diff --git a/dbms/src/Processors/Formats/IRowOutputFormat.cpp b/dbms/src/Processors/Formats/IRowOutputFormat.cpp index 6db2802f232..9d40f056cd3 100644 --- a/dbms/src/Processors/Formats/IRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/IRowOutputFormat.cpp @@ -20,6 +20,9 @@ void IRowOutputFormat::consume(DB::Chunk chunk) first_row = false; write(columns, row); + + if (write_single_row_callback) + write_single_row_callback(); } } @@ -96,6 +99,3 @@ void IRowOutputFormat::writeTotals(const DB::Columns & columns, size_t row_num) } } - - - diff --git a/dbms/src/Processors/Formats/IRowOutputFormat.h b/dbms/src/Processors/Formats/IRowOutputFormat.h index 09bfe17a1b5..a702714a77d 100644 --- a/dbms/src/Processors/Formats/IRowOutputFormat.h +++ b/dbms/src/Processors/Formats/IRowOutputFormat.h @@ -1,8 +1,10 @@ #pragma once -#include +#include #include +#include + namespace DB { @@ -22,8 +24,8 @@ protected: void finalize() override; public: - IRowOutputFormat(const Block & header, WriteBuffer & out_) - : IOutputFormat(header, out_), types(header.getDataTypes()) + IRowOutputFormat(const Block & header, WriteBuffer & out_, FormatFactory::WriteCallback callback) + : IOutputFormat(header, out_), types(header.getDataTypes()), write_single_row_callback(callback) { } @@ -57,6 +59,9 @@ private: bool prefix_written = false; bool suffix_written = false; + // Callback used to indicate that another row is written. + FormatFactory::WriteCallback write_single_row_callback; + void writePrefixIfNot() { if (!prefix_written) @@ -76,5 +81,3 @@ private: }; } - - diff --git a/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp index fb728d1293b..1bbdfbf93fc 100644 --- a/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp @@ -9,8 +9,8 @@ namespace DB { -BinaryRowOutputFormat::BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_) - : IRowOutputFormat(header, out_), with_names(with_names_), with_types(with_types_) +BinaryRowOutputFormat::BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_, FormatFactory::WriteCallback callback) + : IRowOutputFormat(header, out_, callback), with_names(with_names_), with_types(with_types_) { } @@ -53,18 +53,20 @@ void registerOutputFormatProcessorRowBinary(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings &) { - return std::make_shared(buf, sample, false, false); + return std::make_shared(buf, sample, false, false, callback); }); factory.registerOutputFormatProcessor("RowBinaryWithNamesAndTypes", []( WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings &) { - return std::make_shared(buf, sample, true, true); + return std::make_shared(buf, sample, true, true, callback); }); } diff --git a/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.h index 23e7be0a558..0452a1e3e0f 100644 --- a/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.h @@ -17,7 +17,7 @@ class WriteBuffer; class BinaryRowOutputFormat: public IRowOutputFormat { public: - BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_); + BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_, FormatFactory::WriteCallback callback); String getName() const override { return "BinaryRowOutputFormat"; } @@ -32,4 +32,3 @@ protected: }; } - diff --git a/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp index 8f17b8f15c8..081dcf890a0 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp @@ -8,8 +8,8 @@ namespace DB { -CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_), with_names(with_names_), format_settings(format_settings_) +CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, callback), with_names(with_names_), format_settings(format_settings_) { auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -77,9 +77,10 @@ void registerOutputFormatProcessorCSV(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, with_names, format_settings); + return std::make_shared(buf, sample, with_names, callback, format_settings); }); } } diff --git a/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.h index 803d3aa80a6..28bd8edf31a 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.h @@ -20,7 +20,7 @@ public: /** with_names - output in the first line a header with column names * with_types - output in the next line header with the names of the types */ - CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, const FormatSettings & format_settings_); + CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_); String getName() const override { return "CSVRowOutputFormat"; } @@ -45,4 +45,3 @@ protected: }; } - diff --git a/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp index 72292cc0a26..011054dfce7 100644 --- a/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp @@ -8,8 +8,8 @@ namespace DB { JSONCompactRowOutputFormat::JSONCompactRowOutputFormat( - WriteBuffer & out_, const Block & header, const FormatSettings & settings_) - : JSONRowOutputFormat(out_, header, settings_) + WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) + : JSONRowOutputFormat(out_, header, callback, settings_) { } @@ -81,9 +81,10 @@ void registerOutputFormatProcessorJSONCompact(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, format_settings); + return std::make_shared(buf, sample, callback, format_settings); }); } diff --git a/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h index e2b5ce76f88..f4002f74287 100644 --- a/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h @@ -16,7 +16,7 @@ struct FormatSettings; class JSONCompactRowOutputFormat : public JSONRowOutputFormat { public: - JSONCompactRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & settings_); + JSONCompactRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); String getName() const override { return "JSONCompactRowOutputFormat"; } diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp index cf23e06c9a6..513420d29b4 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -8,8 +8,8 @@ namespace DB { -JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & settings_) - : IRowOutputFormat(header_, out_), settings(settings_) +JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_) + : IRowOutputFormat(header_, out_, callback), settings(settings_) { auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -57,9 +57,10 @@ void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, format_settings); + return std::make_shared(buf, sample, callback, format_settings); }); } diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h index 66b3fa88652..d3931e080f5 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -15,7 +15,7 @@ namespace DB class JSONEachRowRowOutputFormat : public IRowOutputFormat { public: - JSONEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & settings_); + JSONEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_); String getName() const override { return "JSONEachRowRowOutputFormat"; } @@ -37,4 +37,3 @@ private: }; } - diff --git a/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index f0d791a6554..90cff3f1498 100644 --- a/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -2,14 +2,13 @@ #include #include #include -#include namespace DB { -JSONRowOutputFormat::JSONRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & settings_) - : IRowOutputFormat(header, out_), settings(settings_) +JSONRowOutputFormat::JSONRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) + : IRowOutputFormat(header, out_, callback), settings(settings_) { auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); @@ -248,9 +247,10 @@ void registerOutputFormatProcessorJSON(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, format_settings); + return std::make_shared(buf, sample, callback, format_settings); }); } diff --git a/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.h index 5df98475ea3..f9aea3a3e8b 100644 --- a/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.h @@ -16,7 +16,7 @@ namespace DB class JSONRowOutputFormat : public IRowOutputFormat { public: - JSONRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & settings_); + JSONRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); String getName() const override { return "JSONRowOutputFormat"; } @@ -81,4 +81,3 @@ protected: }; } - diff --git a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index f046c810fbd..f7ba96a63bd 100644 --- a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -107,10 +107,12 @@ void MySQLOutputFormat::flush() void registerOutputFormatProcessorMySQLWrite(FormatFactory & factory) { factory.registerOutputFormatProcessor( - "MySQLWire", [](WriteBuffer & buf, const Block & sample, const Context & context, const FormatSettings & settings) - { - return std::make_shared(buf, sample, context, settings); - }); + "MySQLWire", + [](WriteBuffer & buf, + const Block & sample, + const Context & context, + FormatFactory::WriteCallback, + const FormatSettings & settings) { return std::make_shared(buf, sample, context, settings); }); } } diff --git a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.h b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.h index d5691936862..39d04818dee 100644 --- a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.h @@ -40,4 +40,3 @@ private: }; } - diff --git a/dbms/src/Processors/Formats/Impl/NativeFormat.cpp b/dbms/src/Processors/Formats/Impl/NativeFormat.cpp index 0abab1e964d..6ac0e30e0a8 100644 --- a/dbms/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/NativeFormat.cpp @@ -161,6 +161,7 @@ void registerOutputFormatProcessorNative(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback, const FormatSettings &) { return std::make_shared(sample, buf); diff --git a/dbms/src/Processors/Formats/Impl/NullFormat.cpp b/dbms/src/Processors/Formats/Impl/NullFormat.cpp index be754294c45..cf1dc7186ab 100644 --- a/dbms/src/Processors/Formats/Impl/NullFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/NullFormat.cpp @@ -22,6 +22,7 @@ void registerOutputFormatProcessorNull(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback, const FormatSettings &) { return std::make_shared(sample, buf); diff --git a/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp index 7f30c5bfdc4..9ada3d16516 100644 --- a/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp @@ -107,7 +107,7 @@ void ODBCDriver2BlockOutputFormat::writePrefix() void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory) { factory.registerOutputFormatProcessor( - "ODBCDriver2", [](WriteBuffer & buf, const Block & sample, const Context &, const FormatSettings & format_settings) + "ODBCDriver2", [](WriteBuffer & buf, const Block & sample, const Context &, FormatFactory::WriteCallback, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); }); diff --git a/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.cpp index 3f84bacbfaf..467bfb614e3 100644 --- a/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.cpp @@ -70,6 +70,7 @@ void registerOutputFormatProcessorODBCDriver(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 224b0e6d2d8..1956403654e 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -423,7 +423,12 @@ void ParquetBlockOutputFormat::finalize() void registerOutputFormatProcessorParquet(FormatFactory & factory) { factory.registerOutputFormatProcessor( - "Parquet", [](WriteBuffer & buf, const Block & sample, const Context & /*context*/, const FormatSettings & format_settings) + "Parquet", + [](WriteBuffer & buf, + const Block & sample, + const Context & /*context*/, + FormatFactory::WriteCallback, + const FormatSettings & format_settings) { auto impl = std::make_shared(buf, sample, format_settings); /// TODO diff --git a/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 84c4fc7cbc3..f6a8222d854 100644 --- a/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -261,6 +261,7 @@ void registerOutputFormatProcessorPretty(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); @@ -270,6 +271,7 @@ void registerOutputFormatProcessorPretty(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback, const FormatSettings & format_settings) { FormatSettings changed_settings = format_settings; diff --git a/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 53fef75a97c..15652dec08d 100644 --- a/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -134,6 +134,7 @@ void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); @@ -143,6 +144,7 @@ void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback, const FormatSettings & format_settings) { FormatSettings changed_settings = format_settings; diff --git a/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 2b75d867327..a420f438b19 100644 --- a/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -103,6 +103,7 @@ void registerOutputFormatProcessorPrettySpace(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); @@ -112,6 +113,7 @@ void registerOutputFormatProcessorPrettySpace(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback, const FormatSettings & format_settings) { FormatSettings changed_settings = format_settings; diff --git a/dbms/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index 30e27c68f1e..96b19337b3d 100644 --- a/dbms/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -23,8 +23,9 @@ namespace ErrorCodes ProtobufRowOutputFormat::ProtobufRowOutputFormat( WriteBuffer & out_, const Block & header, + FormatFactory::WriteCallback callback, const FormatSchemaInfo & format_schema) - : IRowOutputFormat(header, out_) + : IRowOutputFormat(header, out_, callback) , data_types(header.getDataTypes()) , writer(out, ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), header.getNames()) { @@ -46,9 +47,14 @@ void ProtobufRowOutputFormat::write(const Columns & columns, size_t row_num) void registerOutputFormatProcessorProtobuf(FormatFactory & factory) { factory.registerOutputFormatProcessor( - "Protobuf", [](WriteBuffer & buf, const Block & header, const Context & context, const FormatSettings &) + "Protobuf", + [](WriteBuffer & buf, + const Block & header, + const Context & context, + FormatFactory::WriteCallback callback, + const FormatSettings &) { - return std::make_shared(buf, header, FormatSchemaInfo(context, "Protobuf")); + return std::make_shared(buf, header, callback, FormatSchemaInfo(context, "Protobuf")); }); } diff --git a/dbms/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h index 96bd85e317d..ae8ebbf28f4 100644 --- a/dbms/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h @@ -35,6 +35,7 @@ public: ProtobufRowOutputFormat( WriteBuffer & out_, const Block & header, + FormatFactory::WriteCallback callback, const FormatSchemaInfo & format_schema); String getName() const override { return "ProtobufRowOutputFormat"; } diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp index f1c13af9c5d..f1fbc72b9ee 100644 --- a/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp @@ -8,8 +8,8 @@ namespace DB { -TSKVRowOutputFormat::TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings_) - : TabSeparatedRowOutputFormat(out_, header, false, false, format_settings_) +TSKVRowOutputFormat::TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_) + : TabSeparatedRowOutputFormat(out_, header, false, false, callback, format_settings_) { auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); @@ -46,9 +46,10 @@ void registerOutputFormatProcessorTSKV(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings & settings) { - return std::make_shared(buf, sample, settings); + return std::make_shared(buf, sample, callback, settings); }); } diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.h index 607fa5fdc9b..7166176e2f4 100644 --- a/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.h @@ -14,7 +14,7 @@ namespace DB class TSKVRowOutputFormat: public TabSeparatedRowOutputFormat { public: - TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings); + TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & format_settings); String getName() const override { return "TSKVRowOutputFormat"; } @@ -27,4 +27,3 @@ protected: }; } - diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h index c2cf31fc196..42ebd846dfc 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h @@ -13,8 +13,16 @@ namespace DB class TabSeparatedRawRowOutputFormat : public TabSeparatedRowOutputFormat { public: - TabSeparatedRawRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) - : TabSeparatedRowOutputFormat(out_, header_, with_names_, with_types_, format_settings_) {} + TabSeparatedRawRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + bool with_names_, + bool with_types_, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings_) + : TabSeparatedRowOutputFormat(out_, header_, with_names_, with_types_, callback, format_settings_) + { + } String getName() const override { return "TabSeparatedRawRowOutputFormat"; } @@ -25,4 +33,3 @@ public: }; } - diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index 92058323102..ae3a1fb9af0 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -6,10 +6,14 @@ namespace DB { - TabSeparatedRowOutputFormat::TabSeparatedRowOutputFormat( - WriteBuffer & out_, const Block & header_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) + WriteBuffer & out_, + const Block & header_, + bool with_names_, + bool with_types_, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, callback), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) { } @@ -75,9 +79,10 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings & settings) { - return std::make_shared(buf, sample, false, false, settings); + return std::make_shared(buf, sample, false, false, callback, settings); }); } @@ -87,9 +92,10 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings & settings) { - return std::make_shared(buf, sample, false, false, settings); + return std::make_shared(buf, sample, false, false, callback, settings); }); } @@ -99,9 +105,10 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings & settings) { - return std::make_shared(buf, sample, true, false, settings); + return std::make_shared(buf, sample, true, false, callback, settings); }); } @@ -111,9 +118,10 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings & settings) { - return std::make_shared(buf, sample, true, true, settings); + return std::make_shared(buf, sample, true, true, callback, settings); }); } } diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h index a00cd2d8fdd..3d3f82e7833 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h @@ -18,7 +18,13 @@ public: /** with_names - output in the first line a header with column names * with_types - output the next line header with the names of the types */ - TabSeparatedRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, bool with_types_, const FormatSettings & format_settings_); + TabSeparatedRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + bool with_names_, + bool with_types_, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings_); String getName() const override { return "TabSeparatedRowOutputFormat"; } @@ -40,4 +46,3 @@ protected: }; } - diff --git a/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp index 80c4135dfb3..e773cbf167c 100644 --- a/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp @@ -10,8 +10,8 @@ namespace DB { -ValuesRowOutputFormat::ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_), format_settings(format_settings_) +ValuesRowOutputFormat::ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, callback), format_settings(format_settings_) { } @@ -47,9 +47,10 @@ void registerOutputFormatProcessorValues(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings & settings) { - return std::make_shared(buf, sample, settings); + return std::make_shared(buf, sample, callback, settings); }); } diff --git a/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.h index f7a28002c92..b507a157e57 100644 --- a/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.h @@ -15,7 +15,7 @@ class WriteBuffer; class ValuesRowOutputFormat : public IRowOutputFormat { public: - ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); + ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_); String getName() const override { return "ValuesRowOutputFormat"; } @@ -30,4 +30,3 @@ private: }; } - diff --git a/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp index 744ad2d0953..6e794c9a79b 100644 --- a/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp @@ -11,8 +11,8 @@ namespace DB { VerticalRowOutputFormat::VerticalRowOutputFormat( - WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_), format_settings(format_settings_) + WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, callback), format_settings(format_settings_) { auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -169,9 +169,10 @@ void registerOutputFormatProcessorVertical(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings & settings) { - return std::make_shared(buf, sample, settings); + return std::make_shared(buf, sample, callback, settings); }); } diff --git a/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.h index 2a0d248bab8..aa40a6a5ad6 100644 --- a/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.h @@ -18,7 +18,7 @@ class Context; class VerticalRowOutputFormat : public IRowOutputFormat { public: - VerticalRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); + VerticalRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_); String getName() const override { return "VerticalRowOutputFormat"; } @@ -50,4 +50,3 @@ protected: }; } - diff --git a/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index 545f80692cc..07335fc6c49 100644 --- a/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp @@ -7,8 +7,8 @@ namespace DB { -XMLRowOutputFormat::XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_), format_settings(format_settings_) +XMLRowOutputFormat::XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, callback), format_settings(format_settings_) { auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); @@ -246,9 +246,10 @@ void registerOutputFormatProcessorXML(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings & settings) { - return std::make_shared(buf, sample, settings); + return std::make_shared(buf, sample, callback, settings); }); } diff --git a/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.h index b2370090c32..8c3da59cfd1 100644 --- a/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.h @@ -16,7 +16,7 @@ namespace DB class XMLRowOutputFormat : public IRowOutputFormat { public: - XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); + XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_); String getName() const override { return "XMLRowOutputFormat"; } @@ -75,4 +75,3 @@ protected: }; } - diff --git a/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp b/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp index d13df2ae7bc..1962e4fbc63 100644 --- a/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp +++ b/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp @@ -33,7 +33,7 @@ KafkaBlockInputStream::~KafkaBlockInputStream() buffer->reset(); } - storage.pushBuffer(buffer); + storage.pushReadBuffer(buffer); } Block KafkaBlockInputStream::getHeader() const @@ -43,11 +43,12 @@ Block KafkaBlockInputStream::getHeader() const void KafkaBlockInputStream::readPrefixImpl() { - buffer = storage.tryClaimBuffer(context.getSettingsRef().queue_max_wait_ms.totalMilliseconds()); + auto timeout = std::chrono::milliseconds(context.getSettingsRef().queue_max_wait_ms.totalMilliseconds()); + buffer = storage.popReadBuffer(timeout); claimed = !!buffer; if (!buffer) - buffer = storage.createBuffer(); + return; buffer->subBufferAs()->subscribe(storage.getTopics()); @@ -80,6 +81,9 @@ void KafkaBlockInputStream::readPrefixImpl() Block KafkaBlockInputStream::readImpl() { + if (!buffer) + return Block(); + Block block = children.back()->read(); if (!block) return block; @@ -99,6 +103,9 @@ Block KafkaBlockInputStream::readImpl() void KafkaBlockInputStream::readSuffixImpl() { + if (!buffer) + return; + buffer->subBufferAs()->commit(); broken = false; diff --git a/dbms/src/Storages/Kafka/KafkaBlockInputStream.h b/dbms/src/Storages/Kafka/KafkaBlockInputStream.h index a7e82034eb9..fef7f8d0469 100644 --- a/dbms/src/Storages/Kafka/KafkaBlockInputStream.h +++ b/dbms/src/Storages/Kafka/KafkaBlockInputStream.h @@ -27,7 +27,7 @@ private: Names column_names; UInt64 max_block_size; - BufferPtr buffer; + ConsumerBufferPtr buffer; MutableColumns virtual_columns; bool broken = true, claimed = false; }; diff --git a/dbms/src/Storages/Kafka/KafkaBlockOutputStream.cpp b/dbms/src/Storages/Kafka/KafkaBlockOutputStream.cpp new file mode 100644 index 00000000000..574b3bd9fd1 --- /dev/null +++ b/dbms/src/Storages/Kafka/KafkaBlockOutputStream.cpp @@ -0,0 +1,54 @@ +#include "KafkaBlockOutputStream.h" + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + +extern int CANNOT_CREATE_IO_BUFFER; + +} + +KafkaBlockOutputStream::KafkaBlockOutputStream(StorageKafka & storage_, const Context & context_) : storage(storage_), context(context_) +{ +} + +KafkaBlockOutputStream::~KafkaBlockOutputStream() +{ +} + +Block KafkaBlockOutputStream::getHeader() const +{ + return storage.getSampleBlockNonMaterialized(); +} + +void KafkaBlockOutputStream::writePrefix() +{ + buffer = storage.createWriteBuffer(); + if (!buffer) + throw Exception("Failed to create Kafka producer!", ErrorCodes::CANNOT_CREATE_IO_BUFFER); + + child = FormatFactory::instance().getOutput(storage.getFormatName(), *buffer, getHeader(), context, [this]{ buffer->count_row(); }); +} + +void KafkaBlockOutputStream::write(const Block & block) +{ + child->write(block); +} + +void KafkaBlockOutputStream::writeSuffix() +{ + child->writeSuffix(); + flush(); +} + +void KafkaBlockOutputStream::flush() +{ + if (buffer) + buffer->flush(); +} + +} diff --git a/dbms/src/Storages/Kafka/KafkaBlockOutputStream.h b/dbms/src/Storages/Kafka/KafkaBlockOutputStream.h new file mode 100644 index 00000000000..daa1c7bd98b --- /dev/null +++ b/dbms/src/Storages/Kafka/KafkaBlockOutputStream.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class KafkaBlockOutputStream : public IBlockOutputStream +{ +public: + explicit KafkaBlockOutputStream(StorageKafka & storage_, const Context & context_); + ~KafkaBlockOutputStream() override; + + Block getHeader() const override; + + void writePrefix() override; + void write(const Block & block) override; + void writeSuffix() override; + + void flush() override; + +private: + StorageKafka & storage; + Context context; + ProducerBufferPtr buffer; + BlockOutputStreamPtr child; +}; + +} diff --git a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h index a8295152d91..8f78eae23e1 100644 --- a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h +++ b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h @@ -10,7 +10,7 @@ namespace DB { -using BufferPtr = std::shared_ptr; +using ConsumerBufferPtr = std::shared_ptr; using ConsumerPtr = std::shared_ptr; class ReadBufferFromKafkaConsumer : public ReadBuffer diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 7cfcb4caf10..e429ef505de 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -106,7 +107,7 @@ StorageKafka::StorageKafka( , skip_broken(skip_broken_) , intermediate_commit(intermediate_commit_) { - task = global_context.getSchedulePool().createTask(log->name(), [this]{ streamThread(); }); + task = global_context.getSchedulePool().createTask(log->name(), [this]{ threadFunc(); }); task->deactivate(); } @@ -140,14 +141,21 @@ BlockInputStreams StorageKafka::read( } +BlockOutputStreamPtr StorageKafka::write(const ASTPtr &, const Context & context) +{ + if (topics.size() > 1) + throw Exception("Can't write to Kafka table with multiple topics!", ErrorCodes::NOT_IMPLEMENTED); + return std::make_shared(*this, context); +} + + void StorageKafka::startup() { for (size_t i = 0; i < num_consumers; ++i) { - // Make buffer available try { - pushBuffer(createBuffer()); + pushReadBuffer(createReadBuffer()); ++num_created_consumers; } catch (const cppkafka::Exception &) @@ -169,7 +177,7 @@ void StorageKafka::shutdown() // Close all consumers for (size_t i = 0; i < num_created_consumers; ++i) { - auto buffer = claimBuffer(); + auto buffer = popReadBuffer(); // FIXME: not sure if we really close consumers here, and if we really need to close them here. } @@ -193,10 +201,70 @@ void StorageKafka::updateDependencies() } -BufferPtr StorageKafka::createBuffer() +void StorageKafka::pushReadBuffer(ConsumerBufferPtr buffer) { + std::lock_guard lock(mutex); + buffers.push_back(buffer); + semaphore.set(); +} + + +ConsumerBufferPtr StorageKafka::popReadBuffer() +{ + return popReadBuffer(std::chrono::milliseconds::zero()); +} + + +ConsumerBufferPtr StorageKafka::popReadBuffer(std::chrono::milliseconds timeout) +{ + // Wait for the first free buffer + if (timeout == std::chrono::milliseconds::zero()) + semaphore.wait(); + else + { + if (!semaphore.tryWait(timeout.count())) + return nullptr; + } + + // Take the first available buffer from the list + std::lock_guard lock(mutex); + auto buffer = buffers.back(); + buffers.pop_back(); + return buffer; +} + + +ProducerBufferPtr StorageKafka::createWriteBuffer() +{ + cppkafka::Configuration conf; + conf.set("metadata.broker.list", brokers); + conf.set("group.id", group); + conf.set("client.id", VERSION_FULL); + // TODO: fill required settings + updateConfiguration(conf); + + auto producer = std::make_shared(conf); + const Settings & settings = global_context.getSettingsRef(); + size_t poll_timeout = settings.stream_poll_timeout_ms.totalMilliseconds(); + + return std::make_shared( + producer, topics[0], row_delimiter ? std::optional{row_delimiter} : std::optional(), 1, 1024, std::chrono::milliseconds(poll_timeout)); +} + + +ConsumerBufferPtr StorageKafka::createReadBuffer() +{ + cppkafka::Configuration conf; + conf.set("metadata.broker.list", brokers); + conf.set("group.id", group); + conf.set("client.id", VERSION_FULL); + conf.set("auto.offset.reset", "smallest"); // If no offset stored for this group, read all messages from the start + conf.set("enable.auto.commit", "false"); // We manually commit offsets after a stream successfully finished + conf.set("enable.partition.eof", "false"); // Ignore EOF messages + updateConfiguration(conf); + // Create a consumer and subscribe to topics - auto consumer = std::make_shared(createConsumerConfiguration()); + auto consumer = std::make_shared(conf); // Limit the number of batched messages to allow early cancellations const Settings & settings = global_context.getSettingsRef(); @@ -209,61 +277,9 @@ BufferPtr StorageKafka::createBuffer() std::make_unique(consumer, log, batch_size, poll_timeout, intermediate_commit), row_delimiter); } -BufferPtr StorageKafka::claimBuffer() + +void StorageKafka::updateConfiguration(cppkafka::Configuration & conf) { - return tryClaimBuffer(-1L); -} - -BufferPtr StorageKafka::tryClaimBuffer(long wait_ms) -{ - // Wait for the first free buffer - if (wait_ms >= 0) - { - if (!semaphore.tryWait(wait_ms)) - return nullptr; - } - else - semaphore.wait(); - - // Take the first available buffer from the list - std::lock_guard lock(mutex); - auto buffer = buffers.back(); - buffers.pop_back(); - return buffer; -} - -void StorageKafka::pushBuffer(BufferPtr buffer) -{ - std::lock_guard lock(mutex); - buffers.push_back(buffer); - semaphore.set(); -} - - -cppkafka::Configuration StorageKafka::createConsumerConfiguration() -{ - cppkafka::Configuration conf; - - LOG_TRACE(log, "Setting brokers: " << brokers); - conf.set("metadata.broker.list", brokers); - - LOG_TRACE(log, "Setting Group ID: " << group << " Client ID: clickhouse"); - conf.set("group.id", group); - - conf.set("client.id", VERSION_FULL); - - // If no offset stored for this group, read all messages from the start - conf.set("auto.offset.reset", "smallest"); - - // We manually commit offsets after a stream successfully finished - conf.set("enable.auto.commit", "false"); - - // Ignore EOF messages - conf.set("enable.partition.eof", "false"); - - // for debug logs inside rdkafka - // conf.set("debug", "consumer,cgrp,topic,fetch"); - // Update consumer configuration from the configuration const auto & config = global_context.getConfigRef(); if (config.has(CONFIG_PREFIX)) @@ -276,8 +292,6 @@ cppkafka::Configuration StorageKafka::createConsumerConfiguration() if (config.has(topic_config_key)) loadFromConfig(conf, config, topic_config_key); } - - return conf; } bool StorageKafka::checkDependencies(const String & current_database_name, const String & current_table_name) @@ -307,7 +321,7 @@ bool StorageKafka::checkDependencies(const String & current_database_name, const return true; } -void StorageKafka::streamThread() +void StorageKafka::threadFunc() { try { diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index f9b6609def5..b42ce93e668 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -5,11 +5,11 @@ #include #include #include -#include +#include + #include #include -#include #include namespace DB @@ -36,14 +36,20 @@ public: size_t max_block_size, unsigned num_streams) override; + BlockOutputStreamPtr write( + const ASTPtr & query, + const Context & context + ) override; + void rename(const String & /* new_path_to_db */, const String & new_database_name, const String & new_table_name) override; void updateDependencies() override; - BufferPtr createBuffer(); - BufferPtr claimBuffer(); - BufferPtr tryClaimBuffer(long wait_ms); - void pushBuffer(BufferPtr buf); + void pushReadBuffer(ConsumerBufferPtr buf); + ConsumerBufferPtr popReadBuffer(); + ConsumerBufferPtr popReadBuffer(std::chrono::milliseconds timeout); + + ProducerBufferPtr createWriteBuffer(); const auto & getTopics() const { return topics; } const auto & getFormatName() const { return format_name; } @@ -84,7 +90,7 @@ private: // Consumer list Poco::Semaphore semaphore; std::mutex mutex; - std::vector buffers; /// available buffers for Kafka consumers + std::vector buffers; /// available buffers for Kafka consumers size_t skip_broken; @@ -94,9 +100,12 @@ private: BackgroundSchedulePool::TaskHolder task; std::atomic stream_cancelled{false}; - cppkafka::Configuration createConsumerConfiguration(); + ConsumerBufferPtr createReadBuffer(); - void streamThread(); + // Update Kafka configuration with values from CH user configuration. + void updateConfiguration(cppkafka::Configuration & conf); + + void threadFunc(); bool streamToViews(); bool checkDependencies(const String & database_name, const String & table_name); }; diff --git a/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp new file mode 100644 index 00000000000..f88b7eaad5c --- /dev/null +++ b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp @@ -0,0 +1,90 @@ +#include "WriteBufferToKafkaProducer.h" + +namespace DB +{ +WriteBufferToKafkaProducer::WriteBufferToKafkaProducer( + ProducerPtr producer_, + const std::string & topic_, + std::optional delimiter, + size_t rows_per_message, + size_t chunk_size_, + std::chrono::milliseconds poll_timeout) + : WriteBuffer(nullptr, 0) + , producer(producer_) + , topic(topic_) + , delim(delimiter) + , max_rows(rows_per_message) + , chunk_size(chunk_size_) + , timeout(poll_timeout) +{ +} + +WriteBufferToKafkaProducer::~WriteBufferToKafkaProducer() +{ + assert(rows == 0 && chunks.empty()); +} + +void WriteBufferToKafkaProducer::count_row() +{ + if (++rows % max_rows == 0) + { + std::string payload; + payload.reserve((chunks.size() - 1) * chunk_size + offset()); + for (auto i = chunks.begin(), e = --chunks.end(); i != e; ++i) + payload.append(*i); + int trunk_delim = delim && chunks.back()[offset() - 1] == delim ? 1 : 0; + payload.append(chunks.back(), 0, offset() - trunk_delim); + + while (true) + { + try + { + producer->produce(cppkafka::MessageBuilder(topic).payload(payload)); + } + catch (cppkafka::HandleException & e) + { + if (e.get_error() == RD_KAFKA_RESP_ERR__QUEUE_FULL) + { + producer->poll(timeout); + continue; + } + throw e; + } + + break; + } + + rows = 0; + chunks.clear(); + set(nullptr, 0); + } +} + +void WriteBufferToKafkaProducer::flush() +{ + // For unknown reason we may hit some internal timeout when inserting for the first time. + while (true) + { + try + { + producer->flush(timeout); + } + catch (cppkafka::HandleException & e) + { + if (e.get_error() == RD_KAFKA_RESP_ERR__TIMED_OUT) + continue; + throw e; + } + + break; + } +} + +void WriteBufferToKafkaProducer::nextImpl() +{ + chunks.push_back(std::string()); + chunks.back().resize(chunk_size); + set(chunks.back().data(), chunk_size); +} + +} diff --git a/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.h b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.h new file mode 100644 index 00000000000..d11766d7f35 --- /dev/null +++ b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.h @@ -0,0 +1,46 @@ +#pragma once + +#include + +#include + +#include + +namespace DB +{ + +class WriteBufferToKafkaProducer; + +using ProducerBufferPtr = std::shared_ptr; +using ProducerPtr = std::shared_ptr; + +class WriteBufferToKafkaProducer : public WriteBuffer +{ +public: + WriteBufferToKafkaProducer( + ProducerPtr producer_, + const std::string & topic_, + std::optional delimiter, + size_t rows_per_message, + size_t chunk_size_, + std::chrono::milliseconds poll_timeout); + ~WriteBufferToKafkaProducer() override; + + void count_row(); + void flush(); + +private: + void nextImpl() override; + + ProducerPtr producer; + const std::string topic; + const std::optional delim; + const size_t max_rows; + const size_t chunk_size; + const std::chrono::milliseconds timeout; + + size_t rows = 0; + std::list chunks; +}; + +} diff --git a/dbms/tests/integration/test_storage_kafka/test.py b/dbms/tests/integration/test_storage_kafka/test.py index 09b13d884f6..0f0cf6996ef 100644 --- a/dbms/tests/integration/test_storage_kafka/test.py +++ b/dbms/tests/integration/test_storage_kafka/test.py @@ -1,14 +1,17 @@ import os.path as p +import random +import threading import time import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV +from helpers.client import QueryRuntimeException import json import subprocess import kafka.errors -from kafka import KafkaAdminClient, KafkaProducer +from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer from google.protobuf.internal.encoder import _VarintBytes """ @@ -69,6 +72,17 @@ def kafka_produce(topic, messages, timestamp=None): print ("Produced {} messages for topic {}".format(len(messages), topic)) +def kafka_consume(topic): + consumer = KafkaConsumer(bootstrap_servers="localhost:9092", auto_offset_reset="earliest") + consumer.subscribe(topics=(topic)) + for toppar, messages in consumer.poll(5000).items(): + if toppar.topic == topic: + for message in messages: + yield message.value + consumer.unsubscribe() + consumer.close() + + def kafka_produce_protobuf_messages(topic, start_index, num_messages): data = '' for i in range(start_index, start_index + num_messages): @@ -490,6 +504,105 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster): kafka_check_result(result, True, 'test_kafka_virtual2.reference') +def test_kafka_insert(kafka_cluster): + instance.query(''' + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'insert1', + kafka_group_name = 'insert1', + kafka_format = 'TSV', + kafka_row_delimiter = '\\n'; + ''') + + values = [] + for i in range(50): + values.append("({i}, {i})".format(i=i)) + values = ','.join(values) + + while True: + try: + instance.query("INSERT INTO test.kafka VALUES {}".format(values)) + break + except QueryRuntimeException as e: + if 'Local: Timed out.' in str(e): + continue + else: + raise + + messages = [] + while True: + messages.extend(kafka_consume('insert1')) + if len(messages) == 50: + break + + result = '\n'.join(messages) + kafka_check_result(result, True) + + +def test_kafka_produce_consume(kafka_cluster): + instance.query(''' + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'insert2', + kafka_group_name = 'insert2', + kafka_format = 'TSV', + kafka_row_delimiter = '\\n'; + ''') + + messages_num = 10000 + def insert(): + values = [] + for i in range(messages_num): + values.append("({i}, {i})".format(i=i)) + values = ','.join(values) + + while True: + try: + instance.query("INSERT INTO test.kafka VALUES {}".format(values)) + break + except QueryRuntimeException as e: + if 'Local: Timed out.' in str(e): + continue + else: + raise + + threads = [] + threads_num = 16 + for _ in range(threads_num): + threads.append(threading.Thread(target=insert)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; + ''') + + while True: + result = instance.query('SELECT count() FROM test.view') + time.sleep(1) + if int(result) == messages_num * threads_num: + break + + instance.query(''' + DROP TABLE test.consumer; + DROP TABLE test.view; + ''') + + for thread in threads: + thread.join() + + assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + + if __name__ == '__main__': cluster.start() raw_input("Cluster created, press any key to destroy...") diff --git a/libs/libcommon/include/ext/shared_ptr_helper.h b/libs/libcommon/include/ext/shared_ptr_helper.h index 7e882e03391..f7fd7c38ace 100644 --- a/libs/libcommon/include/ext/shared_ptr_helper.h +++ b/libs/libcommon/include/ext/shared_ptr_helper.h @@ -12,7 +12,7 @@ namespace ext * * Downsides: * - your class cannot be final; - * - bad compilation error messages; + * - awful compilation error messages; * - bad code navigation. * - different dynamic type of created object, you cannot use typeid. */ diff --git a/utils/kafka/consume.py b/utils/kafka/consume.py index ce2a0ef6a96..34a1997d3e6 100755 --- a/utils/kafka/consume.py +++ b/utils/kafka/consume.py @@ -26,6 +26,7 @@ def main(): 'bootstrap_servers': f'{args.server}:{args.port}', 'client_id': args.client, 'group_id': args.group, + 'auto_offset_reset': 'earliest', } client = kafka.KafkaConsumer(**config) From 8d0c2eacfd163e2f3c9db670c95ddc23cfeb8ca0 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Tue, 20 Aug 2019 13:18:08 +0200 Subject: [PATCH 269/509] remove doubled const TABLE_IS_READ_ONLY --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 3781b9efea5..7547a37af74 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -91,7 +91,6 @@ namespace ErrorCodes extern const int NOT_FOUND_NODE; extern const int NO_ACTIVE_REPLICAS; extern const int LEADERSHIP_CHANGED; - extern const int TABLE_IS_READ_ONLY; extern const int TABLE_WAS_NOT_DROPPED; extern const int PARTITION_ALREADY_EXISTS; extern const int TOO_MANY_RETRIES_TO_FETCH_PARTS; From e986d6ba530e005cc3c2a6a5e84de1cdb9134c97 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 20 Aug 2019 15:06:52 +0300 Subject: [PATCH 270/509] DOCAPI-7459: skip_unavailable_shards docs (#6361) * Link fix. * DOCAPI-7459: skip_unavailable_shards docs * Update dbms/src/Core/Settings.h Co-Authored-By: Ivan Blinkov * Update docs/en/operations/settings/settings.md Co-Authored-By: Ivan Blinkov * Update docs/en/operations/settings/settings.md Co-Authored-By: Ivan Blinkov * Update docs/en/operations/settings/settings.md Co-Authored-By: Ivan Blinkov * DOCAPI-7459: Markup fixes. --- dbms/src/Core/Settings.h | 2 +- .../en/operations/server_settings/settings.md | 9 ++++--- docs/en/operations/settings/settings.md | 26 +++++++++++++++++++ docs/en/operations/table_engines/buffer.md | 19 ++++++++++---- docs/en/operations/table_engines/mergetree.md | 1 + 5 files changed, 47 insertions(+), 10 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 86e8746e854..877f99856c9 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -99,7 +99,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, parallel_replicas_count, 0, "") \ M(SettingUInt64, parallel_replica_offset, 0, "") \ \ - M(SettingBool, skip_unavailable_shards, false, "Silently skip unavailable shards.") \ + M(SettingBool, skip_unavailable_shards, false, "If 1, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.") \ \ M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.") \ M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.") \ diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index bf2a430f7d8..2be36a6e0c6 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -555,11 +555,9 @@ If the table doesn't exist, ClickHouse will create it. If the structure of the q ``` -## remote_servers +## remote_servers {#server_settings_remote_servers} -Configuration of clusters used by the Distributed table engine. - -For more information, see the section "[Table engines/Distributed](../../operations/table_engines/distributed.md)". +Configuration of clusters used by the [Distributed](../../operations/table_engines/distributed.md) table engine and by the `cluster` table function. **Example** @@ -569,6 +567,9 @@ For more information, see the section "[Table engines/Distributed](../../operati For the value of the `incl` attribute, see the section "[Configuration files](../configuration_files.md#configuration_files)". +**See Also** + +- [skip_unavailable_shards](../settings/settings.md#settings-skip_unavailable_shards) ## timezone diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9e298583b38..02d5871894b 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -836,3 +836,29 @@ Possible values: Default value: `uniqExact`. [Original article](https://clickhouse.yandex/docs/en/operations/settings/settings/) + + +## skip_unavailable_shards {#settings-skip_unavailable_shards} + +Enables or disables silent skipping of: + +- Node, if its name cannot be resolved through DNS. + + When skipping is disabled, ClickHouse requires that all the nodes in the [cluster configuration](../server_settings/settings.md#server_settings_remote_servers) can be resolvable through DNS. Otherwise, ClickHouse throws an exception when trying to perform a query on the cluster. + + If skipping is enabled, ClickHouse considers unresolved nodes as unavailable and tries to resolve them at every connection attempt. Such behavior creates the risk of wrong cluster configuration because a user can specify the wrong node name, and ClickHouse doesn't report about it. However, this can be useful in systems with dynamic DNS, for example, [Kubernetes](https://kubernetes.io), where nodes can be unresolvable during downtime, and this is not an error. + +- Shard, if there are no available replicas of the shard. + + When skipping is disabled, ClickHouse throws an exception. + + When skipping is enabled, ClickHouse returns a partial answer and doesn't report about issues with nodes availability. + +Possible values: + +- 1 — skipping enabled. +- 0 — skipping disabled. + +Default value: 0. + +[Original article](https://clickhouse.yandex/docs/en/operations/settings/settings/) diff --git a/docs/en/operations/table_engines/buffer.md b/docs/en/operations/table_engines/buffer.md index d8e240b8013..18024918835 100644 --- a/docs/en/operations/table_engines/buffer.md +++ b/docs/en/operations/table_engines/buffer.md @@ -2,17 +2,26 @@ Buffers the data to write in RAM, periodically flushing it to another table. During the read operation, data is read from the buffer and the other table simultaneously. -``` +```sql Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes) ``` -Engine parameters:database, table – The table to flush data to. Instead of the database name, you can use a constant expression that returns a string. num_layers – Parallelism layer. Physically, the table will be represented as 'num_layers' of independent buffers. Recommended value: 16. min_time, max_time, min_rows, max_rows, min_bytes, and max_bytes are conditions for flushing data from the buffer. +Engine parameters: -Data is flushed from the buffer and written to the destination table if all the 'min' conditions or at least one 'max' condition are met.min_time, max_time – Condition for the time in seconds from the moment of the first write to the buffer. min_rows, max_rows – Condition for the number of rows in the buffer. min_bytes, max_bytes – Condition for the number of bytes in the buffer. +- `database` – Database name. Instead of the database name, you can use a constant expression that returns a string. +- `table` – Table to flush data to. +- `num_layers` – Parallelism layer. Physically, the table will be represented as 'num_layers' of independent buffers. Recommended value: 16. +- `min_time`, `max_time`, `min_rows`, `max_rows`, `min_bytes`, and `max_bytes` – Conditions for flushing data from the buffer. -During the write operation, data is inserted to a 'num_layers' number of random buffers. Or, if the data part to insert is large enough (greater than 'max_rows' or 'max_bytes'), it is written directly to the destination table, omitting the buffer. +Data is flushed from the buffer and written to the destination table if all the `min*` conditions or at least one `max*` condition are met. -The conditions for flushing the data are calculated separately for each of the 'num_layers' buffers. For example, if num_layers = 16 and max_bytes = 100000000, the maximum RAM consumption is 1.6 GB. +- `min_time`, `max_time` – Condition for the time in seconds from the moment of the first write to the buffer. +- `min_rows`, `max_rows` – Condition for the number of rows in the buffer. +- `min_bytes`, `max_bytes` – Condition for the number of bytes in the buffer. + +During the write operation, data is inserted to a `num_layers` number of random buffers. Or, if the data part to insert is large enough (greater than `max_rows` or `max_bytes`), it is written directly to the destination table, omitting the buffer. + +The conditions for flushing the data are calculated separately for each of the `num_layers` buffers. For example, if `num_layers = 16` and `max_bytes = 100000000`, the maximum RAM consumption is 1.6 GB. Example: diff --git a/docs/en/operations/table_engines/mergetree.md b/docs/en/operations/table_engines/mergetree.md index d28c5595942..a3646d3d0e9 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -172,6 +172,7 @@ The number of columns in the primary key is not explicitly limited. Depending on - Improve the performance of an index. If the primary key is `(a, b)`, then adding another column `c` will improve the performance if the following conditions are met: + - There are queries with a condition on column `c`. - Long data ranges (several times longer than the `index_granularity`) with identical values for `(a, b)` are common. In other words, when adding another column allows you to skip quite long data ranges. From 0ab886ce9ddcecd9831ca697c740cb31a8f127fa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Aug 2019 15:28:36 +0300 Subject: [PATCH 271/509] Materialize constant in arrayWithConstant function. --- dbms/src/Functions/array/arrayWithConstant.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/array/arrayWithConstant.cpp b/dbms/src/Functions/array/arrayWithConstant.cpp index efdcc7d6b72..b23d76b694d 100644 --- a/dbms/src/Functions/array/arrayWithConstant.cpp +++ b/dbms/src/Functions/array/arrayWithConstant.cpp @@ -72,7 +72,7 @@ public: offsets.push_back(offset); } - block.getByPosition(result).column = ColumnArray::create(col_value->replicate(offsets), std::move(offsets_col)); + block.getByPosition(result).column = ColumnArray::create(col_value->replicate(offsets)->convertToFullColumnIfConst(), std::move(offsets_col)); } }; From 17a7cb8735cc51684f5c519a8ccc26f66e774c68 Mon Sep 17 00:00:00 2001 From: Rafael David Tinoco Date: Tue, 20 Aug 2019 02:38:47 +0000 Subject: [PATCH 272/509] MySQL 8 integration requires previous declaration checks C and C++ differ in the form of types being defined. While C++ structs are defined also as new types, in C you have to explicitly typedef the struct to have a new type. Fir this case, it was enough to check if MySQL header was already defined in order not to re-declare MYSQL, MYSQL_RES, MYSQL_ROW and MYSQL_FIELD. Signed-off-by: Rafael David Tinoco --- libs/libmysqlxx/include/mysqlxx/Types.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/libs/libmysqlxx/include/mysqlxx/Types.h b/libs/libmysqlxx/include/mysqlxx/Types.h index 30abdeb9fd0..b5ed70916fa 100644 --- a/libs/libmysqlxx/include/mysqlxx/Types.h +++ b/libs/libmysqlxx/include/mysqlxx/Types.h @@ -3,6 +3,8 @@ #include #include +#ifndef _mysql_h + struct st_mysql; using MYSQL = st_mysql; @@ -14,7 +16,7 @@ using MYSQL_ROW = char**; struct st_mysql_field; using MYSQL_FIELD = st_mysql_field; - +#endif namespace mysqlxx { From bdd46bd5853055ebcf9f33b55b4aeeffd672b79e Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 20 Aug 2019 15:58:23 +0300 Subject: [PATCH 273/509] DOCAPI-7991: Typo fix. --- docs/en/operations/table_engines/log_family.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/table_engines/log_family.md b/docs/en/operations/table_engines/log_family.md index bc55f5ccce4..aef0e21f08c 100644 --- a/docs/en/operations/table_engines/log_family.md +++ b/docs/en/operations/table_engines/log_family.md @@ -14,7 +14,7 @@ Engines: - Store data on a disk. - Append data to the end of file when writing. -- Support locs for concurrent data access. +- Support locks for concurrent data access. During `INSERT` query the table is locked, and other queries for reading and writing data both wait for unlocking. If there are no writing data queries, any number of reading data queries can be performed concurrently. From eecc8b012132365569a22082711de21dae70c7f7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Aug 2019 16:33:25 +0300 Subject: [PATCH 274/509] Update tests. --- dbms/tests/performance/general_purpose_hashes.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/tests/performance/general_purpose_hashes.xml b/dbms/tests/performance/general_purpose_hashes.xml index 14da22e6c97..b7a1b915ff0 100644 --- a/dbms/tests/performance/general_purpose_hashes.xml +++ b/dbms/tests/performance/general_purpose_hashes.xml @@ -1,4 +1,6 @@ + loop + 3 @@ -10,7 +12,6 @@ - From 852a06203d63fcbafa5b9d17560b4fb586f41f59 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 20 Aug 2019 17:08:09 +0300 Subject: [PATCH 275/509] Move input_format_defaults_for_omitted_fields to incompatible changes --- CHANGELOG.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 58c38dc120a..607f650deeb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,6 +9,9 @@ * `RENAME` queries now work with all storages. [#5953](https://github.com/yandex/ClickHouse/pull/5953) ([Ivan](https://github.com/abyss7)) * Now client receive logs from server with any desired level by setting `send_logs_level` regardless to the log level specified in server settings. [#5964](https://github.com/yandex/ClickHouse/pull/5964) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +### Backward Incompatible Change +* The setting `input_format_defaults_for_omitted_fields` is enabled by default. Inserts in Distibuted tables need this setting to be the same on cluster (you need to set it before rolling update). It enables calculation of complex default expressions for omitted fields in `JSONEachRow` and `CSV*` formats. It should be the expected behaviour but may lead to negligible performance difference. [#6043](https://github.com/yandex/ClickHouse/pull/6043) ([Artem Zuikov](https://github.com/4ertus2)), [#5625](https://github.com/yandex/ClickHouse/pull/5625) ([akuzm](https://github.com/akuzm)) + ### Experimental features * New query processing pipeline. Use `experimental_use_processors=1` option to enable it. Use for your own trouble. [#4914](https://github.com/yandex/ClickHouse/pull/4914) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) @@ -18,7 +21,6 @@ * Fixed overestimation of `max_rows_to_read` if the setting `merge_tree_uniform_read_distribution` is set to 0. [#6019](https://github.com/yandex/ClickHouse/pull/6019) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### Improvement -* The setting `input_format_defaults_for_omitted_fields` is enabled by default. It enables calculation of complex default expressions for omitted fields in `JSONEachRow` and `CSV*` formats. It should be the expected behaviour but may lead to negligible performance difference or subtle incompatibilities. [#6043](https://github.com/yandex/ClickHouse/pull/6043) ([Artem Zuikov](https://github.com/4ertus2)), [#5625](https://github.com/yandex/ClickHouse/pull/5625) ([akuzm](https://github.com/akuzm)) * Throws an exception if `config.d` file doesn't have the corresponding root element as the config file [#6123](https://github.com/yandex/ClickHouse/pull/6123) ([dimarub2000](https://github.com/dimarub2000)) ### Performance Improvement From 7bd748c96917753954979b9f78872bbb39c42bd9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Aug 2019 17:13:06 +0300 Subject: [PATCH 276/509] Renamed NumbersMtBlockInputStream to NumbersMultiThreadedBlockInputStream. --- dbms/src/Storages/System/StorageSystemNumbers.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemNumbers.cpp b/dbms/src/Storages/System/StorageSystemNumbers.cpp index b08e074af60..78e25e0c375 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.cpp +++ b/dbms/src/Storages/System/StorageSystemNumbers.cpp @@ -51,10 +51,10 @@ struct NumbersMtState using NumbersMtStatePtr = std::shared_ptr; -class NumbersMtBlockInputStream : public IBlockInputStream +class NumbersMultiThreadedBlockInputStream : public IBlockInputStream { public: - NumbersMtBlockInputStream(NumbersMtStatePtr state_, UInt64 block_size_, UInt64 max_counter_) + NumbersMultiThreadedBlockInputStream(NumbersMtStatePtr state_, UInt64 block_size_, UInt64 max_counter_) : state(std::move(state_)), counter(state->counter), block_size(block_size_), max_counter(max_counter_) {} String getName() const override { return "NumbersMt"; } @@ -131,7 +131,7 @@ BlockInputStreams StorageSystemNumbers::read( UInt64 max_counter = offset + *limit; for (size_t i = 0; i < num_streams; ++i) - res[i] = std::make_shared(state, max_block_size, max_counter); + res[i] = std::make_shared(state, max_block_size, max_counter); return res; } From 06bb0af3869b44ceff7ce4c69c6a23fa3bb4942f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Aug 2019 18:03:35 +0300 Subject: [PATCH 277/509] Fix build. --- .../Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp index 6d448b00203..33fc1c9920b 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp @@ -34,9 +34,10 @@ void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factor WriteBuffer & buf, const Block & sample, const Context &, + FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, format_settings); + return std::make_shared(buf, sample, callback, format_settings); }); } From c43f9030da6fa0464b08a57972048f1a3f31d2f7 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 20 Aug 2019 18:36:08 +0300 Subject: [PATCH 278/509] DOCAPI-7460: Clarifications. --- .../agg_functions/parametric_functions.md | 43 ++++++++++++++++--- .../functions/other_functions.md | 2 +- 2 files changed, 38 insertions(+), 7 deletions(-) diff --git a/docs/en/query_language/agg_functions/parametric_functions.md b/docs/en/query_language/agg_functions/parametric_functions.md index d27cb5d9431..84898a61133 100644 --- a/docs/en/query_language/agg_functions/parametric_functions.md +++ b/docs/en/query_language/agg_functions/parametric_functions.md @@ -4,18 +4,18 @@ Some aggregate functions can accept not only argument columns (used for compress ## histogram -Calculates a histogram. +Calculates an adaptive histogram. It doesn't guarantee precise results. ``` histogram(number_of_bins)(values) ``` - -The functions uses [A Streaming Parallel Decision Tree Algorithm](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf). It calculates the borders of histogram bins automatically, and in common case the widths of bins are not equal. + +The functions uses [A Streaming Parallel Decision Tree Algorithm](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf). The borders of histogram bins are adjusted as a new data enters a function, and in common case the widths of bins are not equal. **Parameters** -`number_of_bins` — Number of bins for the histogram. -`values` — [Expression](../syntax.md#syntax-expressions) resulting in a data sample. +`number_of_bins` — Upper limit for a number of bins for the histogram. Function automatically calculates the number of bins. It tries to reach the specified number of bins, but if it fails, it uses less number of bins. +`values` — [Expression](../syntax.md#syntax-expressions) resulting in input values. **Returned values** @@ -32,7 +32,12 @@ The functions uses [A Streaming Parallel Decision Tree Algorithm](http://jmlr.or **Example** ```sql -SELECT histogram(5)(number + 1) FROM (SELECT * FROM system.numbers LIMIT 20) +SELECT histogram(5)(number + 1) +FROM ( + SELECT * + FROM system.numbers + LIMIT 20 +) ``` ```text ┌─histogram(5)(plus(number, 1))───────────────────────────────────────────┐ @@ -40,6 +45,32 @@ SELECT histogram(5)(number + 1) FROM (SELECT * FROM system.numbers LIMIT 20) └─────────────────────────────────────────────────────────────────────────┘ ``` +You can visualize a histogram with the [bar](../other_functions.md#function-bar) function, for example: + +```sql +WITH histogram(5)(rand() % 100) AS hist +SELECT + arrayJoin(hist).3 AS height, + bar(height, 0, 6, 5) AS bar +FROM +( + SELECT * + FROM system.numbers + LIMIT 20 +) +``` +```text +┌─height─┬─bar───┐ +│ 2.125 │ █▋ │ +│ 3.25 │ ██▌ │ +│ 5.625 │ ████▏ │ +│ 5.625 │ ████▏ │ +│ 3.375 │ ██▌ │ +└────────┴───────┘ +``` + +In this case you should remember, that you don't know the borders of histogram bins. + ## sequenceMatch(pattern)(time, cond1, cond2, ...) Pattern matching for event chains. diff --git a/docs/en/query_language/functions/other_functions.md b/docs/en/query_language/functions/other_functions.md index 007f1352775..268c245d24f 100644 --- a/docs/en/query_language/functions/other_functions.md +++ b/docs/en/query_language/functions/other_functions.md @@ -120,7 +120,7 @@ Accepts constant strings: database name, table name, and column name. Returns a The function throws an exception if the table does not exist. For elements in a nested data structure, the function checks for the existence of a column. For the nested data structure itself, the function returns 0. -## bar +## bar {#function-bar} Allows building a unicode-art diagram. From 822fe63f3bdd832ef8d039223de27da339a1f65b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Aug 2019 19:13:18 +0300 Subject: [PATCH 279/509] Added comment. --- dbms/src/Storages/System/StorageSystemNumbers.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/src/Storages/System/StorageSystemNumbers.h b/dbms/src/Storages/System/StorageSystemNumbers.h index 75e717c0f60..5efc23a1125 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.h +++ b/dbms/src/Storages/System/StorageSystemNumbers.h @@ -19,6 +19,9 @@ class Context; * If multithreaded is specified, numbers will be generated in several streams * (and result could be out of order). If both multithreaded and limit are specified, * the table could give you not exactly 1..limit range, but some arbitrary 'limit' numbers. + * + * In multithreaded case, if even_distributed is False, implementation with atomic is used, + * and result is always in [0 ... limit - 1] range. */ class StorageSystemNumbers : public ext::shared_ptr_helper, public IStorage { From 3785d945ff732436164eb8f4c746315e913cfc8c Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 20 Aug 2019 20:42:31 +0300 Subject: [PATCH 280/509] Sync RPM packages instructions to other docs languages (#6568) --- docs/en/getting_started/index.md | 4 ++-- docs/fa/getting_started/index.md | 27 +++++++++++++++++++++++++ docs/ru/getting_started/index.md | 22 ++++++++++++++++----- docs/zh/getting_started/index.md | 34 ++++++++++++++++++++++++-------- 4 files changed, 72 insertions(+), 15 deletions(-) diff --git a/docs/en/getting_started/index.md b/docs/en/getting_started/index.md index 8f6308cd0ab..8cdbae86e5e 100644 --- a/docs/en/getting_started/index.md +++ b/docs/en/getting_started/index.md @@ -37,9 +37,9 @@ You can also download and install packages manually from here: :ﺪﯿﻨﮐ ﺐﺼﻧ ﻭ ﯼﺮﯿﮔﺭﺎﺑ ﺎﺠﻨ + + Docker Image ﺯﺍ ### + +.ﺪﻨﻨﮐ ﯽﻣ ﻩﺩﺎﻔﺘﺳﺍ ﻞﺧﺍﺩ ﺭﺩ "deb" ﯽﻤﺳﺭ ﯼﺎﻫ ﻪﺘﺴﺑ ﺯﺍ ﺮﯾﻭﺎﺼﺗ ﻦﯾﺍ .ﺪﯿﻨﮐ ﻝﺎﺒﻧﺩ ﺍﺭ (/ht + + ### نصب از طریق Source برای Compile، دستورالعمل های فایل build.md را دنبال کنید: diff --git a/docs/ru/getting_started/index.md b/docs/ru/getting_started/index.md index 8091f297019..e3fb2ab0985 100644 --- a/docs/ru/getting_started/index.md +++ b/docs/ru/getting_started/index.md @@ -37,13 +37,25 @@ sudo apt-get install clickhouse-client clickhouse-server ### Из RPM пакетов -Яндекс не использует ClickHouse на поддерживающих `rpm` дистрибутивах Linux, а также `rpm` пакеты менее тщательно тестируются. Таким образом, использовать их стоит на свой страх и риск, но, тем не менее, многие другие компании успешно работают на них в production без каких-либо серьезных проблем. +Команда ClickHouse в Яндексе рекомендует использовать официальные предкомпилированные `rpm` пакеты для CentOS, RedHad и всех остальных дистрибутивов Linux, основанных на rpm. -Для CentOS, RHEL и Fedora возможны следующие варианты: +Сначала нужно подключить официальный репозиторий: +```bash +sudo yum install yum-utils +sudo rpm --import https://repo.yandex.ru/clickhouse/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.yandex.ru/clickhouse/rpm/stable/x86_64 +``` -* Пакеты из генерируются на основе официальных `deb` пакетов от Яндекса и содержат в точности тот же исполняемый файл. -* Пакеты из собираются независимой компанией Altinity, но широко используются без каких-либо нареканий. -* Либо можно использовать Docker (см. ниже). +Для использования наиболее свежих версий нужно заменить `stable` на `testing` (рекомендуется для тестовых окружений). + +Then run these commands to actually install packages: +Для, собственно, установки пакетов необходимо выполнить следующие команды: + +```bash +sudo yum install clickhouse-server clickhouse-client +``` + +Также есть возможность установить пакеты вручную, скачав отсюда: . ### Из Docker образа diff --git a/docs/zh/getting_started/index.md b/docs/zh/getting_started/index.md index 20d3c8ff9b1..f51323ce7e8 100644 --- a/docs/zh/getting_started/index.md +++ b/docs/zh/getting_started/index.md @@ -43,6 +43,32 @@ ClickHouse包含访问控制配置,它们位于`users.xml`文件中(与'config 默认情况下,允许从任何地方使用默认的‘default’用户无密码的访问ClickHouse。参考‘user/default/networks’。 有关更多信息,请参考"Configuration files"部分。 +###来自RPM包 + +Yandex ClickHouse团队建议使用官方预编译的`rpm`软件包,用于CentOS,RedHat和所有其他基于rpm的Linux发行版。 + +首先,您需要添加官方存储库: + +```bash +sudo yum install yum-utils +sudo rpm --import https://repo.yandex.ru/clickhouse/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.yandex.ru/clickhouse/rpm/stable/x86_64 +``` + +如果您想使用最新版本,请将`stable`替换为`testing`(建议您在测试环境中使用)。 + +然后运行这些命令以实际安装包: + +```bash +sudo yum install clickhouse-server clickhouse-client +``` + +您也可以从此处手动下载和安装软件包:。 + +###来自Docker + +要在Docker中运行ClickHouse,请遵循[Docker Hub](https://hub.docker.com/r/yandex/clickhouse-server/)上的指南。那些图像使用官方的`deb`包。 + ### 使用源码安装 具体编译方式可以参考build.md。 @@ -67,14 +93,6 @@ Server: dbms/programs/clickhouse-server 日志的路径可以在server config (src/dbms/programs/server/config.xml)中配置。 -### 其他的安装方法 - -Docker image: - -CentOS或RHEL安装包: - -Gentoo:`emerge clickhouse` - ## 启动 可以运行如下命令在后台启动服务: From 182708e4b1ba46c42844bba5ee308e424b555698 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Aug 2019 20:58:31 +0300 Subject: [PATCH 281/509] Fixed typo --- ....referece => 00993_system_parts_race_condition_drop.reference} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{00993_system_parts_race_condition_drop.referece => 00993_system_parts_race_condition_drop.reference} (100%) diff --git a/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.referece b/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.referece rename to dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.reference From 4f1c9ee197ba94d39565eddd23c6f8370592972e Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 20 Aug 2019 21:05:01 +0300 Subject: [PATCH 282/509] CLICKHOUSE-3885 copier: use partition_key in check query --- dbms/programs/copier/ClusterCopier.cpp | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 435d06da854..7ac03dfdd3b 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1932,15 +1932,13 @@ protected: TaskTable & task_table = task_shard.task_table; - String query; - { - WriteBufferFromOwnString wb; - wb << "SELECT 1" - << " FROM "<< getQuotedTable(task_shard.table_read_shard) - << " WHERE " << queryToString(task_table.engine_push_partition_key_ast) << " = " << partition_quoted_name - << " LIMIT 1"; - query = wb.str(); - } + std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + + query += " LIMIT 1"; LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " << partition_quoted_name << " existence, executing query: " << query); From d8683a33c86ad3a189fc15e61a91706746b98c11 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Aug 2019 21:05:05 +0300 Subject: [PATCH 283/509] Lowered test scale because it is too slow in debug build --- .../0_stateless/00899_long_attach_memory_limit.reference | 4 ++-- .../queries/0_stateless/00899_long_attach_memory_limit.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.reference b/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.reference index 4e65c4e0d23..e3038e03530 100644 --- a/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.reference +++ b/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.reference @@ -1,2 +1,2 @@ -10000000 -10000000 +5000000 +5000000 diff --git a/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.sql b/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.sql index fc355bda462..0c8b8874293 100644 --- a/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.sql +++ b/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.sql @@ -1,9 +1,9 @@ DROP TABLE IF EXISTS index_memory; CREATE TABLE index_memory (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity = 1; -INSERT INTO index_memory SELECT * FROM system.numbers LIMIT 10000000; +INSERT INTO index_memory SELECT * FROM system.numbers LIMIT 5000000; SELECT count() FROM index_memory; DETACH TABLE index_memory; -SET max_memory_usage = 79000000; +SET max_memory_usage = 39000000; ATTACH TABLE index_memory; SELECT count() FROM index_memory; DROP TABLE index_memory; From 074853ac0e4fbd2511b3392097ea8ed97c58d859 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Aug 2019 21:11:39 +0300 Subject: [PATCH 284/509] Fixed flappy test --- .../queries/0_stateless/00704_drop_truncate_memory_table.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh b/dbms/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh index 9081580ddfc..170259f0e24 100755 --- a/dbms/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh +++ b/dbms/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh @@ -20,7 +20,7 @@ INSERT INTO memory SELECT * FROM numbers(1000);" ${CLICKHOUSE_CLIENT} --multiquery --query=" SET max_threads = 1; -SELECT count() FROM memory WHERE NOT ignore(sleep(0.0001));" 2>&1 | grep -c -P '^1000$|Table .+? doesn.t exist' & +SELECT count() FROM memory WHERE NOT ignore(sleep(0.0001));" 2>&1 | grep -c -P '^1000$|^0$|Table .+? doesn.t exist' & sleep 0.05; From 295aafb13fec3077a14af429f6e07ae7f8a18d2c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Aug 2019 21:18:37 +0300 Subject: [PATCH 285/509] Parallel ALTER modify: don't use ThreadPool if the number of threads is one --- dbms/src/Storages/StorageMergeTree.cpp | 23 ++++++++++++++++------- 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 11d378ecfa0..4003a51e301 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -206,20 +206,29 @@ std::vector StorageMergeTree::prepar const Settings & settings_ = context.getSettingsRef(); size_t thread_pool_size = std::min(parts.size(), settings_.max_alter_threads); - ThreadPool thread_pool(thread_pool_size); + + std::optional thread_pool; + + if (thread_pool_size > 1) + thread_pool.emplace(thread_pool_size); for (const auto & part : parts) { transactions.push_back(std::make_unique(part)); - thread_pool.schedule( - [this, & transaction = transactions.back(), & columns_for_parts, & new_indices = new_indices.indices] - { - this->alterDataPart(columns_for_parts, new_indices, false, transaction); - }); + auto job = [this, & transaction = transactions.back(), & columns_for_parts, & new_indices = new_indices.indices] + { + this->alterDataPart(columns_for_parts, new_indices, false, transaction); + }; + + if (thread_pool) + thread_pool->schedule(job); + else + job(); } - thread_pool.wait(); + if (thread_pool) + thread_pool->wait(); auto erase_pos = std::remove_if(transactions.begin(), transactions.end(), [](const MergeTreeData::AlterDataPartTransactionPtr & transaction) From 3fa67f4b7274cfd33bbadf89aa7c6111f949d9f5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Aug 2019 21:55:05 +0300 Subject: [PATCH 286/509] Checkpoint --- ...ition.sh => 00991_system_parts_race_condition.sh.disabled} | 4 ++++ 1 file changed, 4 insertions(+) rename dbms/tests/queries/0_stateless/{00991_system_parts_race_condition.sh => 00991_system_parts_race_condition.sh.disabled} (91%) diff --git a/dbms/tests/queries/0_stateless/00991_system_parts_race_condition.sh b/dbms/tests/queries/0_stateless/00991_system_parts_race_condition.sh.disabled similarity index 91% rename from dbms/tests/queries/0_stateless/00991_system_parts_race_condition.sh rename to dbms/tests/queries/0_stateless/00991_system_parts_race_condition.sh.disabled index 28dbcbdd938..f8cd9fd7b36 100755 --- a/dbms/tests/queries/0_stateless/00991_system_parts_race_condition.sh +++ b/dbms/tests/queries/0_stateless/00991_system_parts_race_condition.sh.disabled @@ -1,5 +1,9 @@ #!/usr/bin/env bash +# This test is disabled because it triggers internal assert in Thread Sanitizer. +# Thread Sanitizer does not support for more than 64 mutexes to be locked in a single thread. +# https://github.com/google/sanitizers/issues/950 + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh From db55780d21948c2c262e726310de9d6e8553f7aa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Aug 2019 22:53:03 +0300 Subject: [PATCH 287/509] Added a test --- .../0_stateless/00994_table_function_numbers_mt.reference | 3 +++ .../queries/0_stateless/00994_table_function_numbers_mt.sql | 3 +++ 2 files changed, 6 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.reference create mode 100644 dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.sql diff --git a/dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.reference b/dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.reference new file mode 100644 index 00000000000..062dd5ddd57 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.reference @@ -0,0 +1,3 @@ +0 9999999 49999995000000 +10000000 14999999 62499997500000 +10000000 14999999 62499997500000 diff --git a/dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.sql b/dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.sql new file mode 100644 index 00000000000..f2bbbb438dd --- /dev/null +++ b/dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.sql @@ -0,0 +1,3 @@ +SELECT min(number), max(number), sum(number) FROM numbers_mt(10000000); +SELECT min(number), max(number), sum(number) FROM numbers(10000000, 5000000); +SELECT min(number), max(number), sum(number) FROM numbers_mt(10000000, 5000000); From c1de51dc5bcb44024953efe7b6e78b1f424dfca2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Aug 2019 22:53:27 +0300 Subject: [PATCH 288/509] Better code --- .../Storages/System/StorageSystemNumbers.cpp | 23 +++++++++++-------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemNumbers.cpp b/dbms/src/Storages/System/StorageSystemNumbers.cpp index 78e25e0c375..2afe2a7c018 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.cpp +++ b/dbms/src/Storages/System/StorageSystemNumbers.cpp @@ -5,9 +5,13 @@ #include #include + namespace DB { +namespace +{ + class NumbersBlockInputStream : public IBlockInputStream { public: @@ -43,19 +47,19 @@ private: }; -struct NumbersMtState +struct NumbersMultiThreadedState { std::atomic counter; - explicit NumbersMtState(UInt64 offset) : counter(offset) {} + explicit NumbersMultiThreadedState(UInt64 offset) : counter(offset) {} }; -using NumbersMtStatePtr = std::shared_ptr; +using NumbersMultiThreadedStatePtr = std::shared_ptr; class NumbersMultiThreadedBlockInputStream : public IBlockInputStream { public: - NumbersMultiThreadedBlockInputStream(NumbersMtStatePtr state_, UInt64 block_size_, UInt64 max_counter_) - : state(std::move(state_)), counter(state->counter), block_size(block_size_), max_counter(max_counter_) {} + NumbersMultiThreadedBlockInputStream(NumbersMultiThreadedStatePtr state_, UInt64 block_size_, UInt64 max_counter_) + : state(std::move(state_)), block_size(block_size_), max_counter(max_counter_) {} String getName() const override { return "NumbersMt"; } @@ -70,7 +74,7 @@ protected: if (block_size == 0) return {}; - UInt64 curr = counter.fetch_add(block_size, std::memory_order_acquire); + UInt64 curr = state->counter.fetch_add(block_size, std::memory_order_acquire); if (curr >= max_counter) return {}; @@ -90,13 +94,14 @@ protected: } private: - NumbersMtStatePtr state; - std::atomic & counter; + NumbersMultiThreadedStatePtr state; UInt64 block_size; UInt64 max_counter; }; +} + StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multithreaded_, std::optional limit_, UInt64 offset_, bool even_distribution_) : name(name_), multithreaded(multithreaded_), even_distribution(even_distribution_), limit(limit_), offset(offset_) @@ -127,7 +132,7 @@ BlockInputStreams StorageSystemNumbers::read( if (num_streams > 1 && !even_distribution && *limit) { - auto state = std::make_shared(offset); + auto state = std::make_shared(offset); UInt64 max_counter = offset + *limit; for (size_t i = 0; i < num_streams; ++i) From dd739960737efc0b48aa367f377e25e95b914684 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 01:52:57 +0300 Subject: [PATCH 289/509] Added instruction for "list_backports" script --- utils/list_backports.sh | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/utils/list_backports.sh b/utils/list_backports.sh index 552b560d5c6..6eba3410c95 100755 --- a/utils/list_backports.sh +++ b/utils/list_backports.sh @@ -1,5 +1,21 @@ #!/bin/sh +# sudo apt install python3-pip +# sudo pip3 install gitpython termcolor + +# Go to GitHub. +# In top-right corner, click to your profile icon. +# Choose "Settings". +# Choose "Developer settings". +# Choose "Personal access tokens". +# Choose "Generate new token". + +# Don't check any checkboxes. + +# Run as: +# ./list_backports.sh --token your-token + + set -e SCRIPTPATH=$(readlink -f "$0") SCRIPTDIR=$(dirname "$SCRIPTPATH") From 1e6437e4ee6c171c7405c7f2b5a7b56175cf1a5a Mon Sep 17 00:00:00 2001 From: alex-zaitsev Date: Wed, 21 Aug 2019 03:35:41 +0300 Subject: [PATCH 290/509] WIP: Fix MacOS build (#6429) * Fix broken MacOS build * Fix MacOS build * Update ReadBufferFromFile.cpp * Update WriteBufferFromFile.cpp * Update Elf.h * Update Elf.h --- cmake/find_parquet.cmake | 2 +- dbms/src/Common/Elf.h | 31 +++++++++++++++++++++++++++++ dbms/src/IO/ReadBufferFromFile.cpp | 2 +- dbms/src/IO/WriteBufferFromFile.cpp | 2 +- 4 files changed, 34 insertions(+), 3 deletions(-) diff --git a/cmake/find_parquet.cmake b/cmake/find_parquet.cmake index 39af93955f4..63f589a9ea5 100644 --- a/cmake/find_parquet.cmake +++ b/cmake/find_parquet.cmake @@ -2,7 +2,7 @@ option (ENABLE_PARQUET "Enable parquet" ON) if (ENABLE_PARQUET) -if (NOT OS_FREEBSD) # Freebsd: ../contrib/arrow/cpp/src/arrow/util/bit-util.h:27:10: fatal error: endian.h: No such file or directory +if (NOT OS_FREEBSD AND NOT APPLE) # Freebsd: ../contrib/arrow/cpp/src/arrow/util/bit-util.h:27:10: fatal error: endian.h: No such file or directory option(USE_INTERNAL_PARQUET_LIBRARY "Set to FALSE to use system parquet library instead of bundled" ${NOT_UNBUNDLED}) endif() diff --git a/dbms/src/Common/Elf.h b/dbms/src/Common/Elf.h index 869b869b530..9377fb44677 100644 --- a/dbms/src/Common/Elf.h +++ b/dbms/src/Common/Elf.h @@ -6,9 +6,40 @@ #include #include +#ifdef __APPLE__ + +#include + +typedef Elf64_Addr GElf_Addr; +typedef Elf64_Half GElf_Half; +typedef Elf64_Off GElf_Off; +typedef Elf64_Sword GElf_Sword; +typedef Elf64_Word GElf_Word; +typedef Elf64_Sxword GElf_Sxword; +typedef Elf64_Xword GElf_Xword; + +typedef Elf64_Ehdr GElf_Ehdr; +typedef Elf64_Phdr GElf_Phdr; +typedef Elf64_Shdr GElf_Shdr; +typedef Elf64_Dyn GElf_Dyn; +typedef Elf64_Rel GElf_Rel; +typedef Elf64_Rela GElf_Rela; +typedef Elf64_Sym GElf_Sym; + +#define DT_GNU_HASH 0x6ffffef5 /* GNU-style hash table. */ + +#define __ELF_NATIVE_CLASS __WORDSIZE +#define ElfW(type) _ElfW(Elf, __ELF_NATIVE_CLASS, type) +#define _ElfW(e, w, t) _ElfW_1(e, w, _##t) +#define _ElfW_1(e, w, t) e##w##t + +#else + #include #include +#endif + using ElfAddr = ElfW(Addr); using ElfEhdr = ElfW(Ehdr); diff --git a/dbms/src/IO/ReadBufferFromFile.cpp b/dbms/src/IO/ReadBufferFromFile.cpp index b9cd7caf155..37225c0a36e 100644 --- a/dbms/src/IO/ReadBufferFromFile.cpp +++ b/dbms/src/IO/ReadBufferFromFile.cpp @@ -47,7 +47,7 @@ ReadBufferFromFile::ReadBufferFromFile( if (o_direct) { if (fcntl(fd, F_NOCACHE, 1) == -1) - throwFromErrno("Cannot set F_NOCACHE on file " + file_name, file_name, ErrorCodes::CANNOT_OPEN_FILE); + throwFromErrnoWithPath("Cannot set F_NOCACHE on file " + file_name, file_name, ErrorCodes::CANNOT_OPEN_FILE); } #endif } diff --git a/dbms/src/IO/WriteBufferFromFile.cpp b/dbms/src/IO/WriteBufferFromFile.cpp index 5e6fd7d6fe1..70aff23a420 100644 --- a/dbms/src/IO/WriteBufferFromFile.cpp +++ b/dbms/src/IO/WriteBufferFromFile.cpp @@ -51,7 +51,7 @@ WriteBufferFromFile::WriteBufferFromFile( if (o_direct) { if (fcntl(fd, F_NOCACHE, 1) == -1) - throwFromErrno("Cannot set F_NOCACHE on file " + file_name, file_name, ErrorCodes::CANNOT_OPEN_FILE); + throwFromErrnoWithPath("Cannot set F_NOCACHE on file " + file_name, file_name, ErrorCodes::CANNOT_OPEN_FILE); } #endif } From f277d0ebbfb61850178684a44fd7535f5dbdf342 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 03:37:55 +0300 Subject: [PATCH 291/509] Style --- dbms/src/Common/Elf.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/Elf.h b/dbms/src/Common/Elf.h index 9377fb44677..c9e87c4fea9 100644 --- a/dbms/src/Common/Elf.h +++ b/dbms/src/Common/Elf.h @@ -26,7 +26,7 @@ typedef Elf64_Rel GElf_Rel; typedef Elf64_Rela GElf_Rela; typedef Elf64_Sym GElf_Sym; -#define DT_GNU_HASH 0x6ffffef5 /* GNU-style hash table. */ +#define DT_GNU_HASH 0x6ffffef5 /* GNU-style hash table. */ #define __ELF_NATIVE_CLASS __WORDSIZE #define ElfW(type) _ElfW(Elf, __ELF_NATIVE_CLASS, type) From 17e93b8f2d9672616a2e5b15148258467886a423 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 03:48:34 +0300 Subject: [PATCH 292/509] Disable ELF on Mac OS #6429 --- dbms/src/Common/Dwarf.cpp | 4 +++ dbms/src/Common/Dwarf.h | 4 +++ dbms/src/Common/Elf.cpp | 4 +++ dbms/src/Common/Elf.h | 35 +++----------------------- dbms/src/Common/StackTrace.cpp | 13 ++++++++++ dbms/src/Common/SymbolIndex.cpp | 4 +++ dbms/src/Common/SymbolIndex.h | 4 +++ dbms/src/Common/tests/symbol_index.cpp | 7 ++++++ dbms/src/Functions/addressToLine.cpp | 4 +++ dbms/src/Functions/addressToSymbol.cpp | 4 +++ 10 files changed, 52 insertions(+), 31 deletions(-) diff --git a/dbms/src/Common/Dwarf.cpp b/dbms/src/Common/Dwarf.cpp index 9d4a2f05632..b7dc6a6eaa9 100644 --- a/dbms/src/Common/Dwarf.cpp +++ b/dbms/src/Common/Dwarf.cpp @@ -1,3 +1,5 @@ +#ifdef __ELF__ + /* * Copyright 2012-present Facebook, Inc. * @@ -1031,3 +1033,5 @@ bool Dwarf::LineNumberVM::findAddress(uintptr_t target, Path & file, uint64_t & } } + +#endif diff --git a/dbms/src/Common/Dwarf.h b/dbms/src/Common/Dwarf.h index 5bc358df863..216b9bca1a9 100644 --- a/dbms/src/Common/Dwarf.h +++ b/dbms/src/Common/Dwarf.h @@ -1,5 +1,7 @@ #pragma once +#ifdef __ELF__ + /* * Copyright 2012-present Facebook, Inc. * @@ -285,3 +287,5 @@ private: }; } + +#endif diff --git a/dbms/src/Common/Elf.cpp b/dbms/src/Common/Elf.cpp index 035477d0243..594264c0099 100644 --- a/dbms/src/Common/Elf.cpp +++ b/dbms/src/Common/Elf.cpp @@ -1,3 +1,5 @@ +#ifdef __ELF__ + #include #include @@ -128,3 +130,5 @@ size_t Elf::Section::size() const } } + +#endif diff --git a/dbms/src/Common/Elf.h b/dbms/src/Common/Elf.h index c9e87c4fea9..f0576ab7d58 100644 --- a/dbms/src/Common/Elf.h +++ b/dbms/src/Common/Elf.h @@ -1,45 +1,16 @@ #pragma once +#ifdef __ELF__ + #include #include #include #include -#ifdef __APPLE__ - -#include - -typedef Elf64_Addr GElf_Addr; -typedef Elf64_Half GElf_Half; -typedef Elf64_Off GElf_Off; -typedef Elf64_Sword GElf_Sword; -typedef Elf64_Word GElf_Word; -typedef Elf64_Sxword GElf_Sxword; -typedef Elf64_Xword GElf_Xword; - -typedef Elf64_Ehdr GElf_Ehdr; -typedef Elf64_Phdr GElf_Phdr; -typedef Elf64_Shdr GElf_Shdr; -typedef Elf64_Dyn GElf_Dyn; -typedef Elf64_Rel GElf_Rel; -typedef Elf64_Rela GElf_Rela; -typedef Elf64_Sym GElf_Sym; - -#define DT_GNU_HASH 0x6ffffef5 /* GNU-style hash table. */ - -#define __ELF_NATIVE_CLASS __WORDSIZE -#define ElfW(type) _ElfW(Elf, __ELF_NATIVE_CLASS, type) -#define _ElfW(e, w, t) _ElfW_1(e, w, _##t) -#define _ElfW_1(e, w, t) e##w##t - -#else - #include #include -#endif - using ElfAddr = ElfW(Addr); using ElfEhdr = ElfW(Ehdr); @@ -92,3 +63,5 @@ private: }; } + +#endif diff --git a/dbms/src/Common/StackTrace.cpp b/dbms/src/Common/StackTrace.cpp index 9792f2fb1f1..a642ec2b73a 100644 --- a/dbms/src/Common/StackTrace.cpp +++ b/dbms/src/Common/StackTrace.cpp @@ -250,6 +250,7 @@ static void toStringEveryLineImpl(const StackTrace::Frames & frames, size_t offs if (size == 0) return callback(""); +#ifdef __ELF__ const DB::SymbolIndex & symbol_index = DB::SymbolIndex::instance(); std::unordered_map dwarfs; @@ -290,6 +291,18 @@ static void toStringEveryLineImpl(const StackTrace::Frames & frames, size_t offs callback(out.str()); out.str({}); } +#else + std::stringstream out; + + for (size_t i = offset; i < size; ++i) + { + const void * addr = frames[i]; + out << i << ". " << addr; + + callback(out.str()); + out.str({}); + } +#endif } static std::string toStringImpl(const StackTrace::Frames & frames, size_t offset, size_t size) diff --git a/dbms/src/Common/SymbolIndex.cpp b/dbms/src/Common/SymbolIndex.cpp index d4cf41342a0..05268c4b6f1 100644 --- a/dbms/src/Common/SymbolIndex.cpp +++ b/dbms/src/Common/SymbolIndex.cpp @@ -1,3 +1,5 @@ +#ifdef __ELF__ + #include #include @@ -316,3 +318,5 @@ const SymbolIndex::Object * SymbolIndex::findObject(const void * address) const } } + +#endif diff --git a/dbms/src/Common/SymbolIndex.h b/dbms/src/Common/SymbolIndex.h index 41a773f5f4a..01afe9bf7fa 100644 --- a/dbms/src/Common/SymbolIndex.h +++ b/dbms/src/Common/SymbolIndex.h @@ -1,5 +1,7 @@ #pragma once +#ifdef __ELF__ + #include #include #include @@ -53,3 +55,5 @@ private: }; } + +#endif diff --git a/dbms/src/Common/tests/symbol_index.cpp b/dbms/src/Common/tests/symbol_index.cpp index 6c0d303fe35..dde5ce185ae 100644 --- a/dbms/src/Common/tests/symbol_index.cpp +++ b/dbms/src/Common/tests/symbol_index.cpp @@ -16,6 +16,7 @@ using namespace DB; int main(int argc, char ** argv) { +#ifdef __ELF__ if (argc < 2) { std::cerr << "Usage: ./symbol_index address\n"; @@ -53,6 +54,12 @@ int main(int argc, char ** argv) std::cerr << "\n"; std::cerr << StackTrace().toString() << "\n"; +#else + (void)argc; + (void)argv; + + std::cerr << "This test does not make sense for non-ELF objects.\n"; +#endif return 0; } diff --git a/dbms/src/Functions/addressToLine.cpp b/dbms/src/Functions/addressToLine.cpp index 7f7bd609dee..498ab6e7a12 100644 --- a/dbms/src/Functions/addressToLine.cpp +++ b/dbms/src/Functions/addressToLine.cpp @@ -1,3 +1,5 @@ +#ifdef __ELF__ + #include #include #include @@ -149,3 +151,5 @@ void registerFunctionAddressToLine(FunctionFactory & factory) } } + +#endif diff --git a/dbms/src/Functions/addressToSymbol.cpp b/dbms/src/Functions/addressToSymbol.cpp index ceb641e457c..80b197c4429 100644 --- a/dbms/src/Functions/addressToSymbol.cpp +++ b/dbms/src/Functions/addressToSymbol.cpp @@ -1,3 +1,5 @@ +#ifdef __ELF__ + #include #include #include @@ -92,3 +94,5 @@ void registerFunctionAddressToSymbol(FunctionFactory & factory) } } + +#endif From a6b64d6ba0118ebd7c32bcfbf28fe09339a3bd38 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 04:00:13 +0300 Subject: [PATCH 293/509] Parquet: Fix reading boolean columns --- dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 3b7ab9e6c31..4f42597eac5 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -111,7 +111,7 @@ static void fillColumnWithStringData(std::shared_ptr & arrow_colu static void fillColumnWithBooleanData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { auto & column_data = static_cast &>(*internal_column).getData(); - column_data.resize(arrow_column->length()); + column_data.reserve(arrow_column->length()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) { @@ -120,7 +120,7 @@ static void fillColumnWithBooleanData(std::shared_ptr & arrow_col std::shared_ptr buffer = chunk.data()->buffers[1]; for (size_t bool_i = 0; bool_i != static_cast(chunk.length()); ++bool_i) - column_data[bool_i] = chunk.Value(bool_i); + column_data.emplace_back(chunk.Value(bool_i)); } } From 562bffb337ffdf577008c1fbeab402ab23e4bfa3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 04:14:25 +0300 Subject: [PATCH 294/509] Fixed nullIf function --- dbms/src/Functions/nullIf.cpp | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/dbms/src/Functions/nullIf.cpp b/dbms/src/Functions/nullIf.cpp index a68537c9c7e..86e44c58906 100644 --- a/dbms/src/Functions/nullIf.cpp +++ b/dbms/src/Functions/nullIf.cpp @@ -47,14 +47,13 @@ public: Block temp_block = block; - size_t res_pos = temp_block.columns(); - temp_block.insert({nullptr, std::make_shared(), ""}); + auto not_equals_func = FunctionFactory::instance().get("notEquals", context)->build( + {temp_block.getByPosition(arguments[0]), temp_block.getByPosition(arguments[1])}); - { - auto equals_func = FunctionFactory::instance().get("notEquals", context)->build( - {temp_block.getByPosition(arguments[0]), temp_block.getByPosition(arguments[1])}); - equals_func->execute(temp_block, {arguments[0], arguments[1]}, res_pos, input_rows_count); - } + size_t not_equals_res_pos = temp_block.columns(); + temp_block.insert({nullptr, not_equals_func->getReturnType(), ""}); + + not_equals_func->execute(temp_block, {arguments[0], arguments[1]}, not_equals_res_pos, input_rows_count); /// Argument corresponding to the NULL value. size_t null_pos = temp_block.columns(); @@ -68,15 +67,14 @@ public: temp_block.insert(null_elem); auto func_if = FunctionFactory::instance().get("if", context)->build( - {temp_block.getByPosition(res_pos), temp_block.getByPosition(null_pos), temp_block.getByPosition(arguments[0])}); - func_if->execute(temp_block, {res_pos, arguments[0], null_pos}, result, input_rows_count); + {temp_block.getByPosition(not_equals_res_pos), temp_block.getByPosition(arguments[0]), temp_block.getByPosition(null_pos)}); + func_if->execute(temp_block, {not_equals_res_pos, arguments[0], null_pos}, result, input_rows_count); block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column); } }; - void registerFunctionNullIf(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); From 449dc4757aad399a342afda206d8919760624d20 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 04:20:06 +0300 Subject: [PATCH 295/509] Fixed nullIf function --- dbms/src/Functions/nullIf.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/src/Functions/nullIf.cpp b/dbms/src/Functions/nullIf.cpp index 86e44c58906..ca4df53917f 100644 --- a/dbms/src/Functions/nullIf.cpp +++ b/dbms/src/Functions/nullIf.cpp @@ -43,17 +43,17 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { - /// nullIf(col1, col2) == if(col1 != col2, col1, NULL) + /// nullIf(col1, col2) == if(col1 = col2, NULL, 1) Block temp_block = block; - auto not_equals_func = FunctionFactory::instance().get("notEquals", context)->build( + auto equals_func = FunctionFactory::instance().get("equals", context)->build( {temp_block.getByPosition(arguments[0]), temp_block.getByPosition(arguments[1])}); size_t not_equals_res_pos = temp_block.columns(); - temp_block.insert({nullptr, not_equals_func->getReturnType(), ""}); + temp_block.insert({nullptr, equals_func->getReturnType(), ""}); - not_equals_func->execute(temp_block, {arguments[0], arguments[1]}, not_equals_res_pos, input_rows_count); + equals_func->execute(temp_block, {arguments[0], arguments[1]}, not_equals_res_pos, input_rows_count); /// Argument corresponding to the NULL value. size_t null_pos = temp_block.columns(); @@ -67,8 +67,8 @@ public: temp_block.insert(null_elem); auto func_if = FunctionFactory::instance().get("if", context)->build( - {temp_block.getByPosition(not_equals_res_pos), temp_block.getByPosition(arguments[0]), temp_block.getByPosition(null_pos)}); - func_if->execute(temp_block, {not_equals_res_pos, arguments[0], null_pos}, result, input_rows_count); + {temp_block.getByPosition(not_equals_res_pos), temp_block.getByPosition(null_pos), temp_block.getByPosition(arguments[0])}); + func_if->execute(temp_block, {not_equals_res_pos, null_pos, arguments[0]}, result, input_rows_count); block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column); } From 69be2c4658575a536e04a6542b385fcfadf6ddd5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 04:28:55 +0300 Subject: [PATCH 296/509] Fixed nullIf function --- dbms/src/Functions/nullIf.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Functions/nullIf.cpp b/dbms/src/Functions/nullIf.cpp index ca4df53917f..3726850098d 100644 --- a/dbms/src/Functions/nullIf.cpp +++ b/dbms/src/Functions/nullIf.cpp @@ -50,10 +50,10 @@ public: auto equals_func = FunctionFactory::instance().get("equals", context)->build( {temp_block.getByPosition(arguments[0]), temp_block.getByPosition(arguments[1])}); - size_t not_equals_res_pos = temp_block.columns(); + size_t equals_res_pos = temp_block.columns(); temp_block.insert({nullptr, equals_func->getReturnType(), ""}); - equals_func->execute(temp_block, {arguments[0], arguments[1]}, not_equals_res_pos, input_rows_count); + equals_func->execute(temp_block, {arguments[0], arguments[1]}, equals_res_pos, input_rows_count); /// Argument corresponding to the NULL value. size_t null_pos = temp_block.columns(); @@ -67,8 +67,8 @@ public: temp_block.insert(null_elem); auto func_if = FunctionFactory::instance().get("if", context)->build( - {temp_block.getByPosition(not_equals_res_pos), temp_block.getByPosition(null_pos), temp_block.getByPosition(arguments[0])}); - func_if->execute(temp_block, {not_equals_res_pos, null_pos, arguments[0]}, result, input_rows_count); + {temp_block.getByPosition(equals_res_pos), temp_block.getByPosition(null_pos), temp_block.getByPosition(arguments[0])}); + func_if->execute(temp_block, {equals_res_pos, null_pos, arguments[0]}, result, input_rows_count); block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column); } From b768bef4e38f9f9e3c99cf72c67ca57ebf1bd394 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 04:43:19 +0300 Subject: [PATCH 297/509] Fixed nullIf function --- dbms/src/Functions/nullIf.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/nullIf.cpp b/dbms/src/Functions/nullIf.cpp index 3726850098d..91fb311f444 100644 --- a/dbms/src/Functions/nullIf.cpp +++ b/dbms/src/Functions/nullIf.cpp @@ -70,7 +70,7 @@ public: {temp_block.getByPosition(equals_res_pos), temp_block.getByPosition(null_pos), temp_block.getByPosition(arguments[0])}); func_if->execute(temp_block, {equals_res_pos, null_pos, arguments[0]}, result, input_rows_count); - block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column); + block.getByPosition(result).column = makeNullable(std::move(temp_block.getByPosition(result).column)); } }; From 89665208011c62430120c8f9247cb8cd7c1c10c7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 05:28:04 +0300 Subject: [PATCH 298/509] Added "assert_cast" function as a replacement for static_cast for columns --- dbms/programs/obfuscator/Obfuscator.cpp | 23 +++---- .../odbc-bridge/ODBCBlockInputStream.cpp | 31 +++++----- .../AggregateFunctionArray.h | 7 ++- .../AggregateFunctionBitwise.h | 5 +- .../AggregateFunctionBoundingRatio.h | 3 +- .../AggregateFunctionCount.h | 11 ++-- .../AggregateFunctionEntropy.h | 5 +- .../AggregateFunctionForEach.h | 9 +-- .../AggregateFunctionGroupArray.h | 15 ++--- .../AggregateFunctionGroupArrayInsertAt.h | 3 +- .../AggregateFunctionGroupArrayMoving.h | 3 +- .../AggregateFunctionGroupBitmap.h | 6 +- .../AggregateFunctionGroupUniqArray.h | 9 +-- .../AggregateFunctionHistogram.h | 13 ++-- .../AggregateFunctions/AggregateFunctionIf.h | 3 +- .../AggregateFunctionMLMethod.cpp | 5 +- .../AggregateFunctionMaxIntersections.h | 9 +-- .../AggregateFunctionMerge.h | 3 +- .../AggregateFunctionMinMaxAny.h | 25 ++++---- .../AggregateFunctionNull.h | 7 ++- .../AggregateFunctionQuantile.h | 7 ++- .../AggregateFunctionResample.h | 5 +- .../AggregateFunctionRetention.h | 7 ++- .../AggregateFunctionSequenceMatch.h | 9 +-- .../AggregateFunctionSimpleLinearRegression.h | 11 ++-- .../AggregateFunctionState.h | 3 +- .../AggregateFunctionStatistics.h | 14 +++-- .../AggregateFunctionSumMap.h | 13 ++-- .../AggregateFunctionTimeSeriesGroupSum.h | 13 ++-- .../AggregateFunctionTopK.h | 12 ++-- .../AggregateFunctionUniq.h | 9 +-- .../AggregateFunctionUniqCombined.h | 7 ++- .../AggregateFunctionUniqUpTo.h | 9 +-- .../AggregateFunctionWindowFunnel.h | 7 ++- .../src/AggregateFunctions/UniqVariadicHash.h | 5 +- dbms/src/Columns/ColumnAggregateFunction.cpp | 7 ++- dbms/src/Columns/ColumnArray.cpp | 40 ++++++------ dbms/src/Columns/ColumnArray.h | 5 +- dbms/src/Columns/ColumnConst.h | 3 +- dbms/src/Columns/ColumnDecimal.cpp | 3 +- dbms/src/Columns/ColumnFixedString.cpp | 7 ++- dbms/src/Columns/ColumnFixedString.h | 3 +- dbms/src/Columns/ColumnLowCardinality.cpp | 4 +- dbms/src/Columns/ColumnLowCardinality.h | 9 +-- dbms/src/Columns/ColumnNullable.cpp | 9 +-- dbms/src/Columns/ColumnNullable.h | 7 ++- dbms/src/Columns/ColumnString.cpp | 5 +- dbms/src/Columns/ColumnString.h | 5 +- dbms/src/Columns/ColumnTuple.cpp | 7 ++- dbms/src/Columns/ColumnUnique.h | 7 ++- dbms/src/Columns/ColumnVector.cpp | 3 +- dbms/src/Columns/FilterDescription.cpp | 3 +- dbms/src/Columns/ReverseIndex.h | 1 + dbms/src/Columns/getLeastSuperColumn.cpp | 5 +- dbms/src/Common/ColumnsHashing.h | 13 ++-- dbms/src/Common/ColumnsHashingImpl.h | 4 +- dbms/src/Common/assert_cast.h | 44 +++++++++++++ dbms/src/Core/Block.cpp | 6 +- dbms/src/Core/SortCursor.h | 3 +- .../CollapsingSortedBlockInputStream.cpp | 3 +- .../ConvertingBlockInputStream.cpp | 3 +- .../SummingSortedBlockInputStream.cpp | 5 +- .../TotalsHavingBlockInputStream.cpp | 3 +- ...sionedCollapsingSortedBlockInputStream.cpp | 3 +- .../DataTypes/DataTypeAggregateFunction.cpp | 13 ++-- dbms/src/DataTypes/DataTypeArray.cpp | 17 ++--- dbms/src/DataTypes/DataTypeDate.cpp | 16 ++--- dbms/src/DataTypes/DataTypeDateTime.cpp | 15 ++--- dbms/src/DataTypes/DataTypeEnum.cpp | 33 +++++----- dbms/src/DataTypes/DataTypeFixedString.cpp | 21 ++++--- dbms/src/DataTypes/DataTypeLowCardinality.cpp | 11 ++-- .../DataTypeLowCardinalityHelpers.cpp | 5 +- dbms/src/DataTypes/DataTypeNullable.cpp | 25 ++++---- dbms/src/DataTypes/DataTypeNumberBase.cpp | 17 ++--- dbms/src/DataTypes/DataTypeString.cpp | 23 +++---- dbms/src/DataTypes/DataTypeTuple.cpp | 5 +- dbms/src/DataTypes/DataTypeUUID.cpp | 15 ++--- dbms/src/DataTypes/DataTypesDecimal.cpp | 16 ++--- dbms/src/DataTypes/NestedUtils.cpp | 7 ++- .../Dictionaries/MongoDBBlockInputStream.cpp | 57 ++++++++--------- dbms/src/Dictionaries/TrieDictionary.cpp | 6 +- dbms/src/Formats/MySQLBlockInputStream.cpp | 47 +++++++------- dbms/src/Functions/FunctionBinaryArithmetic.h | 7 ++- dbms/src/Functions/FunctionHelpers.cpp | 5 +- dbms/src/Functions/FunctionHelpers.h | 3 +- dbms/src/Functions/FunctionsBitmap.h | 3 +- dbms/src/Functions/FunctionsComparison.h | 11 ++-- dbms/src/Functions/FunctionsConversion.h | 3 +- .../Functions/FunctionsExternalDictionaries.h | 7 ++- .../src/Functions/FunctionsExternalModels.cpp | 4 +- dbms/src/Functions/FunctionsJSON.h | 35 ++++++----- dbms/src/Functions/FunctionsRound.h | 3 +- dbms/src/Functions/FunctionsStringArray.h | 5 +- dbms/src/Functions/IFunction.cpp | 9 +-- .../Functions/appendTrailingCharIfAbsent.cpp | 3 +- .../src/Functions/array/FunctionArrayMapped.h | 3 +- dbms/src/Functions/array/arrayDistinct.cpp | 3 +- dbms/src/Functions/array/arrayElement.cpp | 3 +- .../Functions/array/arrayEnumerateRanked.cpp | 3 +- dbms/src/Functions/array/arrayIndex.h | 17 ++--- dbms/src/Functions/array/arrayIntersect.cpp | 7 ++- dbms/src/Functions/array/arrayReverse.cpp | 3 +- .../Functions/array/emptyArrayToSingle.cpp | 7 ++- dbms/src/Functions/greatCircleDistance.cpp | 13 ++-- dbms/src/Functions/if.cpp | 27 ++++---- dbms/src/Functions/isNotNull.cpp | 3 +- dbms/src/Functions/multiIf.cpp | 9 +-- dbms/src/Functions/pointInEllipses.cpp | 9 +-- dbms/src/Functions/runningDifference.h | 5 +- dbms/src/Functions/sleep.h | 3 +- dbms/src/Functions/tupleElement.cpp | 3 +- dbms/src/Functions/visibleWidth.cpp | 3 +- dbms/src/Interpreters/AggregationCommon.h | 29 ++++----- dbms/src/Interpreters/Aggregator.cpp | 5 +- dbms/src/Interpreters/Aggregator.h | 7 ++- dbms/src/Interpreters/ExpressionJIT.cpp | 5 +- dbms/src/Interpreters/Join.cpp | 9 +-- dbms/src/Interpreters/NullableUtils.cpp | 5 +- dbms/src/Interpreters/SetVariants.cpp | 4 +- dbms/src/Interpreters/SetVariants.h | 4 +- dbms/src/Interpreters/createBlockSelector.cpp | 3 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 17 ++--- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 62 +++++++++---------- dbms/src/Storages/StorageJoin.cpp | 5 +- 124 files changed, 720 insertions(+), 549 deletions(-) create mode 100644 dbms/src/Common/assert_cast.h diff --git a/dbms/programs/obfuscator/Obfuscator.cpp b/dbms/programs/obfuscator/Obfuscator.cpp index a96c10072dc..5149566465c 100644 --- a/dbms/programs/obfuscator/Obfuscator.cpp +++ b/dbms/programs/obfuscator/Obfuscator.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -263,11 +264,11 @@ public: ColumnPtr generate(const IColumn & column) override { - const auto & src_data = static_cast &>(column).getData(); + const auto & src_data = assert_cast &>(column).getData(); size_t size = src_data.size(); auto res_column = ColumnVector::create(size); - auto & res_data = static_cast &>(*res_column).getData(); + auto & res_data = assert_cast &>(*res_column).getData(); for (size_t i = 0; i < size; ++i) { @@ -355,7 +356,7 @@ public: ColumnPtr generate(const IColumn & column) override { - const ColumnFixedString & column_fixed_string = static_cast(column); + const ColumnFixedString & column_fixed_string = assert_cast(column); const size_t string_size = column_fixed_string.getN(); const auto & src_data = column_fixed_string.getChars(); @@ -392,11 +393,11 @@ public: ColumnPtr generate(const IColumn & column) override { - const auto & src_data = static_cast &>(column).getData(); + const auto & src_data = assert_cast &>(column).getData(); size_t size = src_data.size(); auto res_column = ColumnVector::create(size); - auto & res_data = static_cast &>(*res_column).getData(); + auto & res_data = assert_cast &>(*res_column).getData(); for (size_t i = 0; i < size; ++i) { @@ -749,7 +750,7 @@ public: void train(const IColumn & column) override { - const ColumnString & column_string = static_cast(column); + const ColumnString & column_string = assert_cast(column); size_t size = column_string.size(); for (size_t i = 0; i < size; ++i) @@ -766,7 +767,7 @@ public: ColumnPtr generate(const IColumn & column) override { - const ColumnString & column_string = static_cast(column); + const ColumnString & column_string = assert_cast(column); size_t size = column_string.size(); auto res_column = ColumnString::create(); @@ -801,7 +802,7 @@ public: void train(const IColumn & column) override { - const ColumnArray & column_array = static_cast(column); + const ColumnArray & column_array = assert_cast(column); const IColumn & nested_column = column_array.getData(); nested_model->train(nested_column); @@ -814,7 +815,7 @@ public: ColumnPtr generate(const IColumn & column) override { - const ColumnArray & column_array = static_cast(column); + const ColumnArray & column_array = assert_cast(column); const IColumn & nested_column = column_array.getData(); ColumnPtr new_nested_column = nested_model->generate(nested_column); @@ -834,7 +835,7 @@ public: void train(const IColumn & column) override { - const ColumnNullable & column_nullable = static_cast(column); + const ColumnNullable & column_nullable = assert_cast(column); const IColumn & nested_column = column_nullable.getNestedColumn(); nested_model->train(nested_column); @@ -847,7 +848,7 @@ public: ColumnPtr generate(const IColumn & column) override { - const ColumnNullable & column_nullable = static_cast(column); + const ColumnNullable & column_nullable = assert_cast(column); const IColumn & nested_column = column_nullable.getNestedColumn(); ColumnPtr new_nested_column = nested_model->generate(nested_column); diff --git a/dbms/programs/odbc-bridge/ODBCBlockInputStream.cpp b/dbms/programs/odbc-bridge/ODBCBlockInputStream.cpp index 70aaba3f137..8aa93c43c2b 100644 --- a/dbms/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/dbms/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -44,46 +45,46 @@ namespace switch (type) { case ValueType::vtUInt8: - static_cast(column).insertValue(value.convert()); + assert_cast(column).insertValue(value.convert()); break; case ValueType::vtUInt16: - static_cast(column).insertValue(value.convert()); + assert_cast(column).insertValue(value.convert()); break; case ValueType::vtUInt32: - static_cast(column).insertValue(value.convert()); + assert_cast(column).insertValue(value.convert()); break; case ValueType::vtUInt64: - static_cast(column).insertValue(value.convert()); + assert_cast(column).insertValue(value.convert()); break; case ValueType::vtInt8: - static_cast(column).insertValue(value.convert()); + assert_cast(column).insertValue(value.convert()); break; case ValueType::vtInt16: - static_cast(column).insertValue(value.convert()); + assert_cast(column).insertValue(value.convert()); break; case ValueType::vtInt32: - static_cast(column).insertValue(value.convert()); + assert_cast(column).insertValue(value.convert()); break; case ValueType::vtInt64: - static_cast(column).insertValue(value.convert()); + assert_cast(column).insertValue(value.convert()); break; case ValueType::vtFloat32: - static_cast(column).insertValue(value.convert()); + assert_cast(column).insertValue(value.convert()); break; case ValueType::vtFloat64: - static_cast(column).insertValue(value.convert()); + assert_cast(column).insertValue(value.convert()); break; case ValueType::vtString: - static_cast(column).insert(value.convert()); + assert_cast(column).insert(value.convert()); break; case ValueType::vtDate: - static_cast(column).insertValue(UInt16{LocalDate{value.convert()}.getDayNum()}); + assert_cast(column).insertValue(UInt16{LocalDate{value.convert()}.getDayNum()}); break; case ValueType::vtDateTime: - static_cast(column).insertValue(time_t{LocalDateTime{value.convert()}}); + assert_cast(column).insertValue(time_t{LocalDateTime{value.convert()}}); break; case ValueType::vtUUID: - static_cast(column).insert(parse(value.convert())); + assert_cast(column).insert(parse(value.convert())); break; } } @@ -114,7 +115,7 @@ Block ODBCBlockInputStream::readImpl() { if (description.types[idx].second) { - ColumnNullable & column_nullable = static_cast(*columns[idx]); + ColumnNullable & column_nullable = assert_cast(*columns[idx]); insertValue(column_nullable.getNestedColumn(), description.types[idx].first, value); column_nullable.getNullMapData().emplace_back(0); } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionArray.h b/dbms/src/AggregateFunctions/AggregateFunctionArray.h index 08fa7c13bc3..2a4780b6c9d 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionArray.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -81,9 +82,9 @@ public: const IColumn * nested[num_arguments]; for (size_t i = 0; i < num_arguments; ++i) - nested[i] = &static_cast(*columns[i]).getData(); + nested[i] = &assert_cast(*columns[i]).getData(); - const ColumnArray & first_array_column = static_cast(*columns[0]); + const ColumnArray & first_array_column = assert_cast(*columns[0]); const IColumn::Offsets & offsets = first_array_column.getOffsets(); size_t begin = offsets[row_num - 1]; @@ -92,7 +93,7 @@ public: /// Sanity check. NOTE We can implement specialization for a case with single argument, if the check will hurt performance. for (size_t i = 1; i < num_arguments; ++i) { - const ColumnArray & ith_column = static_cast(*columns[i]); + const ColumnArray & ith_column = assert_cast(*columns[i]); const IColumn::Offsets & ith_offsets = ith_column.getOffsets(); if (ith_offsets[row_num] != end || (row_num != 0 && ith_offsets[row_num - 1] != begin)) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionBitwise.h b/dbms/src/AggregateFunctions/AggregateFunctionBitwise.h index 2788fdccd51..23bd5e30d3f 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionBitwise.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionBitwise.h @@ -5,6 +5,7 @@ #include #include +#include #include @@ -55,7 +56,7 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - this->data(place).update(static_cast &>(*columns[0]).getData()[row_num]); + this->data(place).update(assert_cast &>(*columns[0]).getData()[row_num]); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override @@ -75,7 +76,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast &>(to).getData().push_back(this->data(place).value); + assert_cast &>(to).getData().push_back(this->data(place).value); } const char * getHeaderFilePath() const override { return __FILE__; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h index 5966993dc65..def1c2eafe4 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -151,7 +152,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(getBoundingRatio(data(place))); + assert_cast(to).getData().push_back(getBoundingRatio(data(place))); } const char * getHeaderFilePath() const override diff --git a/dbms/src/AggregateFunctions/AggregateFunctionCount.h b/dbms/src/AggregateFunctions/AggregateFunctionCount.h index c1b96a4fe4a..6cd9c0c7487 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionCount.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionCount.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -59,7 +60,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(data(place).count); + assert_cast(to).getData().push_back(data(place).count); } const char * getHeaderFilePath() const override { return __FILE__; } @@ -86,7 +87,7 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - data(place).count += !static_cast(*columns[0]).isNullAt(row_num); + data(place).count += !assert_cast(*columns[0]).isNullAt(row_num); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override @@ -106,7 +107,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(data(place).count); + assert_cast(to).getData().push_back(data(place).count); } const char * getHeaderFilePath() const override { return __FILE__; } @@ -143,7 +144,7 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { for (size_t i = 0; i < number_of_arguments; ++i) - if (is_nullable[i] && static_cast(*columns[i]).isNullAt(row_num)) + if (is_nullable[i] && assert_cast(*columns[i]).isNullAt(row_num)) return; ++data(place).count; @@ -166,7 +167,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(data(place).count); + assert_cast(to).getData().push_back(data(place).count); } const char * getHeaderFilePath() const override { return __FILE__; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionEntropy.h b/dbms/src/AggregateFunctions/AggregateFunctionEntropy.h index 97433a8de02..23f08a081db 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionEntropy.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionEntropy.h @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -115,7 +116,7 @@ public: if constexpr (!std::is_same_v) { /// Here we manage only with numerical types - const auto & column = static_cast &>(*columns[0]); + const auto & column = assert_cast &>(*columns[0]); this->data(place).add(column.getData()[row_num]); } else @@ -141,7 +142,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - auto & column = static_cast &>(to); + auto & column = assert_cast &>(to); column.getData().push_back(this->data(place).get()); } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionForEach.h b/dbms/src/AggregateFunctions/AggregateFunctionForEach.h index 5b095d1825f..dc89976dbd9 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionForEach.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionForEach.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -149,9 +150,9 @@ public: const IColumn * nested[num_arguments]; for (size_t i = 0; i < num_arguments; ++i) - nested[i] = &static_cast(*columns[i]).getData(); + nested[i] = &assert_cast(*columns[i]).getData(); - const ColumnArray & first_array_column = static_cast(*columns[0]); + const ColumnArray & first_array_column = assert_cast(*columns[0]); const IColumn::Offsets & offsets = first_array_column.getOffsets(); size_t begin = offsets[row_num - 1]; @@ -160,7 +161,7 @@ public: /// Sanity check. NOTE We can implement specialization for a case with single argument, if the check will hurt performance. for (size_t i = 1; i < num_arguments; ++i) { - const ColumnArray & ith_column = static_cast(*columns[i]); + const ColumnArray & ith_column = assert_cast(*columns[i]); const IColumn::Offsets & ith_offsets = ith_column.getOffsets(); if (ith_offsets[row_num] != end || (row_num != 0 && ith_offsets[row_num - 1] != begin)) @@ -228,7 +229,7 @@ public: { const AggregateFunctionForEachData & state = data(place); - ColumnArray & arr_to = static_cast(to); + ColumnArray & arr_to = assert_cast(to); ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); IColumn & elems_to = arr_to.getData(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h index f578cee9d00..f5d4410b186 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -12,6 +12,7 @@ #include #include +#include #include @@ -68,7 +69,7 @@ public: if (limit_num_elems && this->data(place).value.size() >= max_elems) return; - this->data(place).value.push_back(static_cast &>(*columns[0]).getData()[row_num], arena); + this->data(place).value.push_back(assert_cast &>(*columns[0]).getData()[row_num], arena); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override @@ -119,14 +120,14 @@ public: const auto & value = this->data(place).value; size_t size = value.size(); - ColumnArray & arr_to = static_cast(to); + ColumnArray & arr_to = assert_cast(to); ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); offsets_to.push_back(offsets_to.back() + size); if (size) { - typename ColumnVector::Container & data_to = static_cast &>(arr_to.getData()).getData(); + typename ColumnVector::Container & data_to = assert_cast &>(arr_to.getData()).getData(); data_to.insert(this->data(place).value.begin(), this->data(place).value.end()); } } @@ -191,7 +192,7 @@ struct GroupArrayListNodeString : public GroupArrayListNodeBase(column).getDataAt(row_num); + StringRef string = assert_cast(column).getDataAt(row_num); Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + string.size, alignof(Node))); node->next = nullptr; @@ -203,7 +204,7 @@ struct GroupArrayListNodeString : public GroupArrayListNodeBase(column).insertData(data(), size); + assert_cast(column).insertData(data(), size); } }; @@ -374,7 +375,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - auto & column_array = static_cast(to); + auto & column_array = assert_cast(to); auto & offsets = column_array.getOffsets(); offsets.push_back(offsets.back() + data(place).elems); @@ -383,7 +384,7 @@ public: if (std::is_same_v) { - auto & string_offsets = static_cast(column_data).getOffsets(); + auto & string_offsets = assert_cast(column_data).getOffsets(); string_offsets.reserve(string_offsets.size() + data(place).elems); } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h index c7dab21a4cb..de551beab97 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h @@ -10,6 +10,7 @@ #include #include +#include #include #include @@ -180,7 +181,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - ColumnArray & to_array = static_cast(to); + ColumnArray & to_array = assert_cast(to); IColumn & to_data = to_array.getData(); ColumnArray::Offsets & to_offsets = to_array.getOffsets(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h index c2ecbbcdfb1..85c9b9afcda 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h @@ -11,6 +11,7 @@ #include #include +#include #include @@ -164,7 +165,7 @@ public: const auto & data = this->data(place); size_t size = data.value.size(); - ColumnArray & arr_to = static_cast(to); + ColumnArray & arr_to = assert_cast(to); ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); offsets_to.push_back(offsets_to.back() + size); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmap.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmap.h index ccc851efbec..3322c3baf8d 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmap.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include @@ -26,7 +26,7 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - this->data(place).rbs.add(static_cast &>(*columns[0]).getData()[row_num]); + this->data(place).rbs.add(assert_cast &>(*columns[0]).getData()[row_num]); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override @@ -46,7 +46,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast &>(to).getData().push_back(this->data(place).rbs.size()); + assert_cast &>(to).getData().push_back(this->data(place).rbs.size()); } const char * getHeaderFilePath() const override { return __FILE__; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index 029bf6efe83..4b52f1e6fd9 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -10,6 +10,7 @@ #include #include +#include #include @@ -63,7 +64,7 @@ public: { if (limit_num_elems && this->data(place).value.size() >= max_elems) return; - this->data(place).value.insert(static_cast &>(*columns[0]).getData()[row_num]); + this->data(place).value.insert(assert_cast &>(*columns[0]).getData()[row_num]); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override @@ -100,7 +101,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - ColumnArray & arr_to = static_cast(to); + ColumnArray & arr_to = assert_cast(to); ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); const typename State::Set & set = this->data(place).value; @@ -108,7 +109,7 @@ public: offsets_to.push_back(offsets_to.back() + size); - typename ColumnVector::Container & data_to = static_cast &>(arr_to.getData()).getData(); + typename ColumnVector::Container & data_to = assert_cast &>(arr_to.getData()).getData(); size_t old_size = data_to.size(); data_to.resize(old_size + size); @@ -254,7 +255,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - ColumnArray & arr_to = static_cast(to); + ColumnArray & arr_to = assert_cast(to); ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); IColumn & data_to = arr_to.getData(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h index 04aa88a806c..9d2e0ea1331 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -333,7 +334,7 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - auto val = static_cast &>(*columns[0]).getData()[row_num]; + auto val = assert_cast &>(*columns[0]).getData()[row_num]; this->data(place).add(static_cast(val), 1, max_bins); } @@ -356,13 +357,13 @@ public: { auto & data = this->data(const_cast(place)); - auto & to_array = static_cast(to); + auto & to_array = assert_cast(to); ColumnArray::Offsets & offsets_to = to_array.getOffsets(); - auto & to_tuple = static_cast(to_array.getData()); + auto & to_tuple = assert_cast(to_array.getData()); - auto & to_lower = static_cast &>(to_tuple.getColumn(0)); - auto & to_upper = static_cast &>(to_tuple.getColumn(1)); - auto & to_weights = static_cast &>(to_tuple.getColumn(2)); + auto & to_lower = assert_cast &>(to_tuple.getColumn(0)); + auto & to_upper = assert_cast &>(to_tuple.getColumn(1)); + auto & to_weights = assert_cast &>(to_tuple.getColumn(2)); data.insertResultInto(to_lower, to_upper, to_weights, max_bins); offsets_to.push_back(to_tuple.size()); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionIf.h b/dbms/src/AggregateFunctions/AggregateFunctionIf.h index 8daf9505ae6..c6d2e218ccc 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionIf.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionIf.h @@ -2,6 +2,7 @@ #include #include +#include #include @@ -75,7 +76,7 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override { - if (static_cast(*columns[num_arguments - 1]).getData()[row_num]) + if (assert_cast(*columns[num_arguments - 1]).getData()[row_num]) nested_func->add(place, columns, row_num, arena); } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMLMethod.cpp b/dbms/src/AggregateFunctions/AggregateFunctionMLMethod.cpp index 2f4962f26a2..8aa639c88ea 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMLMethod.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionMLMethod.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include "AggregateFunctionFactory.h" #include "FactoryHelpers.h" #include "Helpers.h" @@ -147,14 +148,14 @@ void LinearModelData::returnWeights(IColumn & to) const { size_t size = weights.size() + 1; - ColumnArray & arr_to = static_cast(to); + ColumnArray & arr_to = assert_cast(to); ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); size_t old_size = offsets_to.back(); offsets_to.push_back(old_size + size); typename ColumnFloat64::Container & val_to - = static_cast(arr_to.getData()).getData(); + = assert_cast(arr_to.getData()).getData(); val_to.reserve(old_size + size); for (size_t i = 0; i + 1 < size; ++i) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 8b93f15fd9d..69cd6c0f3ec 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -10,6 +10,7 @@ #include #include +#include #include @@ -88,8 +89,8 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override { - PointType left = static_cast &>(*columns[0]).getData()[row_num]; - PointType right = static_cast &>(*columns[1]).getData()[row_num]; + PointType left = assert_cast &>(*columns[0]).getData()[row_num]; + PointType right = assert_cast &>(*columns[1]).getData()[row_num]; if (!isNaN(left)) this->data(place).value.push_back(std::make_pair(left, Int64(1)), arena); @@ -152,12 +153,12 @@ public: if (kind == AggregateFunctionIntersectionsKind::Count) { - auto & result_column = static_cast(to).getData(); + auto & result_column = assert_cast(to).getData(); result_column.push_back(max_intersections); } else { - auto & result_column = static_cast &>(to).getData(); + auto & result_column = assert_cast &>(to).getData(); result_column.push_back(position_of_max_intersections); } } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMerge.h b/dbms/src/AggregateFunctions/AggregateFunctionMerge.h index c94d4d3cf3c..c34f2fc1869 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMerge.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionMerge.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -70,7 +71,7 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override { - nested_func->merge(place, static_cast(*columns[0]).getData()[row_num], arena); + nested_func->merge(place, assert_cast(*columns[0]).getData()[row_num], arena); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index 019968994b1..00869c846d4 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -38,9 +39,9 @@ public: void insertResultInto(IColumn & to) const { if (has()) - static_cast &>(to).getData().push_back(value); + assert_cast &>(to).getData().push_back(value); else - static_cast &>(to).insertDefault(); + assert_cast &>(to).insertDefault(); } void write(WriteBuffer & buf, const IDataType & /*data_type*/) const @@ -61,7 +62,7 @@ public: void change(const IColumn & column, size_t row_num, Arena *) { has_value = true; - value = static_cast &>(column).getData()[row_num]; + value = assert_cast &>(column).getData()[row_num]; } /// Assuming to.has() @@ -112,7 +113,7 @@ public: bool changeIfLess(const IColumn & column, size_t row_num, Arena * arena) { - if (!has() || static_cast &>(column).getData()[row_num] < value) + if (!has() || assert_cast &>(column).getData()[row_num] < value) { change(column, row_num, arena); return true; @@ -134,7 +135,7 @@ public: bool changeIfGreater(const IColumn & column, size_t row_num, Arena * arena) { - if (!has() || static_cast &>(column).getData()[row_num] > value) + if (!has() || assert_cast &>(column).getData()[row_num] > value) { change(column, row_num, arena); return true; @@ -161,7 +162,7 @@ public: bool isEqualTo(const IColumn & column, size_t row_num) const { - return has() && static_cast &>(column).getData()[row_num] == value; + return has() && assert_cast &>(column).getData()[row_num] == value; } }; @@ -204,9 +205,9 @@ public: void insertResultInto(IColumn & to) const { if (has()) - static_cast(to).insertDataWithTerminatingZero(getData(), size); + assert_cast(to).insertDataWithTerminatingZero(getData(), size); else - static_cast(to).insertDefault(); + assert_cast(to).insertDefault(); } void write(WriteBuffer & buf, const IDataType & /*data_type*/) const @@ -281,7 +282,7 @@ public: void change(const IColumn & column, size_t row_num, Arena * arena) { - changeImpl(static_cast(column).getDataAtWithTerminatingZero(row_num), arena); + changeImpl(assert_cast(column).getDataAtWithTerminatingZero(row_num), arena); } void change(const Self & to, Arena * arena) @@ -330,7 +331,7 @@ public: bool changeIfLess(const IColumn & column, size_t row_num, Arena * arena) { - if (!has() || static_cast(column).getDataAtWithTerminatingZero(row_num) < getStringRef()) + if (!has() || assert_cast(column).getDataAtWithTerminatingZero(row_num) < getStringRef()) { change(column, row_num, arena); return true; @@ -352,7 +353,7 @@ public: bool changeIfGreater(const IColumn & column, size_t row_num, Arena * arena) { - if (!has() || static_cast(column).getDataAtWithTerminatingZero(row_num) > getStringRef()) + if (!has() || assert_cast(column).getDataAtWithTerminatingZero(row_num) > getStringRef()) { change(column, row_num, arena); return true; @@ -379,7 +380,7 @@ public: bool isEqualTo(const IColumn & column, size_t row_num) const { - return has() && static_cast(column).getDataAtWithTerminatingZero(row_num) == getStringRef(); + return has() && assert_cast(column).getDataAtWithTerminatingZero(row_num) == getStringRef(); } }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionNull.h b/dbms/src/AggregateFunctions/AggregateFunctionNull.h index ab4b5b27844..e3128043ba6 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionNull.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionNull.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -149,7 +150,7 @@ public: { if (result_is_nullable) { - ColumnNullable & to_concrete = static_cast(to); + ColumnNullable & to_concrete = assert_cast(to); if (getFlag(place)) { nested_function->insertResultInto(nestedPlace(place), to_concrete.getNestedColumn()); @@ -194,7 +195,7 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override { - const ColumnNullable * column = static_cast(columns[0]); + const ColumnNullable * column = assert_cast(columns[0]); if (!column->isNullAt(row_num)) { this->setFlag(place); @@ -233,7 +234,7 @@ public: { if (is_nullable[i]) { - const ColumnNullable & nullable_col = static_cast(*columns[i]); + const ColumnNullable & nullable_col = assert_cast(*columns[i]); if (nullable_col.isNullAt(row_num)) { /// If at least one column has a null value in the current row, diff --git a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h index d6b123eac9e..78758a93298 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -21,6 +21,7 @@ #include #include #include +#include #include @@ -143,7 +144,7 @@ public: if constexpr (returns_many) { - ColumnArray & arr_to = static_cast(to); + ColumnArray & arr_to = assert_cast(to); ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); size_t size = levels.size(); @@ -154,7 +155,7 @@ public: if constexpr (returns_float) { - auto & data_to = static_cast &>(arr_to.getData()).getData(); + auto & data_to = assert_cast &>(arr_to.getData()).getData(); size_t old_size = data_to.size(); data_to.resize(data_to.size() + size); @@ -172,7 +173,7 @@ public: else { if constexpr (returns_float) - static_cast &>(to).getData().push_back(data.getFloat(level)); + assert_cast &>(to).getData().push_back(data.getFloat(level)); else static_cast(to).getData().push_back(data.get(level)); } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionResample.h b/dbms/src/AggregateFunctions/AggregateFunctionResample.h index 894e0e18f51..39f1e8a35c4 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionResample.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionResample.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -169,8 +170,8 @@ public: ConstAggregateDataPtr place, IColumn & to) const override { - auto & col = static_cast(to); - auto & col_offsets = static_cast(col.getOffsetsColumn()); + auto & col = assert_cast(to); + auto & col_offsets = assert_cast(col.getOffsetsColumn()); for (size_t i = 0; i < total; ++i) nested_function->insertResultInto(place + i * sod, col.getData()); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionRetention.h b/dbms/src/AggregateFunctions/AggregateFunctionRetention.h index fb35b7bbb0c..806b34dd543 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionRetention.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionRetention.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -100,7 +101,7 @@ public: { for (const auto i : ext::range(0, events_size)) { - auto event = static_cast *>(columns[i])->getData()[row_num]; + auto event = assert_cast *>(columns[i])->getData()[row_num]; if (event) { this->data(place).add(i); @@ -125,8 +126,8 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - auto & data_to = static_cast(static_cast(to).getData()).getData(); - auto & offsets_to = static_cast(to).getOffsets(); + auto & data_to = assert_cast(assert_cast(to).getData()).getData(); + auto & offsets_to = assert_cast(to).getOffsets(); ColumnArray::Offset current_offset = data_to.size(); data_to.resize(current_offset + events_size); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h b/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h index e4b6985316f..903d5ce3493 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -152,12 +153,12 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override { - const auto timestamp = static_cast *>(columns[0])->getData()[row_num]; + const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; typename Data::Events events; for (const auto i : ext::range(1, arg_count)) { - const auto event = static_cast(columns[i])->getData()[row_num]; + const auto event = assert_cast(columns[i])->getData()[row_num]; events.set(i - 1, event); } @@ -574,7 +575,7 @@ public: auto events_it = events_begin; bool match = this->pattern_has_time ? this->backtrackingMatch(events_it, events_end) : this->dfaMatch(events_it, events_end); - static_cast(to).getData().push_back(match); + assert_cast(to).getData().push_back(match); } }; @@ -594,7 +595,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { const_cast(this->data(place)).sort(); - static_cast(to).getData().push_back(count(place)); + assert_cast(to).getData().push_back(count(place)); } private: diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h b/dbms/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h index 95732c23bda..cc5346c6f01 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -120,8 +121,8 @@ public: Arena * ) const override { - auto col_x = static_cast *>(columns[0]); - auto col_y = static_cast *>(columns[1]); + auto col_x = assert_cast *>(columns[0]); + auto col_y = assert_cast *>(columns[1]); X x = col_x->getData()[row_num]; Y y = col_y->getData()[row_num]; @@ -181,9 +182,9 @@ public: Ret k = this->data(place).getK(); Ret b = this->data(place).getB(k); - auto & col_tuple = static_cast(to); - auto & col_k = static_cast &>(col_tuple.getColumn(0)); - auto & col_b = static_cast &>(col_tuple.getColumn(1)); + auto & col_tuple = assert_cast(to); + auto & col_k = assert_cast &>(col_tuple.getColumn(0)); + auto & col_b = assert_cast &>(col_tuple.getColumn(1)); col_k.getData().push_back(k); col_b.getData().push_back(b); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionState.h b/dbms/src/AggregateFunctions/AggregateFunctionState.h index 1f49ac80db9..d558d93571a 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionState.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionState.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -81,7 +82,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(const_cast(place)); + assert_cast(to).getData().push_back(const_cast(place)); } /// Aggregate function or aggregate function state. diff --git a/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h b/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h index d1112ec0831..284e6dfb8bd 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h @@ -5,9 +5,11 @@ #include #include #include +#include #include + namespace DB { @@ -52,7 +54,7 @@ class AggregateFunctionVarianceData public: void update(const IColumn & column, size_t row_num) { - T received = static_cast &>(column).getData()[row_num]; + T received = assert_cast &>(column).getData()[row_num]; Float64 val = static_cast(received); Float64 delta = val - mean; @@ -95,7 +97,7 @@ public: void publish(IColumn & to) const { - static_cast(to).getData().push_back(Op::apply(m2, count)); + assert_cast(to).getData().push_back(Op::apply(m2, count)); } private: @@ -265,11 +267,11 @@ private: public: void update(const IColumn & column_left, const IColumn & column_right, size_t row_num) { - T left_received = static_cast &>(column_left).getData()[row_num]; + T left_received = assert_cast &>(column_left).getData()[row_num]; Float64 left_val = static_cast(left_received); Float64 left_delta = left_val - left_mean; - U right_received = static_cast &>(column_right).getData()[row_num]; + U right_received = assert_cast &>(column_right).getData()[row_num]; Float64 right_val = static_cast(right_received); Float64 right_delta = right_val - right_mean; @@ -345,9 +347,9 @@ public: void publish(IColumn & to) const { if constexpr (compute_marginal_moments) - static_cast(to).getData().push_back(Op::apply(co_moment, Base::left_m2, Base::right_m2, count)); + assert_cast(to).getData().push_back(Op::apply(co_moment, Base::left_m2, Base::right_m2, count)); else - static_cast(to).getData().push_back(Op::apply(co_moment, count)); + assert_cast(to).getData().push_back(Op::apply(co_moment, count)); } private: diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index 34bc92edfa0..9a9ba9e8832 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -12,6 +12,7 @@ #include #include +#include #include #include @@ -83,7 +84,7 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override { // Column 0 contains array of keys of known type - const ColumnArray & array_column0 = static_cast(*columns[0]); + const ColumnArray & array_column0 = assert_cast(*columns[0]); const IColumn::Offsets & offsets0 = array_column0.getOffsets(); const auto & keys_vec = static_cast(array_column0.getData()); const size_t keys_vec_offset = offsets0[row_num - 1]; @@ -94,7 +95,7 @@ public: for (size_t col = 0, size = values_types.size(); col < size; ++col) { Field value; - const ColumnArray & array_column = static_cast(*columns[col + 1]); + const ColumnArray & array_column = assert_cast(*columns[col + 1]); const IColumn::Offsets & offsets = array_column.getOffsets(); const size_t values_vec_offset = offsets[row_num - 1]; const size_t values_vec_size = (offsets[row_num] - values_vec_offset); @@ -228,8 +229,8 @@ public: size_t size = merged_maps.size(); - auto & to_tuple = static_cast(to); - auto & to_keys_arr = static_cast(to_tuple.getColumn(0)); + auto & to_tuple = assert_cast(to); + auto & to_keys_arr = assert_cast(to_tuple.getColumn(0)); auto & to_keys_col = to_keys_arr.getData(); // Advance column offsets @@ -239,7 +240,7 @@ public: for (size_t col = 0; col < values_types.size(); ++col) { - auto & to_values_arr = static_cast(to_tuple.getColumn(col + 1)); + auto & to_values_arr = assert_cast(to_tuple.getColumn(col + 1)); auto & to_values_offsets = to_values_arr.getOffsets(); to_values_offsets.push_back(to_values_offsets.back() + size); to_values_arr.getData().reserve(size); @@ -254,7 +255,7 @@ public: // Write 0..n arrays of values for (size_t col = 0; col < values_types.size(); ++col) { - auto & to_values_col = static_cast(to_tuple.getColumn(col + 1)).getData(); + auto & to_values_col = assert_cast(to_tuple.getColumn(col + 1)).getData(); to_values_col.insert(elem.second[col]); } } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h b/dbms/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h index 5e2a9b15f4e..94a64ed6331 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h @@ -16,6 +16,7 @@ #include #include #include +#include #include #include "IAggregateFunction.h" @@ -235,9 +236,9 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override { - auto uid = static_cast *>(columns[0])->getData()[row_num]; - auto ts = static_cast *>(columns[1])->getData()[row_num]; - auto val = static_cast *>(columns[2])->getData()[row_num]; + auto uid = assert_cast *>(columns[0])->getData()[row_num]; + auto ts = assert_cast *>(columns[1])->getData()[row_num]; + auto val = assert_cast *>(columns[2])->getData()[row_num]; if (uid && ts && val) { this->data(place).add(uid, ts, val); @@ -255,7 +256,7 @@ public: const auto & value = this->data(place).result; size_t size = value.size(); - ColumnArray & arr_to = static_cast(to); + ColumnArray & arr_to = assert_cast(to); ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); size_t old_size = offsets_to.back(); @@ -264,9 +265,9 @@ public: if (size) { typename ColumnInt64::Container & ts_to - = static_cast(static_cast(arr_to.getData()).getColumn(0)).getData(); + = assert_cast(assert_cast(arr_to.getData()).getColumn(0)).getData(); typename ColumnFloat64::Container & val_to - = static_cast(static_cast(arr_to.getData()).getColumn(1)).getData(); + = assert_cast(assert_cast(arr_to.getData()).getColumn(1)).getData(); ts_to.reserve(old_size + size); val_to.reserve(old_size + size); size_t i = 0; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h index 9bcfa07b78e..84eeb50d189 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h @@ -10,8 +10,8 @@ #include #include - #include +#include #include @@ -62,9 +62,9 @@ public: set.resize(reserved); if constexpr (is_weighted) - set.insert(static_cast &>(*columns[0]).getData()[row_num], columns[1]->getUInt(row_num)); + set.insert(assert_cast &>(*columns[0]).getData()[row_num], columns[1]->getUInt(row_num)); else - set.insert(static_cast &>(*columns[0]).getData()[row_num]); + set.insert(assert_cast &>(*columns[0]).getData()[row_num]); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override @@ -86,7 +86,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - ColumnArray & arr_to = static_cast(to); + ColumnArray & arr_to = assert_cast(to); ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); const typename State::Set & set = this->data(place).value; @@ -95,7 +95,7 @@ public: offsets_to.push_back(offsets_to.back() + size); - typename ColumnVector::Container & data_to = static_cast &>(arr_to.getData()).getData(); + typename ColumnVector::Container & data_to = assert_cast &>(arr_to.getData()).getData(); size_t old_size = data_to.size(); data_to.resize(old_size + size); @@ -215,7 +215,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - ColumnArray & arr_to = static_cast(to); + ColumnArray & arr_to = assert_cast(to); ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); IColumn & data_to = arr_to.getData(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniq.h b/dbms/src/AggregateFunctions/AggregateFunctionUniq.h index bb292091788..4ecd5693cc2 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniq.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -170,7 +171,7 @@ struct OneAdder { if constexpr (!std::is_same_v) { - const auto & value = static_cast &>(column).getData()[row_num]; + const auto & value = assert_cast &>(column).getData()[row_num]; data.set.insert(AggregateFunctionUniqTraits::hash(value)); } else @@ -183,7 +184,7 @@ struct OneAdder { if constexpr (!std::is_same_v) { - data.set.insert(static_cast &>(column).getData()[row_num]); + data.set.insert(assert_cast &>(column).getData()[row_num]); } else { @@ -240,7 +241,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(this->data(place).set.size()); + assert_cast(to).getData().push_back(this->data(place).set.size()); } const char * getHeaderFilePath() const override { return __FILE__; } @@ -296,7 +297,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(this->data(place).set.size()); + assert_cast(to).getData().push_back(this->data(place).set.size()); } const char * getHeaderFilePath() const override { return __FILE__; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h index 7206da3a6aa..8d90e72a149 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -131,7 +132,7 @@ public: { if constexpr (!std::is_same_v) { - const auto & value = static_cast &>(*columns[0]).getData()[row_num]; + const auto & value = assert_cast &>(*columns[0]).getData()[row_num]; this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits::hash(value)); } else @@ -158,7 +159,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(this->data(place).set.size()); + assert_cast(to).getData().push_back(this->data(place).set.size()); } const char * getHeaderFilePath() const override @@ -222,7 +223,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(this->data(place).set.size()); + assert_cast(to).getData().push_back(this->data(place).set.size()); } const char * getHeaderFilePath() const override diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.h index 88e16f330ce..ddb8efe3eb2 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -101,7 +102,7 @@ struct __attribute__((__packed__)) AggregateFunctionUniqUpToData void add(const IColumn & column, size_t row_num, UInt8 threshold) { - insert(static_cast &>(column).getData()[row_num], threshold); + insert(assert_cast &>(column).getData()[row_num], threshold); } }; @@ -123,7 +124,7 @@ struct AggregateFunctionUniqUpToData : AggregateFunctionUniqUpToData &>(column).getData()[row_num]; + UInt128 value = assert_cast &>(column).getData()[row_num]; insert(sipHash64(value), threshold); } }; @@ -176,7 +177,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(this->data(place).size()); + assert_cast(to).getData().push_back(this->data(place).size()); } const char * getHeaderFilePath() const override { return __FILE__; } @@ -240,7 +241,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(this->data(place).size()); + assert_cast(to).getData().push_back(this->data(place).size()); } const char * getHeaderFilePath() const override { return __FILE__; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index df93ec5c0ba..16d9ac548ad 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -9,6 +9,7 @@ #include #include #include +#include #include @@ -215,11 +216,11 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override { - const auto timestamp = static_cast *>(columns[0])->getData()[row_num]; + const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; // reverse iteration and stable sorting are needed for events that are qualified by more than one condition. for (auto i = events_size; i > 0; --i) { - auto event = static_cast *>(columns[i])->getData()[row_num]; + auto event = assert_cast *>(columns[i])->getData()[row_num]; if (event) this->data(place).add(timestamp, i); } @@ -242,7 +243,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(getEventLevel(this->data(place))); + assert_cast(to).getData().push_back(getEventLevel(this->data(place))); } const char * getHeaderFilePath() const override diff --git a/dbms/src/AggregateFunctions/UniqVariadicHash.h b/dbms/src/AggregateFunctions/UniqVariadicHash.h index eb38768f776..d6e584a2e1e 100644 --- a/dbms/src/AggregateFunctions/UniqVariadicHash.h +++ b/dbms/src/AggregateFunctions/UniqVariadicHash.h @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -67,7 +68,7 @@ struct UniqVariadicHash { UInt64 hash; - const auto & tuple_columns = static_cast(columns[0])->getColumns(); + const auto & tuple_columns = assert_cast(columns[0])->getColumns(); const auto * column = tuple_columns.data(); const auto * columns_end = column + num_args; @@ -116,7 +117,7 @@ struct UniqVariadicHash { static inline UInt128 apply(size_t num_args, const IColumn ** columns, size_t row_num) { - const auto & tuple_columns = static_cast(columns[0])->getColumns(); + const auto & tuple_columns = assert_cast(columns[0])->getColumns(); const auto * column = tuple_columns.data(); const auto * columns_end = column + num_args; diff --git a/dbms/src/Columns/ColumnAggregateFunction.cpp b/dbms/src/Columns/ColumnAggregateFunction.cpp index e919556c01f..845ae47e623 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.cpp +++ b/dbms/src/Columns/ColumnAggregateFunction.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -158,7 +159,7 @@ void ColumnAggregateFunction::ensureOwnership() void ColumnAggregateFunction::insertRangeFrom(const IColumn & from, size_t start, size_t length) { - const ColumnAggregateFunction & from_concrete = static_cast(from); + const ColumnAggregateFunction & from_concrete = assert_cast(from); if (start + length > from_concrete.data.size()) throw Exception("Parameters start = " + toString(start) + ", length = " + toString(length) @@ -353,7 +354,7 @@ void ColumnAggregateFunction::insertMergeFrom(ConstAggregateDataPtr place) void ColumnAggregateFunction::insertMergeFrom(const IColumn & from, size_t n) { - insertMergeFrom(static_cast(from).data[n]); + insertMergeFrom(assert_cast(from).data[n]); } Arena & ColumnAggregateFunction::createOrGetArena() @@ -492,7 +493,7 @@ MutableColumns ColumnAggregateFunction::scatter(IColumn::ColumnIndex num_columns } for (size_t i = 0; i < num_rows; ++i) - static_cast(*columns[selector[i]]).data.push_back(data[i]); + assert_cast(*columns[selector[i]]).data.push_back(data[i]); return columns; } diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index b825342ee5f..b51340662b6 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -16,7 +16,7 @@ #include #include #include - +#include namespace DB @@ -227,7 +227,7 @@ void ColumnArray::insert(const Field & x) void ColumnArray::insertFrom(const IColumn & src_, size_t n) { - const ColumnArray & src = static_cast(src_); + const ColumnArray & src = assert_cast(src_); size_t size = src.sizeAt(n); size_t offset = src.offsetAt(n); @@ -257,7 +257,7 @@ void ColumnArray::popBack(size_t n) int ColumnArray::compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const { - const ColumnArray & rhs = static_cast(rhs_); + const ColumnArray & rhs = assert_cast(rhs_); /// Suboptimal size_t lhs_size = sizeAt(n); @@ -373,7 +373,7 @@ void ColumnArray::insertRangeFrom(const IColumn & src, size_t start, size_t leng if (length == 0) return; - const ColumnArray & src_concrete = static_cast(src); + const ColumnArray & src_concrete = assert_cast(src); if (start + length > src_concrete.getOffsets().size()) throw Exception("Parameter out of bound in ColumnArray::insertRangeFrom method. [start(" + std::to_string(start) + ") + length(" + std::to_string(length) + ") > offsets.size(" + std::to_string(src_concrete.getOffsets().size()) + ")]", @@ -429,10 +429,10 @@ ColumnPtr ColumnArray::filterNumber(const Filter & filt, ssize_t result_size_hin auto res = ColumnArray::create(data->cloneEmpty()); - auto & res_elems = static_cast &>(res->getData()).getData(); + auto & res_elems = assert_cast &>(res->getData()).getData(); Offsets & res_offsets = res->getOffsets(); - filterArraysImpl(static_cast &>(*data).getData(), getOffsets(), res_elems, res_offsets, filt, result_size_hint); + filterArraysImpl(assert_cast &>(*data).getData(), getOffsets(), res_elems, res_offsets, filt, result_size_hint); return res; } @@ -554,11 +554,11 @@ ColumnPtr ColumnArray::filterNullable(const Filter & filt, ssize_t result_size_h if (getOffsets().size() == 0) return ColumnArray::create(data); - const ColumnNullable & nullable_elems = static_cast(*data); + const ColumnNullable & nullable_elems = assert_cast(*data); auto array_of_nested = ColumnArray::create(nullable_elems.getNestedColumnPtr(), offsets); auto filtered_array_of_nested_owner = array_of_nested->filter(filt, result_size_hint); - auto & filtered_array_of_nested = static_cast(*filtered_array_of_nested_owner); + auto & filtered_array_of_nested = assert_cast(*filtered_array_of_nested_owner); auto & filtered_offsets = filtered_array_of_nested.getOffsetsPtr(); auto res_null_map = ColumnUInt8::create(); @@ -577,7 +577,7 @@ ColumnPtr ColumnArray::filterTuple(const Filter & filt, ssize_t result_size_hint if (getOffsets().size() == 0) return ColumnArray::create(data); - const ColumnTuple & tuple = static_cast(*data); + const ColumnTuple & tuple = assert_cast(*data); /// Make temporary arrays for each components of Tuple, then filter and collect back. @@ -593,11 +593,11 @@ ColumnPtr ColumnArray::filterTuple(const Filter & filt, ssize_t result_size_hint Columns tuple_columns(tuple_size); for (size_t i = 0; i < tuple_size; ++i) - tuple_columns[i] = static_cast(*temporary_arrays[i]).getDataPtr(); + tuple_columns[i] = assert_cast(*temporary_arrays[i]).getDataPtr(); return ColumnArray::create( ColumnTuple::create(tuple_columns), - static_cast(*temporary_arrays.front()).getOffsetsPtr()); + assert_cast(*temporary_arrays.front()).getOffsetsPtr()); } @@ -789,7 +789,7 @@ ColumnPtr ColumnArray::replicateString(const Offsets & replicate_offsets) const if (0 == col_size) return res; - ColumnArray & res_ = static_cast(*res); + ColumnArray & res_ = assert_cast(*res); const ColumnString & src_string = typeid_cast(*data); const ColumnString::Chars & src_chars = src_string.getChars(); @@ -901,7 +901,7 @@ ColumnPtr ColumnArray::replicateGeneric(const Offsets & replicate_offsets) const throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); MutableColumnPtr res = cloneEmpty(); - ColumnArray & res_concrete = static_cast(*res); + ColumnArray & res_concrete = assert_cast(*res); if (0 == col_size) return res; @@ -922,7 +922,7 @@ ColumnPtr ColumnArray::replicateGeneric(const Offsets & replicate_offsets) const ColumnPtr ColumnArray::replicateNullable(const Offsets & replicate_offsets) const { - const ColumnNullable & nullable = static_cast(*data); + const ColumnNullable & nullable = assert_cast(*data); /// Make temporary arrays for each components of Nullable. Then replicate them independently and collect back to result. /// NOTE Offsets are calculated twice and it is redundant. @@ -934,15 +934,15 @@ ColumnPtr ColumnArray::replicateNullable(const Offsets & replicate_offsets) cons return ColumnArray::create( ColumnNullable::create( - static_cast(*array_of_nested).getDataPtr(), - static_cast(*array_of_null_map).getDataPtr()), - static_cast(*array_of_nested).getOffsetsPtr()); + assert_cast(*array_of_nested).getDataPtr(), + assert_cast(*array_of_null_map).getDataPtr()), + assert_cast(*array_of_nested).getOffsetsPtr()); } ColumnPtr ColumnArray::replicateTuple(const Offsets & replicate_offsets) const { - const ColumnTuple & tuple = static_cast(*data); + const ColumnTuple & tuple = assert_cast(*data); /// Make temporary arrays for each components of Tuple. In the same way as for Nullable. @@ -958,11 +958,11 @@ ColumnPtr ColumnArray::replicateTuple(const Offsets & replicate_offsets) const Columns tuple_columns(tuple_size); for (size_t i = 0; i < tuple_size; ++i) - tuple_columns[i] = static_cast(*temporary_arrays[i]).getDataPtr(); + tuple_columns[i] = assert_cast(*temporary_arrays[i]).getDataPtr(); return ColumnArray::create( ColumnTuple::create(tuple_columns), - static_cast(*temporary_arrays.front()).getOffsetsPtr()); + assert_cast(*temporary_arrays.front()).getOffsetsPtr()); } diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index f3f7f1f4a1a..7d1ff63a62e 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -89,12 +90,12 @@ public: Offsets & ALWAYS_INLINE getOffsets() { - return static_cast(*offsets).getData(); + return assert_cast(*offsets).getData(); } const Offsets & ALWAYS_INLINE getOffsets() const { - return static_cast(*offsets).getData(); + return assert_cast(*offsets).getData(); } const ColumnPtr & getDataPtr() const { return data; } diff --git a/dbms/src/Columns/ColumnConst.h b/dbms/src/Columns/ColumnConst.h index 6b320f12f28..5da6cc59527 100644 --- a/dbms/src/Columns/ColumnConst.h +++ b/dbms/src/Columns/ColumnConst.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -175,7 +176,7 @@ public: int compareAt(size_t, size_t, const IColumn & rhs, int nan_direction_hint) const override { - return data->compareAt(0, 0, *static_cast(rhs).data, nan_direction_hint); + return data->compareAt(0, 0, *assert_cast(rhs).data, nan_direction_hint); } MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; diff --git a/dbms/src/Columns/ColumnDecimal.cpp b/dbms/src/Columns/ColumnDecimal.cpp index 5e475b64dfa..ab80f7832a8 100644 --- a/dbms/src/Columns/ColumnDecimal.cpp +++ b/dbms/src/Columns/ColumnDecimal.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include @@ -133,7 +134,7 @@ void ColumnDecimal::insertData(const char * src, size_t /*length*/) template void ColumnDecimal::insertRangeFrom(const IColumn & src, size_t start, size_t length) { - const ColumnDecimal & src_vec = static_cast(src); + const ColumnDecimal & src_vec = assert_cast(src); if (start + length > src_vec.data.size()) throw Exception("Parameters start = " + toString(start) + ", length = " + toString(length) + diff --git a/dbms/src/Columns/ColumnFixedString.cpp b/dbms/src/Columns/ColumnFixedString.cpp index 686223f712f..02df8d010ea 100644 --- a/dbms/src/Columns/ColumnFixedString.cpp +++ b/dbms/src/Columns/ColumnFixedString.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -33,7 +34,7 @@ MutableColumnPtr ColumnFixedString::cloneResized(size_t size) const if (size > 0) { - auto & new_col = static_cast(*new_col_holder); + auto & new_col = assert_cast(*new_col_holder); new_col.chars.resize(size * n); size_t count = std::min(this->size(), size); @@ -60,7 +61,7 @@ void ColumnFixedString::insert(const Field & x) void ColumnFixedString::insertFrom(const IColumn & src_, size_t index) { - const ColumnFixedString & src = static_cast(src_); + const ColumnFixedString & src = assert_cast(src_); if (n != src.getN()) throw Exception("Size of FixedString doesn't match", ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH); @@ -140,7 +141,7 @@ void ColumnFixedString::getPermutation(bool reverse, size_t limit, int /*nan_dir void ColumnFixedString::insertRangeFrom(const IColumn & src, size_t start, size_t length) { - const ColumnFixedString & src_concrete = static_cast(src); + const ColumnFixedString & src_concrete = assert_cast(src); if (start + length > src_concrete.size()) throw Exception("Parameters start = " diff --git a/dbms/src/Columns/ColumnFixedString.h b/dbms/src/Columns/ColumnFixedString.h index 2b06d19d7ca..3a91d4d6ff9 100644 --- a/dbms/src/Columns/ColumnFixedString.h +++ b/dbms/src/Columns/ColumnFixedString.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -102,7 +103,7 @@ public: int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int /*nan_direction_hint*/) const override { - const ColumnFixedString & rhs = static_cast(rhs_); + const ColumnFixedString & rhs = assert_cast(rhs_); return memcmpSmallAllowOverflow15(chars.data() + p1 * n, rhs.chars.data() + p2 * n, n); } diff --git a/dbms/src/Columns/ColumnLowCardinality.cpp b/dbms/src/Columns/ColumnLowCardinality.cpp index 152570f1a6f..fa713f76e5e 100644 --- a/dbms/src/Columns/ColumnLowCardinality.cpp +++ b/dbms/src/Columns/ColumnLowCardinality.cpp @@ -3,6 +3,8 @@ #include #include #include +#include + namespace DB { @@ -244,7 +246,7 @@ MutableColumnPtr ColumnLowCardinality::cloneResized(size_t size) const int ColumnLowCardinality::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const { - const auto & low_cardinality_column = static_cast(rhs); + const auto & low_cardinality_column = assert_cast(rhs); size_t n_index = getIndexes().getUInt(n); size_t m_index = low_cardinality_column.getIndexes().getUInt(m); return getDictionary().compareAt(n_index, m_index, low_cardinality_column.getDictionary(), nan_direction_hint); diff --git a/dbms/src/Columns/ColumnLowCardinality.h b/dbms/src/Columns/ColumnLowCardinality.h index 9081938e2c6..74ea04cb08f 100644 --- a/dbms/src/Columns/ColumnLowCardinality.h +++ b/dbms/src/Columns/ColumnLowCardinality.h @@ -2,6 +2,7 @@ #include #include #include +#include #include #include "ColumnsNumber.h" @@ -166,10 +167,10 @@ public: switch (idx.getSizeOfIndexType()) { - case sizeof(UInt8): return static_cast(indexes)->getElement(row); - case sizeof(UInt16): return static_cast(indexes)->getElement(row); - case sizeof(UInt32): return static_cast(indexes)->getElement(row); - case sizeof(UInt64): return static_cast(indexes)->getElement(row); + case sizeof(UInt8): return assert_cast(indexes)->getElement(row); + case sizeof(UInt16): return assert_cast(indexes)->getElement(row); + case sizeof(UInt32): return assert_cast(indexes)->getElement(row); + case sizeof(UInt64): return assert_cast(indexes)->getElement(row); default: throw Exception("Unexpected size of index type for low cardinality column.", ErrorCodes::LOGICAL_ERROR); } } diff --git a/dbms/src/Columns/ColumnNullable.cpp b/dbms/src/Columns/ColumnNullable.cpp index c5cdffba84d..716b583f9a2 100644 --- a/dbms/src/Columns/ColumnNullable.cpp +++ b/dbms/src/Columns/ColumnNullable.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -129,7 +130,7 @@ const char * ColumnNullable::deserializeAndInsertFromArena(const char * pos) void ColumnNullable::insertRangeFrom(const IColumn & src, size_t start, size_t length) { - const ColumnNullable & nullable_col = static_cast(src); + const ColumnNullable & nullable_col = assert_cast(src); getNullMapColumn().insertRangeFrom(*nullable_col.null_map, start, length); getNestedColumn().insertRangeFrom(*nullable_col.nested_column, start, length); } @@ -150,7 +151,7 @@ void ColumnNullable::insert(const Field & x) void ColumnNullable::insertFrom(const IColumn & src, size_t n) { - const ColumnNullable & src_concrete = static_cast(src); + const ColumnNullable & src_concrete = assert_cast(src); getNestedColumn().insertFrom(src_concrete.getNestedColumn(), n); getNullMapData().push_back(src_concrete.getNullMapData()[n]); } @@ -190,7 +191,7 @@ int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null /// the ordering specified by either NULLS FIRST or NULLS LAST in the /// ORDER BY construction. - const ColumnNullable & nullable_rhs = static_cast(rhs_); + const ColumnNullable & nullable_rhs = assert_cast(rhs_); bool lval_is_null = isNullAt(n); bool rval_is_null = nullable_rhs.isNullAt(m); @@ -458,7 +459,7 @@ ColumnPtr makeNullable(const ColumnPtr & column) return column; if (isColumnConst(*column)) - return ColumnConst::create(makeNullable(static_cast(*column).getDataColumnPtr()), column->size()); + return ColumnConst::create(makeNullable(assert_cast(*column).getDataColumnPtr()), column->size()); return ColumnNullable::create(column, ColumnUInt8::create(column->size(), 0)); } diff --git a/dbms/src/Columns/ColumnNullable.h b/dbms/src/Columns/ColumnNullable.h index b4db692b61d..f93a227abae 100644 --- a/dbms/src/Columns/ColumnNullable.h +++ b/dbms/src/Columns/ColumnNullable.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -45,7 +46,7 @@ public: std::string getName() const override { return "Nullable(" + nested_column->getName() + ")"; } MutableColumnPtr cloneResized(size_t size) const override; size_t size() const override { return nested_column->size(); } - bool isNullAt(size_t n) const override { return static_cast(*null_map).getData()[n] != 0;} + bool isNullAt(size_t n) const override { return assert_cast(*null_map).getData()[n] != 0;} Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; bool getBool(size_t n) const override { return isNullAt(n) ? 0 : nested_column->getBool(n); } @@ -116,8 +117,8 @@ public: /// Return the column that represents the byte map. const ColumnPtr & getNullMapColumnPtr() const { return null_map; } - ColumnUInt8 & getNullMapColumn() { return static_cast(*null_map); } - const ColumnUInt8 & getNullMapColumn() const { return static_cast(*null_map); } + ColumnUInt8 & getNullMapColumn() { return assert_cast(*null_map); } + const ColumnUInt8 & getNullMapColumn() const { return assert_cast(*null_map); } NullMap & getNullMapData() { return getNullMapColumn().getData(); } const NullMap & getNullMapData() const { return getNullMapColumn().getData(); } diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index 78d2f3ce045..8d3f3e66587 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -68,7 +69,7 @@ void ColumnString::insertRangeFrom(const IColumn & src, size_t start, size_t len if (length == 0) return; - const ColumnString & src_concrete = static_cast(src); + const ColumnString & src_concrete = assert_cast(src); if (start + length > src_concrete.offsets.size()) throw Exception("Parameter out of bound in IColumnString::insertRangeFrom method.", @@ -360,7 +361,7 @@ void ColumnString::getExtremes(Field & min, Field & max) const int ColumnString::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, const Collator & collator) const { - const ColumnString & rhs = static_cast(rhs_); + const ColumnString & rhs = assert_cast(rhs_); return collator.compare( reinterpret_cast(&chars[offsetAt(n)]), sizeAt(n), diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index 398077ce964..489fb0b4f66 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -8,6 +8,7 @@ #include #include #include +#include class Collator; @@ -121,7 +122,7 @@ public: void insertFrom(const IColumn & src_, size_t n) override { - const ColumnString & src = static_cast(src_); + const ColumnString & src = assert_cast(src_); const size_t size_to_append = src.offsets[n] - src.offsets[n - 1]; /// -1th index is Ok, see PaddedPODArray. if (size_to_append == 1) @@ -204,7 +205,7 @@ public: int compareAt(size_t n, size_t m, const IColumn & rhs_, int /*nan_direction_hint*/) const override { - const ColumnString & rhs = static_cast(rhs_); + const ColumnString & rhs = assert_cast(rhs_); return memcmpSmallAllowOverflow15(chars.data() + offsetAt(n), sizeAt(n) - 1, rhs.chars.data() + rhs.offsetAt(m), rhs.sizeAt(m) - 1); } diff --git a/dbms/src/Columns/ColumnTuple.cpp b/dbms/src/Columns/ColumnTuple.cpp index bef717361df..07599b3456f 100644 --- a/dbms/src/Columns/ColumnTuple.cpp +++ b/dbms/src/Columns/ColumnTuple.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -118,7 +119,7 @@ void ColumnTuple::insert(const Field & x) void ColumnTuple::insertFrom(const IColumn & src_, size_t n) { - const ColumnTuple & src = static_cast(src_); + const ColumnTuple & src = assert_cast(src_); const size_t tuple_size = columns.size(); if (src.columns.size() != tuple_size) @@ -172,7 +173,7 @@ void ColumnTuple::insertRangeFrom(const IColumn & src, size_t start, size_t leng const size_t tuple_size = columns.size(); for (size_t i = 0; i < tuple_size; ++i) columns[i]->insertRangeFrom( - *static_cast(src).columns[i], + *assert_cast(src).columns[i], start, length); } @@ -245,7 +246,7 @@ int ColumnTuple::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_dire { const size_t tuple_size = columns.size(); for (size_t i = 0; i < tuple_size; ++i) - if (int res = columns[i]->compareAt(n, m, *static_cast(rhs).columns[i], nan_direction_hint)) + if (int res = columns[i]->compareAt(n, m, *assert_cast(rhs).columns[i], nan_direction_hint)) return res; return 0; diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 154bb457f01..62a468e5821 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -11,6 +11,7 @@ #include #include +#include #include #include @@ -140,8 +141,8 @@ private: static size_t numSpecialValues(bool is_nullable) { return is_nullable ? 2 : 1; } size_t numSpecialValues() const { return numSpecialValues(is_nullable); } - ColumnType * getRawColumnPtr() { return static_cast(column_holder.get()); } - const ColumnType * getRawColumnPtr() const { return static_cast(column_holder.get()); } + ColumnType * getRawColumnPtr() { return assert_cast(column_holder.get()); } + const ColumnType * getRawColumnPtr() const { return assert_cast(column_holder.get()); } template MutableColumnPtr uniqueInsertRangeImpl( @@ -232,7 +233,7 @@ void ColumnUnique::updateNullMask() size_t size = getRawColumnPtr()->size(); if (nested_null_mask->size() != size) - static_cast(*nested_null_mask).getData().resize_fill(size); + assert_cast(*nested_null_mask).getData().resize_fill(size); } } diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index a2d6de9df80..d2edbe496e7 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -224,7 +225,7 @@ Float64 ColumnVector::getFloat64(size_t n) const template void ColumnVector::insertRangeFrom(const IColumn & src, size_t start, size_t length) { - const ColumnVector & src_vec = static_cast(src); + const ColumnVector & src_vec = assert_cast(src); if (start + length > src_vec.data.size()) throw Exception("Parameters start = " diff --git a/dbms/src/Columns/FilterDescription.cpp b/dbms/src/Columns/FilterDescription.cpp index 405ed974c6c..272cc4d92de 100644 --- a/dbms/src/Columns/FilterDescription.cpp +++ b/dbms/src/Columns/FilterDescription.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -25,7 +26,7 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column) if (isColumnConst(column)) { - const ColumnConst & column_const = static_cast(column); + const ColumnConst & column_const = assert_cast(column); ColumnPtr column_nested = column_const.getDataColumnPtr()->convertToFullColumnIfLowCardinality(); if (!typeid_cast(column_nested.get())) diff --git a/dbms/src/Columns/ReverseIndex.h b/dbms/src/Columns/ReverseIndex.h index 8fa4e87680b..157deef46b8 100644 --- a/dbms/src/Columns/ReverseIndex.h +++ b/dbms/src/Columns/ReverseIndex.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include diff --git a/dbms/src/Columns/getLeastSuperColumn.cpp b/dbms/src/Columns/getLeastSuperColumn.cpp index d442c926b23..11c5b1b58fd 100644 --- a/dbms/src/Columns/getLeastSuperColumn.cpp +++ b/dbms/src/Columns/getLeastSuperColumn.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include @@ -14,7 +15,7 @@ namespace ErrorCodes static bool sameConstants(const IColumn & a, const IColumn & b) { - return static_cast(a).getField() == static_cast(b).getField(); + return assert_cast(a).getField() == assert_cast(b).getField(); } ColumnWithTypeAndName getLeastSuperColumn(std::vector columns) @@ -57,7 +58,7 @@ ColumnWithTypeAndName getLeastSuperColumn(std::vectorcreateColumnConst(0, static_cast(*columns[0]->column).getField()); + result.column = result.type->createColumnConst(0, assert_cast(*columns[0]->column).getField()); else result.column = result.type->createColumn(); diff --git a/dbms/src/Common/ColumnsHashing.h b/dbms/src/Common/ColumnsHashing.h index bf564738f7a..d6b328a9253 100644 --- a/dbms/src/Common/ColumnsHashing.h +++ b/dbms/src/Common/ColumnsHashing.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -80,7 +81,7 @@ struct HashMethodString HashMethodString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) { const IColumn & column = *key_columns[0]; - const ColumnString & column_string = static_cast(column); + const ColumnString & column_string = assert_cast(column); offsets = column_string.getOffsets().data(); chars = column_string.getChars().data(); } @@ -120,7 +121,7 @@ struct HashMethodFixedString HashMethodFixedString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) { const IColumn & column = *key_columns[0]; - const ColumnFixedString & column_string = static_cast(column); + const ColumnFixedString & column_string = assert_cast(column); n = column_string.getN(); chars = &column_string.getChars(); } @@ -307,10 +308,10 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod { switch (size_of_index_type) { - case sizeof(UInt8): return static_cast(positions)->getElement(row); - case sizeof(UInt16): return static_cast(positions)->getElement(row); - case sizeof(UInt32): return static_cast(positions)->getElement(row); - case sizeof(UInt64): return static_cast(positions)->getElement(row); + case sizeof(UInt8): return assert_cast(positions)->getElement(row); + case sizeof(UInt16): return assert_cast(positions)->getElement(row); + case sizeof(UInt32): return assert_cast(positions)->getElement(row); + case sizeof(UInt64): return assert_cast(positions)->getElement(row); default: throw Exception("Unexpected size of index type for low cardinality column.", ErrorCodes::LOGICAL_ERROR); } } diff --git a/dbms/src/Common/ColumnsHashingImpl.h b/dbms/src/Common/ColumnsHashingImpl.h index d980a3f1b64..b53239d8bc9 100644 --- a/dbms/src/Common/ColumnsHashingImpl.h +++ b/dbms/src/Common/ColumnsHashingImpl.h @@ -1,8 +1,10 @@ #pragma once #include +#include #include + namespace DB { @@ -310,7 +312,7 @@ protected: { if (null_maps[k] != nullptr) { - const auto & null_map = static_cast(*null_maps[k]).getData(); + const auto & null_map = assert_cast(*null_maps[k]).getData(); if (null_map[row] == 1) { size_t bucket = k / 8; diff --git a/dbms/src/Common/assert_cast.h b/dbms/src/Common/assert_cast.h new file mode 100644 index 00000000000..93f213a0a23 --- /dev/null +++ b/dbms/src/Common/assert_cast.h @@ -0,0 +1,44 @@ +#pragma once + +#include +#include +#include +#include + +#include +#include + + +namespace DB +{ + namespace ErrorCodes + { + extern const int BAD_CAST; + } +} + + +/** Perform static_cast in release build. + * Checks type by comparing typeid and throw an exception in debug build. + * The exact match of the type is checked. That is, cast to the ancestor will be unsuccessful. + */ +template +To assert_cast(From && from) +{ +#ifndef NDEBUG + try + { + if (typeid(from) == typeid(To)) + return static_cast(from); + } + catch (const std::exception & e) + { + throw DB::Exception(e.what(), DB::ErrorCodes::BAD_CAST); + } + + throw DB::Exception("Bad cast from type " + demangle(typeid(from).name()) + " to " + demangle(typeid(To).name()), + DB::ErrorCodes::BAD_CAST); +#else + return static_cast(from); +#endif +} diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index 5a1f9753c13..b045b9ec1ff 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -5,7 +5,9 @@ #include #include + #include +#include #include @@ -484,8 +486,8 @@ static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, cons if (isColumnConst(*actual.column) && isColumnConst(*expected.column)) { - Field actual_value = static_cast(*actual.column).getField(); - Field expected_value = static_cast(*expected.column).getField(); + Field actual_value = assert_cast(*actual.column).getField(); + Field expected_value = assert_cast(*expected.column).getField(); if (actual_value != expected_value) return on_error("Block structure mismatch in " + context_description + " stream: different values of constants, actual: " diff --git a/dbms/src/Core/SortCursor.h b/dbms/src/Core/SortCursor.h index 3a2b64cd8b6..c898ee71b8f 100644 --- a/dbms/src/Core/SortCursor.h +++ b/dbms/src/Core/SortCursor.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -167,7 +168,7 @@ struct SortCursorWithCollation int res; if (impl->need_collation[i]) { - const ColumnString & column_string = static_cast(*impl->sort_columns[i]); + const ColumnString & column_string = assert_cast(*impl->sort_columns[i]); res = column_string.compareAtWithCollation(lhs_pos, rhs_pos, *(rhs.impl->sort_columns[i]), *impl->desc[i].collator); } else diff --git a/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp b/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp index cfc165aa4d2..7e4ad04b806 100644 --- a/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -118,7 +119,7 @@ void CollapsingSortedBlockInputStream::merge(MutableColumns & merged_columns, st if (current_key.empty()) setPrimaryKeyRef(current_key, current); - Int8 sign = static_cast(*current->all_columns[sign_column_number]).getData()[current->pos]; + Int8 sign = assert_cast(*current->all_columns[sign_column_number]).getData()[current->pos]; setPrimaryKeyRef(next_key, current); bool key_differs = next_key != current_key; diff --git a/dbms/src/DataStreams/ConvertingBlockInputStream.cpp b/dbms/src/DataStreams/ConvertingBlockInputStream.cpp index 0ab250ac7ee..320bb35f5b3 100644 --- a/dbms/src/DataStreams/ConvertingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ConvertingBlockInputStream.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include @@ -75,7 +76,7 @@ ConvertingBlockInputStream::ConvertingBlockInputStream( throw Exception("Cannot convert column " + backQuoteIfNeed(res_elem.name) + " because it is non constant in source stream but must be constant in result", ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE); - else if (static_cast(*src_elem.column).getField() != static_cast(*res_elem.column).getField()) + else if (assert_cast(*src_elem.column).getField() != assert_cast(*res_elem.column).getField()) throw Exception("Cannot convert column " + backQuoteIfNeed(res_elem.name) + " because it is constant but values of constants are different in source and result", ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE); diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index cda13a29fd2..35fdc948239 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -297,7 +298,7 @@ Block SummingSortedBlockInputStream::readImpl() /// Unpack tuple into block. size_t tuple_size = desc.column_numbers.size(); for (size_t i = 0; i < tuple_size; ++i) - res.getByPosition(desc.column_numbers[i]).column = static_cast(*desc.merged_column).getColumnPtr(i); + res.getByPosition(desc.column_numbers[i]).column = assert_cast(*desc.merged_column).getColumnPtr(i); } else res.getByPosition(desc.column_numbers[0]).column = std::move(desc.merged_column); @@ -491,7 +492,7 @@ void SummingSortedBlockInputStream::addRow(SortCursor & cursor) { // desc.state is not used for AggregateFunction types auto & col = cursor->all_columns[desc.column_numbers[0]]; - static_cast(*desc.merged_column).insertMergeFrom(*col, cursor->pos); + assert_cast(*desc.merged_column).insertMergeFrom(*col, cursor->pos); } else { diff --git a/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp b/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp index 6282f950e46..058381ad6bd 100644 --- a/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp +++ b/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -155,7 +156,7 @@ void TotalsHavingBlockInputStream::addToTotals(const Block & source_block, const continue; } - auto & totals_column = static_cast(*current_totals[i]); + auto & totals_column = assert_cast(*current_totals[i]); assert(totals_column.size() == 1); /// Accumulate all aggregate states from a column of a source block into diff --git a/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp b/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp index c3615e68b68..ee6690f2f0d 100644 --- a/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -112,7 +113,7 @@ void VersionedCollapsingSortedBlockInputStream::merge(MutableColumns & merged_co RowRef next_key; - Int8 sign = static_cast(*current->all_columns[sign_column_number]).getData()[current->pos]; + Int8 sign = assert_cast(*current->all_columns[sign_column_number]).getData()[current->pos]; setPrimaryKeyRef(next_key, current); diff --git a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp index b53b7529f95..1855522c713 100644 --- a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -77,12 +78,12 @@ void DataTypeAggregateFunction::deserializeBinary(Field & field, ReadBuffer & is void DataTypeAggregateFunction::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - function->serialize(static_cast(column).getData()[row_num], ostr); + function->serialize(assert_cast(column).getData()[row_num], ostr); } void DataTypeAggregateFunction::deserializeBinary(IColumn & column, ReadBuffer & istr) const { - ColumnAggregateFunction & column_concrete = static_cast(column); + ColumnAggregateFunction & column_concrete = assert_cast(column); Arena & arena = column_concrete.createOrGetArena(); size_t size_of_state = function->sizeOfData(); @@ -155,13 +156,13 @@ void DataTypeAggregateFunction::deserializeBinaryBulk(IColumn & column, ReadBuff static String serializeToString(const AggregateFunctionPtr & function, const IColumn & column, size_t row_num) { WriteBufferFromOwnString buffer; - function->serialize(static_cast(column).getData()[row_num], buffer); + function->serialize(assert_cast(column).getData()[row_num], buffer); return buffer.str(); } static void deserializeFromString(const AggregateFunctionPtr & function, IColumn & column, const String & s) { - ColumnAggregateFunction & column_concrete = static_cast(column); + ColumnAggregateFunction & column_concrete = assert_cast(column); Arena & arena = column_concrete.createOrGetArena(); size_t size_of_state = function->sizeOfData(); @@ -262,13 +263,13 @@ void DataTypeAggregateFunction::serializeProtobuf(const IColumn & column, size_t if (value_index) return; value_index = static_cast( - protobuf.writeAggregateFunction(function, static_cast(column).getData()[row_num])); + protobuf.writeAggregateFunction(function, assert_cast(column).getData()[row_num])); } void DataTypeAggregateFunction::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const { row_added = false; - ColumnAggregateFunction & column_concrete = static_cast(column); + ColumnAggregateFunction & column_concrete = assert_cast(column); Arena & arena = column_concrete.createOrGetArena(); size_t size_of_state = function->sizeOfData(); AggregateDataPtr place = arena.alignedAlloc(size_of_state, function->alignOfData()); diff --git a/dbms/src/DataTypes/DataTypeArray.cpp b/dbms/src/DataTypes/DataTypeArray.cpp index 7b5de251a0d..dc4cddb5e62 100644 --- a/dbms/src/DataTypes/DataTypeArray.cpp +++ b/dbms/src/DataTypes/DataTypeArray.cpp @@ -14,6 +14,7 @@ #include #include +#include namespace DB @@ -57,7 +58,7 @@ void DataTypeArray::deserializeBinary(Field & field, ReadBuffer & istr) const void DataTypeArray::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - const ColumnArray & column_array = static_cast(column); + const ColumnArray & column_array = assert_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); size_t offset = offsets[row_num - 1]; @@ -74,7 +75,7 @@ void DataTypeArray::serializeBinary(const IColumn & column, size_t row_num, Writ void DataTypeArray::deserializeBinary(IColumn & column, ReadBuffer & istr) const { - ColumnArray & column_array = static_cast(column); + ColumnArray & column_array = assert_cast(column); ColumnArray::Offsets & offsets = column_array.getOffsets(); size_t size; @@ -278,7 +279,7 @@ void DataTypeArray::deserializeBinaryBulkWithMultipleStreams( template static void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, Writer && write_nested) { - const ColumnArray & column_array = static_cast(column); + const ColumnArray & column_array = assert_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); size_t offset = offsets[row_num - 1]; @@ -300,7 +301,7 @@ static void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffe template static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && read_nested) { - ColumnArray & column_array = static_cast(column); + ColumnArray & column_array = assert_cast(column); ColumnArray::Offsets & offsets = column_array.getOffsets(); IColumn & nested_column = column_array.getData(); @@ -367,7 +368,7 @@ void DataTypeArray::deserializeText(IColumn & column, ReadBuffer & istr, const F void DataTypeArray::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const ColumnArray & column_array = static_cast(column); + const ColumnArray & column_array = assert_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); size_t offset = offsets[row_num - 1]; @@ -394,7 +395,7 @@ void DataTypeArray::deserializeTextJSON(IColumn & column, ReadBuffer & istr, con void DataTypeArray::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const ColumnArray & column_array = static_cast(column); + const ColumnArray & column_array = assert_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); size_t offset = offsets[row_num - 1]; @@ -433,7 +434,7 @@ void DataTypeArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr, cons void DataTypeArray::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const { - const ColumnArray & column_array = static_cast(column); + const ColumnArray & column_array = assert_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); size_t offset = offsets[row_num - 1] + value_index; size_t next_offset = offsets[row_num]; @@ -453,7 +454,7 @@ void DataTypeArray::serializeProtobuf(const IColumn & column, size_t row_num, Pr void DataTypeArray::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const { row_added = false; - ColumnArray & column_array = static_cast(column); + ColumnArray & column_array = assert_cast(column); IColumn & nested_column = column_array.getData(); ColumnArray::Offsets & offsets = column_array.getOffsets(); size_t old_size = offsets.size(); diff --git a/dbms/src/DataTypes/DataTypeDate.cpp b/dbms/src/DataTypes/DataTypeDate.cpp index 0b1f502b694..2c1dfcbb0fe 100644 --- a/dbms/src/DataTypes/DataTypeDate.cpp +++ b/dbms/src/DataTypes/DataTypeDate.cpp @@ -7,13 +7,15 @@ #include #include +#include + namespace DB { void DataTypeDate::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeDateText(DayNum(static_cast(column).getData()[row_num]), ostr); + writeDateText(DayNum(assert_cast(column).getData()[row_num]), ostr); } void DataTypeDate::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -25,7 +27,7 @@ void DataTypeDate::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, c { DayNum x; readDateText(x, istr); - static_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(x); } void DataTypeDate::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -46,7 +48,7 @@ void DataTypeDate::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, co assertChar('\'', istr); readDateText(x, istr); assertChar('\'', istr); - static_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. + assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. } void DataTypeDate::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -62,7 +64,7 @@ void DataTypeDate::deserializeTextJSON(IColumn & column, ReadBuffer & istr, cons assertChar('"', istr); readDateText(x, istr); assertChar('"', istr); - static_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(x); } void DataTypeDate::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -76,14 +78,14 @@ void DataTypeDate::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const { LocalDate value; readCSV(value, istr); - static_cast(column).getData().push_back(value.getDayNum()); + assert_cast(column).getData().push_back(value.getDayNum()); } void DataTypeDate::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const { if (value_index) return; - value_index = static_cast(protobuf.writeDate(DayNum(static_cast(column).getData()[row_num]))); + value_index = static_cast(protobuf.writeDate(DayNum(assert_cast(column).getData()[row_num]))); } void DataTypeDate::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const @@ -93,7 +95,7 @@ void DataTypeDate::deserializeProtobuf(IColumn & column, ProtobufReader & protob if (!protobuf.readDate(d)) return; - auto & container = static_cast(column).getData(); + auto & container = assert_cast(column).getData(); if (allow_add_row) { container.emplace_back(d); diff --git a/dbms/src/DataTypes/DataTypeDateTime.cpp b/dbms/src/DataTypes/DataTypeDateTime.cpp index a6b8f0da92a..867833b4b2d 100644 --- a/dbms/src/DataTypes/DataTypeDateTime.cpp +++ b/dbms/src/DataTypes/DataTypeDateTime.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -39,7 +40,7 @@ std::string DataTypeDateTime::doGetName() const void DataTypeDateTime::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeDateTimeText(static_cast(column).getData()[row_num], ostr, time_zone); + writeDateTimeText(assert_cast(column).getData()[row_num], ostr, time_zone); } void DataTypeDateTime::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -71,7 +72,7 @@ void DataTypeDateTime::deserializeTextEscaped(IColumn & column, ReadBuffer & ist { time_t x; readText(x, istr, settings, time_zone, utc_time_zone); - static_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(x); } void DataTypeDateTime::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -93,7 +94,7 @@ void DataTypeDateTime::deserializeTextQuoted(IColumn & column, ReadBuffer & istr { readIntText(x, istr); } - static_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. + assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. } void DataTypeDateTime::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -115,7 +116,7 @@ void DataTypeDateTime::deserializeTextJSON(IColumn & column, ReadBuffer & istr, { readIntText(x, istr); } - static_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(x); } void DataTypeDateTime::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -142,14 +143,14 @@ void DataTypeDateTime::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c if (maybe_quote == '\'' || maybe_quote == '\"') assertChar(maybe_quote, istr); - static_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(x); } void DataTypeDateTime::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const { if (value_index) return; - value_index = static_cast(protobuf.writeDateTime(static_cast(column).getData()[row_num])); + value_index = static_cast(protobuf.writeDateTime(assert_cast(column).getData()[row_num])); } void DataTypeDateTime::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const @@ -159,7 +160,7 @@ void DataTypeDateTime::deserializeProtobuf(IColumn & column, ProtobufReader & pr if (!protobuf.readDateTime(t)) return; - auto & container = static_cast(column).getData(); + auto & container = assert_cast(column).getData(); if (allow_add_row) { container.emplace_back(t); diff --git a/dbms/src/DataTypes/DataTypeEnum.cpp b/dbms/src/DataTypes/DataTypeEnum.cpp index 36a26540cda..cffc29feaf8 100644 --- a/dbms/src/DataTypes/DataTypeEnum.cpp +++ b/dbms/src/DataTypes/DataTypeEnum.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -120,7 +121,7 @@ void DataTypeEnum::deserializeBinary(Field & field, ReadBuffer & istr) con template void DataTypeEnum::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - writeBinary(static_cast(column).getData()[row_num], ostr); + writeBinary(assert_cast(column).getData()[row_num], ostr); } template @@ -128,19 +129,19 @@ void DataTypeEnum::deserializeBinary(IColumn & column, ReadBuffer & istr) { typename ColumnType::value_type x; readBinary(x, istr); - static_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(x); } template void DataTypeEnum::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeString(getNameForValue(static_cast(column).getData()[row_num]), ostr); + writeString(getNameForValue(assert_cast(column).getData()[row_num]), ostr); } template void DataTypeEnum::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeEscapedString(getNameForValue(static_cast(column).getData()[row_num]), ostr); + writeEscapedString(getNameForValue(assert_cast(column).getData()[row_num]), ostr); } template @@ -149,13 +150,13 @@ void DataTypeEnum::deserializeTextEscaped(IColumn & column, ReadBuffer & i /// NOTE It would be nice to do without creating a temporary object - at least extract std::string out. std::string field_name; readEscapedString(field_name, istr); - static_cast(column).getData().push_back(getValue(StringRef(field_name))); + assert_cast(column).getData().push_back(getValue(StringRef(field_name))); } template void DataTypeEnum::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeQuotedString(getNameForValue(static_cast(column).getData()[row_num]), ostr); + writeQuotedString(getNameForValue(assert_cast(column).getData()[row_num]), ostr); } template @@ -163,7 +164,7 @@ void DataTypeEnum::deserializeTextQuoted(IColumn & column, ReadBuffer & is { std::string field_name; readQuotedStringWithSQLStyle(field_name, istr); - static_cast(column).getData().push_back(getValue(StringRef(field_name))); + assert_cast(column).getData().push_back(getValue(StringRef(field_name))); } template @@ -171,19 +172,19 @@ void DataTypeEnum::deserializeWholeText(IColumn & column, ReadBuffer & ist { std::string field_name; readString(field_name, istr); - static_cast(column).getData().push_back(getValue(StringRef(field_name))); + assert_cast(column).getData().push_back(getValue(StringRef(field_name))); } template void DataTypeEnum::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - writeJSONString(getNameForValue(static_cast(column).getData()[row_num]), ostr, settings); + writeJSONString(getNameForValue(assert_cast(column).getData()[row_num]), ostr, settings); } template void DataTypeEnum::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeXMLString(getNameForValue(static_cast(column).getData()[row_num]), ostr); + writeXMLString(getNameForValue(assert_cast(column).getData()[row_num]), ostr); } template @@ -191,13 +192,13 @@ void DataTypeEnum::deserializeTextJSON(IColumn & column, ReadBuffer & istr { std::string field_name; readJSONString(field_name, istr); - static_cast(column).getData().push_back(getValue(StringRef(field_name))); + assert_cast(column).getData().push_back(getValue(StringRef(field_name))); } template void DataTypeEnum::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeCSVString(getNameForValue(static_cast(column).getData()[row_num]), ostr); + writeCSVString(getNameForValue(assert_cast(column).getData()[row_num]), ostr); } template @@ -205,7 +206,7 @@ void DataTypeEnum::deserializeTextCSV(IColumn & column, ReadBuffer & istr, { std::string field_name; readCSVString(field_name, istr, settings.csv); - static_cast(column).getData().push_back(getValue(StringRef(field_name))); + assert_cast(column).getData().push_back(getValue(StringRef(field_name))); } template @@ -238,7 +239,7 @@ void DataTypeEnum::serializeProtobuf(const IColumn & column, size_t row_nu if (value_index) return; protobuf.prepareEnumMapping(values); - value_index = static_cast(protobuf.writeEnum(static_cast(column).getData()[row_num])); + value_index = static_cast(protobuf.writeEnum(assert_cast(column).getData()[row_num])); } template @@ -250,7 +251,7 @@ void DataTypeEnum::deserializeProtobuf(IColumn & column, ProtobufReader & if (!protobuf.readEnum(value)) return; - auto & container = static_cast(column).getData(); + auto & container = assert_cast(column).getData(); if (allow_add_row) { container.emplace_back(value); @@ -269,7 +270,7 @@ Field DataTypeEnum::getDefault() const template void DataTypeEnum::insertDefaultInto(IColumn & column) const { - static_cast(column).getData().push_back(values.front().second); + assert_cast(column).getData().push_back(values.front().second); } template diff --git a/dbms/src/DataTypes/DataTypeFixedString.cpp b/dbms/src/DataTypes/DataTypeFixedString.cpp index 34970fdaae9..7de245b4589 100644 --- a/dbms/src/DataTypes/DataTypeFixedString.cpp +++ b/dbms/src/DataTypes/DataTypeFixedString.cpp @@ -17,6 +17,7 @@ #include #include +#include namespace DB @@ -58,13 +59,13 @@ void DataTypeFixedString::deserializeBinary(Field & field, ReadBuffer & istr) co void DataTypeFixedString::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - ostr.write(reinterpret_cast(&static_cast(column).getChars()[n * row_num]), n); + ostr.write(reinterpret_cast(&assert_cast(column).getChars()[n * row_num]), n); } void DataTypeFixedString::deserializeBinary(IColumn & column, ReadBuffer & istr) const { - ColumnFixedString::Chars & data = static_cast(column).getChars(); + ColumnFixedString::Chars & data = assert_cast(column).getChars(); size_t old_size = data.size(); data.resize(old_size + n); try @@ -112,13 +113,13 @@ void DataTypeFixedString::deserializeBinaryBulk(IColumn & column, ReadBuffer & i void DataTypeFixedString::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeString(reinterpret_cast(&static_cast(column).getChars()[n * row_num]), n, ostr); + writeString(reinterpret_cast(&assert_cast(column).getChars()[n * row_num]), n, ostr); } void DataTypeFixedString::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - const char * pos = reinterpret_cast(&static_cast(column).getChars()[n * row_num]); + const char * pos = reinterpret_cast(&assert_cast(column).getChars()[n * row_num]); writeAnyEscapedString<'\''>(pos, pos + n, ostr); } @@ -165,7 +166,7 @@ void DataTypeFixedString::deserializeTextEscaped(IColumn & column, ReadBuffer & void DataTypeFixedString::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - const char * pos = reinterpret_cast(&static_cast(column).getChars()[n * row_num]); + const char * pos = reinterpret_cast(&assert_cast(column).getChars()[n * row_num]); writeAnyQuotedString<'\''>(pos, pos + n, ostr); } @@ -184,7 +185,7 @@ void DataTypeFixedString::deserializeWholeText(IColumn & column, ReadBuffer & is void DataTypeFixedString::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const char * pos = reinterpret_cast(&static_cast(column).getChars()[n * row_num]); + const char * pos = reinterpret_cast(&assert_cast(column).getChars()[n * row_num]); writeJSONString(pos, pos + n, ostr, settings); } @@ -197,14 +198,14 @@ void DataTypeFixedString::deserializeTextJSON(IColumn & column, ReadBuffer & ist void DataTypeFixedString::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - const char * pos = reinterpret_cast(&static_cast(column).getChars()[n * row_num]); + const char * pos = reinterpret_cast(&assert_cast(column).getChars()[n * row_num]); writeXMLString(pos, pos + n, ostr); } void DataTypeFixedString::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - const char * pos = reinterpret_cast(&static_cast(column).getChars()[n * row_num]); + const char * pos = reinterpret_cast(&assert_cast(column).getChars()[n * row_num]); writeCSVString(pos, pos + n, ostr); } @@ -219,7 +220,7 @@ void DataTypeFixedString::serializeProtobuf(const IColumn & column, size_t row_n { if (value_index) return; - const char * pos = reinterpret_cast(&static_cast(column).getChars()[n * row_num]); + const char * pos = reinterpret_cast(&assert_cast(column).getChars()[n * row_num]); value_index = static_cast(protobuf.writeString(StringRef(pos, n))); } @@ -227,7 +228,7 @@ void DataTypeFixedString::serializeProtobuf(const IColumn & column, size_t row_n void DataTypeFixedString::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const { row_added = false; - auto & column_string = static_cast(column); + auto & column_string = assert_cast(column); ColumnFixedString::Chars & data = column_string.getChars(); size_t old_size = data.size(); try diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.cpp b/dbms/src/DataTypes/DataTypeLowCardinality.cpp index 812c5d04032..afb0a206a7a 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.cpp +++ b/dbms/src/DataTypes/DataTypeLowCardinality.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -800,7 +801,7 @@ namespace void operator()() { if (typeid_cast *>(&keys_type)) - column = creator(static_cast *>(nullptr)); + column = creator(assert_cast *>(nullptr)); } }; } @@ -814,13 +815,13 @@ MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDat type = nullable_type->getNestedType().get(); if (isString(type)) - return creator(static_cast(nullptr)); + return creator(assert_cast(nullptr)); if (isFixedString(type)) - return creator(static_cast(nullptr)); + return creator(assert_cast(nullptr)); if (typeid_cast(type)) - return creator(static_cast *>(nullptr)); + return creator(assert_cast *>(nullptr)); if (typeid_cast(type)) - return creator(static_cast *>(nullptr)); + return creator(assert_cast *>(nullptr)); if (isColumnedAsNumber(type)) { MutableColumnUniquePtr column; diff --git a/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp b/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp index e04b3a834d5..6ebd6586c18 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp +++ b/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp @@ -7,6 +7,9 @@ #include #include +#include + + namespace DB { @@ -110,7 +113,7 @@ ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const Data if (from_type->equals(*low_cardinality_type->getDictionaryType())) { auto col = low_cardinality_type->createColumn(); - static_cast(*col).insertRangeFromFullColumn(*column, 0, column->size()); + assert_cast(*col).insertRangeFromFullColumn(*column, 0, column->size()); return col; } } diff --git a/dbms/src/DataTypes/DataTypeNullable.cpp b/dbms/src/DataTypes/DataTypeNullable.cpp index 44df56b3d3a..376b4b6dad0 100644 --- a/dbms/src/DataTypes/DataTypeNullable.cpp +++ b/dbms/src/DataTypes/DataTypeNullable.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -84,7 +85,7 @@ void DataTypeNullable::serializeBinaryBulkWithMultipleStreams( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - const ColumnNullable & col = static_cast(column); + const ColumnNullable & col = assert_cast(column); col.checkConsistency(); /// First serialize null map. @@ -105,7 +106,7 @@ void DataTypeNullable::deserializeBinaryBulkWithMultipleStreams( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { - ColumnNullable & col = static_cast(column); + ColumnNullable & col = assert_cast(column); settings.path.push_back(Substream::NullMap); if (auto stream = settings.getter(settings.path)) @@ -119,7 +120,7 @@ void DataTypeNullable::deserializeBinaryBulkWithMultipleStreams( void DataTypeNullable::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - const ColumnNullable & col = static_cast(column); + const ColumnNullable & col = assert_cast(column); bool is_null = col.isNullAt(row_num); writeBinary(is_null, ostr); @@ -134,7 +135,7 @@ static void safeDeserialize( IColumn & column, CheckForNull && check_for_null, DeserializeNested && deserialize_nested) { - ColumnNullable & col = static_cast(column); + ColumnNullable & col = assert_cast(column); if (check_for_null()) { @@ -167,7 +168,7 @@ void DataTypeNullable::deserializeBinary(IColumn & column, ReadBuffer & istr) co void DataTypeNullable::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const ColumnNullable & col = static_cast(column); + const ColumnNullable & col = assert_cast(column); if (col.isNullAt(row_num)) writeCString("\\N", ostr); @@ -235,7 +236,7 @@ void DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & ist void DataTypeNullable::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const ColumnNullable & col = static_cast(column); + const ColumnNullable & col = assert_cast(column); if (col.isNullAt(row_num)) writeCString("NULL", ostr); @@ -262,7 +263,7 @@ void DataTypeNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, void DataTypeNullable::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const ColumnNullable & col = static_cast(column); + const ColumnNullable & col = assert_cast(column); if (col.isNullAt(row_num)) writeCString("\\N", ostr); @@ -342,7 +343,7 @@ void DataTypeNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c void DataTypeNullable::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const ColumnNullable & col = static_cast(column); + const ColumnNullable & col = assert_cast(column); /// In simple text format (like 'Pretty' format) (these formats are suitable only for output and cannot be parsed back), /// data is printed without escaping. @@ -358,7 +359,7 @@ void DataTypeNullable::serializeText(const IColumn & column, size_t row_num, Wri void DataTypeNullable::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const ColumnNullable & col = static_cast(column); + const ColumnNullable & col = assert_cast(column); if (col.isNullAt(row_num)) writeCString("null", ostr); @@ -375,7 +376,7 @@ void DataTypeNullable::deserializeTextJSON(IColumn & column, ReadBuffer & istr, void DataTypeNullable::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const ColumnNullable & col = static_cast(column); + const ColumnNullable & col = assert_cast(column); if (col.isNullAt(row_num)) writeCString("\\N", ostr); @@ -385,14 +386,14 @@ void DataTypeNullable::serializeTextXML(const IColumn & column, size_t row_num, void DataTypeNullable::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const { - const ColumnNullable & col = static_cast(column); + const ColumnNullable & col = assert_cast(column); if (!col.isNullAt(row_num)) nested_data_type->serializeProtobuf(col.getNestedColumn(), row_num, protobuf, value_index); } void DataTypeNullable::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const { - ColumnNullable & col = static_cast(column); + ColumnNullable & col = assert_cast(column); IColumn & nested_column = col.getNestedColumn(); size_t old_size = nested_column.size(); try diff --git a/dbms/src/DataTypes/DataTypeNumberBase.cpp b/dbms/src/DataTypes/DataTypeNumberBase.cpp index d00846f6b13..ea0494cfee0 100644 --- a/dbms/src/DataTypes/DataTypeNumberBase.cpp +++ b/dbms/src/DataTypes/DataTypeNumberBase.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -17,7 +18,7 @@ namespace DB template void DataTypeNumberBase::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeText(static_cast &>(column).getData()[row_num], ostr); + writeText(assert_cast &>(column).getData()[row_num], ostr); } template @@ -30,7 +31,7 @@ void DataTypeNumberBase::deserializeText(IColumn & column, ReadBuffer & istr, else readText(x, istr); - static_cast &>(column).getData().push_back(x); + assert_cast &>(column).getData().push_back(x); } template @@ -64,7 +65,7 @@ static inline void writeDenormalNumber(T x, WriteBuffer & ostr) template void DataTypeNumberBase::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - auto x = static_cast &>(column).getData()[row_num]; + auto x = assert_cast &>(column).getData()[row_num]; bool is_finite = isFinite(x); const bool need_quote = (std::is_integral_v && (sizeof(T) == 8) && settings.json.quote_64bit_integers) @@ -132,7 +133,7 @@ void DataTypeNumberBase::deserializeTextJSON(IColumn & column, ReadBuffer & i assertChar('"', istr); } - static_cast &>(column).getData().push_back(x); + assert_cast &>(column).getData().push_back(x); } template @@ -140,7 +141,7 @@ void DataTypeNumberBase::deserializeTextCSV(IColumn & column, ReadBuffer & is { FieldType x; readCSV(x, istr); - static_cast &>(column).getData().push_back(x); + assert_cast &>(column).getData().push_back(x); } template @@ -168,7 +169,7 @@ void DataTypeNumberBase::deserializeBinary(Field & field, ReadBuffer & istr) template void DataTypeNumberBase::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - writeBinary(static_cast &>(column).getData()[row_num], ostr); + writeBinary(assert_cast &>(column).getData()[row_num], ostr); } template @@ -176,7 +177,7 @@ void DataTypeNumberBase::deserializeBinary(IColumn & column, ReadBuffer & ist { typename ColumnVector::value_type x; readBinary(x, istr); - static_cast &>(column).getData().push_back(x); + assert_cast &>(column).getData().push_back(x); } template @@ -209,7 +210,7 @@ void DataTypeNumberBase::serializeProtobuf(const IColumn & column, size_t row { if (value_index) return; - value_index = static_cast(protobuf.writeNumber(static_cast &>(column).getData()[row_num])); + value_index = static_cast(protobuf.writeNumber(assert_cast &>(column).getData()[row_num])); } diff --git a/dbms/src/DataTypes/DataTypeString.cpp b/dbms/src/DataTypes/DataTypeString.cpp index 5d104c76fef..dde0140c7af 100644 --- a/dbms/src/DataTypes/DataTypeString.cpp +++ b/dbms/src/DataTypes/DataTypeString.cpp @@ -5,6 +5,7 @@ #include #include +#include #include #include @@ -45,7 +46,7 @@ void DataTypeString::deserializeBinary(Field & field, ReadBuffer & istr) const void DataTypeString::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - const StringRef & s = static_cast(column).getDataAt(row_num); + const StringRef & s = assert_cast(column).getDataAt(row_num); writeVarUInt(s.size, ostr); writeString(s, ostr); } @@ -53,7 +54,7 @@ void DataTypeString::serializeBinary(const IColumn & column, size_t row_num, Wri void DataTypeString::deserializeBinary(IColumn & column, ReadBuffer & istr) const { - ColumnString & column_string = static_cast(column); + ColumnString & column_string = assert_cast(column); ColumnString::Chars & data = column_string.getChars(); ColumnString::Offsets & offsets = column_string.getOffsets(); @@ -211,20 +212,20 @@ void DataTypeString::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, void DataTypeString::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeString(static_cast(column).getDataAt(row_num), ostr); + writeString(assert_cast(column).getDataAt(row_num), ostr); } void DataTypeString::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeEscapedString(static_cast(column).getDataAt(row_num), ostr); + writeEscapedString(assert_cast(column).getDataAt(row_num), ostr); } template static inline void read(IColumn & column, Reader && reader) { - ColumnString & column_string = static_cast(column); + ColumnString & column_string = assert_cast(column); ColumnString::Chars & data = column_string.getChars(); ColumnString::Offsets & offsets = column_string.getOffsets(); size_t old_chars_size = data.size(); @@ -258,7 +259,7 @@ void DataTypeString::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, void DataTypeString::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeQuotedString(static_cast(column).getDataAt(row_num), ostr); + writeQuotedString(assert_cast(column).getDataAt(row_num), ostr); } @@ -270,7 +271,7 @@ void DataTypeString::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, void DataTypeString::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - writeJSONString(static_cast(column).getDataAt(row_num), ostr, settings); + writeJSONString(assert_cast(column).getDataAt(row_num), ostr, settings); } @@ -282,13 +283,13 @@ void DataTypeString::deserializeTextJSON(IColumn & column, ReadBuffer & istr, co void DataTypeString::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeXMLString(static_cast(column).getDataAt(row_num), ostr); + writeXMLString(assert_cast(column).getDataAt(row_num), ostr); } void DataTypeString::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeCSVString<>(static_cast(column).getDataAt(row_num), ostr); + writeCSVString<>(assert_cast(column).getDataAt(row_num), ostr); } @@ -302,14 +303,14 @@ void DataTypeString::serializeProtobuf(const IColumn & column, size_t row_num, P { if (value_index) return; - value_index = static_cast(protobuf.writeString(static_cast(column).getDataAt(row_num))); + value_index = static_cast(protobuf.writeString(assert_cast(column).getDataAt(row_num))); } void DataTypeString::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const { row_added = false; - auto & column_string = static_cast(column); + auto & column_string = assert_cast(column); ColumnString::Chars & data = column_string.getChars(); ColumnString::Offsets & offsets = column_string.getOffsets(); size_t old_size = offsets.size(); diff --git a/dbms/src/DataTypes/DataTypeTuple.cpp b/dbms/src/DataTypes/DataTypeTuple.cpp index 8f52b5fd3ff..39af1ead0fd 100644 --- a/dbms/src/DataTypes/DataTypeTuple.cpp +++ b/dbms/src/DataTypes/DataTypeTuple.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -88,12 +89,12 @@ std::string DataTypeTuple::doGetName() const static inline IColumn & extractElementColumn(IColumn & column, size_t idx) { - return static_cast(column).getColumn(idx); + return assert_cast(column).getColumn(idx); } static inline const IColumn & extractElementColumn(const IColumn & column, size_t idx) { - return static_cast(column).getColumn(idx); + return assert_cast(column).getColumn(idx); } diff --git a/dbms/src/DataTypes/DataTypeUUID.cpp b/dbms/src/DataTypes/DataTypeUUID.cpp index ff2ac0db0d3..87e306cf477 100644 --- a/dbms/src/DataTypes/DataTypeUUID.cpp +++ b/dbms/src/DataTypes/DataTypeUUID.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -12,14 +13,14 @@ namespace DB void DataTypeUUID::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeText(UUID(static_cast(column).getData()[row_num]), ostr); + writeText(UUID(assert_cast(column).getData()[row_num]), ostr); } void DataTypeUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { UUID x; readText(x, istr); - static_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(x); } void DataTypeUUID::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -40,7 +41,7 @@ void DataTypeUUID::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, co assertChar('\'', istr); readText(x, istr); assertChar('\'', istr); - static_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. + assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. } void DataTypeUUID::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -56,7 +57,7 @@ void DataTypeUUID::deserializeTextJSON(IColumn & column, ReadBuffer & istr, cons assertChar('"', istr); readText(x, istr); assertChar('"', istr); - static_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(x); } void DataTypeUUID::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -70,14 +71,14 @@ void DataTypeUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const { UUID value; readCSV(value, istr); - static_cast(column).getData().push_back(value); + assert_cast(column).getData().push_back(value); } void DataTypeUUID::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const { if (value_index) return; - value_index = static_cast(protobuf.writeUUID(UUID(static_cast(column).getData()[row_num]))); + value_index = static_cast(protobuf.writeUUID(UUID(assert_cast(column).getData()[row_num]))); } void DataTypeUUID::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const @@ -87,7 +88,7 @@ void DataTypeUUID::deserializeProtobuf(IColumn & column, ProtobufReader & protob if (!protobuf.readUUID(uuid)) return; - auto & container = static_cast(column).getData(); + auto & container = assert_cast(column).getData(); if (allow_add_row) { container.emplace_back(uuid); diff --git a/dbms/src/DataTypes/DataTypesDecimal.cpp b/dbms/src/DataTypes/DataTypesDecimal.cpp index 920d7b360bd..be5042fa57e 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.cpp +++ b/dbms/src/DataTypes/DataTypesDecimal.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -11,6 +12,7 @@ #include #include + namespace DB { @@ -47,7 +49,7 @@ bool DataTypeDecimal::equals(const IDataType & rhs) const template void DataTypeDecimal::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - T value = static_cast(column).getData()[row_num]; + T value = assert_cast(column).getData()[row_num]; writeText(value, scale, ostr); } @@ -76,7 +78,7 @@ void DataTypeDecimal::deserializeText(IColumn & column, ReadBuffer & istr, co { T x; readText(x, istr); - static_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(x); } template @@ -84,7 +86,7 @@ void DataTypeDecimal::deserializeTextCSV(IColumn & column, ReadBuffer & istr, { T x; readText(x, istr, true); - static_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(x); } template @@ -109,7 +111,7 @@ void DataTypeDecimal::serializeBinary(const Field & field, WriteBuffer & ostr template void DataTypeDecimal::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - const FieldType & x = static_cast(column).getData()[row_num]; + const FieldType & x = assert_cast(column).getData()[row_num]; writeBinary(x, ostr); } @@ -140,7 +142,7 @@ void DataTypeDecimal::deserializeBinary(IColumn & column, ReadBuffer & istr) { typename FieldType::NativeType x; readBinary(x, istr); - static_cast(column).getData().push_back(FieldType(x)); + assert_cast(column).getData().push_back(FieldType(x)); } template @@ -159,7 +161,7 @@ void DataTypeDecimal::serializeProtobuf(const IColumn & column, size_t row_nu { if (value_index) return; - value_index = static_cast(protobuf.writeDecimal(static_cast(column).getData()[row_num], scale)); + value_index = static_cast(protobuf.writeDecimal(assert_cast(column).getData()[row_num], scale)); } @@ -171,7 +173,7 @@ void DataTypeDecimal::deserializeProtobuf(IColumn & column, ProtobufReader & if (!protobuf.readDecimal(decimal, precision, scale)) return; - auto & container = static_cast(column).getData(); + auto & container = assert_cast(column).getData(); if (allow_add_row) { container.emplace_back(decimal); diff --git a/dbms/src/DataTypes/NestedUtils.cpp b/dbms/src/DataTypes/NestedUtils.cpp index b3399ddfc1c..e988490a487 100644 --- a/dbms/src/DataTypes/NestedUtils.cpp +++ b/dbms/src/DataTypes/NestedUtils.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include @@ -94,7 +95,7 @@ Block flatten(const Block & block) bool is_const = isColumnConst(*elem.column); const ColumnArray * column_array; if (is_const) - column_array = typeid_cast(&static_cast(*elem.column).getDataColumn()); + column_array = typeid_cast(&assert_cast(*elem.column).getDataColumn()); else column_array = typeid_cast(elem.column.get()); @@ -181,8 +182,8 @@ void validateArraySizes(const Block & block) /// It's not the first column of Nested data structure. if (!inserted) { - const ColumnArray & first_array_column = static_cast(*block.getByPosition(it->second).column); - const ColumnArray & another_array_column = static_cast(*elem.column); + const ColumnArray & first_array_column = assert_cast(*block.getByPosition(it->second).column); + const ColumnArray & another_array_column = assert_cast(*elem.column); if (!first_array_column.hasEqualOffsets(another_array_column)) throw Exception("Elements '" + block.getByPosition(it->second).name diff --git a/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp b/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp index 363147c484e..b1c2f67cc86 100644 --- a/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp +++ b/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp @@ -1,24 +1,25 @@ #include "config_core.h" #if USE_POCO_MONGODB -# include -# include -# include +#include +#include +#include -# include -# include -# include -# include +#include +#include +#include +#include -# include -# include -# include -# include -# include -# include -# include -# include "DictionaryStructure.h" -# include "MongoDBBlockInputStream.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include "DictionaryStructure.h" +#include "MongoDBBlockInputStream.h" namespace DB @@ -54,26 +55,26 @@ namespace switch (value.type()) { case Poco::MongoDB::ElementTraits::TypeId: - static_cast &>(column).getData().push_back( + assert_cast &>(column).getData().push_back( static_cast &>(value).value()); break; case Poco::MongoDB::ElementTraits::TypeId: - static_cast &>(column).getData().push_back( + assert_cast &>(column).getData().push_back( static_cast &>(value).value()); break; case Poco::MongoDB::ElementTraits::TypeId: - static_cast &>(column).getData().push_back( + assert_cast &>(column).getData().push_back( static_cast &>(value).value()); break; case Poco::MongoDB::ElementTraits::TypeId: - static_cast &>(column).getData().push_back( + assert_cast &>(column).getData().push_back( static_cast &>(value).value()); break; case Poco::MongoDB::ElementTraits::TypeId: - static_cast &>(column).getData().emplace_back(); + assert_cast &>(column).getData().emplace_back(); break; case Poco::MongoDB::ElementTraits::TypeId: - static_cast &>(column).getData().push_back( + assert_cast &>(column).getData().push_back( parse(static_cast &>(value).value())); break; default: @@ -123,13 +124,13 @@ namespace if (value.type() == Poco::MongoDB::ElementTraits::TypeId) { std::string string_id = value.toString(); - static_cast(column).insertDataWithTerminatingZero(string_id.data(), string_id.size() + 1); + assert_cast(column).insertDataWithTerminatingZero(string_id.data(), string_id.size() + 1); break; } else if (value.type() == Poco::MongoDB::ElementTraits::TypeId) { String string = static_cast &>(value).value(); - static_cast(column).insertDataWithTerminatingZero(string.data(), string.size() + 1); + assert_cast(column).insertDataWithTerminatingZero(string.data(), string.size() + 1); break; } @@ -143,7 +144,7 @@ namespace throw Exception{"Type mismatch, expected Timestamp, got type id = " + toString(value.type()) + " for column " + name, ErrorCodes::TYPE_MISMATCH}; - static_cast(column).getData().push_back(UInt16{DateLUT::instance().toDayNum( + assert_cast(column).getData().push_back(UInt16{DateLUT::instance().toDayNum( static_cast &>(value).value().epochTime())}); break; } @@ -154,7 +155,7 @@ namespace throw Exception{"Type mismatch, expected Timestamp, got type id = " + toString(value.type()) + " for column " + name, ErrorCodes::TYPE_MISMATCH}; - static_cast(column).getData().push_back( + assert_cast(column).getData().push_back( static_cast &>(value).value().epochTime()); break; } @@ -163,7 +164,7 @@ namespace if (value.type() == Poco::MongoDB::ElementTraits::TypeId) { String string = static_cast &>(value).value(); - static_cast(column).getData().push_back(parse(string)); + assert_cast(column).getData().push_back(parse(string)); } else throw Exception{"Type mismatch, expected String (UUID), got type id = " + toString(value.type()) + " for column " @@ -209,7 +210,7 @@ Block MongoDBBlockInputStream::readImpl() { if (description.types[idx].second) { - ColumnNullable & column_nullable = static_cast(*columns[idx]); + ColumnNullable & column_nullable = assert_cast(*columns[idx]); insertValue(column_nullable.getNestedColumn(), description.types[idx].first, *value, name); column_nullable.getNullMapData().emplace_back(0); } diff --git a/dbms/src/Dictionaries/TrieDictionary.cpp b/dbms/src/Dictionaries/TrieDictionary.cpp index 8ab7d2f34af..7a5adee63b6 100644 --- a/dbms/src/Dictionaries/TrieDictionary.cpp +++ b/dbms/src/Dictionaries/TrieDictionary.cpp @@ -1,8 +1,8 @@ #include "TrieDictionary.h" -#include #include #include #include +#include #include #include #include @@ -731,8 +731,8 @@ BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_nam auto getView = [](const Columns & columns, const std::vector & dict_attributes) { auto column = ColumnString::create(); - const auto & ip_column = static_cast(*columns.front()); - const auto & mask_column = static_cast &>(*columns.back()); + const auto & ip_column = assert_cast(*columns.front()); + const auto & mask_column = assert_cast &>(*columns.back()); char buffer[48]; for (size_t row : ext::range(0, ip_column.size())) { diff --git a/dbms/src/Formats/MySQLBlockInputStream.cpp b/dbms/src/Formats/MySQLBlockInputStream.cpp index 1896dbcc4b6..dcf609f62ce 100644 --- a/dbms/src/Formats/MySQLBlockInputStream.cpp +++ b/dbms/src/Formats/MySQLBlockInputStream.cpp @@ -1,14 +1,15 @@ #include "config_core.h" #if USE_MYSQL -# include -# include -# include -# include -# include -# include -# include -# include "MySQLBlockInputStream.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include "MySQLBlockInputStream.h" namespace DB @@ -41,46 +42,46 @@ namespace switch (type) { case ValueType::vtUInt8: - static_cast(column).insertValue(value.getUInt()); + assert_cast(column).insertValue(value.getUInt()); break; case ValueType::vtUInt16: - static_cast(column).insertValue(value.getUInt()); + assert_cast(column).insertValue(value.getUInt()); break; case ValueType::vtUInt32: - static_cast(column).insertValue(value.getUInt()); + assert_cast(column).insertValue(value.getUInt()); break; case ValueType::vtUInt64: - static_cast(column).insertValue(value.getUInt()); + assert_cast(column).insertValue(value.getUInt()); break; case ValueType::vtInt8: - static_cast(column).insertValue(value.getInt()); + assert_cast(column).insertValue(value.getInt()); break; case ValueType::vtInt16: - static_cast(column).insertValue(value.getInt()); + assert_cast(column).insertValue(value.getInt()); break; case ValueType::vtInt32: - static_cast(column).insertValue(value.getInt()); + assert_cast(column).insertValue(value.getInt()); break; case ValueType::vtInt64: - static_cast(column).insertValue(value.getInt()); + assert_cast(column).insertValue(value.getInt()); break; case ValueType::vtFloat32: - static_cast(column).insertValue(value.getDouble()); + assert_cast(column).insertValue(value.getDouble()); break; case ValueType::vtFloat64: - static_cast(column).insertValue(value.getDouble()); + assert_cast(column).insertValue(value.getDouble()); break; case ValueType::vtString: - static_cast(column).insertData(value.data(), value.size()); + assert_cast(column).insertData(value.data(), value.size()); break; case ValueType::vtDate: - static_cast(column).insertValue(UInt16(value.getDate().getDayNum())); + assert_cast(column).insertValue(UInt16(value.getDate().getDayNum())); break; case ValueType::vtDateTime: - static_cast(column).insertValue(UInt32(value.getDateTime())); + assert_cast(column).insertValue(UInt32(value.getDateTime())); break; case ValueType::vtUUID: - static_cast(column).insert(parse(value.data(), value.size())); + assert_cast(column).insert(parse(value.data(), value.size())); break; } } @@ -113,7 +114,7 @@ Block MySQLBlockInputStream::readImpl() { if (description.types[idx].second) { - ColumnNullable & column_nullable = static_cast(*columns[idx]); + ColumnNullable & column_nullable = assert_cast(*columns[idx]); insertValue(column_nullable.getNestedColumn(), description.types[idx].first, value); column_nullable.getNullMapData().emplace_back(0); } diff --git a/dbms/src/Functions/FunctionBinaryArithmetic.h b/dbms/src/Functions/FunctionBinaryArithmetic.h index 5dfc2a10c5a..afe65a8adad 100644 --- a/dbms/src/Functions/FunctionBinaryArithmetic.h +++ b/dbms/src/Functions/FunctionBinaryArithmetic.h @@ -23,6 +23,7 @@ #include "castTypeToEither.h" #include "FunctionFactory.h" #include +#include #include #if USE_EMBEDDED_COMPILER @@ -549,7 +550,7 @@ class FunctionBinaryArithmetic : public IFunction const IColumn & agg_state_column = *block.getByPosition(new_arguments[0]).column; bool agg_state_is_const = isColumnConst(agg_state_column); const ColumnAggregateFunction & column = typeid_cast( - agg_state_is_const ? static_cast(agg_state_column).getDataColumn() : agg_state_column); + agg_state_is_const ? assert_cast(agg_state_column).getDataColumn() : agg_state_column); AggregateFunctionPtr function = column.getAggregateFunction(); @@ -613,9 +614,9 @@ class FunctionBinaryArithmetic : public IFunction bool rhs_is_const = isColumnConst(rhs_column); const ColumnAggregateFunction & lhs = typeid_cast( - lhs_is_const ? static_cast(lhs_column).getDataColumn() : lhs_column); + lhs_is_const ? assert_cast(lhs_column).getDataColumn() : lhs_column); const ColumnAggregateFunction & rhs = typeid_cast( - rhs_is_const ? static_cast(rhs_column).getDataColumn() : rhs_column); + rhs_is_const ? assert_cast(rhs_column).getDataColumn() : rhs_column); AggregateFunctionPtr function = lhs.getAggregateFunction(); diff --git a/dbms/src/Functions/FunctionHelpers.cpp b/dbms/src/Functions/FunctionHelpers.cpp index 1f85b062b85..212a107e37c 100644 --- a/dbms/src/Functions/FunctionHelpers.cpp +++ b/dbms/src/Functions/FunctionHelpers.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -22,7 +23,7 @@ const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * co if (!isColumnConst(*column)) return {}; - const ColumnConst * res = static_cast(column); + const ColumnConst * res = assert_cast(column); if (checkColumn(&res->getDataColumn()) || checkColumn(&res->getDataColumn())) @@ -34,7 +35,7 @@ const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * co Columns convertConstTupleToConstantElements(const ColumnConst & column) { - const ColumnTuple & src_tuple = static_cast(column.getDataColumn()); + const ColumnTuple & src_tuple = assert_cast(column.getDataColumn()); const auto & src_tuple_columns = src_tuple.getColumns(); size_t tuple_size = src_tuple_columns.size(); size_t rows = column.size(); diff --git a/dbms/src/Functions/FunctionHelpers.h b/dbms/src/Functions/FunctionHelpers.h index 2d5e77b10b8..ac116510b7e 100644 --- a/dbms/src/Functions/FunctionHelpers.h +++ b/dbms/src/Functions/FunctionHelpers.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -28,7 +29,7 @@ const ColumnConst * checkAndGetColumnConst(const IColumn * column) if (!column || !isColumnConst(*column)) return {}; - const ColumnConst * res = static_cast(column); + const ColumnConst * res = assert_cast(column); if (!checkColumn(&res->getDataColumn())) return {}; diff --git a/dbms/src/Functions/FunctionsBitmap.h b/dbms/src/Functions/FunctionsBitmap.h index 22964605b8b..e87adae8064 100644 --- a/dbms/src/Functions/FunctionsBitmap.h +++ b/dbms/src/Functions/FunctionsBitmap.h @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB @@ -195,7 +196,7 @@ public: // input data const auto & return_type = block.getByPosition(result).type; auto res_ptr = return_type->createColumn(); - ColumnArray & res = static_cast(*res_ptr); + ColumnArray & res = assert_cast(*res_ptr); IColumn & res_data = res.getData(); ColumnArray::Offsets & res_offsets = res.getOffsets(); diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index c3d29339d55..efe39bde5b9 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -838,7 +839,7 @@ private: throw Exception("String is too long for Date: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); ColumnPtr parsed_const_date_holder = DataTypeDate().createColumnConst(input_rows_count, date); - const ColumnConst * parsed_const_date = static_cast(parsed_const_date_holder.get()); + const ColumnConst * parsed_const_date = assert_cast(parsed_const_date_holder.get()); executeNumLeftType(block, result, left_is_num ? col_left_untyped : parsed_const_date, left_is_num ? parsed_const_date : col_right_untyped); @@ -852,7 +853,7 @@ private: throw Exception("String is too long for DateTime: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); ColumnPtr parsed_const_date_time_holder = DataTypeDateTime().createColumnConst(input_rows_count, UInt64(date_time)); - const ColumnConst * parsed_const_date_time = static_cast(parsed_const_date_time_holder.get()); + const ColumnConst * parsed_const_date_time = assert_cast(parsed_const_date_time_holder.get()); executeNumLeftType(block, result, left_is_num ? col_left_untyped : parsed_const_date_time, left_is_num ? parsed_const_date_time : col_right_untyped); @@ -866,7 +867,7 @@ private: throw Exception("String is too long for UUID: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); ColumnPtr parsed_const_uuid_holder = DataTypeUUID().createColumnConst(input_rows_count, uuid); - const ColumnConst * parsed_const_uuid = static_cast(parsed_const_uuid_holder.get()); + const ColumnConst * parsed_const_uuid = assert_cast(parsed_const_uuid_holder.get()); executeNumLeftType(block, result, left_is_num ? col_left_untyped : parsed_const_uuid, left_is_num ? parsed_const_uuid : col_right_untyped); @@ -932,12 +933,12 @@ private: if (x_const) x_columns = convertConstTupleToConstantElements(*x_const); else - x_columns = static_cast(*c0.column).getColumnsCopy(); + x_columns = assert_cast(*c0.column).getColumnsCopy(); if (y_const) y_columns = convertConstTupleToConstantElements(*y_const); else - y_columns = static_cast(*c1.column).getColumnsCopy(); + y_columns = assert_cast(*c1.column).getColumnsCopy(); for (size_t i = 0; i < tuple_size; ++i) { diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index f903210ef4f..3baca412497 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -2053,7 +2054,7 @@ private: if (!skip_not_null_check) { const auto & col = block.getByPosition(arguments[0]).column; - const auto & nullable_col = static_cast(*col); + const auto & nullable_col = assert_cast(*col); const auto & null_map = nullable_col.getNullMapData(); if (!memoryIsZero(null_map.data(), null_map.size())) diff --git a/dbms/src/Functions/FunctionsExternalDictionaries.h b/dbms/src/Functions/FunctionsExternalDictionaries.h index d86bc291212..1ae0c6d3198 100644 --- a/dbms/src/Functions/FunctionsExternalDictionaries.h +++ b/dbms/src/Functions/FunctionsExternalDictionaries.h @@ -10,6 +10,7 @@ #include #include +#include #include #include @@ -168,7 +169,7 @@ private: if (checkColumn(key_col.get())) { - const auto & key_columns = static_cast(*key_col).getColumnsCopy(); + const auto & key_columns = assert_cast(*key_col).getColumnsCopy(); const auto & key_types = static_cast(*key_col_with_type.type).getElements(); auto out = ColumnUInt8::create(key_col_with_type.column->size()); @@ -354,7 +355,7 @@ private: if (checkColumn(key_col.get())) { - const auto & key_columns = static_cast(*key_col).getColumnsCopy(); + const auto & key_columns = assert_cast(*key_col).getColumnsCopy(); const auto & key_types = static_cast(*key_col_with_type.type).getElements(); auto out = ColumnString::create(); @@ -891,7 +892,7 @@ private: if (checkColumn(key_col.get())) { - const auto & key_columns = static_cast(*key_col).getColumnsCopy(); + const auto & key_columns = assert_cast(*key_col).getColumnsCopy(); const auto & key_types = static_cast(*key_col_with_type.type).getElements(); typename ColVec::MutablePtr out; diff --git a/dbms/src/Functions/FunctionsExternalModels.cpp b/dbms/src/Functions/FunctionsExternalModels.cpp index 89418d8620e..6c96e09bad2 100644 --- a/dbms/src/Functions/FunctionsExternalModels.cpp +++ b/dbms/src/Functions/FunctionsExternalModels.cpp @@ -14,6 +14,8 @@ #include #include #include +#include + namespace DB { @@ -100,7 +102,7 @@ void FunctionModelEvaluate::executeImpl(Block & block, const ColumnNumbers & arg { auto mut_null_map = (*std::move(null_map)).mutate(); - NullMap & result_null_map = static_cast(*mut_null_map).getData(); + NullMap & result_null_map = assert_cast(*mut_null_map).getData(); const NullMap & src_null_map = col_nullable->getNullMapColumn().getData(); for (size_t i = 0, size = result_null_map.size(); i < size; ++i) diff --git a/dbms/src/Functions/FunctionsJSON.h b/dbms/src/Functions/FunctionsJSON.h index 0d229f8e2f3..cae88a40b59 100644 --- a/dbms/src/Functions/FunctionsJSON.h +++ b/dbms/src/Functions/FunctionsJSON.h @@ -7,6 +7,7 @@ #include "config_functions.h" #include #include +#include #include #include #include @@ -215,7 +216,7 @@ private: if (isColumnConst(*column.column)) { - const auto & column_const = static_cast(*column.column); + const auto & column_const = assert_cast(*column.column); if (isString(column.type)) moves.emplace_back(MoveType::ConstKey, column_const.getField().get()); else @@ -300,7 +301,7 @@ public: using Iterator = typename JSONParser::Iterator; static bool addValueToColumn(IColumn & dest, const Iterator &) { - ColumnVector & col_vec = static_cast &>(dest); + ColumnVector & col_vec = assert_cast &>(dest); col_vec.insertValue(1); return true; } @@ -330,7 +331,7 @@ public: else return false; - ColumnVector & col_vec = static_cast &>(dest); + ColumnVector & col_vec = assert_cast &>(dest); col_vec.insertValue(size); return true; } @@ -355,7 +356,7 @@ public: if (!JSONParser::isObjectMember(it)) return false; StringRef key = JSONParser::getKey(it); - ColumnString & col_str = static_cast(dest); + ColumnString & col_str = assert_cast(dest); col_str.insertData(key.data, key.size); return true; } @@ -407,7 +408,7 @@ public: else return false; - ColumnVector & col_vec = static_cast &>(dest); + ColumnVector & col_vec = assert_cast &>(dest); col_vec.insertValue(type); return true; } @@ -451,7 +452,7 @@ public: else return false; - auto & col_vec = static_cast &>(dest); + auto & col_vec = assert_cast &>(dest); col_vec.insertValue(value); return true; } @@ -497,7 +498,7 @@ public: if (!JSONParser::isBool(it)) return false; - auto & col_vec = static_cast &>(dest); + auto & col_vec = assert_cast &>(dest); col_vec.insertValue(static_cast(JSONParser::getBool(it))); return true; } @@ -523,7 +524,7 @@ public: return false; StringRef str = JSONParser::getString(it); - ColumnString & col_str = static_cast(dest); + ColumnString & col_str = assert_cast(dest); col_str.insertData(str.data, str.size); return true; } @@ -573,7 +574,7 @@ struct JSONExtractTree { if (!JSONParser::isString(it)) return false; - auto & col_str = static_cast(dest); + auto & col_str = assert_cast(dest); StringRef str = JSONParser::getString(it); if (str.size > col_str.getN()) return false; @@ -597,7 +598,7 @@ struct JSONExtractTree bool addValueToColumn(IColumn & dest, const Iterator & it) override { - auto & col_vec = static_cast &>(dest); + auto & col_vec = assert_cast &>(dest); if (JSONParser::isInt64(it)) { @@ -642,7 +643,7 @@ struct JSONExtractTree bool addValueToColumn(IColumn & dest, const Iterator & it) override { - ColumnNullable & col_null = static_cast(dest); + ColumnNullable & col_null = assert_cast(dest); if (!nested->addValueToColumn(col_null.getNestedColumn(), it)) return false; col_null.getNullMapColumn().insertValue(0); @@ -667,7 +668,7 @@ struct JSONExtractTree if (!JSONParser::firstArrayElement(it2)) return false; - ColumnArray & col_arr = static_cast(dest); + ColumnArray & col_arr = assert_cast(dest); auto & data = col_arr.getData(); size_t old_size = data.size(); bool were_valid_elements = false; @@ -706,7 +707,7 @@ struct JSONExtractTree bool addValueToColumn(IColumn & dest, const Iterator & it) override { - ColumnTuple & tuple = static_cast(dest); + ColumnTuple & tuple = assert_cast(dest); size_t old_size = dest.size(); bool were_valid_elements = false; @@ -916,10 +917,10 @@ public: if (!JSONParser::isObject(it)) return false; - auto & col_arr = static_cast(dest); - auto & col_tuple = static_cast(col_arr.getData()); + auto & col_arr = assert_cast(dest); + auto & col_tuple = assert_cast(col_arr.getData()); size_t old_size = col_tuple.size(); - auto & col_key = static_cast(col_tuple.getColumn(0)); + auto & col_key = assert_cast(col_tuple.getColumn(0)); auto & col_value = col_tuple.getColumn(1); StringRef key; @@ -958,7 +959,7 @@ public: using Iterator = typename JSONParser::Iterator; static bool addValueToColumn(IColumn & dest, const Iterator & it) { - ColumnString & col_str = static_cast(dest); + ColumnString & col_str = assert_cast(dest); auto & chars = col_str.getChars(); WriteBufferFromVector buf(chars, WriteBufferFromVector::AppendModeTag()); traverse(it, buf); diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index bc276435cc8..fb91455fcb8 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -10,6 +10,7 @@ #include #include "IFunction.h" #include +#include #include #include #include @@ -515,7 +516,7 @@ public: if (!isColumnConst(scale_column)) throw Exception("Scale argument for rounding functions must be constant.", ErrorCodes::ILLEGAL_COLUMN); - Field scale_field = static_cast(scale_column).getField(); + Field scale_field = assert_cast(scale_column).getField(); if (scale_field.getType() != Field::Types::UInt64 && scale_field.getType() != Field::Types::Int64) throw Exception("Scale argument for rounding functions must have integer type.", ErrorCodes::ILLEGAL_COLUMN); diff --git a/dbms/src/Functions/FunctionsStringArray.h b/dbms/src/Functions/FunctionsStringArray.h index fb89dfceda2..55438ba41e7 100644 --- a/dbms/src/Functions/FunctionsStringArray.h +++ b/dbms/src/Functions/FunctionsStringArray.h @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -551,8 +552,8 @@ public: } else { - const ColumnArray & col_arr = static_cast(*block.getByPosition(arguments[0]).column); - const ColumnString & col_string = static_cast(col_arr.getData()); + const ColumnArray & col_arr = assert_cast(*block.getByPosition(arguments[0]).column); + const ColumnString & col_string = assert_cast(col_arr.getData()); auto col_res = ColumnString::create(); diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index f4dbac7c638..a86ea724f7a 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -141,8 +142,8 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, const Block & block, const Colum { MutableColumnPtr mutable_result_null_map_column = (*std::move(result_null_map_column)).mutate(); - NullMap & result_null_map = static_cast(*mutable_result_null_map_column).getData(); - const NullMap & src_null_map = static_cast(*null_map_column).getData(); + NullMap & result_null_map = assert_cast(*mutable_result_null_map_column).getData(); + const NullMap & src_null_map = assert_cast(*null_map_column).getData(); for (size_t i = 0, size = result_null_map.size(); i < size; ++i) if (src_null_map[i]) @@ -238,7 +239,7 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo else { have_converted_columns = true; - temporary_block.insert({ static_cast(column.column.get())->getDataColumnPtr(), column.type, column.name }); + temporary_block.insert({ assert_cast(column.column.get())->getDataColumnPtr(), column.type, column.name }); } } @@ -584,7 +585,7 @@ DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & ar { bool is_const = arg.column && isColumnConst(*arg.column); if (is_const) - arg.column = static_cast(*arg.column).removeLowCardinality(); + arg.column = assert_cast(*arg.column).removeLowCardinality(); if (auto * low_cardinality_type = typeid_cast(arg.type.get())) { diff --git a/dbms/src/Functions/appendTrailingCharIfAbsent.cpp b/dbms/src/Functions/appendTrailingCharIfAbsent.cpp index fe28a6026bd..36bd6a8f643 100644 --- a/dbms/src/Functions/appendTrailingCharIfAbsent.cpp +++ b/dbms/src/Functions/appendTrailingCharIfAbsent.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -60,7 +61,7 @@ private: if (!checkColumnConst(column_char.get())) throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN}; - String trailing_char_str = static_cast(*column_char).getValue(); + String trailing_char_str = assert_cast(*column_char).getValue(); if (trailing_char_str.size() != 1) throw Exception{"Second argument of function " + getName() + " must be a one-character string", ErrorCodes::BAD_ARGUMENTS}; diff --git a/dbms/src/Functions/array/FunctionArrayMapped.h b/dbms/src/Functions/array/FunctionArrayMapped.h index dc25b228ce9..e27b9e28420 100644 --- a/dbms/src/Functions/array/FunctionArrayMapped.h +++ b/dbms/src/Functions/array/FunctionArrayMapped.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -146,7 +147,7 @@ public: if (!column_const_array) throw Exception("Expected array column, found " + column_array_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); column_array_ptr = column_const_array->convertToFullColumn(); - column_array = static_cast(column_array_ptr.get()); + column_array = assert_cast(column_array_ptr.get()); } block.getByPosition(result).column = Impl::execute(*column_array, column_array->getDataPtr()); diff --git a/dbms/src/Functions/array/arrayDistinct.cpp b/dbms/src/Functions/array/arrayDistinct.cpp index 173b0e1334c..aa4e8aec4f1 100644 --- a/dbms/src/Functions/array/arrayDistinct.cpp +++ b/dbms/src/Functions/array/arrayDistinct.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -92,7 +93,7 @@ void FunctionArrayDistinct::executeImpl(Block & block, const ColumnNumbers & arg const auto & return_type = block.getByPosition(result).type; auto res_ptr = return_type->createColumn(); - ColumnArray & res = static_cast(*res_ptr); + ColumnArray & res = assert_cast(*res_ptr); const IColumn & src_data = array->getData(); const ColumnArray::Offsets & offsets = array->getOffsets(); diff --git a/dbms/src/Functions/array/arrayElement.cpp b/dbms/src/Functions/array/arrayElement.cpp index c9293da9433..59f275692aa 100644 --- a/dbms/src/Functions/array/arrayElement.cpp +++ b/dbms/src/Functions/array/arrayElement.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -800,7 +801,7 @@ void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & argu else { /// ColumnConst(ColumnArray(ColumnNullable(...))) - const auto & nullable_col = static_cast(col_const_array->getData()); + const auto & nullable_col = assert_cast(col_const_array->getData()); const auto & nested_col = nullable_col.getNestedColumnPtr(); source_block = diff --git a/dbms/src/Functions/array/arrayEnumerateRanked.cpp b/dbms/src/Functions/array/arrayEnumerateRanked.cpp index c148664b664..7be0cbc44ce 100644 --- a/dbms/src/Functions/array/arrayEnumerateRanked.cpp +++ b/dbms/src/Functions/array/arrayEnumerateRanked.cpp @@ -1,5 +1,6 @@ #include #include +#include #include "arrayEnumerateRanked.h" @@ -49,7 +50,7 @@ ArraysDepths getArraysDepths(const ColumnsWithTypeAndName & arguments) if (depth_column && isColumnConst(*depth_column)) { - UInt64 value = static_cast(*depth_column).getValue(); + UInt64 value = assert_cast(*depth_column).getValue(); if (!value) throw Exception("Incorrect arguments for function arrayEnumerateUniqRanked or arrayEnumerateDenseRanked: depth (" + std::to_string(value) + ") cannot be less or equal 0.", diff --git a/dbms/src/Functions/array/arrayIndex.h b/dbms/src/Functions/array/arrayIndex.h index 12a6bef8672..70e73f1a7e9 100644 --- a/dbms/src/Functions/array/arrayIndex.h +++ b/dbms/src/Functions/array/arrayIndex.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -650,11 +651,11 @@ private: { const auto & null_map1 = block.getByPosition(arguments[2]).column; if (null_map1) - null_map_data = &static_cast(*null_map1).getData(); + null_map_data = &assert_cast(*null_map1).getData(); const auto & null_map2 = block.getByPosition(arguments[3]).column; if (null_map2) - null_map_item = &static_cast(*null_map2).getData(); + null_map_item = &assert_cast(*null_map2).getData(); } const auto item_arg = block.getByPosition(arguments[1]).column.get(); @@ -698,11 +699,11 @@ private: { const auto & col1 = block.getByPosition(arguments[2]).column; if (col1) - null_map_data = &static_cast(*col1).getData(); + null_map_data = &assert_cast(*col1).getData(); const auto & col2 = block.getByPosition(arguments[3]).column; if (col2) - null_map_item = &static_cast(*col2).getData(); + null_map_item = &assert_cast(*col2).getData(); } const auto item_arg = block.getByPosition(arguments[1]).column.get(); @@ -778,7 +779,7 @@ private: { const auto & col = block.getByPosition(arguments[3]).column; if (col) - null_map = &static_cast(*col).getData(); + null_map = &assert_cast(*col).getData(); } const auto size = item_arg->size(); @@ -838,11 +839,11 @@ private: { const auto & null_map1 = block.getByPosition(arguments[2]).column; if (null_map1) - null_map_data = &static_cast(*null_map1).getData(); + null_map_data = &assert_cast(*null_map1).getData(); const auto & null_map2 = block.getByPosition(arguments[3]).column; if (null_map2) - null_map_item = &static_cast(*null_map2).getData(); + null_map_item = &assert_cast(*null_map2).getData(); } if (item_arg.onlyNull()) @@ -850,7 +851,7 @@ private: col_res->getData(), null_map_data); else if (isColumnConst(item_arg)) ArrayIndexGenericImpl::vector(col_nested, col_array->getOffsets(), - static_cast(item_arg).getDataColumn(), col_res->getData(), /// TODO This is wrong. + assert_cast(item_arg).getDataColumn(), col_res->getData(), /// TODO This is wrong. null_map_data, nullptr); else { diff --git a/dbms/src/Functions/array/arrayIntersect.cpp b/dbms/src/Functions/array/arrayIntersect.cpp index b735d5497d6..e6cc24a49ba 100644 --- a/dbms/src/Functions/array/arrayIntersect.cpp +++ b/dbms/src/Functions/array/arrayIntersect.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -397,11 +398,11 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable all_nullable = false; } - auto & result_data = static_cast(*result_data_ptr); + auto & result_data = assert_cast(*result_data_ptr); auto result_offsets_ptr = ColumnArray::ColumnOffsets::create(rows); - auto & result_offsets = static_cast(*result_offsets_ptr); + auto & result_offsets = assert_cast(*result_offsets_ptr); auto null_map_column = ColumnUInt8::create(); - NullMap & null_map = static_cast(*null_map_column).getData(); + NullMap & null_map = assert_cast(*null_map_column).getData(); Arena arena; diff --git a/dbms/src/Functions/array/arrayReverse.cpp b/dbms/src/Functions/array/arrayReverse.cpp index 87862a5e052..0ef6f10fce8 100644 --- a/dbms/src/Functions/array/arrayReverse.cpp +++ b/dbms/src/Functions/array/arrayReverse.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -61,7 +62,7 @@ void FunctionArrayReverse::executeImpl(Block & block, const ColumnNumbers & argu ErrorCodes::ILLEGAL_COLUMN); auto res_ptr = array->cloneEmpty(); - ColumnArray & res = static_cast(*res_ptr); + ColumnArray & res = assert_cast(*res_ptr); res.getOffsetsPtr() = array->getOffsetsPtr(); const IColumn & src_data = array->getData(); diff --git a/dbms/src/Functions/array/emptyArrayToSingle.cpp b/dbms/src/Functions/array/emptyArrayToSingle.cpp index 2a98faea32a..2c2b17bcae4 100644 --- a/dbms/src/Functions/array/emptyArrayToSingle.cpp +++ b/dbms/src/Functions/array/emptyArrayToSingle.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -83,7 +84,7 @@ namespace if (const ColumnVector * src_data_concrete = checkAndGetColumn>(&src_data)) { const PaddedPODArray & src_data_vec = src_data_concrete->getData(); - PaddedPODArray & res_data = static_cast &>(res_data_col).getData(); + PaddedPODArray & res_data = assert_cast &>(res_data_col).getData(); size_t size = src_offsets.size(); res_offsets.resize(size); @@ -379,7 +380,7 @@ void FunctionEmptyArrayToSingle::executeImpl(Block & block, const ColumnNumbers ErrorCodes::ILLEGAL_COLUMN); MutableColumnPtr res_ptr = array->cloneEmpty(); - ColumnArray & res = static_cast(*res_ptr); + ColumnArray & res = assert_cast(*res_ptr); const IColumn & src_data = array->getData(); const ColumnArray::Offsets & src_offsets = array->getOffsets(); @@ -398,7 +399,7 @@ void FunctionEmptyArrayToSingle::executeImpl(Block & block, const ColumnNumbers inner_col = &nullable_col->getNestedColumn(); src_null_map = &nullable_col->getNullMapData(); - auto & nullable_res_col = static_cast(res_data); + auto & nullable_res_col = assert_cast(res_data); inner_res_col = &nullable_res_col.getNestedColumn(); res_null_map = &nullable_res_col.getNullMapData(); } diff --git a/dbms/src/Functions/greatCircleDistance.cpp b/dbms/src/Functions/greatCircleDistance.cpp index 593334c6cfb..06aa60c7798 100644 --- a/dbms/src/Functions/greatCircleDistance.cpp +++ b/dbms/src/Functions/greatCircleDistance.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -124,10 +125,10 @@ private: if (result_is_const) { - const auto & colLon1 = static_cast(block.getByPosition(arguments[0]).column.get())->getValue(); - const auto & colLat1 = static_cast(block.getByPosition(arguments[1]).column.get())->getValue(); - const auto & colLon2 = static_cast(block.getByPosition(arguments[2]).column.get())->getValue(); - const auto & colLat2 = static_cast(block.getByPosition(arguments[3]).column.get())->getValue(); + const auto & colLon1 = assert_cast(block.getByPosition(arguments[0]).column.get())->getValue(); + const auto & colLat1 = assert_cast(block.getByPosition(arguments[1]).column.get())->getValue(); + const auto & colLon2 = assert_cast(block.getByPosition(arguments[2]).column.get())->getValue(); + const auto & colLat2 = assert_cast(block.getByPosition(arguments[3]).column.get())->getValue(); Float64 res = greatCircleDistance(colLon1, colLat1, colLon2, colLat2); block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(size, res); @@ -143,9 +144,9 @@ private: for (const auto idx : ext::range(0, instrs.size())) { if (instr_type::get_float_64 == instrs[idx].first) - vals[idx] = static_cast *>(instrs[idx].second)->getData()[row]; + vals[idx] = assert_cast *>(instrs[idx].second)->getData()[row]; else if (instr_type::get_const_float_64 == instrs[idx].first) - vals[idx] = static_cast(instrs[idx].second)->getValue(); + vals[idx] = assert_cast(instrs[idx].second)->getValue(); else throw Exception{"Unknown instruction type in implementation of greatCircleDistance function", ErrorCodes::LOGICAL_ERROR}; } diff --git a/dbms/src/Functions/if.cpp b/dbms/src/Functions/if.cpp index a406b63be8b..99740b26914 100644 --- a/dbms/src/Functions/if.cpp +++ b/dbms/src/Functions/if.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -279,7 +280,7 @@ private: conditional( NumericArraySource(*col_left_array), NumericArraySource(*col_right_array), - NumericArraySink(static_cast(*res), input_rows_count), + NumericArraySink(assert_cast(*res), input_rows_count), cond_col->getData()); block.getByPosition(result).column = std::move(res); @@ -296,7 +297,7 @@ private: conditional( NumericArraySource(*col_left_array), ConstSource>(*col_right_const_array), - NumericArraySink(static_cast(*res), input_rows_count), + NumericArraySink(assert_cast(*res), input_rows_count), cond_col->getData()); block.getByPosition(result).column = std::move(res); @@ -336,7 +337,7 @@ private: conditional( ConstSource>(*col_left_const_array), NumericArraySource(*col_right_array), - NumericArraySink(static_cast(*res), input_rows_count), + NumericArraySink(assert_cast(*res), input_rows_count), cond_col->getData()); block.getByPosition(result).column = std::move(res); @@ -353,7 +354,7 @@ private: conditional( ConstSource>(*col_left_const_array), ConstSource>(*col_right_const_array), - NumericArraySink(static_cast(*res), input_rows_count), + NumericArraySink(assert_cast(*res), input_rows_count), cond_col->getData()); block.getByPosition(result).column = std::move(res); @@ -396,7 +397,7 @@ private: } else if (auto col_const_arr_left = checkAndGetColumnConst(col_left_untyped)) { - if (checkColumn(&static_cast(col_const_arr_left->getDataColumn()).getData())) + if (checkColumn(&assert_cast(col_const_arr_left->getDataColumn()).getData())) { left_ok = true; right_ok = executeConstRightTypeArray( @@ -438,7 +439,7 @@ private: /// The result is FixedString. auto col_res_untyped = block.getByPosition(result).type->createColumn(); - ColumnFixedString * col_res = static_cast(col_res_untyped.get()); + ColumnFixedString * col_res = assert_cast(col_res_untyped.get()); auto sink = FixedStringSink(*col_res, rows); if (col_then_fixed && col_else_fixed) @@ -514,7 +515,7 @@ private: && (col_arr_else || col_arr_else_const)) { auto res = block.getByPosition(result).type->createColumn(); - auto col_res = static_cast(res.get()); + auto col_res = assert_cast(res.get()); if (col_arr_then && col_arr_else) conditional(GenericArraySource(*col_arr_then), GenericArraySource(*col_arr_else), GenericArraySink(*col_res, rows), cond_data); @@ -612,8 +613,8 @@ private: if (then_is_const && else_is_const) { - const IColumn & then_nested_column = static_cast(*col_then).getDataColumn(); - const IColumn & else_nested_column = static_cast(*col_else).getDataColumn(); + const IColumn & then_nested_column = assert_cast(*col_then).getDataColumn(); + const IColumn & else_nested_column = assert_cast(*col_else).getDataColumn(); for (size_t i = 0; i < input_rows_count; ++i) { @@ -625,7 +626,7 @@ private: } else if (then_is_const) { - const IColumn & then_nested_column = static_cast(*col_then).getDataColumn(); + const IColumn & then_nested_column = assert_cast(*col_then).getDataColumn(); for (size_t i = 0; i < input_rows_count; ++i) { @@ -637,7 +638,7 @@ private: } else if (else_is_const) { - const IColumn & else_nested_column = static_cast(*col_else).getDataColumn(); + const IColumn & else_nested_column = assert_cast(*col_else).getDataColumn(); for (size_t i = 0; i < input_rows_count; ++i) { @@ -817,7 +818,7 @@ private: { auto arg_else_column = arg_else.column; auto result_column = (*std::move(arg_else_column)).mutate(); - static_cast(*result_column).applyNullMap(static_cast(*arg_cond.column)); + assert_cast(*result_column).applyNullMap(assert_cast(*arg_cond.column)); block.getByPosition(result).column = std::move(result_column); } else @@ -859,7 +860,7 @@ private: { auto arg_then_column = arg_then.column; auto result_column = (*std::move(arg_then_column)).mutate(); - static_cast(*result_column).applyNegatedNullMap(static_cast(*arg_cond.column)); + assert_cast(*result_column).applyNegatedNullMap(assert_cast(*arg_cond.column)); block.getByPosition(result).column = std::move(result_column); } else diff --git a/dbms/src/Functions/isNotNull.cpp b/dbms/src/Functions/isNotNull.cpp index 546bf4a160a..5762dbdbfcc 100644 --- a/dbms/src/Functions/isNotNull.cpp +++ b/dbms/src/Functions/isNotNull.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -43,7 +44,7 @@ public: /// Return the negated null map. auto res_column = ColumnUInt8::create(input_rows_count); const auto & src_data = nullable->getNullMapData(); - auto & res_data = static_cast(*res_column).getData(); + auto & res_data = assert_cast(*res_column).getData(); for (size_t i = 0; i < input_rows_count; ++i) res_data[i] = !src_data[i]; diff --git a/dbms/src/Functions/multiIf.cpp b/dbms/src/Functions/multiIf.cpp index f56889f7a01..bf64fd9af4c 100644 --- a/dbms/src/Functions/multiIf.cpp +++ b/dbms/src/Functions/multiIf.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -192,11 +193,11 @@ public: if (instruction.condition_always_true) insert = true; else if (!instruction.condition_is_nullable) - insert = static_cast(*instruction.condition).getData()[i]; + insert = assert_cast(*instruction.condition).getData()[i]; else { - const ColumnNullable & condition_nullable = static_cast(*instruction.condition); - const ColumnUInt8 & condition_nested = static_cast(condition_nullable.getNestedColumn()); + const ColumnNullable & condition_nullable = assert_cast(*instruction.condition); + const ColumnUInt8 & condition_nested = assert_cast(condition_nullable.getNestedColumn()); const NullMap & condition_null_map = condition_nullable.getNullMapData(); insert = !condition_null_map[i] && condition_nested.getData()[i]; @@ -207,7 +208,7 @@ public: if (!instruction.source_is_constant) res->insertFrom(*instruction.source, i); else - res->insertFrom(static_cast(*instruction.source).getDataColumn(), 0); + res->insertFrom(assert_cast(*instruction.source).getDataColumn(), 0); break; } diff --git a/dbms/src/Functions/pointInEllipses.cpp b/dbms/src/Functions/pointInEllipses.cpp index 2958d6171f1..8cfe1acf887 100644 --- a/dbms/src/Functions/pointInEllipses.cpp +++ b/dbms/src/Functions/pointInEllipses.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -131,8 +132,8 @@ private: const auto col_y = block.getByPosition(arguments[1]).column.get(); if (const_cnt == 0) { - const auto col_vec_x = static_cast *> (col_x); - const auto col_vec_y = static_cast *> (col_y); + const auto col_vec_x = assert_cast *> (col_x); + const auto col_vec_y = assert_cast *> (col_y); auto dst = ColumnVector::create(); auto & dst_data = dst->getData(); @@ -148,8 +149,8 @@ private: } else if (const_cnt == 2) { - const auto col_const_x = static_cast (col_x); - const auto col_const_y = static_cast (col_y); + const auto col_const_x = assert_cast (col_x); + const auto col_const_y = assert_cast (col_y); size_t start_index = 0; UInt8 res = isPointInEllipses(col_const_x->getValue(), col_const_y->getValue(), ellipses.data(), ellipses_count, start_index); block.getByPosition(result).column = DataTypeUInt8().createColumnConst(size, res); diff --git a/dbms/src/Functions/runningDifference.h b/dbms/src/Functions/runningDifference.h index 35987ca3aeb..374142311e9 100644 --- a/dbms/src/Functions/runningDifference.h +++ b/dbms/src/Functions/runningDifference.h @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -188,8 +189,8 @@ public: { using SrcFieldType = decltype(field_type_tag); - process(static_cast &>(*src_column).getData(), - static_cast> &>(*res_column).getData(), null_map); + process(assert_cast &>(*src_column).getData(), + assert_cast> &>(*res_column).getData(), null_map); }); if (null_map_column) diff --git a/dbms/src/Functions/sleep.h b/dbms/src/Functions/sleep.h index 5e9732d59f4..c8ef267e4c6 100644 --- a/dbms/src/Functions/sleep.h +++ b/dbms/src/Functions/sleep.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -73,7 +74,7 @@ public: if (!isColumnConst(*col)) throw Exception("The argument of function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN); - Float64 seconds = applyVisitor(FieldVisitorConvertToNumber(), static_cast(*col).getField()); + Float64 seconds = applyVisitor(FieldVisitorConvertToNumber(), assert_cast(*col).getField()); if (seconds < 0) throw Exception("Cannot sleep negative amount of time (not implemented)", ErrorCodes::BAD_ARGUMENTS); diff --git a/dbms/src/Functions/tupleElement.cpp b/dbms/src/Functions/tupleElement.cpp index 99bcc300259..a26c976e18f 100644 --- a/dbms/src/Functions/tupleElement.cpp +++ b/dbms/src/Functions/tupleElement.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -87,7 +88,7 @@ public: const IColumn * tuple_col = first_arg.column.get(); while (const DataTypeArray * array_type = checkAndGetDataType(tuple_type)) { - const ColumnArray * array_col = static_cast(tuple_col); + const ColumnArray * array_col = assert_cast(tuple_col); tuple_type = array_type->getNestedType().get(); tuple_col = &array_col->getData(); diff --git a/dbms/src/Functions/visibleWidth.cpp b/dbms/src/Functions/visibleWidth.cpp index c03bdff071d..8d229e878a8 100644 --- a/dbms/src/Functions/visibleWidth.cpp +++ b/dbms/src/Functions/visibleWidth.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -50,7 +51,7 @@ public: size_t size = input_rows_count; auto res_col = ColumnUInt64::create(size); - auto & res_data = static_cast(*res_col).getData(); + auto & res_data = assert_cast(*res_col).getData(); /// For simplicity reasons, function is implemented by serializing into temporary buffer. diff --git a/dbms/src/Interpreters/AggregationCommon.h b/dbms/src/Interpreters/AggregationCommon.h index 74836d4463d..af36d14c51e 100644 --- a/dbms/src/Interpreters/AggregationCommon.h +++ b/dbms/src/Interpreters/AggregationCommon.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -90,10 +91,10 @@ static inline T ALWAYS_INLINE packFixed( { switch ((*low_cardinality_sizes)[j]) { - case sizeof(UInt8): index = static_cast(positions)->getElement(i); break; - case sizeof(UInt16): index = static_cast(positions)->getElement(i); break; - case sizeof(UInt32): index = static_cast(positions)->getElement(i); break; - case sizeof(UInt64): index = static_cast(positions)->getElement(i); break; + case sizeof(UInt8): index = assert_cast(positions)->getElement(i); break; + case sizeof(UInt16): index = assert_cast(positions)->getElement(i); break; + case sizeof(UInt32): index = assert_cast(positions)->getElement(i); break; + case sizeof(UInt64): index = assert_cast(positions)->getElement(i); break; default: throw Exception("Unexpected size of index type for low cardinality column.", ErrorCodes::LOGICAL_ERROR); } } @@ -102,23 +103,23 @@ static inline T ALWAYS_INLINE packFixed( switch (key_sizes[j]) { case 1: - memcpy(bytes + offset, static_cast(column)->getRawDataBegin<1>() + index, 1); + memcpy(bytes + offset, assert_cast(column)->getRawDataBegin<1>() + index, 1); offset += 1; break; case 2: - memcpy(bytes + offset, static_cast(column)->getRawDataBegin<2>() + index * 2, 2); + memcpy(bytes + offset, assert_cast(column)->getRawDataBegin<2>() + index * 2, 2); offset += 2; break; case 4: - memcpy(bytes + offset, static_cast(column)->getRawDataBegin<4>() + index * 4, 4); + memcpy(bytes + offset, assert_cast(column)->getRawDataBegin<4>() + index * 4, 4); offset += 4; break; case 8: - memcpy(bytes + offset, static_cast(column)->getRawDataBegin<8>() + index * 8, 8); + memcpy(bytes + offset, assert_cast(column)->getRawDataBegin<8>() + index * 8, 8); offset += 8; break; default: - memcpy(bytes + offset, static_cast(column)->getRawDataBegin<1>() + index * key_sizes[j], key_sizes[j]); + memcpy(bytes + offset, assert_cast(column)->getRawDataBegin<1>() + index * key_sizes[j], key_sizes[j]); offset += key_sizes[j]; } } @@ -168,23 +169,23 @@ static inline T ALWAYS_INLINE packFixed( switch (key_sizes[j]) { case 1: - memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin<1>() + i, 1); + memcpy(bytes + offset, assert_cast(key_columns[j])->getRawDataBegin<1>() + i, 1); offset += 1; break; case 2: - memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin<2>() + i * 2, 2); + memcpy(bytes + offset, assert_cast(key_columns[j])->getRawDataBegin<2>() + i * 2, 2); offset += 2; break; case 4: - memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin<4>() + i * 4, 4); + memcpy(bytes + offset, assert_cast(key_columns[j])->getRawDataBegin<4>() + i * 4, 4); offset += 4; break; case 8: - memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin<8>() + i * 8, 8); + memcpy(bytes + offset, assert_cast(key_columns[j])->getRawDataBegin<8>() + i * 8, 8); offset += 8; break; default: - memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin<1>() + i * key_sizes[j], key_sizes[j]); + memcpy(bytes + offset, assert_cast(key_columns[j])->getRawDataBegin<1>() + i * key_sizes[j], key_sizes[j]); offset += key_sizes[j]; } } diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index f367908a0a5..33fbb903497 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #if __has_include() @@ -1212,7 +1213,7 @@ Block Aggregator::prepareBlockAndFill( aggregate_columns[i] = header.safeGetByPosition(i + params.keys_size).type->createColumn(); /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states. - ColumnAggregateFunction & column_aggregate_func = static_cast(*aggregate_columns[i]); + ColumnAggregateFunction & column_aggregate_func = assert_cast(*aggregate_columns[i]); for (size_t j = 0; j < data_variants.aggregates_pools.size(); ++j) column_aggregate_func.addArena(data_variants.aggregates_pools[j]); @@ -1228,7 +1229,7 @@ Block Aggregator::prepareBlockAndFill( if (aggregate_functions[i]->isState()) { /// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states. - ColumnAggregateFunction & column_aggregate_func = static_cast(*final_aggregate_columns[i]); + ColumnAggregateFunction & column_aggregate_func = assert_cast(*final_aggregate_columns[i]); for (size_t j = 0; j < data_variants.aggregates_pools.size(); ++j) column_aggregate_func.addArena(data_variants.aggregates_pools[j]); diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index df7354d8294..41fd957345e 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -270,7 +271,7 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns_low_cardinality, const Sizes & /*key_sizes*/) { auto ref = BaseState::getValueRef(value); - static_cast(key_columns_low_cardinality[0].get())->insertData(ref.data, ref.size); + assert_cast(key_columns_low_cardinality[0].get())->insertData(ref.data, ref.size); } }; @@ -318,9 +319,9 @@ struct AggregationMethodKeysFixed /// If we have a nullable column, get its nested column and its null map. if (column_nullable) { - ColumnNullable & nullable_col = static_cast(*key_columns[i]); + ColumnNullable & nullable_col = assert_cast(*key_columns[i]); observed_column = &nullable_col.getNestedColumn(); - null_map = static_cast(&nullable_col.getNullMapColumn()); + null_map = assert_cast(&nullable_col.getNullMapColumn()); } else { diff --git a/dbms/src/Interpreters/ExpressionJIT.cpp b/dbms/src/Interpreters/ExpressionJIT.cpp index 1b030fbd3cd..972cfe38c17 100644 --- a/dbms/src/Interpreters/ExpressionJIT.cpp +++ b/dbms/src/Interpreters/ExpressionJIT.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -418,9 +419,9 @@ static llvm::Constant * getNativeValue(llvm::Type * type, const IColumn & column return value ? llvm::ConstantStruct::get(static_cast(type), value, is_null) : nullptr; } if (type->isFloatTy()) - return llvm::ConstantFP::get(type, static_cast &>(column).getElement(i)); + return llvm::ConstantFP::get(type, assert_cast &>(column).getElement(i)); if (type->isDoubleTy()) - return llvm::ConstantFP::get(type, static_cast &>(column).getElement(i)); + return llvm::ConstantFP::get(type, assert_cast &>(column).getElement(i)); if (type->isIntegerTy()) return llvm::ConstantInt::get(type, column.getUInt(i)); /// TODO: if (type->isVectorTy()) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 0ee93122275..8e91424bc21 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -18,6 +18,7 @@ #include #include +#include #include @@ -73,7 +74,7 @@ static ColumnWithTypeAndName correctNullability(ColumnWithTypeAndName && column, if (negative_null_map.size()) { MutableColumnPtr mutable_column = (*std::move(column.column)).mutate(); - static_cast(*mutable_column).applyNegatedNullMap(negative_null_map); + assert_cast(*mutable_column).applyNegatedNullMap(negative_null_map); column.column = std::move(mutable_column); } } @@ -140,7 +141,7 @@ Join::Type Join::chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_siz /// If there is single string key, use hash table of it's values. if (keys_size == 1 && (typeid_cast(key_columns[0]) - || (isColumnConst(*key_columns[0]) && typeid_cast(&static_cast(key_columns[0])->getDataColumn())))) + || (isColumnConst(*key_columns[0]) && typeid_cast(&assert_cast(key_columns[0])->getDataColumn())))) return Type::key_string; if (keys_size == 1 && typeid_cast(key_columns[0])) @@ -872,7 +873,7 @@ void Join::joinBlockImpl( { /// Some trash to represent IColumn::Filter as ColumnUInt8 needed for ColumnNullable::applyNullMap() auto null_map_filter_ptr = ColumnUInt8::create(); - ColumnUInt8 & null_map_filter = static_cast(*null_map_filter_ptr); + ColumnUInt8 & null_map_filter = assert_cast(*null_map_filter_ptr); null_map_filter.getData().swap(row_filter); const IColumn::Filter & filter = null_map_filter.getData(); @@ -1399,7 +1400,7 @@ private: for (auto & it = *nulls_position; it != end && rows_added < max_block_size; ++it) { const Block * block = it->first; - const NullMap & nullmap = static_cast(*it->second).getData(); + const NullMap & nullmap = assert_cast(*it->second).getData(); for (size_t row = 0; row < nullmap.size(); ++row) { diff --git a/dbms/src/Interpreters/NullableUtils.cpp b/dbms/src/Interpreters/NullableUtils.cpp index 9299a591afd..fe2801f5d11 100644 --- a/dbms/src/Interpreters/NullableUtils.cpp +++ b/dbms/src/Interpreters/NullableUtils.cpp @@ -1,3 +1,4 @@ +#include #include @@ -34,7 +35,7 @@ ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullM { MutableColumnPtr mutable_null_map_holder = (*std::move(null_map_holder)).mutate(); - PaddedPODArray & mutable_null_map = static_cast(*mutable_null_map_holder).getData(); + PaddedPODArray & mutable_null_map = assert_cast(*mutable_null_map_holder).getData(); const PaddedPODArray & other_null_map = column_nullable->getNullMapData(); for (size_t i = 0, size = mutable_null_map.size(); i < size; ++i) mutable_null_map[i] |= other_null_map[i]; @@ -44,7 +45,7 @@ ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullM } } - null_map = null_map_holder ? &static_cast(*null_map_holder).getData() : nullptr; + null_map = null_map_holder ? &assert_cast(*null_map_holder).getData() : nullptr; } return null_map_holder; diff --git a/dbms/src/Interpreters/SetVariants.cpp b/dbms/src/Interpreters/SetVariants.cpp index 281304f886e..56f2ff04230 100644 --- a/dbms/src/Interpreters/SetVariants.cpp +++ b/dbms/src/Interpreters/SetVariants.cpp @@ -1,8 +1,10 @@ #include #include #include +#include #include + namespace DB { @@ -155,7 +157,7 @@ typename SetVariantsTemplate::Type SetVariantsTemplate::choose /// If there is single string key, use hash table of it's values. if (keys_size == 1 && (typeid_cast(nested_key_columns[0]) - || (isColumnConst(*nested_key_columns[0]) && typeid_cast(&static_cast(nested_key_columns[0])->getDataColumn())))) + || (isColumnConst(*nested_key_columns[0]) && typeid_cast(&assert_cast(nested_key_columns[0])->getDataColumn())))) return Type::key_string; if (keys_size == 1 && typeid_cast(nested_key_columns[0])) diff --git a/dbms/src/Interpreters/SetVariants.h b/dbms/src/Interpreters/SetVariants.h index 1ff63291a06..2d84e80156c 100644 --- a/dbms/src/Interpreters/SetVariants.h +++ b/dbms/src/Interpreters/SetVariants.h @@ -4,7 +4,7 @@ #include #include #include - +#include #include #include #include @@ -109,7 +109,7 @@ protected: { if (null_maps[k] != nullptr) { - const auto & null_map = static_cast(*null_maps[k]).getData(); + const auto & null_map = assert_cast(*null_maps[k]).getData(); if (null_map[row] == 1) { size_t bucket = k / 8; diff --git a/dbms/src/Interpreters/createBlockSelector.cpp b/dbms/src/Interpreters/createBlockSelector.cpp index 9db1b435edb..2b08ca0845c 100644 --- a/dbms/src/Interpreters/createBlockSelector.cpp +++ b/dbms/src/Interpreters/createBlockSelector.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include @@ -32,7 +33,7 @@ IColumn::Selector createBlockSelector( /// const columns contain only one value, therefore we do not need to read it at every iteration if (isColumnConst(column)) { - const auto data = static_cast(column).getValue(); + const auto data = assert_cast(column).getValue(); const auto shard_num = slots[static_cast(data) % total_weight]; selector.assign(num_rows, shard_num); } diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 4f42597eac5..a02ba33d9e7 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -72,8 +73,8 @@ static void fillColumnWithNumericData(std::shared_ptr & arrow_col /// Also internal strings are null terminated. static void fillColumnWithStringData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { - PaddedPODArray & column_chars_t = static_cast(*internal_column).getChars(); - PaddedPODArray & column_offsets = static_cast(*internal_column).getOffsets(); + PaddedPODArray & column_chars_t = assert_cast(*internal_column).getChars(); + PaddedPODArray & column_offsets = assert_cast(*internal_column).getOffsets(); size_t chars_t_size = 0; for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) @@ -110,7 +111,7 @@ static void fillColumnWithStringData(std::shared_ptr & arrow_colu static void fillColumnWithBooleanData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { - auto & column_data = static_cast &>(*internal_column).getData(); + auto & column_data = assert_cast &>(*internal_column).getData(); column_data.reserve(arrow_column->length()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) @@ -127,7 +128,7 @@ static void fillColumnWithBooleanData(std::shared_ptr & arrow_col /// Arrow stores Parquet::DATE in Int32, while ClickHouse stores Date in UInt16. Therefore, it should be checked before saving static void fillColumnWithDate32Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { - PaddedPODArray & column_data = static_cast &>(*internal_column).getData(); + PaddedPODArray & column_data = assert_cast &>(*internal_column).getData(); column_data.reserve(arrow_column->length()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) @@ -155,7 +156,7 @@ static void fillColumnWithDate32Data(std::shared_ptr & arrow_colu /// Arrow stores Parquet::DATETIME in Int64, while ClickHouse stores DateTime in UInt32. Therefore, it should be checked before saving static void fillColumnWithDate64Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { - auto & column_data = static_cast &>(*internal_column).getData(); + auto & column_data = assert_cast &>(*internal_column).getData(); column_data.reserve(arrow_column->length()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) @@ -171,7 +172,7 @@ static void fillColumnWithDate64Data(std::shared_ptr & arrow_colu static void fillColumnWithTimestampData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { - auto & column_data = static_cast &>(*internal_column).getData(); + auto & column_data = assert_cast &>(*internal_column).getData(); column_data.reserve(arrow_column->length()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) @@ -207,7 +208,7 @@ static void fillColumnWithTimestampData(std::shared_ptr & arrow_c static void fillColumnWithDecimalData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { - auto & column = static_cast &>(*internal_column); + auto & column = assert_cast &>(*internal_column); auto & column_data = column.getData(); column_data.reserve(arrow_column->length()); @@ -224,7 +225,7 @@ static void fillColumnWithDecimalData(std::shared_ptr & arrow_col /// Creates a null bytemap from arrow's null bitmap static void fillByteMapFromArrowColumn(std::shared_ptr & arrow_column, MutableColumnPtr & bytemap) { - PaddedPODArray & bytemap_data = static_cast &>(*bytemap).getData(); + PaddedPODArray & bytemap_data = assert_cast &>(*bytemap).getData(); bytemap_data.reserve(arrow_column->length()); for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->data()->num_chunks()); ++chunk_i) diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 1956403654e..f7092bc9a5b 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -3,28 +3,28 @@ #if USE_PARQUET // TODO: clean includes -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include -# include // REMOVE ME namespace DB { @@ -49,7 +49,7 @@ template static void fillArrowArrayWithNumericColumnData( ColumnPtr write_column, std::shared_ptr & arrow_array, const PaddedPODArray * null_bytemap) { - const PaddedPODArray & internal_data = static_cast &>(*write_column).getData(); + const PaddedPODArray & internal_data = assert_cast &>(*write_column).getData(); ArrowBuilderType builder; arrow::Status status; @@ -76,7 +76,7 @@ template static void fillArrowArrayWithStringColumnData( ColumnPtr write_column, std::shared_ptr & arrow_array, const PaddedPODArray * null_bytemap) { - const auto & internal_column = static_cast(*write_column); + const auto & internal_column = assert_cast(*write_column); arrow::StringBuilder builder; arrow::Status status; @@ -102,7 +102,7 @@ static void fillArrowArrayWithStringColumnData( static void fillArrowArrayWithDateColumnData( ColumnPtr write_column, std::shared_ptr & arrow_array, const PaddedPODArray * null_bytemap) { - const PaddedPODArray & internal_data = static_cast &>(*write_column).getData(); + const PaddedPODArray & internal_data = assert_cast &>(*write_column).getData(); //arrow::Date32Builder date_builder; arrow::UInt16Builder builder; arrow::Status status; @@ -124,7 +124,7 @@ static void fillArrowArrayWithDateColumnData( static void fillArrowArrayWithDateTimeColumnData( ColumnPtr write_column, std::shared_ptr & arrow_array, const PaddedPODArray * null_bytemap) { - auto & internal_data = static_cast &>(*write_column).getData(); + auto & internal_data = assert_cast &>(*write_column).getData(); //arrow::Date64Builder builder; arrow::UInt32Builder builder; arrow::Status status; @@ -195,7 +195,7 @@ static void fillArrowArrayWithDecimalColumnData( */ } -# define FOR_INTERNAL_NUMERIC_TYPES(M) \ +#define FOR_INTERNAL_NUMERIC_TYPES(M) \ M(UInt8, arrow::UInt8Builder) \ M(Int8, arrow::Int8Builder) \ M(UInt16, arrow::UInt16Builder) \ @@ -232,8 +232,8 @@ const std::unordered_map> internal_type static const PaddedPODArray * extractNullBytemapPtr(ColumnPtr column) { - ColumnPtr null_column = static_cast(*column).getNullMapColumnPtr(); - const PaddedPODArray & null_bytemap = static_cast &>(*null_column).getData(); + ColumnPtr null_column = assert_cast(*column).getNullMapColumnPtr(); + const PaddedPODArray & null_bytemap = assert_cast &>(*null_column).getData(); return &null_bytemap; } @@ -318,7 +318,7 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) std::shared_ptr arrow_array; ColumnPtr nested_column - = is_column_nullable ? static_cast(*column.column).getNestedColumnPtr() : column.column; + = is_column_nullable ? assert_cast(*column.column).getNestedColumnPtr() : column.column; const PaddedPODArray * null_bytemap = is_column_nullable ? extractNullBytemapPtr(column.column) : nullptr; if ("String" == column_nested_type_name) @@ -357,14 +357,14 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) }; callOnIndexAndDataType(column_nested_type->getTypeId(), fill_decimal); } -# define DISPATCH(CPP_NUMERIC_TYPE, ARROW_BUILDER_TYPE) \ +#define DISPATCH(CPP_NUMERIC_TYPE, ARROW_BUILDER_TYPE) \ else if (#CPP_NUMERIC_TYPE == column_nested_type_name) \ { \ fillArrowArrayWithNumericColumnData(nested_column, arrow_array, null_bytemap); \ } FOR_INTERNAL_NUMERIC_TYPES(DISPATCH) -# undef DISPATCH +#undef DISPATCH else { throw Exception{"Internal type \"" + column_nested_type_name + "\" of a column \"" + column.name diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index 3de820a5f4e..b9a6e28afb7 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include /// toLower #include @@ -264,7 +265,7 @@ private: if (key_pos == i) { // unwrap null key column - ColumnNullable & nullable_col = static_cast(*columns[i]); + ColumnNullable & nullable_col = assert_cast(*columns[i]); columns[i] = nullable_col.getNestedColumnPtr()->assumeMutable(); } else @@ -300,7 +301,7 @@ private: res.getByPosition(i).column = makeNullable(std::move(columns[i])); else { - const ColumnNullable & nullable_col = static_cast(*columns[i]); + const ColumnNullable & nullable_col = assert_cast(*columns[i]); res.getByPosition(i).column = nullable_col.getNestedColumnPtr(); } } From 6e26d8ccdf9825bb88aeda507e3db3b2c7b228cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 05:42:50 +0300 Subject: [PATCH 299/509] Addition to prev. revision --- dbms/src/DataTypes/DataTypeLowCardinality.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.cpp b/dbms/src/DataTypes/DataTypeLowCardinality.cpp index afb0a206a7a..b0f9d312773 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.cpp +++ b/dbms/src/DataTypes/DataTypeLowCardinality.cpp @@ -801,7 +801,7 @@ namespace void operator()() { if (typeid_cast *>(&keys_type)) - column = creator(assert_cast *>(nullptr)); + column = creator(static_cast *>(nullptr)); } }; } @@ -815,13 +815,13 @@ MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDat type = nullable_type->getNestedType().get(); if (isString(type)) - return creator(assert_cast(nullptr)); + return creator(static_cast(nullptr)); if (isFixedString(type)) - return creator(assert_cast(nullptr)); + return creator(static_cast(nullptr)); if (typeid_cast(type)) - return creator(assert_cast *>(nullptr)); + return creator(static_cast *>(nullptr)); if (typeid_cast(type)) - return creator(assert_cast *>(nullptr)); + return creator(static_cast *>(nullptr)); if (isColumnedAsNumber(type)) { MutableColumnUniquePtr column; From 40454aa7a0fa99d5b0ee5c25cb7b73a2ea4c21c8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 05:46:54 +0300 Subject: [PATCH 300/509] Addition to prev. revision --- dbms/src/Common/assert_cast.h | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/assert_cast.h b/dbms/src/Common/assert_cast.h index 93f213a0a23..7f9a19805bb 100644 --- a/dbms/src/Common/assert_cast.h +++ b/dbms/src/Common/assert_cast.h @@ -28,8 +28,16 @@ To assert_cast(From && from) #ifndef NDEBUG try { - if (typeid(from) == typeid(To)) - return static_cast(from); + if constexpr (std::is_pointer_v) + { + if (typeid(*from) == typeid(std::remove_pointer_t)) + return static_cast(from); + } + else + { + if (typeid(from) == typeid(To)) + return static_cast(from); + } } catch (const std::exception & e) { From c7850d65377bdfb132e6fb35b6478220d9463849 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 06:06:43 +0300 Subject: [PATCH 301/509] Removed obsolete trash from fuzz test --- dbms/tests/queries/0_stateless/00746_sql_fuzzy.pl | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00746_sql_fuzzy.pl b/dbms/tests/queries/0_stateless/00746_sql_fuzzy.pl index dd12d9ffdfa..101245b9f60 100755 --- a/dbms/tests/queries/0_stateless/00746_sql_fuzzy.pl +++ b/dbms/tests/queries/0_stateless/00746_sql_fuzzy.pl @@ -133,7 +133,6 @@ sub main { split /[\s;,]+/, $ENV{SQL_FUZZY_FUNCTIONS} || file_read($ENV{SQL_FUZZY_FILE_FUNCTIONS} || 'clickhouse-functions') - || '__inner_restore_projection__ __inner_build_projection_composition__ convertCharset one_or_zero findClusterValue findClusterIndex toNullable coalesce isNotNull geoToH3 pointInEllipses transform pow acos asin tan cos tgamma lgamma erfc erf sqrt log10 exp10 e visitParamExtractFloat visitParamExtractUInt decodeURLComponent cutURLParameter cutQueryStringAndFragment cutFragment cutWWW URLPathHierarchy URLHierarchy extractURLParameterNames extractURLParameter queryStringAndFragment pathFull sin topLevelDomain domainWithoutWWW domain protocol greatCircleDistance extract match positionCaseInsensitiveUTF8 positionCaseInsensitive positionUTF8 position replaceRegexpAll replaceRegexpOne arrayStringConcat splitByString splitByChar alphaTokens endsWith startsWith appendTrailingCharIfAbsent substringUTF8 concatAssumeInjective reverseUTF8 upperUTF8 __inner_project__ upper lower CRC32 length notEmpty trunc round roundAge roundDuration roundToExp2 reinterpretAsString reinterpretAsDateTime reinterpretAsDate reinterpretAsFloat64 reinterpretAsFloat32 reinterpretAsInt64 reinterpretAsInt8 reinterpretAsUInt32 toStartOfFiveMinute toISOYear toISOWeek concat toDecimal64 ifNull toStartOfDay toSecond addSeconds sleepEachRow materialize visitParamExtractInt toStartOfMinute toDayOfWeek toDayOfMonth bitShiftLeft emptyArrayUInt8 parseDateTimeBestEffort toTime toDateTimeOrNull toFloat32OrNull toInt16 IPv6NumToString atan substring arrayIntersect isInfinite toRelativeHourNum hex arrayEnumerateDense toUInt8OrZero toRelativeSecondNum toUInt64OrNull MACNumToString toInt32OrNull toDayOfYear toUnixTimestamp toString toDateOrZero subtractDays toMinute murmurHash3_64 murmurHash2_32 toUInt64 toUInt8 dictGetDateTime empty isFinite caseWithoutExpression caseWithoutExpr visitParamExtractRaw queryString dictGetInt32OrDefault caseWithExpression toInt8OrZero multiIf if intExp10 bitShiftRight less toUInt8OrNull toInt8OrNull bitmaskToArray toIntervalYear toFloat64OrZero dateDiff generateUUIDv4 arrayPopBack toIntervalMonth toUUID notEquals toInt16OrNull murmurHash2_64 hasAny toIntervalMinute isNull tupleElement replaceAll parseDateTimeBestEffortOrZero toFloat32OrZero lowerUTF8 notIn gcd like regionToPopulation MACStringToOUI notLike toStringCutToZero lcm parseDateTimeBestEffortOrNull not toInt32OrZero arrayFilter toInt16OrZero range equals now toTypeName toUInt32OrNull emptyArrayString dictGetDateTimeOrDefault bitRotateRight cutIPv6 toUInt32OrZero timezone reverse runningDifferenceStartingWithFirstValue toDateTime arrayPopFront toInt32 intHash64 extractURLParameters lowCardinalityIndices toStartOfMonth toYear hasAll rowNumberInAllBlocks bitTestAll arrayCount arraySort abs bitNot intDiv intDivOrZero firstSignificantSubdomain dictGetFloat32OrDefault reinterpretAsUInt16 toHour minus regionToArea unhex IPv4StringToNum toIntervalHour toInt8 dictGetFloat32 log IPv4NumToString modulo arrayEnumerate cutQueryString reinterpretAsFixedString countEqual bitTest toDecimal128 plus or reinterpretAsUInt64 toMonth visitParamExtractBool emptyArrayUInt64 replaceOne arrayReverseSort toFloat32 toRelativeMonthNum emptyArrayInt32 toRelativeYearNum arrayElement log2 array arrayReverse toUInt64OrZero emptyArrayFloat64 negate arrayPushBack subtractWeeks bitTestAny bitAnd toDecimal32 arrayPushFront lessOrEquals intExp2 toUInt16OrZero arrayConcat arrayCumSum arraySlice addDays dictGetUInt8 toUInt32 bitOr caseWithExpr toStartOfYear toIntervalDay MD5 emptyArrayUInt32 emptyArrayInt8 toMonday addMonths arrayUniq SHA256 arrayExists multiply toUInt16OrNull dictGetInt8 visitParamHas emptyArrayInt64 toIntervalSecond toDate sleep emptyArrayToSingle path toInt64OrZero SHA1 extractAll emptyArrayDate dumpColumnStructure toInt64 lengthUTF8 greatest arrayEnumerateUniq arrayDistinct arrayFirst toFixedString IPv4NumToStringClassC toFloat64OrNull IPv4ToIPv6 identity ceil toStartOfQuarter dictGetInt8OrDefault MACStringToNum emptyArrayUInt16 UUIDStringToNum dictGetUInt16 toStartOfFifteenMinutes toStartOfHour sumburConsistentHash toStartOfISOYear toRelativeQuarterNum toRelativeWeekNum toRelativeDayNum cbrt yesterday bitXor timeSlot timeSlots emptyArrayInt16 dictGetInt16 toYYYYMM toYYYYMMDDhhmmss toUInt16 addMinutes addHours addWeeks nullIf subtractSeconds subtractMinutes toIntervalWeek subtractHours isNaN subtractMonths toDateOrNull subtractYears toTimeZone formatDateTime has cityHash64 intHash32 fragment regionToCity indexOf regionToDistrict regionToCountry visibleWidth regionToContinent regionToTopContinent toColumnTypeName regionHierarchy CHAR_LENGTH least divide SEHierarchy dictGetDate OSToRoot SEToRoot OSIn SEIn regionToName dictGetStringOrDefault OSHierarchy exp floor dictGetUInt8OrDefault dictHas dictGetUInt64 cutToFirstSignificantSubdomain dictGetInt32 pointInPolygon dictGetInt64 blockNumber IPv6StringToNum dictGetString dictGetFloat64 dictGetUUID CHARACTER_LENGTH toQuarter dictGetHierarchy toFloat64 arraySum toInt64OrNull dictIsIn dictGetUInt16OrDefault dictGetUInt32OrDefault emptyArrayDateTime greater jumpConsistentHash dictGetUInt64OrDefault dictGetInt16OrDefault dictGetInt64OrDefault reinterpretAsInt32 dictGetUInt32 murmurHash3_32 bar dictGetUUIDOrDefault rand modelEvaluate arrayReduce farmHash64 bitmaskToList formatReadableSize halfMD5 SHA224 arrayMap sipHash64 dictGetFloat64OrDefault sipHash128 metroHash64 murmurHash3_128 yandexConsistentHash emptyArrayFloat32 arrayAll toYYYYMMDD today arrayFirstIndex greaterOrEquals arrayDifference visitParamExtractString toDateTimeOrZero globalNotIn throwIf and xor currentDatabase hostName URLHash getSizeOfEnumType defaultValueOfArgumentType blockSize tuple arrayCumSumNonNegative rowNumberInBlock arrayResize ignore toRelativeMinuteNum indexHint reinterpretAsInt16 addYears arrayJoin replicate hasColumnInTable version regionIn uptime runningAccumulate runningDifference assumeNotNull pi finalizeAggregation toLowCardinality exp2 lowCardinalityKeys in globalIn dictGetDateOrDefault rand64 CAST bitRotateLeft randConstant UUIDNumToString reinterpretAsUInt8 truncate ceiling retention maxIntersections groupBitXor groupBitOr uniqUpTo uniqCombined uniqExact uniq covarPop stddevPop varPop covarSamp varSamp sumMap corrStable corr quantileTiming quantileDeterministic quantilesExact uniqHLL12 quantilesTiming covarPopStable stddevSampStable quantilesExactWeighted quantileExactWeighted quantileTimingWeighted quantileExact quantilesDeterministic quantiles topK sumWithOverflow count groupArray stddevSamp groupArrayInsertAt quantile quantilesTimingWeighted quantileTDigest quantilesTDigest windowFunnel min argMax varSampStable maxIntersectionsPosition quantilesTDigestWeighted groupUniqArray sequenceCount sumKahan any anyHeavy histogram quantileTDigestWeighted max groupBitAnd argMin varPopStable avg sequenceMatch stddevPopStable sum anyLast covarSampStable BIT_XOR medianExactWeighted medianTiming medianExact median medianDeterministic VAR_SAMP STDDEV_POP medianTDigest VAR_POP medianTDigestWeighted BIT_OR STDDEV_SAMP medianTimingWeighted COVAR_SAMP COVAR_POP BIT_AND' ]; # $functions = [grep { not $_ ~~ [qw( )] } @$functions]; # will be removed # select name from system.table_functions format TSV; @@ -141,7 +140,6 @@ sub main { split /[\s;,]+/, $ENV{SQL_FUZZY_TABLE_FUNCTIONS} || file_read($ENV{SQL_FUZZY_FILE_TABLE_FUNCTIONS} || 'clickhouse-table-functions') - || 'mysql jdbc odbc remote catBoostPool merge file cluster shardByHash url numbers' ]; $table_functions = [grep { not $_ ~~ [qw(numbers)] } @$table_functions]; # too slow say one_of({}, $query), ';' for 1 .. ($ENV{SQL_FUZZY_LINES} || 100); From 6e8389bff51fcf0ae6b373bcb21c6202e8150b6f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 06:07:53 +0300 Subject: [PATCH 302/509] Fixed fuzz test --- dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh b/dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh index 913963c7318..f417f0146c2 100755 --- a/dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh +++ b/dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh @@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -q "select name from system.table_functions format TSV;" > $S # if you want long run use: env SQL_FUZZY_RUNS=100000 clickhouse-test sql_fuzzy for SQL_FUZZY_RUN in $(seq ${SQL_FUZZY_RUNS:=10}); do - env SQL_FUZZY_RUN=$SQL_FUZZY_RUN $CURDIR/00746_sql_fuzzy.pl | $CLICKHOUSE_CLIENT -n --ignore-error >/dev/null 2>&1 + env SQL_FUZZY_RUN=$SQL_FUZZY_RUN $CURDIR/00746_sql_fuzzy.pl | $CLICKHOUSE_CLIENT --max_execution_time 10 -n --ignore-error >/dev/null 2>&1 if [[ `$CLICKHOUSE_CLIENT -q "SELECT 'Still alive'"` != 'Still alive' ]]; then break fi From d8480098273339589d5f4775591bad4a20913351 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 06:16:59 +0300 Subject: [PATCH 303/509] Fixed false data race in "MergeTreeDataPart::is_frozen" field --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- dbms/src/Storages/MergeTree/MergeTreeDataPart.h | 4 ++-- dbms/src/Storages/System/StorageSystemParts.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index d8871b9e1a8..db063593652 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -2876,7 +2876,7 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & String backup_part_absolute_path = part_absolute_path; backup_part_absolute_path.replace(0, clickhouse_path.size(), backup_path); localBackup(part_absolute_path, backup_part_absolute_path); - part->is_frozen = true; + part->is_frozen.store(true, std::memory_order_relaxed); ++parts_processed; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h index e46a64f0546..fe9cc330f0e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h @@ -94,8 +94,8 @@ struct MergeTreeDataPart /// If true it means that there are no ZooKeeper node for this part, so it should be deleted only from filesystem bool is_duplicate = false; - /// Frozen by ALTER TABLE ... FREEZE ... - mutable bool is_frozen = false; + /// Frozen by ALTER TABLE ... FREEZE ... It is used for information purposes in system.parts table. + mutable std::atomic is_frozen {false}; /** * Part state is a stage of its lifetime. States are ordered and state of a part could be increased only. diff --git a/dbms/src/Storages/System/StorageSystemParts.cpp b/dbms/src/Storages/System/StorageSystemParts.cpp index 8cf744eaefe..52b85aa0cb6 100644 --- a/dbms/src/Storages/System/StorageSystemParts.cpp +++ b/dbms/src/Storages/System/StorageSystemParts.cpp @@ -103,7 +103,7 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns_, const Sto columns_[i++]->insert(static_cast(part->info.getDataVersion())); columns_[i++]->insert(part->getIndexSizeInBytes()); columns_[i++]->insert(part->getIndexSizeInAllocatedBytes()); - columns_[i++]->insert(part->is_frozen); + columns_[i++]->insert(part->is_frozen.load(std::memory_order_relaxed)); columns_[i++]->insert(info.database); columns_[i++]->insert(info.table); From 3c3ca97218995db9efd5d66587604dd7994cb416 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 08:47:09 +0300 Subject: [PATCH 304/509] Addition to prev. revision --- dbms/src/Interpreters/AggregationCommon.h | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/dbms/src/Interpreters/AggregationCommon.h b/dbms/src/Interpreters/AggregationCommon.h index af36d14c51e..c9343b517cc 100644 --- a/dbms/src/Interpreters/AggregationCommon.h +++ b/dbms/src/Interpreters/AggregationCommon.h @@ -103,23 +103,23 @@ static inline T ALWAYS_INLINE packFixed( switch (key_sizes[j]) { case 1: - memcpy(bytes + offset, assert_cast(column)->getRawDataBegin<1>() + index, 1); + memcpy(bytes + offset, static_cast(column)->getRawDataBegin<1>() + index, 1); offset += 1; break; case 2: - memcpy(bytes + offset, assert_cast(column)->getRawDataBegin<2>() + index * 2, 2); + memcpy(bytes + offset, static_cast(column)->getRawDataBegin<2>() + index * 2, 2); offset += 2; break; case 4: - memcpy(bytes + offset, assert_cast(column)->getRawDataBegin<4>() + index * 4, 4); + memcpy(bytes + offset, static_cast(column)->getRawDataBegin<4>() + index * 4, 4); offset += 4; break; case 8: - memcpy(bytes + offset, assert_cast(column)->getRawDataBegin<8>() + index * 8, 8); + memcpy(bytes + offset, static_cast(column)->getRawDataBegin<8>() + index * 8, 8); offset += 8; break; default: - memcpy(bytes + offset, assert_cast(column)->getRawDataBegin<1>() + index * key_sizes[j], key_sizes[j]); + memcpy(bytes + offset, static_cast(column)->getRawDataBegin<1>() + index * key_sizes[j], key_sizes[j]); offset += key_sizes[j]; } } @@ -169,23 +169,23 @@ static inline T ALWAYS_INLINE packFixed( switch (key_sizes[j]) { case 1: - memcpy(bytes + offset, assert_cast(key_columns[j])->getRawDataBegin<1>() + i, 1); + memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin<1>() + i, 1); offset += 1; break; case 2: - memcpy(bytes + offset, assert_cast(key_columns[j])->getRawDataBegin<2>() + i * 2, 2); + memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin<2>() + i * 2, 2); offset += 2; break; case 4: - memcpy(bytes + offset, assert_cast(key_columns[j])->getRawDataBegin<4>() + i * 4, 4); + memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin<4>() + i * 4, 4); offset += 4; break; case 8: - memcpy(bytes + offset, assert_cast(key_columns[j])->getRawDataBegin<8>() + i * 8, 8); + memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin<8>() + i * 8, 8); offset += 8; break; default: - memcpy(bytes + offset, assert_cast(key_columns[j])->getRawDataBegin<1>() + i * key_sizes[j], key_sizes[j]); + memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin<1>() + i * key_sizes[j], key_sizes[j]); offset += key_sizes[j]; } } From 9a245f7930dabc1a637dae53fadfc22f005a272e Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 21 Aug 2019 10:52:51 +0300 Subject: [PATCH 305/509] DOCAPI-7438: Update of quantileTiming docs. (#6455) * Link fix. * DOCAPI-7438: Update of quantileTiming functions docs. * Update docs/en/query_language/agg_functions/reference.md Co-Authored-By: Ivan Blinkov * Update docs/en/query_language/agg_functions/reference.md Co-Authored-By: Ivan Blinkov * Update docs/en/query_language/agg_functions/reference.md Co-Authored-By: Ivan Blinkov * DOCAPI-7438: Clarifications. * DOCAPI-7438: Links fix. --- docs/en/data_types/float.md | 2 +- .../query_language/agg_functions/reference.md | 54 ++++++++++++++----- docs/fa/data_types/float.md | 2 +- docs/zh/data_types/float.md | 2 +- docs/zh/query_language/select.md | 2 +- 5 files changed, 45 insertions(+), 17 deletions(-) diff --git a/docs/en/data_types/float.md b/docs/en/data_types/float.md index 7c731d9fbc8..29ff604bb02 100644 --- a/docs/en/data_types/float.md +++ b/docs/en/data_types/float.md @@ -27,7 +27,7 @@ SELECT 1 - 0.9 - Floating-point calculations might result in numbers such as infinity (`Inf`) and "not-a-number" (`NaN`). This should be taken into account when processing the results of calculations. - When parsing floating point numbers from text, the result might not be the nearest machine-representable number. -## NaN and Inf +## NaN and Inf {#data_type-float-nan-inf} In contrast to standard SQL, ClickHouse supports the following categories of floating-point numbers: diff --git a/docs/en/query_language/agg_functions/reference.md b/docs/en/query_language/agg_functions/reference.md index 5f3007a60cd..a43e84112b0 100644 --- a/docs/en/query_language/agg_functions/reference.md +++ b/docs/en/query_language/agg_functions/reference.md @@ -819,7 +819,7 @@ Creates an array from different argument values. Memory consumption is the same The second version (with the `max_size` parameter) limits the size of the resulting array to `max_size` elements. For example, `groupUniqArray(1)(x)` is equivalent to `[any(x)]`. -## quantile(level)(x) +## quantile(level)(x) {#agg_function-quantile} Approximates the `level` quantile. `level` is a constant, a floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]` @@ -846,27 +846,55 @@ To achieve this, the function takes a second argument – the "determinator". Th Don't use this function for calculating timings. There is a more suitable function for this purpose: `quantileTiming`. -## quantileTiming(level)(x) +## quantileTiming {#agg_function-quantiletiming} -Computes the quantile of 'level' with a fixed precision. -Works for numbers. Intended for calculating quantiles of page loading time in milliseconds. +Computes the quantile of the specified level with determined precision. The function intended for calculating quantiles of page loading time in milliseconds. -If the value is greater than 30,000 (a page loading time of more than 30 seconds), the result is equated to 30,000. +``` +quantileTiming(level)(expr) +``` -If the total value is not more than about 5670, then the calculation is accurate. +**Parameters** -Otherwise: +- `level` — Quantile level. Range: [0, 1]. +- `expr` — [Expression](../syntax.md#syntax-expressions) returning number in the [Float*](../../data_types/float.md) type. The function expects input values in unix timestamp format in milliseconds, but it doesn't validate format. + + - If negative values are passed to the function, the behavior is undefined. + - If the value is greater than 30,000 (a page loading time of more than 30 seconds), it is assumed to be 30,000. -- if the time is less than 1024 ms, then the calculation is accurate. -- otherwise the calculation is rounded to a multiple of 16 ms. +**Accuracy** -When passing negative values to the function, the behavior is undefined. +The calculation is accurate if: -The returned value has the Float32 type. If no values were passed to the function (when using `quantileTimingIf`), 'nan' is returned. The purpose of this is to differentiate these instances from zeros. See the note on sorting NaNs in "ORDER BY clause". +- Total number of values is not more than about 5670. +- Total number of values is more than about 5670, but the times of page loading is less than 1024ms. -The result is determinate (it doesn't depend on the order of query processing). +Otherwise, the result of a calculation is rounded to the value, multiple of 16 ms. -For its purpose (calculating quantiles of page loading times), using this function is more effective and the result is more accurate than for the `quantile` function. +!! note "Note" + For calculating quantiles of page loading times, this function is more effective and accurate compared to [quantile](#agg_function-quantile). + +**Returned value** + +- Quantile of the specified level. + +Type: `Float32`. + +!!! note "Note" + If no values were passed to the function (when using `quantileTimingIf`), [NaN](../../data_types/float.md#data_type-float-nan-inf) is returned. The purpose of this is to differentiate these cases from the cases which result in zero. See [ORDER BY clause](../select.md#select-order-by) for the note on sorting `NaN` values. + +The result is deterministic (it doesn't depend on the order of query processing). + +**Example** + +```sql +SELECT quantileTiming(0.5)(number / 2) FROM numbers(10) +``` +```text +┌─quantileTiming(0.5)(divide(number, 2))─┐ +│ 2 │ +└────────────────────────────────────────┘ +``` ## quantileTimingWeighted(level)(x, weight) diff --git a/docs/fa/data_types/float.md b/docs/fa/data_types/float.md index c83a4831ff3..df83dabd206 100644 --- a/docs/fa/data_types/float.md +++ b/docs/fa/data_types/float.md @@ -32,7 +32,7 @@ SELECT 1 - 0.9 - محاسبات Float ممکن اسن نتایجی مثل infinity (`inf`) و "Not-a-number" (`Nan`) داشته باشد. این در هنگام پردازش نتایج محاسبات باید مورد توجه قرار گیرد. - هنگام خواندن اعداد float از سطر ها، نتایج ممکن است نزدیک به اعداد machine-representable نباشد. -## NaN و Inf +## NaN و Inf {#data_type-float-nan-inf} در مقابل استاندارد SQL، ClickHouse از موارد زیر مربوط به اعداد float پشتیبانی می کند: diff --git a/docs/zh/data_types/float.md b/docs/zh/data_types/float.md index 113e640e611..2bada8c2410 100644 --- a/docs/zh/data_types/float.md +++ b/docs/zh/data_types/float.md @@ -29,7 +29,7 @@ SELECT 1 - 0.9 - 当一行行阅读浮点数的时候,浮点数的结果可能不是机器最近显示的数值。 -## NaN and Inf +## NaN and Inf {#data_type-float-nan-inf} 与标准SQL相比,ClickHouse 支持以下类别的浮点数: diff --git a/docs/zh/query_language/select.md b/docs/zh/query_language/select.md index 53716c1cfac..4c819db1507 100644 --- a/docs/zh/query_language/select.md +++ b/docs/zh/query_language/select.md @@ -612,7 +612,7 @@ WHERE于HAVING不同之处在于WHERE在聚合前(GROUP BY)执行,HAVING在聚 如果不存在聚合,则不能使用HAVING。 -### ORDER BY 子句 +### ORDER BY 子句 {#select-order-by} 如果存在ORDER BY 子句,则该子句中必须存在一个表达式列表,表达式列表中每一个表达式都可以分配一个DESC或ASC(排序的方向)。如果没有指明排序的方向,将假定以ASC的方式进行排序。其中ASC表示按照升序排序,DESC按照降序排序。示例:`ORDER BY Visits DESC, SearchPhrase` From 086dd815ac19070e119643929164663aa196c035 Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Wed, 21 Aug 2019 11:53:41 +0300 Subject: [PATCH 306/509] T_test.h added, some fixes --- dbms/programs/benchmark/Benchmark.cpp | 229 +++----------------------- dbms/src/Client/ConnectionPool.h | 4 + dbms/src/Common/T_test.h | 212 ++++++++++++++++++++++++ 3 files changed, 242 insertions(+), 203 deletions(-) create mode 100644 dbms/src/Common/T_test.h diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 43e109810ec..cb034a258af 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -32,6 +32,7 @@ #include #include #include +#include /** A tool for evaluating ClickHouse performance. @@ -41,6 +42,8 @@ namespace DB { +using Ports = std::vector; + namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -50,8 +53,7 @@ namespace ErrorCodes class Benchmark : public Poco::Util::Application { public: - Benchmark(unsigned concurrency_, double delay_, - const std::vector & hosts_, const std::vector & ports_, + Benchmark(unsigned concurrency_, double delay_, Strings && hosts_, Ports && ports_, bool cumulative_, bool secure_, const String & default_database_, const String & user_, const String & password_, const String & stage, bool randomize_, size_t max_iterations_, double max_time_, @@ -68,24 +70,17 @@ public: connections.reserve(connections_cnt); comparison_info_total.reserve(connections_cnt); comparison_info_per_interval.reserve(connections_cnt); - comparison_relative.data.resize(connections_cnt); for (size_t i = 0; i < connections_cnt; ++i) { UInt16 cur_port = i >= ports_.size() ? 9000 : ports_[i]; std::string cur_host = i >= hosts_.size() ? "localhost" : hosts_[i]; - connections.emplace_back(std::make_shared(concurrency, cur_host, cur_port, default_database_, user_, password_, "benchmark", Protocol::Compression::Enable, secure)); + connections.emplace_back(std::make_unique(concurrency, cur_host, cur_port, default_database_, user_, password_, "benchmark", Protocol::Compression::Enable, secure)); comparison_info_per_interval.emplace_back(std::make_shared()); comparison_info_total.emplace_back(std::make_shared()); } - if (confidence > 5) - { - std::cerr << "Confidence can't be set to " + toString(confidence) + ". It was set to 5 instead." << '\n'; - confidence = 5; - } - global_context.makeGlobalContext(); std::cerr << std::fixed << std::setprecision(3); @@ -140,7 +135,9 @@ private: using Queue = ConcurrentBoundedQueue; Queue queue; - ConnectionPoolPtrs connections; + using ConnectionPoolUniq = std::unique_ptr; + using ConnectionPoolUniqs = std::vector; + ConnectionPoolUniqs connections; bool randomize; bool cumulative; @@ -192,180 +189,10 @@ private: } }; - struct RelativeAnalysis - { - struct RelativeStats - { - size_t cnt = 0; - double sum = 0; - double squares_sum = 0; - - void add(double seconds) - { - ++cnt; - sum += seconds; - squares_sum += seconds * seconds; - } - - double avg() const - { - return sum / cnt; - } - - double var() const - { - return (squares_sum - (sum * sum / cnt)) / static_cast(cnt - 1); - } - }; - - const std::vector confidence_level = { 80, 90, 95, 98, 99, 99.5 }; - - const std::vector> students_table = { - /* inf */ { 1.282, 1.645, 1.960, 2.326, 2.576, 3.090 }, - /* 1. */ { 3.078, 6.314, 12.706, 31.821, 63.657, 318.313}, - /* 2. */ { 1.886, 2.920, 4.303, 6.965, 9.925, 22.327 }, - /* 3. */ { 1.638, 2.353, 3.182, 4.541, 5.841, 10.215 }, - /* 4. */ { 1.533, 2.132, 2.776, 3.747, 4.604, 7.173 }, - /* 5. */ { 1.476, 2.015, 2.571, 3.365, 4.032, 5.893 }, - /* 6. */ { 1.440, 1.943, 2.447, 3.143, 3.707, 5.208 }, - /* 7. */ { 1.415, 1.895, 2.365, 2.998, 3.499, 4.782 }, - /* 8. */ { 1.397, 1.860, 2.306, 2.896, 3.355, 4.499 }, - /* 9. */ { 1.383, 1.833, 2.262, 2.821, 3.250, 4.296 }, - /* 10. */ { 1.372, 1.812, 2.228, 2.764, 3.169, 4.143 }, - /* 11. */ { 1.363, 1.796, 2.201, 2.718, 3.106, 4.024 }, - /* 12. */ { 1.356, 1.782, 2.179, 2.681, 3.055, 3.929 }, - /* 13. */ { 1.350, 1.771, 2.160, 2.650, 3.012, 3.852 }, - /* 14. */ { 1.345, 1.761, 2.145, 2.624, 2.977, 3.787 }, - /* 15. */ { 1.341, 1.753, 2.131, 2.602, 2.947, 3.733 }, - /* 16. */ { 1.337, 1.746, 2.120, 2.583, 2.921, 3.686 }, - /* 17. */ { 1.333, 1.740, 2.110, 2.567, 2.898, 3.646 }, - /* 18. */ { 1.330, 1.734, 2.101, 2.552, 2.878, 3.610 }, - /* 19. */ { 1.328, 1.729, 2.093, 2.539, 2.861, 3.579 }, - /* 20. */ { 1.325, 1.725, 2.086, 2.528, 2.845, 3.552 }, - /* 21. */ { 1.323, 1.721, 2.080, 2.518, 2.831, 3.527 }, - /* 22. */ { 1.321, 1.717, 2.074, 2.508, 2.819, 3.505 }, - /* 23. */ { 1.319, 1.714, 2.069, 2.500, 2.807, 3.485 }, - /* 24. */ { 1.318, 1.711, 2.064, 2.492, 2.797, 3.467 }, - /* 25. */ { 1.316, 1.708, 2.060, 2.485, 2.787, 3.450 }, - /* 26. */ { 1.315, 1.706, 2.056, 2.479, 2.779, 3.435 }, - /* 27. */ { 1.314, 1.703, 2.052, 2.473, 2.771, 3.421 }, - /* 28. */ { 1.313, 1.701, 2.048, 2.467, 2.763, 3.408 }, - /* 29. */ { 1.311, 1.699, 2.045, 2.462, 2.756, 3.396 }, - /* 30. */ { 1.310, 1.697, 2.042, 2.457, 2.750, 3.385 }, - /* 31. */ { 1.309, 1.696, 2.040, 2.453, 2.744, 3.375 }, - /* 32. */ { 1.309, 1.694, 2.037, 2.449, 2.738, 3.365 }, - /* 33. */ { 1.308, 1.692, 2.035, 2.445, 2.733, 3.356 }, - /* 34. */ { 1.307, 1.691, 2.032, 2.441, 2.728, 3.348 }, - /* 35. */ { 1.306, 1.690, 2.030, 2.438, 2.724, 3.340 }, - /* 36. */ { 1.306, 1.688, 2.028, 2.434, 2.719, 3.333 }, - /* 37. */ { 1.305, 1.687, 2.026, 2.431, 2.715, 3.326 }, - /* 38. */ { 1.304, 1.686, 2.024, 2.429, 2.712, 3.319 }, - /* 39. */ { 1.304, 1.685, 2.023, 2.426, 2.708, 3.313 }, - /* 40. */ { 1.303, 1.684, 2.021, 2.423, 2.704, 3.307 }, - /* 41. */ { 1.303, 1.683, 2.020, 2.421, 2.701, 3.301 }, - /* 42. */ { 1.302, 1.682, 2.018, 2.418, 2.698, 3.296 }, - /* 43. */ { 1.302, 1.681, 2.017, 2.416, 2.695, 3.291 }, - /* 44. */ { 1.301, 1.680, 2.015, 2.414, 2.692, 3.286 }, - /* 45. */ { 1.301, 1.679, 2.014, 2.412, 2.690, 3.281 }, - /* 46. */ { 1.300, 1.679, 2.013, 2.410, 2.687, 3.277 }, - /* 47. */ { 1.300, 1.678, 2.012, 2.408, 2.685, 3.273 }, - /* 48. */ { 1.299, 1.677, 2.011, 2.407, 2.682, 3.269 }, - /* 49. */ { 1.299, 1.677, 2.010, 2.405, 2.680, 3.265 }, - /* 50. */ { 1.299, 1.676, 2.009, 2.403, 2.678, 3.261 }, - /* 51. */ { 1.298, 1.675, 2.008, 2.402, 2.676, 3.258 }, - /* 52. */ { 1.298, 1.675, 2.007, 2.400, 2.674, 3.255 }, - /* 53. */ { 1.298, 1.674, 2.006, 2.399, 2.672, 3.251 }, - /* 54. */ { 1.297, 1.674, 2.005, 2.397, 2.670, 3.248 }, - /* 55. */ { 1.297, 1.673, 2.004, 2.396, 2.668, 3.245 }, - /* 56. */ { 1.297, 1.673, 2.003, 2.395, 2.667, 3.242 }, - /* 57. */ { 1.297, 1.672, 2.002, 2.394, 2.665, 3.239 }, - /* 58. */ { 1.296, 1.672, 2.002, 2.392, 2.663, 3.237 }, - /* 59. */ { 1.296, 1.671, 2.001, 2.391, 2.662, 3.234 }, - /* 60. */ { 1.296, 1.671, 2.000, 2.390, 2.660, 3.232 }, - /* 61. */ { 1.296, 1.670, 2.000, 2.389, 2.659, 3.229 }, - /* 62. */ { 1.295, 1.670, 1.999, 2.388, 2.657, 3.227 }, - /* 63. */ { 1.295, 1.669, 1.998, 2.387, 2.656, 3.225 }, - /* 64. */ { 1.295, 1.669, 1.998, 2.386, 2.655, 3.223 }, - /* 65. */ { 1.295, 1.669, 1.997, 2.385, 2.654, 3.220 }, - /* 66. */ { 1.295, 1.668, 1.997, 2.384, 2.652, 3.218 }, - /* 67. */ { 1.294, 1.668, 1.996, 2.383, 2.651, 3.216 }, - /* 68. */ { 1.294, 1.668, 1.995, 2.382, 2.650, 3.214 }, - /* 69. */ { 1.294, 1.667, 1.995, 2.382, 2.649, 3.213 }, - /* 70. */ { 1.294, 1.667, 1.994, 2.381, 2.648, 3.211 }, - /* 71. */ { 1.294, 1.667, 1.994, 2.380, 2.647, 3.209 }, - /* 72. */ { 1.293, 1.666, 1.993, 2.379, 2.646, 3.207 }, - /* 73. */ { 1.293, 1.666, 1.993, 2.379, 2.645, 3.206 }, - /* 74. */ { 1.293, 1.666, 1.993, 2.378, 2.644, 3.204 }, - /* 75. */ { 1.293, 1.665, 1.992, 2.377, 2.643, 3.202 }, - /* 76. */ { 1.293, 1.665, 1.992, 2.376, 2.642, 3.201 }, - /* 77. */ { 1.293, 1.665, 1.991, 2.376, 2.641, 3.199 }, - /* 78. */ { 1.292, 1.665, 1.991, 2.375, 2.640, 3.198 }, - /* 79. */ { 1.292, 1.664, 1.990, 2.374, 2.640, 3.197 }, - /* 80. */ { 1.292, 1.664, 1.990, 2.374, 2.639, 3.195 }, - /* 81. */ { 1.292, 1.664, 1.990, 2.373, 2.638, 3.194 }, - /* 82. */ { 1.292, 1.664, 1.989, 2.373, 2.637, 3.193 }, - /* 83. */ { 1.292, 1.663, 1.989, 2.372, 2.636, 3.191 }, - /* 84. */ { 1.292, 1.663, 1.989, 2.372, 2.636, 3.190 }, - /* 85. */ { 1.292, 1.663, 1.988, 2.371, 2.635, 3.189 }, - /* 86. */ { 1.291, 1.663, 1.988, 2.370, 2.634, 3.188 }, - /* 87. */ { 1.291, 1.663, 1.988, 2.370, 2.634, 3.187 }, - /* 88. */ { 1.291, 1.662, 1.987, 2.369, 2.633, 3.185 }, - /* 89. */ { 1.291, 1.662, 1.987, 2.369, 2.632, 3.184 }, - /* 90. */ { 1.291, 1.662, 1.987, 2.368, 2.632, 3.183 }, - /* 91. */ { 1.291, 1.662, 1.986, 2.368, 2.631, 3.182 }, - /* 92. */ { 1.291, 1.662, 1.986, 2.368, 2.630, 3.181 }, - /* 93. */ { 1.291, 1.661, 1.986, 2.367, 2.630, 3.180 }, - /* 94. */ { 1.291, 1.661, 1.986, 2.367, 2.629, 3.179 }, - /* 95. */ { 1.291, 1.661, 1.985, 2.366, 2.629, 3.178 }, - /* 96. */ { 1.290, 1.661, 1.985, 2.366, 2.628, 3.177 }, - /* 97. */ { 1.290, 1.661, 1.985, 2.365, 2.627, 3.176 }, - /* 98. */ { 1.290, 1.661, 1.984, 2.365, 2.627, 3.175 }, - /* 99. */ { 1.290, 1.660, 1.984, 2.365, 2.626, 3.175 }, - /* 100. */ { 1.290, 1.660, 1.984, 2.364, 2.626, 3.174 } - }; - - std::vector data; - - bool report(size_t confidence_index) - { - if (data.size() != 2) /// Works for two connections only - return true; - - size_t i = (data[0].cnt - 1) + (data[1].cnt - 1); - - double t = students_table[i > 100 ? 0 : i][confidence_index]; - - double spool = (data[0].cnt - 1) * data[0].var() + (data[1].cnt - 1) * data[1].var(); - spool = sqrt(spool / i); - - double s = spool * sqrt(1.0 / data[0].cnt + 1.0 / data[1].cnt); - - double d = data[1].avg() - data[0].avg(); - - double e = t * s; - - std::cerr << '\n'; - if (fabs(d) > e) - { - std::cerr << std::setprecision(1) << "Difference at " << confidence_level[confidence_index] << "% confidence\n" << std::setprecision(6); - std::cerr << "\t" << d << " +/- " << e << "\n"; - std::cerr << "\t" << d * 100 / data[0].avg() << "% +/- " << e * 100 / data[0].avg() << "%\n"; - std::cerr << "\t(Student's t, pooled s = " << spool << ")\n" << std::setprecision(3); - return false; - } - else - { - std::cerr << std::setprecision(1) << "No difference proven at " << confidence_level[confidence_index] << "% confidence\n" << std::setprecision(3); - return true; - } - } - }; - - RelativeAnalysis comparison_relative; - using MultiStats = std::vector>; MultiStats comparison_info_per_interval; MultiStats comparison_info_total; + T_test t_test; Stopwatch total_watch; Stopwatch delay_watch; @@ -434,8 +261,6 @@ private: { printNumberOfQueriesExecuted(queries_executed); cumulative ? report(comparison_info_total) : report(comparison_info_per_interval); - comparison_relative.report(confidence); - delay_watch.restart(); } } @@ -482,7 +307,6 @@ private: printNumberOfQueriesExecuted(queries_executed); report(comparison_info_total); - comparison_relative.report(confidence); } @@ -549,7 +373,7 @@ private: comparison_info_per_interval[connection_index]->add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes); comparison_info_total[connection_index]->add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes); - comparison_relative.data[connection_index].add(seconds); + t_test.add(connection_index, seconds); } void report(MultiStats & infos) @@ -557,9 +381,9 @@ private: std::lock_guard lock(mutex); std::cerr << "\n"; - for (size_t i = 1; i <= infos.size(); ++i) + for (size_t i = 0; i < infos.size(); ++i) { - const auto & info = infos[i - 1]; + const auto & info = infos[i]; /// Avoid zeros, nans or exceptions if (0 == info->queries) @@ -568,7 +392,7 @@ private: double seconds = info->work_time / concurrency; std::cerr - << "connection " << i << ", " + << connections[i]->getDescription() << ", " << "queries " << info->queries << ", " << "QPS: " << (info->queries / seconds) << ", " << "RPS: " << (info->read_rows / seconds) << ", " @@ -577,13 +401,8 @@ private: << "result MiB/s: " << (info->result_bytes / seconds / 1048576) << "." << "\n"; } - - std::cerr << "\n\t\t"; - - for (size_t i = 1; i <= infos.size(); ++i) - std::cerr << "connection " << i << "\t"; - std::cerr << "\n"; + auto print_percentile = [&](double percent) { std::cerr << percent << "%\t\t"; @@ -602,6 +421,8 @@ private: print_percentile(99.9); print_percentile(99.99); + std::cerr << "\n" << t_test.compareAndReport(confidence).second << "\n"; + if (!cumulative) { for (auto & info : infos) @@ -627,11 +448,11 @@ private: json_out << "{\n"; - for (size_t i = 1; i <= infos.size(); ++i) + for (size_t i = 0; i < infos.size(); ++i) { - const auto & info = infos[i - 1]; + const auto & info = infos[i]; - json_out << double_quote << "connection_" + toString(i) << ": {\n"; + json_out << double_quote << connections[i]->getDescription() << ": {\n"; json_out << double_quote << "statistics" << ": {\n"; print_key_value("QPS", info->queries / info->work_time); @@ -653,7 +474,7 @@ private: print_percentile(*info, 99.99, false); json_out << "}\n"; - json_out << (i == infos.size() ? "}\n" : "},\n"); + json_out << (i == infos.size() - 1 ? "}\n" : "},\n"); } json_out << "}\n"; @@ -693,8 +514,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("timelimit,t", value()->default_value(0.), "stop launch of queries after specified time limit") ("randomize,r", value()->default_value(false), "randomize order of execution") ("json", value()->default_value(""), "write final report to specified file in JSON format") - ("host,h", value>()->default_value(std::vector{"localhost"}, "localhost"), "note that more than one host can be described") - ("port,p", value>()->default_value(std::vector{9000}, "9000"), "note that more than one port can be described") + ("host,h", value()->multitoken(), "") + ("port,p", value()->multitoken(), "") ("cumulative", "prints cumulative data instead of data per interval") ("secure,s", "Use TLS connection") ("user", value()->default_value("default"), "") @@ -721,12 +542,14 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) print_stacktrace = options.count("stacktrace"); UseSSL use_ssl; + Ports ports = options.count("port") ? options["port"].as() : Ports({9000}); + Strings hosts = options.count("host") ? options["host"].as() : Strings({"localhost"}); Benchmark benchmark( options["concurrency"].as(), options["delay"].as(), - options["host"].as>(), - options["port"].as>(), + std::move(hosts), + std::move(ports), options.count("cumulative"), options.count("secure"), options["database"].as(), diff --git a/dbms/src/Client/ConnectionPool.h b/dbms/src/Client/ConnectionPool.h index d18be29b2b3..322bad04794 100644 --- a/dbms/src/Client/ConnectionPool.h +++ b/dbms/src/Client/ConnectionPool.h @@ -88,6 +88,10 @@ public: { return host; } + std::string getDescription() const + { + return host + ":" + toString(port); + } protected: /** Creates a new object to put in the pool. */ diff --git a/dbms/src/Common/T_test.h b/dbms/src/Common/T_test.h new file mode 100644 index 00000000000..456ed464e8f --- /dev/null +++ b/dbms/src/Common/T_test.h @@ -0,0 +1,212 @@ +#pragma once + +#include +#include +#include +#include +#include + +/** + * About: + * This is implementation of Independent two-sample t-test + * Read about it on https://en.wikipedia.org/wiki/Student%27s_t-test (Equal or unequal sample sizes, equal variance) + * + * Usage: + * It's it used to assume with some level of confidence that two distributions don't differ. + * Values can be added with T_test.add(0/1, value) and after compared and reported with compareAndReport(). + */ +struct T_test +{ + struct DistributionData + { + size_t size = 0; + double sum = 0; + double squares_sum = 0; + + void add(double value) + { + ++size; + sum += value; + squares_sum += value * value; + } + + double avg() const + { + return sum / size; + } + + double var() const + { + return (squares_sum - (sum * sum / size)) / static_cast(size - 1); + } + + void clear() + { + size = 0; + sum = 0; + squares_sum = 0; + } + }; + + std::vector data; + + const std::vector> students_table = + { + /* inf */ { 1.282, 1.645, 1.960, 2.326, 2.576, 3.090 }, + /* 1. */ { 3.078, 6.314, 12.706, 31.821, 63.657, 318.313}, + /* 2. */ { 1.886, 2.920, 4.303, 6.965, 9.925, 22.327 }, + /* 3. */ { 1.638, 2.353, 3.182, 4.541, 5.841, 10.215 }, + /* 4. */ { 1.533, 2.132, 2.776, 3.747, 4.604, 7.173 }, + /* 5. */ { 1.476, 2.015, 2.571, 3.365, 4.032, 5.893 }, + /* 6. */ { 1.440, 1.943, 2.447, 3.143, 3.707, 5.208 }, + /* 7. */ { 1.415, 1.895, 2.365, 2.998, 3.499, 4.782 }, + /* 8. */ { 1.397, 1.860, 2.306, 2.896, 3.355, 4.499 }, + /* 9. */ { 1.383, 1.833, 2.262, 2.821, 3.250, 4.296 }, + /* 10. */ { 1.372, 1.812, 2.228, 2.764, 3.169, 4.143 }, + /* 11. */ { 1.363, 1.796, 2.201, 2.718, 3.106, 4.024 }, + /* 12. */ { 1.356, 1.782, 2.179, 2.681, 3.055, 3.929 }, + /* 13. */ { 1.350, 1.771, 2.160, 2.650, 3.012, 3.852 }, + /* 14. */ { 1.345, 1.761, 2.145, 2.624, 2.977, 3.787 }, + /* 15. */ { 1.341, 1.753, 2.131, 2.602, 2.947, 3.733 }, + /* 16. */ { 1.337, 1.746, 2.120, 2.583, 2.921, 3.686 }, + /* 17. */ { 1.333, 1.740, 2.110, 2.567, 2.898, 3.646 }, + /* 18. */ { 1.330, 1.734, 2.101, 2.552, 2.878, 3.610 }, + /* 19. */ { 1.328, 1.729, 2.093, 2.539, 2.861, 3.579 }, + /* 20. */ { 1.325, 1.725, 2.086, 2.528, 2.845, 3.552 }, + /* 21. */ { 1.323, 1.721, 2.080, 2.518, 2.831, 3.527 }, + /* 22. */ { 1.321, 1.717, 2.074, 2.508, 2.819, 3.505 }, + /* 23. */ { 1.319, 1.714, 2.069, 2.500, 2.807, 3.485 }, + /* 24. */ { 1.318, 1.711, 2.064, 2.492, 2.797, 3.467 }, + /* 25. */ { 1.316, 1.708, 2.060, 2.485, 2.787, 3.450 }, + /* 26. */ { 1.315, 1.706, 2.056, 2.479, 2.779, 3.435 }, + /* 27. */ { 1.314, 1.703, 2.052, 2.473, 2.771, 3.421 }, + /* 28. */ { 1.313, 1.701, 2.048, 2.467, 2.763, 3.408 }, + /* 29. */ { 1.311, 1.699, 2.045, 2.462, 2.756, 3.396 }, + /* 30. */ { 1.310, 1.697, 2.042, 2.457, 2.750, 3.385 }, + /* 31. */ { 1.309, 1.696, 2.040, 2.453, 2.744, 3.375 }, + /* 32. */ { 1.309, 1.694, 2.037, 2.449, 2.738, 3.365 }, + /* 33. */ { 1.308, 1.692, 2.035, 2.445, 2.733, 3.356 }, + /* 34. */ { 1.307, 1.691, 2.032, 2.441, 2.728, 3.348 }, + /* 35. */ { 1.306, 1.690, 2.030, 2.438, 2.724, 3.340 }, + /* 36. */ { 1.306, 1.688, 2.028, 2.434, 2.719, 3.333 }, + /* 37. */ { 1.305, 1.687, 2.026, 2.431, 2.715, 3.326 }, + /* 38. */ { 1.304, 1.686, 2.024, 2.429, 2.712, 3.319 }, + /* 39. */ { 1.304, 1.685, 2.023, 2.426, 2.708, 3.313 }, + /* 40. */ { 1.303, 1.684, 2.021, 2.423, 2.704, 3.307 }, + /* 41. */ { 1.303, 1.683, 2.020, 2.421, 2.701, 3.301 }, + /* 42. */ { 1.302, 1.682, 2.018, 2.418, 2.698, 3.296 }, + /* 43. */ { 1.302, 1.681, 2.017, 2.416, 2.695, 3.291 }, + /* 44. */ { 1.301, 1.680, 2.015, 2.414, 2.692, 3.286 }, + /* 45. */ { 1.301, 1.679, 2.014, 2.412, 2.690, 3.281 }, + /* 46. */ { 1.300, 1.679, 2.013, 2.410, 2.687, 3.277 }, + /* 47. */ { 1.300, 1.678, 2.012, 2.408, 2.685, 3.273 }, + /* 48. */ { 1.299, 1.677, 2.011, 2.407, 2.682, 3.269 }, + /* 49. */ { 1.299, 1.677, 2.010, 2.405, 2.680, 3.265 }, + /* 50. */ { 1.299, 1.676, 2.009, 2.403, 2.678, 3.261 }, + /* 51. */ { 1.298, 1.675, 2.008, 2.402, 2.676, 3.258 }, + /* 52. */ { 1.298, 1.675, 2.007, 2.400, 2.674, 3.255 }, + /* 53. */ { 1.298, 1.674, 2.006, 2.399, 2.672, 3.251 }, + /* 54. */ { 1.297, 1.674, 2.005, 2.397, 2.670, 3.248 }, + /* 55. */ { 1.297, 1.673, 2.004, 2.396, 2.668, 3.245 }, + /* 56. */ { 1.297, 1.673, 2.003, 2.395, 2.667, 3.242 }, + /* 57. */ { 1.297, 1.672, 2.002, 2.394, 2.665, 3.239 }, + /* 58. */ { 1.296, 1.672, 2.002, 2.392, 2.663, 3.237 }, + /* 59. */ { 1.296, 1.671, 2.001, 2.391, 2.662, 3.234 }, + /* 60. */ { 1.296, 1.671, 2.000, 2.390, 2.660, 3.232 }, + /* 61. */ { 1.296, 1.670, 2.000, 2.389, 2.659, 3.229 }, + /* 62. */ { 1.295, 1.670, 1.999, 2.388, 2.657, 3.227 }, + /* 63. */ { 1.295, 1.669, 1.998, 2.387, 2.656, 3.225 }, + /* 64. */ { 1.295, 1.669, 1.998, 2.386, 2.655, 3.223 }, + /* 65. */ { 1.295, 1.669, 1.997, 2.385, 2.654, 3.220 }, + /* 66. */ { 1.295, 1.668, 1.997, 2.384, 2.652, 3.218 }, + /* 67. */ { 1.294, 1.668, 1.996, 2.383, 2.651, 3.216 }, + /* 68. */ { 1.294, 1.668, 1.995, 2.382, 2.650, 3.214 }, + /* 69. */ { 1.294, 1.667, 1.995, 2.382, 2.649, 3.213 }, + /* 70. */ { 1.294, 1.667, 1.994, 2.381, 2.648, 3.211 }, + /* 71. */ { 1.294, 1.667, 1.994, 2.380, 2.647, 3.209 }, + /* 72. */ { 1.293, 1.666, 1.993, 2.379, 2.646, 3.207 }, + /* 73. */ { 1.293, 1.666, 1.993, 2.379, 2.645, 3.206 }, + /* 74. */ { 1.293, 1.666, 1.993, 2.378, 2.644, 3.204 }, + /* 75. */ { 1.293, 1.665, 1.992, 2.377, 2.643, 3.202 }, + /* 76. */ { 1.293, 1.665, 1.992, 2.376, 2.642, 3.201 }, + /* 77. */ { 1.293, 1.665, 1.991, 2.376, 2.641, 3.199 }, + /* 78. */ { 1.292, 1.665, 1.991, 2.375, 2.640, 3.198 }, + /* 79. */ { 1.292, 1.664, 1.990, 2.374, 2.640, 3.197 }, + /* 80. */ { 1.292, 1.664, 1.990, 2.374, 2.639, 3.195 }, + /* 81. */ { 1.292, 1.664, 1.990, 2.373, 2.638, 3.194 }, + /* 82. */ { 1.292, 1.664, 1.989, 2.373, 2.637, 3.193 }, + /* 83. */ { 1.292, 1.663, 1.989, 2.372, 2.636, 3.191 }, + /* 84. */ { 1.292, 1.663, 1.989, 2.372, 2.636, 3.190 }, + /* 85. */ { 1.292, 1.663, 1.988, 2.371, 2.635, 3.189 }, + /* 86. */ { 1.291, 1.663, 1.988, 2.370, 2.634, 3.188 }, + /* 87. */ { 1.291, 1.663, 1.988, 2.370, 2.634, 3.187 }, + /* 88. */ { 1.291, 1.662, 1.987, 2.369, 2.633, 3.185 }, + /* 89. */ { 1.291, 1.662, 1.987, 2.369, 2.632, 3.184 }, + /* 90. */ { 1.291, 1.662, 1.987, 2.368, 2.632, 3.183 }, + /* 91. */ { 1.291, 1.662, 1.986, 2.368, 2.631, 3.182 }, + /* 92. */ { 1.291, 1.662, 1.986, 2.368, 2.630, 3.181 }, + /* 93. */ { 1.291, 1.661, 1.986, 2.367, 2.630, 3.180 }, + /* 94. */ { 1.291, 1.661, 1.986, 2.367, 2.629, 3.179 }, + /* 95. */ { 1.291, 1.661, 1.985, 2.366, 2.629, 3.178 }, + /* 96. */ { 1.290, 1.661, 1.985, 2.366, 2.628, 3.177 }, + /* 97. */ { 1.290, 1.661, 1.985, 2.365, 2.627, 3.176 }, + /* 98. */ { 1.290, 1.661, 1.984, 2.365, 2.627, 3.175 }, + /* 99. */ { 1.290, 1.660, 1.984, 2.365, 2.626, 3.175 }, + /* 100. */ { 1.290, 1.660, 1.984, 2.364, 2.626, 3.174 } + }; + const std::vector confidence_level = { 80, 90, 95, 98, 99, 99.5 }; + + T_test() + { + data.resize(2); + } + + void clear() + { + data[0].clear(); + data[1].clear(); + } + + void add(size_t distribution, double value) + { + if (distribution > 1) + return; + data[distribution].add(value); + } + + /// Confidence_level_index can be set in range [0, 5]. Corresponding values can be found above. + std::pair compareAndReport(size_t confidence_level_index = 5) const + { + if (confidence_level_index > 5) + confidence_level_index = 5; + + if (data[0].size == 0 || data[1].size == 0) + return {true, ""}; + + size_t degrees_of_freedom = (data[0].size - 1) + (data[1].size - 1); + + double table_value = students_table[degrees_of_freedom > 100 ? 0 : degrees_of_freedom][confidence_level_index]; + + double pooled_standard_deviation = sqrt(((data[0].size - 1) * data[0].var() + (data[1].size - 1) * data[1].var()) / degrees_of_freedom); + + double t_statistic = pooled_standard_deviation * sqrt(1.0 / data[0].size + 1.0 / data[1].size); + + double mean_difference = fabs(data[0].avg() - data[1].avg()); + + double mean_confidence_interval = table_value * t_statistic; + + std::stringstream ss; + if (mean_difference > mean_confidence_interval && (mean_difference - mean_confidence_interval > 0.0001)) /// difference must be more than 0.0001, to take into account connection latency. + { + ss << "Difference at " << confidence_level[confidence_level_index] << "% confidence : "; + ss << std::fixed << std::setprecision(8) << "mean difference is " << mean_difference << ", but confidence interval is " << mean_confidence_interval; + return {false, ss.str()}; + } + else + { + ss << "No difference proven at " << confidence_level[confidence_level_index] << "% confidence"; + return {true, ss.str()}; + } + } + +}; From 8c5833964d424a076a56ed88bc0527b29de8fda0 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 21 Aug 2019 12:02:28 +0300 Subject: [PATCH 307/509] Update docs/en/query_language/select.md Co-Authored-By: Ivan Blinkov --- docs/en/query_language/select.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index 048284faa6a..8611cccb693 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -555,7 +555,7 @@ You can use the following types of syntax: SELECT expressions_list FROM table_1 ASOF LEFT JOIN table_2 ON equi_cond AND closest_match_cond ``` - You can use any number of equality conditions and exactly one closest match condition. For example, `SELECT count() FROM A ASOF LEFT JOIN B ON A.a == B.b AND B.t <= A.t`. There is just `table_2.some_col <= table_1.some_col` and `table_1.some_col >= table2.some_col` types of conditions are available. You cannot apply other conditions like `>`, `!=`. + You can use any number of equality conditions and exactly one closest match condition. For example, `SELECT count() FROM A ASOF LEFT JOIN B ON A.a == B.b AND B.t <= A.t`. There is just `table_2.some_col <= table_1.some_col` and `table_1.some_col >= table2.some_col` types of conditions are available. You cannot apply other conditions like `>` or `!=`. - `ASOF JOIN ... USING` From fde0a2f51efc0bb8b86d0cbe51bb798ff1e75c03 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 21 Aug 2019 12:04:15 +0300 Subject: [PATCH 308/509] Update docs/en/query_language/select.md Co-Authored-By: Ivan Blinkov --- docs/en/query_language/select.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index 8611cccb693..9e6397720f8 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -552,7 +552,10 @@ You can use the following types of syntax: - `ASOF JOIN ... ON` ```sql - SELECT expressions_list FROM table_1 ASOF LEFT JOIN table_2 ON equi_cond AND closest_match_cond + SELECT expressions_list + FROM table_1 + ASOF LEFT JOIN table_2 + ON equi_cond AND closest_match_cond ``` You can use any number of equality conditions and exactly one closest match condition. For example, `SELECT count() FROM A ASOF LEFT JOIN B ON A.a == B.b AND B.t <= A.t`. There is just `table_2.some_col <= table_1.some_col` and `table_1.some_col >= table2.some_col` types of conditions are available. You cannot apply other conditions like `>` or `!=`. From 52728493d50c57546eaa311feda0f1b97f5fe9d0 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 21 Aug 2019 12:04:33 +0300 Subject: [PATCH 309/509] Update docs/en/query_language/select.md Co-Authored-By: Ivan Blinkov --- docs/en/query_language/select.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index 9e6397720f8..fa024aa7cb3 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -563,7 +563,10 @@ You can use the following types of syntax: - `ASOF JOIN ... USING` ```sql - SELECT expressions_list FROM table_1 ASOF JOIN table_2 USING(equi_column1, ... equi_columnN, asof_column) + SELECT expressions_list + FROM table_1 + ASOF JOIN table_2 + USING (equi_column1, ... equi_columnN, asof_column) ``` `ASOF JOIN` uses `equi_columnX` for joining on equality and `asof_column` for joining on the closest match with the `table_1.asof_column >= table2.asof_column` condition. The `asof_column` column must be the last in the `USING` clause. From bfbed4100eeeabf735496733b7eaa70f6c62e003 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 21 Aug 2019 12:05:23 +0300 Subject: [PATCH 310/509] Update docs/en/query_language/select.md Co-Authored-By: Ivan Blinkov --- docs/en/query_language/select.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index fa024aa7cb3..cb3f2bfa92b 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -588,7 +588,8 @@ event_1_2 | 13:00 | 42 event_2_3 | 13:00 | 42 `ASOF JOIN` can take the timestamp of a user event from `table_1` and find an event in `table_2` where the timestamp is closest (equal or less) to the timestamp of the event from `table_1`. Herewith the `user_id` column can be used for joining on equality and the `ev_time` column can be used for joining on the closest match. In our example, `event_1_1` can be joined with `event_2_1`, `event_1_2` can be joined with `event_2_3`, but `event_2_2` cannot be joined. -- `ASOF` join is not supported in the [Join](../operations/table_engines/join.md) table engine. +!!! note "Note" + `ASOF` join is **not** supported in the [Join](../operations/table_engines/join.md) table engine. To set the default strictness value, use the session configuration parameter [join_default_strictness](../operations/settings/settings.md#settings-join_default_strictness). From 8fcb8022c4c6349e0a448c17fd04baf006e97185 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 21 Aug 2019 12:13:25 +0300 Subject: [PATCH 311/509] Update docs/en/operations/system_tables.md Co-Authored-By: Ivan Blinkov --- docs/en/operations/system_tables.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index 2fd86b52c35..e8c8d6692c6 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -299,7 +299,7 @@ Columns: - `max_block_number` (`UInt64`) – The maximum number of data parts that make up the current part after merging. -- `level` (`UInt32`) – Depth of the merge tree. If a merge was not performed, `level=0`. +- `level` (`UInt32`) – Depth of the merge tree. Zero means that current part was created by insert rather than by merging other parts. - `data_version` (`UInt64`) – Block number that is used to determine which mutations should be applied to the data part (the mutations with the bigger version than `data_version`). From 9a8d1266badf66a9a9c770ed7dce1072397e7e89 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 21 Aug 2019 12:14:00 +0300 Subject: [PATCH 312/509] Update docs/en/operations/system_tables.md Co-Authored-By: Ivan Blinkov --- docs/en/operations/system_tables.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index e8c8d6692c6..b84af27c9f1 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -301,7 +301,7 @@ Columns: - `level` (`UInt32`) – Depth of the merge tree. Zero means that current part was created by insert rather than by merging other parts. -- `data_version` (`UInt64`) – Block number that is used to determine which mutations should be applied to the data part (the mutations with the bigger version than `data_version`). +- `data_version` (`UInt64`) – Number that is used to determine which mutations should be applied to the data part (the mutations with the higher version than `data_version`). - `primary_key_bytes_in_memory` (`UInt64`) – The amount of memory (in bytes) used by primary key values. From 3862dd95df4c8a79ce8e375cf99d3b43389f44d9 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 21 Aug 2019 12:14:38 +0300 Subject: [PATCH 313/509] Update docs/en/operations/system_tables.md Co-Authored-By: Ivan Blinkov --- docs/en/operations/system_tables.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index b84af27c9f1..26fff2cf181 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -405,7 +405,7 @@ Columns: - 4 — Exception during the query execution. - `event_date` (Date) — Event date. - `event_time` (DateTime) — Event time. -- `query_start_time` (DateTime) — Start time of query processing. +- `query_start_time` (DateTime) — Start time of query execution. - `query_duration_ms` (UInt64) — Duration of query processing. - `read_rows` (UInt64) — Number of read rows. - `read_bytes` (UInt64) — Number of read bytes. From 0b273cdf91efbeb6a1548f59f910414bb53050ef Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 21 Aug 2019 12:14:50 +0300 Subject: [PATCH 314/509] Update docs/en/operations/system_tables.md Co-Authored-By: Ivan Blinkov --- docs/en/operations/system_tables.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index 26fff2cf181..3880663eae2 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -406,7 +406,7 @@ Columns: - `event_date` (Date) — Event date. - `event_time` (DateTime) — Event time. - `query_start_time` (DateTime) — Start time of query execution. -- `query_duration_ms` (UInt64) — Duration of query processing. +- `query_duration_ms` (UInt64) — Duration of query execution. - `read_rows` (UInt64) — Number of read rows. - `read_bytes` (UInt64) — Number of read bytes. - `written_rows` (UInt64) — For `INSERT` queries, the number of written rows. For other queries, the column value is 0. From 209af3e1c499bb47ca817f6003e14dad0a0204fa Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 21 Aug 2019 13:18:41 +0300 Subject: [PATCH 315/509] DOCAPI-7442: Clarification --- docs/en/operations/system_tables.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index 3880663eae2..a2d9bc02ebb 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -267,7 +267,7 @@ Columns: - `active` (`UInt8`) – Flag that indicates whether the part is active. If a part is active, it is used in a table; otherwise, it will be deleted. Inactive data parts remain after merging. -- `marks` (`UInt64`) – The number of marks. To get the approximate number of rows in a data part, multiply `marks` by the index granularity (usually 8192). +- `marks` (`UInt64`) – The number of marks. To get the approximate number of rows in a data part, multiply `marks` by the index granularity (usually 8192) (this hint doesn't work for adaptive granularity). - `rows` (`UInt64`) – The number of rows. From bd5dabb15c2fc2c32d27661016fe67a788c35c76 Mon Sep 17 00:00:00 2001 From: Dmitry Rubashkin Date: Wed, 21 Aug 2019 13:32:20 +0300 Subject: [PATCH 316/509] --help size fixed --- .../performance-test/PerformanceTestSuite.cpp | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/dbms/programs/performance-test/PerformanceTestSuite.cpp b/dbms/programs/performance-test/PerformanceTestSuite.cpp index 14ea8882a6b..943ee4fe6c2 100644 --- a/dbms/programs/performance-test/PerformanceTestSuite.cpp +++ b/dbms/programs/performance-test/PerformanceTestSuite.cpp @@ -45,6 +45,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int FILE_DOESNT_EXIST; + extern const int SYSTEM_ERROR; } /** Tests launcher for ClickHouse. @@ -321,10 +322,25 @@ std::unordered_map> getTestQueryIndexes(co int mainEntryClickHousePerformanceTest(int argc, char ** argv) try { + using namespace DB; using po::value; - using Strings = DB::Strings; - po::options_description desc("Allowed options"); + unsigned line_length = po::options_description::m_default_line_length; + unsigned min_description_length = line_length / 2; + winsize terminal_size {}; + + if (isatty(STDIN_FILENO)) + { + if (ioctl(STDIN_FILENO, TIOCGWINSZ, &terminal_size)) + throwFromErrno("Cannot obtain terminal window size (ioctl TIOCGWINSZ)", ErrorCodes::SYSTEM_ERROR); + + line_length = std::max( + static_cast(strlen("--http_native_compression_disable_checksumming_on_decompress ")), + static_cast(terminal_size.ws_col)); + min_description_length = std::min(min_description_length, line_length - 2); + } + + po::options_description desc("Allowed options", line_length, min_description_length); desc.add_options() ("help", "produce help message") ("lite", "use lite version of output") From b5e08fa9293f736dd8f4e939ca4cf7112b1ef43f Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Wed, 21 Aug 2019 14:12:58 +0300 Subject: [PATCH 317/509] Tried to fix --- dbms/programs/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/CMakeLists.txt b/dbms/programs/CMakeLists.txt index 6626d90e5f5..03eba470949 100644 --- a/dbms/programs/CMakeLists.txt +++ b/dbms/programs/CMakeLists.txt @@ -98,7 +98,7 @@ endif() if (CLICKHOUSE_SPLIT_BINARY) set (CLICKHOUSE_ALL_TARGETS clickhouse-server clickhouse-client clickhouse-local clickhouse-benchmark clickhouse-performance-test - clickhouse-extract-from-config clickhouse-compressor clickhouse-format clickhouse-copier) + clickhouse-extract-from-config clickhouse-compressor clickhouse-format clickhouse-obfuscator clickhouse-copier) if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-odbc-bridge) From 2a0a051090871e3531de51c3d4ba29bdf1ba7919 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Aug 2019 14:34:40 +0300 Subject: [PATCH 318/509] Enable processors by default. --- dbms/src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 22b0b5c8d03..52610caf8ed 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -333,7 +333,7 @@ struct Settings : public SettingsCollection M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only for 'mysql' table function.") \ M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.") \ \ - M(SettingBool, experimental_use_processors, false, "Use processors pipeline.") \ + M(SettingBool, experimental_use_processors, true, "Use processors pipeline.") \ \ M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \ M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \ From b9870245fd5936f961487d19f53faef41a805119 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 21 Aug 2019 15:28:13 +0300 Subject: [PATCH 319/509] Add upcoming ClickHouse Meetup in Munich --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index ac579d3f2a3..d5b0bf63165 100644 --- a/README.md +++ b/README.md @@ -14,6 +14,7 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events * [ClickHouse Meetup in Moscow](https://yandex.ru/promo/clickhouse/moscow-2019) on September 5. +* [ClickHouse Meetup in Munich](https://www.meetup.com/ClickHouse-Meetup-Munich/events/264185199/) on September 17. * [ClickHouse Meetup in Paris](https://www.eventbrite.com/e/clickhouse-paris-meetup-2019-registration-68493270215) on October 3. * [ClickHouse Meetup in Hong Kong](https://www.meetup.com/Hong-Kong-Machine-Learning-Meetup/events/263580542/) on October 17. * [ClickHouse Meetup in Shenzhen](https://www.huodongxing.com/event/3483759917300) on October 20. From 5cb8605f61e48255986755089ff56c0a68da5953 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 21 Aug 2019 16:10:33 +0300 Subject: [PATCH 320/509] limit number of mutations in the queue --- .../Storages/MergeTree/MergeTreeSettings.h | 3 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 16 ++-- .../MergeTree/ReplicatedMergeTreeQueue.h | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 16 ++-- .../configs/merge_tree_queue.xml | 7 ++ .../test_replicated_mutations/test.py | 83 ++++++++++++++----- 6 files changed, 93 insertions(+), 34 deletions(-) create mode 100644 dbms/tests/integration/test_replicated_mutations/configs/merge_tree_queue.xml diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index e670000ecc5..36e82b96961 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -30,7 +30,8 @@ struct MergeTreeSettings : public SettingsCollection /** Merge settings. */ \ M(SettingUInt64, max_bytes_to_merge_at_max_space_in_pool, 150ULL * 1024 * 1024 * 1024, "Maximum in total size of parts to merge, when there are maximum free threads in background pool (or entries in replication queue).") \ M(SettingUInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).") \ - M(SettingUInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging parts are allowed simultaneously in ReplicatedMergeTree queue.") \ + M(SettingUInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.") \ + M(SettingUInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.") \ M(SettingUInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.") \ M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.") \ M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.") \ diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 665e8c9bd5c..e6251502576 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1154,17 +1154,21 @@ bool ReplicatedMergeTreeQueue::processEntry( } -size_t ReplicatedMergeTreeQueue::countMergesAndPartMutations() const +std::pair ReplicatedMergeTreeQueue::countMergesAndPartMutations() const { std::lock_guard lock(state_mutex); - size_t count = 0; + size_t count_merges = 0; + size_t count_mutations = 0; for (const auto & entry : queue) - if (entry->type == ReplicatedMergeTreeLogEntry::MERGE_PARTS - || entry->type == ReplicatedMergeTreeLogEntry::MUTATE_PART) - ++count; + { + if (entry->type == ReplicatedMergeTreeLogEntry::MERGE_PARTS) + ++count_merges; + else if (entry->type == ReplicatedMergeTreeLogEntry::MUTATE_PART) + ++count_mutations; + } - return count; + return std::make_pair(count_merges, count_mutations); } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 47d82f4a9a2..5a84cfbb5a6 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -296,7 +296,7 @@ public: bool processEntry(std::function get_zookeeper, LogEntryPtr & entry, const std::function func); /// Count the number of merges and mutations of single parts in the queue. - size_t countMergesAndPartMutations() const; + std::pair countMergesAndPartMutations() const; /// Count the total number of active mutations. size_t countMutations() const; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 8b32cc32704..028235d9eef 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2193,27 +2193,31 @@ void StorageReplicatedMergeTree::mergeSelectingTask() /// If many merges is already queued, then will queue only small enough merges. /// Otherwise merge queue could be filled with only large merges, /// and in the same time, many small parts could be created and won't be merged. - size_t merges_and_mutations_queued = queue.countMergesAndPartMutations(); - if (merges_and_mutations_queued >= settings.max_replicated_merges_in_queue) + auto merges_and_mutations_queued = queue.countMergesAndPartMutations(); + size_t merges_and_mutations_sum = merges_and_mutations_queued.first + merges_and_mutations_queued.second; + if (merges_and_mutations_sum >= settings.max_replicated_merges_in_queue) { - LOG_TRACE(log, "Number of queued merges and part mutations (" << merges_and_mutations_queued - << ") is greater than max_replicated_merges_in_queue (" + LOG_TRACE(log, "Number of queued merges (" << merges_and_mutations_queued.first << ") and part mutations (" + << merges_and_mutations_queued.second << ") is greater than max_replicated_merges_in_queue (" << settings.max_replicated_merges_in_queue << "), so won't select new parts to merge or mutate."); } else { UInt64 max_source_parts_size_for_merge = merger_mutator.getMaxSourcePartsSizeForMerge( - settings.max_replicated_merges_in_queue, merges_and_mutations_queued); + settings.max_replicated_merges_in_queue, merges_and_mutations_sum); UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); FutureMergedMutatedPart future_merged_part; + + /// If there are many mutations in queue it may happen, that we cannot enqueue enough merges to merge all new parts if (max_source_parts_size_for_merge > 0 && merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred)) { success = createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, deduplicate, force_ttl); } - else if (max_source_part_size_for_mutation > 0 && queue.countMutations() > 0) + else if (max_source_part_size_for_mutation > 0 && queue.countMutations() > 0 + && merges_and_mutations_queued.second < settings.max_replicated_mutations_in_queue) { /// Choose a part to mutate. DataPartsVector data_parts = getDataPartsVector(); diff --git a/dbms/tests/integration/test_replicated_mutations/configs/merge_tree_queue.xml b/dbms/tests/integration/test_replicated_mutations/configs/merge_tree_queue.xml new file mode 100644 index 00000000000..ccc63f2eaec --- /dev/null +++ b/dbms/tests/integration/test_replicated_mutations/configs/merge_tree_queue.xml @@ -0,0 +1,7 @@ + + + 50 + 8 + 4 + + \ No newline at end of file diff --git a/dbms/tests/integration/test_replicated_mutations/test.py b/dbms/tests/integration/test_replicated_mutations/test.py index 351ceff3608..dd42a70e280 100644 --- a/dbms/tests/integration/test_replicated_mutations/test.py +++ b/dbms/tests/integration/test_replicated_mutations/test.py @@ -10,21 +10,25 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', with_zookeeper=True) +node1 = cluster.add_instance('node1', macros={'cluster': 'test1'}, with_zookeeper=True) # Check, that limits on max part size for merges doesn`t affect mutations -node2 = cluster.add_instance('node2', main_configs=["configs/merge_tree.xml"], with_zookeeper=True) -nodes = [node1, node2] +node2 = cluster.add_instance('node2', macros={'cluster': 'test1'}, main_configs=["configs/merge_tree.xml"], with_zookeeper=True) + +node3 = cluster.add_instance('node3', macros={'cluster': 'test2'}, main_configs=["configs/merge_tree_queue.xml"], with_zookeeper=True) +node4 = cluster.add_instance('node4', macros={'cluster': 'test2'}, main_configs=["configs/merge_tree_queue.xml"], with_zookeeper=True) + +all_nodes = [node1, node2, node3, node4] @pytest.fixture(scope="module") def started_cluster(): try: cluster.start() - for node in nodes: + for node in all_nodes: node.query("DROP TABLE IF EXISTS test_mutations") - for node in nodes: - node.query("CREATE TABLE test_mutations(d Date, x UInt32, i UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/test_mutations', '{instance}') ORDER BY x PARTITION BY toYYYYMM(d)") + for node in all_nodes: + node.query("CREATE TABLE test_mutations(d Date, x UInt32, i UInt32) ENGINE ReplicatedMergeTree('/clickhouse/{cluster}/tables/test/test_mutations', '{instance}') ORDER BY x PARTITION BY toYYYYMM(d)") yield cluster @@ -33,7 +37,8 @@ def started_cluster(): class Runner: - def __init__(self): + def __init__(self, nodes): + self.nodes = nodes self.mtx = threading.Lock() self.total_inserted_xs = 0 self.total_inserted_rows = 0 @@ -49,6 +54,8 @@ class Runner: self.stop_ev = threading.Event() + self.exceptions = [] + def do_insert(self, thread_num): self.stop_ev.wait(random.random()) @@ -76,7 +83,7 @@ class Runner: try: print 'thread {}: insert for {}: {}'.format(thread_num, date_str, ','.join(str(x) for x in xs)) - random.choice(nodes).query("INSERT INTO test_mutations FORMAT TSV", payload) + random.choice(self.nodes).query("INSERT INTO test_mutations FORMAT TSV", payload) with self.mtx: for x in xs: @@ -86,6 +93,7 @@ class Runner: except Exception, e: print 'Exception while inserting,', e + self.exceptions.append(e) finally: with self.mtx: for x in xs: @@ -113,7 +121,7 @@ class Runner: try: print 'thread {}: delete {} * {}'.format(thread_num, to_delete_count, x) - random.choice(nodes).query("ALTER TABLE test_mutations DELETE WHERE x = {}".format(x)) + random.choice(self.nodes).query("ALTER TABLE test_mutations DELETE WHERE x = {}".format(x)) with self.mtx: self.total_mutations += 1 @@ -130,10 +138,23 @@ class Runner: self.stop_ev.wait(1.0 + random.random() * 2) +def wait_for_mutations(nodes, number_of_mutations): + for i in range(100): # wait for replication 80 seconds max + time.sleep(0.8) + + def get_done_mutations(node): + return int(node.query("SELECT sum(is_done) FROM system.mutations WHERE table = 'test_mutations'").rstrip()) + + if all([get_done_mutations(n) == number_of_mutations for n in nodes]): + return True + return False + + def test_mutations(started_cluster): DURATION_SECONDS = 30 + nodes = [node1, node2] - runner = Runner() + runner = Runner(nodes) threads = [] for thread_num in range(5): @@ -155,16 +176,7 @@ def test_mutations(started_cluster): assert runner.total_inserted_rows > 0 assert runner.total_mutations > 0 - all_done = False - for i in range(100): # wait for replication 80 seconds max - time.sleep(0.8) - - def get_done_mutations(node): - return int(node.query("SELECT sum(is_done) FROM system.mutations WHERE table = 'test_mutations'").rstrip()) - - if all([get_done_mutations(n) == runner.total_mutations for n in nodes]): - all_done = True - break + all_done = wait_for_mutations(nodes, runner.total_mutations) print node1.query("SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations' FORMAT TSVWithNames") assert all_done @@ -174,3 +186,34 @@ def test_mutations(started_cluster): for i, node in enumerate(nodes): actual_sums.append(int(node.query("SELECT sum(x) FROM test_mutations").rstrip())) assert actual_sums[i] == expected_sum + + +def test_mutations_dont_prevent_merges(started_cluster): + nodes = [node3, node4] + for year in range(2000, 2008): + rows = '' + date_str = '{}-01-{}'.format(year, random.randint(1, 10)) + for i in range(10): + rows += '{} {} {}\n'.format(date_str, random.randint(1, 10), i) + node3.query("INSERT INTO test_mutations FORMAT TSV", rows) + + # will run mutations of 8 parts in parallel, mutations will sleep for about 20 seconds + node3.query("ALTER TABLE test_mutations UPDATE i = sleepEachRow(2) WHERE 1") + + runner = Runner(nodes) + threads = [] + for thread_num in range(10): + threads.append(threading.Thread(target=runner.do_insert, args=(thread_num, ))) + + # will insert approx 4-5 new parts per 1 second into each partition + for t in threads: + t.start() + + all_done = wait_for_mutations(nodes, 1) + + runner.stop_ev.set() + for t in threads: + t.join() + + assert all_done + assert all([str(e).find("Too many parts") < 0 for e in runner.exceptions]) From c27cada229416aa8de2671735b20483575b408d6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Aug 2019 16:14:23 +0300 Subject: [PATCH 321/509] Fix ExpressionTransform. --- dbms/src/Processors/Transforms/ExpressionTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Processors/Transforms/ExpressionTransform.cpp b/dbms/src/Processors/Transforms/ExpressionTransform.cpp index 2ae9dd6f57f..90ec1031314 100644 --- a/dbms/src/Processors/Transforms/ExpressionTransform.cpp +++ b/dbms/src/Processors/Transforms/ExpressionTransform.cpp @@ -12,7 +12,7 @@ static Block transformHeader(Block header, const ExpressionActionsPtr & expressi ExpressionTransform::ExpressionTransform(const Block & header_, ExpressionActionsPtr expression_, bool on_totals_, bool default_totals_) - : ISimpleTransform(header_, transformHeader(header_, expression), on_totals_) + : ISimpleTransform(header_, transformHeader(header_, expression_), on_totals_) , expression(std::move(expression_)) , on_totals(on_totals_) , default_totals(default_totals_) From d765671e7e1f4be689157d1ba5ae8237941c88a3 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 21 Aug 2019 17:05:59 +0300 Subject: [PATCH 322/509] Support Oracle Linux (#6585) --- debian/clickhouse-server.postinst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/debian/clickhouse-server.postinst b/debian/clickhouse-server.postinst index 944b1e394f1..c47a8ef4be2 100644 --- a/debian/clickhouse-server.postinst +++ b/debian/clickhouse-server.postinst @@ -20,7 +20,7 @@ OS=${OS=`lsb_release -is 2>/dev/null ||:`} [ -f /usr/share/debconf/confmodule ] && . /usr/share/debconf/confmodule [ -f /etc/default/clickhouse ] && . /etc/default/clickhouse -if [ "$OS" = "rhel" ] || [ "$OS" = "centos" ] || [ "$OS" = "fedora" ] || [ "$OS" = "CentOS" ] || [ "$OS" = "Fedora" ]; then +if [ "$OS" = "rhel" ] || [ "$OS" = "centos" ] || [ "$OS" = "fedora" ] || [ "$OS" = "CentOS" ] || [ "$OS" = "Fedora" ] || [ "$OS" = "ol" ]; then is_rh=1 fi From fe3221d0e4f0c4ed86a2e9e096bc5d8c5734491a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Aug 2019 17:19:18 +0300 Subject: [PATCH 323/509] Fix SortingAggregatedTransform. --- .../Transforms/MergingAggregatedMemoryEfficientTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp b/dbms/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp index b9067e955f4..735dafdee14 100644 --- a/dbms/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp +++ b/dbms/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp @@ -334,7 +334,7 @@ void MergingAggregatedBucketTransform::transform(Chunk & chunk) SortingAggregatedTransform::SortingAggregatedTransform(size_t num_inputs_, AggregatingTransformParamsPtr params_) - : IProcessor(InputPorts(num_inputs_, params->getHeader()), {params_->getHeader()}) + : IProcessor(InputPorts(num_inputs_, params_->getHeader()), {params_->getHeader()}) , num_inputs(num_inputs_) , params(std::move(params_)) , last_bucket_number(num_inputs, -1) From caf0d30a5f01f13d6427d067dfdba7ed2e2438eb Mon Sep 17 00:00:00 2001 From: akonyaev Date: Wed, 21 Aug 2019 17:19:47 +0300 Subject: [PATCH 324/509] ADQM-34 add orc input format --- .gitmodules | 3 + CMakeLists.txt | 1 + cmake/find_orc.cmake | 8 + contrib/CMakeLists.txt | 12 + contrib/arrow-cmake/CMakeLists.txt | 67 +++ contrib/orc | 1 + dbms/src/Formats/FormatFactory.cpp | 2 + dbms/src/Formats/config_formats.h.in | 1 + .../Formats/Impl/ArrowColumnToCHColumn.cpp | 420 +++++++++++++++ .../Formats/Impl/ArrowColumnToCHColumn.h | 45 ++ .../Formats/Impl/ORCBlockInputFormat.cpp | 90 ++++ .../Formats/Impl/ORCBlockInputFormat.h | 40 ++ .../Formats/Impl/ParquetBlockInputFormat.cpp | 489 ++---------------- 13 files changed, 736 insertions(+), 443 deletions(-) create mode 100644 cmake/find_orc.cmake create mode 160000 contrib/orc create mode 100644 dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp create mode 100644 dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h create mode 100644 dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp create mode 100644 dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.h diff --git a/.gitmodules b/.gitmodules index 847abf7d931..f6990fed41f 100644 --- a/.gitmodules +++ b/.gitmodules @@ -103,3 +103,6 @@ [submodule "contrib/fastops"] path = contrib/fastops url = https://github.com/ClickHouse-Extras/fastops +[submodule "contrib/orc"] + path = contrib/orc + url = https://github.com/apache/orc diff --git a/CMakeLists.txt b/CMakeLists.txt index 8466fa5d33d..6ac4d67f6ae 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -481,6 +481,7 @@ include (cmake/find_mimalloc.cmake) include (cmake/find_simdjson.cmake) include (cmake/find_rapidjson.cmake) include (cmake/find_fastops.cmake) +include (cmake/find_orc.cmake) find_contrib_lib(cityhash) find_contrib_lib(farmhash) diff --git a/cmake/find_orc.cmake b/cmake/find_orc.cmake new file mode 100644 index 00000000000..3676bec1b6b --- /dev/null +++ b/cmake/find_orc.cmake @@ -0,0 +1,8 @@ +##TODO replace hardcode to find procedure + +set(USE_ORC 0) +set(USE_INTERNAL_ORC_LIBRARY ON) + +if (ARROW_LIBRARY) + set(USE_ORC 1) +endif() \ No newline at end of file diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index c478311d77a..e652c393141 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -10,6 +10,18 @@ endif () set_property(DIRECTORY PROPERTY EXCLUDE_FROM_ALL 1) +if (USE_INTERNAL_ORC_LIBRARY) + set(BUILD_JAVA OFF) + set (ANALYZE_JAVA OFF) + set (BUILD_CPP_TESTS OFF) + set (BUILD_TOOLS OFF) + option(BUILD_JAVA OFF) + option (ANALYZE_JAVA OFF) + option (BUILD_CPP_TESTS OFF) + option (BUILD_TOOLS OFF) + set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${CMAKE_SOURCE_DIR}/contrib/orc/cmake_modules") + add_subdirectory(orc) +endif() if (USE_INTERNAL_UNWIND_LIBRARY) add_subdirectory (libunwind-cmake) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index a7b6628ea4e..7b94acc9031 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -47,6 +47,71 @@ target_include_directories(${THRIFT_LIBRARY} SYSTEM PUBLIC ${ClickHouse_SOURCE_D target_link_libraries(${THRIFT_LIBRARY} PRIVATE Threads::Threads) +# === orc + +set(ORC_SOURCE_DIR ${ClickHouse_SOURCE_DIR}/contrib/orc/c++) +set(ORC_INCLUDE_DIR ${ORC_SOURCE_DIR}/include) +set(ORC_SOURCE_SRC_DIR ${ORC_SOURCE_DIR}/src) +set(ORC_SOURCE_WRAP_DIR ${ORC_SOURCE_DIR}/wrap) + +set(ORC_BUILD_SRC_DIR ${CMAKE_CURRENT_BINARY_DIR}/../orc/c++/src) +set(ORC_BUILD_INCLUDE_DIR ${CMAKE_CURRENT_BINARY_DIR}/../orc/c++/include) + +set(GOOGLE_PROTOBUF_DIR ${ClickHouse_SOURCE_DIR}/contrib/protobuf/src/) +set(ORC_ADDITION_SOURCE_DIR ${CMAKE_CURRENT_BINARY_DIR}) +set(ARROW_SRC_DIR ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src) + +set(PROTOBUF_EXECUTABLE ${CMAKE_CURRENT_BINARY_DIR}/../protobuf/cmake/protoc) +set(PROTO_DIR ${ORC_SOURCE_DIR}/../proto) + + +add_custom_command(OUTPUT orc_proto.pb.h orc_proto.pb.cc + COMMAND ${PROTOBUF_EXECUTABLE} + -I ${PROTO_DIR} + --cpp_out="${CMAKE_CURRENT_BINARY_DIR}" + "${PROTO_DIR}/orc_proto.proto") + +include_directories(SYSTEM ${ORC_INCLUDE_DIR}) +include_directories(SYSTEM ${ORC_SOURCE_SRC_DIR}) +include_directories(SYSTEM ${ORC_SOURCE_WRAP_DIR}) +include_directories(SYSTEM ${GOOGLE_PROTOBUF_DIR}) +include_directories(SYSTEM ${ORC_BUILD_SRC_DIR}) +include_directories(SYSTEM ${ORC_BUILD_INCLUDE_DIR}) +include_directories(SYSTEM ${ORC_ADDITION_SOURCE_DIR}) +include_directories(SYSTEM ${ARROW_SRC_DIR}) + + +set(ORC_SRCS + ${ARROW_SRC_DIR}/arrow/adapters/orc/adapter.cc + ${ORC_SOURCE_SRC_DIR}/Exceptions.cc + ${ORC_SOURCE_SRC_DIR}/OrcFile.cc + ${ORC_SOURCE_SRC_DIR}/Reader.cc + ${ORC_SOURCE_SRC_DIR}/ByteRLE.cc + ${ORC_SOURCE_SRC_DIR}/ColumnPrinter.cc + ${ORC_SOURCE_SRC_DIR}/ColumnReader.cc + ${ORC_SOURCE_SRC_DIR}/ColumnWriter.cc + ${ORC_SOURCE_SRC_DIR}/Common.cc + ${ORC_SOURCE_SRC_DIR}/Compression.cc + ${ORC_SOURCE_SRC_DIR}/Exceptions.cc + ${ORC_SOURCE_SRC_DIR}/Int128.cc + ${ORC_SOURCE_SRC_DIR}/LzoDecompressor.cc + ${ORC_SOURCE_SRC_DIR}/MemoryPool.cc + ${ORC_SOURCE_SRC_DIR}/OrcFile.cc + ${ORC_SOURCE_SRC_DIR}/Reader.cc + ${ORC_SOURCE_SRC_DIR}/RLE.cc + ${ORC_SOURCE_SRC_DIR}/RLEv1.cc + ${ORC_SOURCE_SRC_DIR}/RLEv2.cc + ${ORC_SOURCE_SRC_DIR}/Statistics.cc + ${ORC_SOURCE_SRC_DIR}/StripeStream.cc + ${ORC_SOURCE_SRC_DIR}/Timezone.cc + ${ORC_SOURCE_SRC_DIR}/TypeImpl.cc + ${ORC_SOURCE_SRC_DIR}/Vector.cc + ${ORC_SOURCE_SRC_DIR}/Writer.cc + ${ORC_SOURCE_SRC_DIR}/io/InputStream.cc + ${ORC_SOURCE_SRC_DIR}/io/OutputStream.cc + ${ORC_ADDITION_SOURCE_DIR}/orc_proto.pb.cc + ) + # === arrow @@ -103,6 +168,7 @@ set(ARROW_SRCS ${LIBRARY_DIR}/util/thread-pool.cc ${LIBRARY_DIR}/util/trie.cc ${LIBRARY_DIR}/util/utf8.cc + ${ORC_SRCS} ) set(ARROW_SRCS ${ARROW_SRCS} @@ -151,6 +217,7 @@ endif() add_library(${ARROW_LIBRARY} ${ARROW_SRCS}) +add_dependencies(${ARROW_LIBRARY} protoc) target_include_directories(${ARROW_LIBRARY} SYSTEM PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}/cpp/src ${Boost_INCLUDE_DIRS}) target_link_libraries(${ARROW_LIBRARY} PRIVATE ${DOUBLE_CONVERSION_LIBRARIES} Threads::Threads) if (ARROW_WITH_LZ4) diff --git a/contrib/orc b/contrib/orc new file mode 160000 index 00000000000..5981208e394 --- /dev/null +++ b/contrib/orc @@ -0,0 +1 @@ +Subproject commit 5981208e39447df84827f6a961d1da76bacb6078 diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 75f30e74761..491363e01b4 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -238,6 +238,7 @@ void registerOutputFormatProcessorTSKV(FormatFactory & factory); void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); void registerInputFormatProcessorParquet(FormatFactory & factory); +void registerInputFormatProcessorORC(FormatFactory & factory); void registerOutputFormatProcessorParquet(FormatFactory & factory); void registerInputFormatProcessorProtobuf(FormatFactory & factory); void registerOutputFormatProcessorProtobuf(FormatFactory & factory); @@ -288,6 +289,7 @@ FormatFactory::FormatFactory() registerInputFormatProcessorProtobuf(*this); registerOutputFormatProcessorProtobuf(*this); registerInputFormatProcessorCapnProto(*this); + registerInputFormatProcessorORC(*this); registerInputFormatProcessorParquet(*this); registerOutputFormatProcessorParquet(*this); diff --git a/dbms/src/Formats/config_formats.h.in b/dbms/src/Formats/config_formats.h.in index 7837bed56d0..1ddd0e18aa9 100644 --- a/dbms/src/Formats/config_formats.h.in +++ b/dbms/src/Formats/config_formats.h.in @@ -5,4 +5,5 @@ #cmakedefine01 USE_CAPNP #cmakedefine01 USE_SNAPPY #cmakedefine01 USE_PARQUET +#cmakedefine01 USE_ORC #cmakedefine01 USE_PROTOBUF diff --git a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp new file mode 100644 index 00000000000..24a144b10b5 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -0,0 +1,420 @@ +#include "config_formats.h" +#include "ArrowColumnToCHColumn.h" + +#if USE_ORC or USE_PARQUET +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + const std::unordered_map> arrow_type_to_internal_type ={ + //{arrow::Type::DECIMAL, std::make_shared()}, + {arrow::Type::UINT8, std::make_shared()}, + {arrow::Type::INT8, std::make_shared()}, + {arrow::Type::UINT16, std::make_shared()}, + {arrow::Type::INT16, std::make_shared()}, + {arrow::Type::UINT32, std::make_shared()}, + {arrow::Type::INT32, std::make_shared()}, + {arrow::Type::UINT64, std::make_shared()}, + {arrow::Type::INT64, std::make_shared()}, + {arrow::Type::HALF_FLOAT, std::make_shared()}, + {arrow::Type::FLOAT, std::make_shared()}, + {arrow::Type::DOUBLE, std::make_shared()}, + + {arrow::Type::BOOL, std::make_shared()}, + //{arrow::Type::DATE32, std::make_shared()}, + {arrow::Type::DATE32, std::make_shared()}, + //{arrow::Type::DATE32, std::make_shared()}, + {arrow::Type::DATE64, std::make_shared()}, + {arrow::Type::TIMESTAMP, std::make_shared()}, + //{arrow::Type::TIME32, std::make_shared()}, + + + {arrow::Type::STRING, std::make_shared()}, + {arrow::Type::BINARY, std::make_shared()}, + //{arrow::Type::FIXED_SIZE_BINARY, std::make_shared()}, + //{arrow::Type::UUID, std::make_shared()}, + + + // TODO: add other types that are convertable to internal ones: + // 0. ENUM? + // 1. UUID -> String + // 2. JSON -> String + // Full list of types: contrib/arrow/cpp/src/arrow/type.h + }; + + namespace ErrorCodes + { + extern const int UNKNOWN_TYPE; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; + extern const int CANNOT_READ_ALL_DATA; + extern const int EMPTY_DATA_PASSED; + extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int CANNOT_CONVERT_TYPE; + extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; + extern const int THERE_IS_NO_COLUMN; + } + + template> + static void + fillColumnWithNumericData(std::shared_ptr &arrow_column, MutableColumnPtr &internal_column) + { + auto &column_data = static_cast(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); + chunk_i < num_chunks; ++chunk_i) + { + std::shared_ptr chunk = arrow_column->data()->chunk(chunk_i); + /// buffers[0] is a null bitmap and buffers[1] are actual values + std::shared_ptr buffer = chunk->data()->buffers[1]; + + const auto *raw_data = reinterpret_cast(buffer->data()); + column_data.insert_assume_reserved(raw_data, raw_data + chunk->length()); + } + } + + void fillColumnWithStringData(std::shared_ptr &arrow_column, MutableColumnPtr &internal_column) + { + PaddedPODArray &column_chars_t = static_cast(*internal_column).getChars(); + PaddedPODArray &column_offsets = static_cast(*internal_column).getOffsets(); + + size_t chars_t_size = 0; + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); + chunk_i < num_chunks; ++chunk_i) + { + arrow::BinaryArray &chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); + const size_t chunk_length = chunk.length(); + + chars_t_size += chunk.value_offset(chunk_length - 1) + chunk.value_length(chunk_length - 1); + chars_t_size += chunk_length; /// additional space for null bytes + } + + column_chars_t.reserve(chars_t_size); + column_offsets.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); + chunk_i < num_chunks; ++chunk_i) + { + arrow::BinaryArray &chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); + std::shared_ptr buffer = chunk.value_data(); + const size_t chunk_length = chunk.length(); + + for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i) + { + if (!chunk.IsNull(offset_i) && buffer) + { + const UInt8 *raw_data = buffer->data() + chunk.value_offset(offset_i); + column_chars_t.insert_assume_reserved(raw_data, raw_data + chunk.value_length(offset_i)); + } + column_chars_t.emplace_back('\0'); + + column_offsets.emplace_back(column_chars_t.size()); + } + } + } + + void + fillColumnWithBooleanData(std::shared_ptr &arrow_column, MutableColumnPtr &internal_column) + { + auto &column_data = static_cast &>(*internal_column).getData(); + column_data.resize(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); + chunk_i < num_chunks; ++chunk_i) + { + arrow::BooleanArray &chunk = static_cast(*(arrow_column->data()->chunk( + chunk_i))); + /// buffers[0] is a null bitmap and buffers[1] are actual values + std::shared_ptr buffer = chunk.data()->buffers[1]; + + for (size_t bool_i = 0; bool_i != static_cast(chunk.length()); ++bool_i) + column_data[bool_i] = chunk.Value(bool_i); + } + } + + void + fillColumnWithDate32Data(std::shared_ptr &arrow_column, MutableColumnPtr &internal_column) + { + PaddedPODArray &column_data = static_cast &>(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); + chunk_i < num_chunks; ++chunk_i) + { + arrow::Date32Array &chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); + + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + UInt32 days_num = static_cast(chunk.Value(value_i)); + if (days_num > DATE_LUT_MAX_DAY_NUM) + { + // TODO: will it rollback correctly? + throw Exception + { + "Input value " + std::to_string(days_num) + " of a column \"" + arrow_column->name() + + "\" is greater than " + "max allowed Date value, which is " + + std::to_string(DATE_LUT_MAX_DAY_NUM), + ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE}; + } + + column_data.emplace_back(days_num); + } + } + } + + void + fillColumnWithDate64Data(std::shared_ptr &arrow_column, MutableColumnPtr &internal_column) + { + auto &column_data = static_cast &>(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); + chunk_i < num_chunks; ++chunk_i) + { + auto &chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + auto timestamp = static_cast(chunk.Value(value_i) / 1000); // Always? in ms + column_data.emplace_back(timestamp); + } + } + } + + void + fillColumnWithTimestampData(std::shared_ptr &arrow_column, MutableColumnPtr &internal_column) + { + auto &column_data = static_cast &>(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); + chunk_i < num_chunks; ++chunk_i) + { + auto &chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); + const auto &type = static_cast(*chunk.type()); + + UInt32 divide = 1; + const auto unit = type.unit(); + switch (unit) + { + case arrow::TimeUnit::SECOND: + divide = 1; + break; + case arrow::TimeUnit::MILLI: + divide = 1000; + break; + case arrow::TimeUnit::MICRO: + divide = 1000000; + break; + case arrow::TimeUnit::NANO: + divide = 1000000000; + break; + } + + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + auto timestamp = static_cast(chunk.Value(value_i) / + divide); // ms! TODO: check other 's' 'ns' ... + column_data.emplace_back(timestamp); + } + } + } + + void + fillColumnWithDecimalData(std::shared_ptr &arrow_column, MutableColumnPtr &internal_column) + { + auto &column = static_cast &>(*internal_column); + auto &column_data = column.getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); + chunk_i < num_chunks; ++chunk_i) + { + auto &chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + column_data.emplace_back( + chunk.IsNull(value_i) ? Decimal128(0) : *reinterpret_cast(chunk.Value( + value_i))); // TODO: copy column + } + } + } + +/// Creates a null bytemap from arrow's null bitmap + void + fillByteMapFromArrowColumn(std::shared_ptr &arrow_column, MutableColumnPtr &bytemap) + { + PaddedPODArray &bytemap_data = static_cast &>(*bytemap).getData(); + bytemap_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->data()->num_chunks()); ++chunk_i) + { + std::shared_ptr chunk = arrow_column->data()->chunk(chunk_i); + + for (size_t value_i = 0; value_i != static_cast(chunk->length()); ++value_i) + bytemap_data.emplace_back(chunk->IsNull(value_i)); + } + } + + void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk &res, std::shared_ptr &table, + arrow::Status &read_status, const Block &header, + int &row_group_current, const Context &context) + { + Columns columns_list; + UInt64 num_rows = 0; + + columns_list.reserve(header.rows()); + + using NameToColumnPtr = std::unordered_map>; + if (!read_status.ok()) + throw Exception{"Error while reading ORC data: " + read_status.ToString(), + ErrorCodes::CANNOT_READ_ALL_DATA}; + + if (0 == table->num_rows()) + throw Exception{"Empty table in input data", ErrorCodes::EMPTY_DATA_PASSED}; + + if (header.columns() > static_cast(table->num_columns())) + // TODO: What if some columns were not presented? Insert NULLs? What if a column is not nullable? + throw Exception{"Number of columns is less than the table has", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH}; + + ++row_group_current; + + NameToColumnPtr name_to_column_ptr; + for (size_t i = 0, num_columns = static_cast(table->num_columns()); i < num_columns; ++i) + { + std::shared_ptr arrow_column = table->column(i); + name_to_column_ptr[arrow_column->name()] = arrow_column; + } + + for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) + { + ColumnWithTypeAndName header_column = header.getByPosition(column_i); + + if (name_to_column_ptr.find(header_column.name) == name_to_column_ptr.end()) + // TODO: What if some columns were not presented? Insert NULLs? What if a column is not nullable? + throw Exception{"Column \"" + header_column.name + "\" is not presented in input data", + ErrorCodes::THERE_IS_NO_COLUMN}; + + std::shared_ptr arrow_column = name_to_column_ptr[header_column.name]; + arrow::Type::type arrow_type = arrow_column->type()->id(); + + // TODO: check if a column is const? + if (!header_column.type->isNullable() && arrow_column->null_count()) + { + throw Exception{"Can not insert NULL data into non-nullable column \"" + header_column.name + "\"", + ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN}; + } + + const bool target_column_is_nullable = header_column.type->isNullable() || arrow_column->null_count(); + + DataTypePtr internal_nested_type; + + if (arrow_type == arrow::Type::DECIMAL) + { + const auto decimal_type = static_cast(arrow_column->type().get()); + internal_nested_type = std::make_shared>(decimal_type->precision(), + decimal_type->scale()); + } else if (arrow_type_to_internal_type.find(arrow_type) != arrow_type_to_internal_type.end()) + { + internal_nested_type = arrow_type_to_internal_type.at(arrow_type); + } + else + { + throw Exception + { + "The type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name() + + "\" is not supported for conversion from a ORC data format", + ErrorCodes::CANNOT_CONVERT_TYPE}; + } + + const DataTypePtr internal_type = target_column_is_nullable ? makeNullable(internal_nested_type) + : internal_nested_type; + const std::string internal_nested_type_name = internal_nested_type->getName(); + + const DataTypePtr column_nested_type = header_column.type->isNullable() + ? static_cast(header_column.type.get())->getNestedType() + : header_column.type; + + const DataTypePtr column_type = header_column.type; + + const std::string column_nested_type_name = column_nested_type->getName(); + + ColumnWithTypeAndName column; + column.name = header_column.name; + column.type = internal_type; + + /// Data + MutableColumnPtr read_column = internal_nested_type->createColumn(); + switch (arrow_type) + { + case arrow::Type::STRING: + case arrow::Type::BINARY: + //case arrow::Type::FIXED_SIZE_BINARY: + fillColumnWithStringData(arrow_column, read_column); + break; + case arrow::Type::BOOL: + fillColumnWithBooleanData(arrow_column, read_column); + break; + case arrow::Type::DATE32: + fillColumnWithDate32Data(arrow_column, read_column); + break; + case arrow::Type::DATE64: + fillColumnWithDate64Data(arrow_column, read_column); + break; + case arrow::Type::TIMESTAMP: + fillColumnWithTimestampData(arrow_column, read_column); + break; + case arrow::Type::DECIMAL: + //fillColumnWithNumericData>(arrow_column, read_column); // Have problems with trash values under NULL, but faster + fillColumnWithDecimalData(arrow_column, read_column /*, internal_nested_type*/); + + break; +# define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ + case ARROW_NUMERIC_TYPE: \ + fillColumnWithNumericData(arrow_column, read_column); \ + break; + + FOR_ARROW_NUMERIC_TYPES(DISPATCH) +# undef DISPATCH + // TODO: support TIMESTAMP_MICROS and TIMESTAMP_MILLIS with truncated micro- and milliseconds? + // TODO: read JSON as a string? + // TODO: read UUID as a string? + default: + throw Exception + { + "Unsupported ORC type \"" + arrow_column->type()->name() + "\" of an input column \"" + + arrow_column->name() + "\"", + ErrorCodes::UNKNOWN_TYPE + }; + } + + + if (column.type->isNullable()) + { + MutableColumnPtr null_bytemap = DataTypeUInt8().createColumn(); + fillByteMapFromArrowColumn(arrow_column, null_bytemap); + column.column = ColumnNullable::create(std::move(read_column), std::move(null_bytemap)); + } + else + column.column = std::move(read_column); + + column.column = castColumn(column, column_type, context); + column.type = column_type; + num_rows = column.column->size(); + columns_list.push_back(std::move(column.column)); + } + + res.setColumns(columns_list, num_rows); + } +} +#endif diff --git a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h new file mode 100644 index 00000000000..cf4f021f8c0 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -0,0 +1,45 @@ +#include "config_formats.h" + +#if USE_ORC or USE_PARQUET + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + + class ArrowColumnToCHColumn + { + private: + +# define FOR_ARROW_NUMERIC_TYPES(M) \ + M(arrow::Type::UINT8, DB::UInt8) \ + M(arrow::Type::INT8, DB::Int8) \ + M(arrow::Type::UINT16, DB::UInt16) \ + M(arrow::Type::INT16, DB::Int16) \ + M(arrow::Type::UINT32, DB::UInt32) \ + M(arrow::Type::INT32, DB::Int32) \ + M(arrow::Type::UINT64, DB::UInt64) \ + M(arrow::Type::INT64, DB::Int64) \ + M(arrow::Type::FLOAT, DB::Float32) \ + M(arrow::Type::DOUBLE, DB::Float64) + + + public: + + static void arrowTableToCHChunk(Chunk &res, std::shared_ptr &table, + arrow::Status &read_status, const Block &header, + int &row_group_current, const Context &context); + }; +} +#endif diff --git a/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp new file mode 100644 index 00000000000..4da6b10f3ab --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -0,0 +1,90 @@ +#include "ORCBlockInputFormat.h" +#if USE_ORC + +#include +#include +#include +#include +#include +#include +#include +#include "ArrowColumnToCHColumn.h" + +namespace DB +{ + ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer &in_, Block header_, const Context &context_) + : IInputFormat(std::move(header_), in_), context{context_} { + } + + Chunk ORCBlockInputFormat::generate() + { + Chunk res; + + auto &header = getPort().getHeader(); + + if (!in.eof()) + { + if (row_group_current < row_group_total) + throw Exception{"Got new data, but data from previous chunks was not read " + + std::to_string(row_group_current) + "/" + std::to_string(row_group_total), + ErrorCodes::CANNOT_READ_ALL_DATA}; + + file_data.clear(); + { + WriteBufferFromString file_buffer(file_data); + copyData(in, file_buffer); + } + + std::unique_ptr local_buffer = std::make_unique(file_data); + + + std::shared_ptr in_stream(new arrow::io::BufferReader(*local_buffer)); + + bool ok = arrow::adapters::orc::ORCFileReader::Open(in_stream, arrow::default_memory_pool(), + &file_reader).ok(); + if (!ok) + return res; + + row_group_total = file_reader->NumberOfRows(); + row_group_current = 0; + + } else + return res; + + if (row_group_current >= row_group_total) + return res; + std::shared_ptr table; + + arrow::Status read_status = file_reader->Read(&table); + + ArrowColumnToCHColumn::arrowTableToCHChunk(res, table, read_status, header, row_group_current, context); + + return res; + } + + void registerInputFormatProcessorORC(FormatFactory &factory) + { + factory.registerInputFormatProcessor( + "ORC", + [](ReadBuffer &buf, + const Block &sample, + const Context &context, + const RowInputFormatParams &, + const FormatSettings & /* settings */) + { + return std::make_shared(buf, sample, context); + }); + } + +} +#else + +namespace DB +{ + class FormatFactory; + void registerInputFormatProcessorORC(FormatFactory &) + { + } +} + +#endif diff --git a/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.h new file mode 100644 index 00000000000..afc65a2e912 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -0,0 +1,40 @@ +#pragma once + +#include "config_formats.h" +#include +#include +#include + +#if USE_ORC + +#include "arrow/adapters/orc/adapter.h" +#include "arrow/io/interfaces.h" + +namespace DB +{ +class Context; + +class ORCBlockInputFormat: public IInputFormat +{ +public: + ORCBlockInputFormat(ReadBuffer & in_, Block header_, const Context & context_); + + String getName() const override { return "ORCBlockInputFormat"; } + +protected: + Chunk generate() override; + +private: + + // TODO: check that this class implements every part of its parent + + const Context & context; + + std::unique_ptr file_reader; + std::string file_data; + int row_group_total = 0; + int row_group_current = 0; +}; + +} +#endif diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 4f42597eac5..32a55c70e55 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -1,479 +1,82 @@ #include "ParquetBlockInputFormat.h" #if USE_PARQUET -#include -#include -#include -// TODO: clear includes -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include #include #include #include -#include -#include -#include #include #include #include +#include "ArrowColumnToCHColumn.h" namespace DB { -namespace ErrorCodes -{ - extern const int UNKNOWN_TYPE; - extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; - extern const int CANNOT_READ_ALL_DATA; - extern const int EMPTY_DATA_PASSED; - extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; - extern const int CANNOT_CONVERT_TYPE; - extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; - extern const int THERE_IS_NO_COLUMN; -} -ParquetBlockInputFormat::ParquetBlockInputFormat(ReadBuffer & in_, Block header_, const Context & context_) - : IInputFormat(std::move(header_), in_), context{context_} -{ -} - -/// Inserts numeric data right into internal column data to reduce an overhead -template > -static void fillColumnWithNumericData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) -{ - auto & column_data = static_cast(*internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - std::shared_ptr chunk = arrow_column->data()->chunk(chunk_i); - /// buffers[0] is a null bitmap and buffers[1] are actual values - std::shared_ptr buffer = chunk->data()->buffers[1]; - - const auto * raw_data = reinterpret_cast(buffer->data()); - column_data.insert_assume_reserved(raw_data, raw_data + chunk->length()); - } -} - -/// Inserts chars and offsets right into internal column data to reduce an overhead. -/// Internal offsets are shifted by one to the right in comparison with Arrow ones. So the last offset should map to the end of all chars. -/// Also internal strings are null terminated. -static void fillColumnWithStringData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) -{ - PaddedPODArray & column_chars_t = static_cast(*internal_column).getChars(); - PaddedPODArray & column_offsets = static_cast(*internal_column).getOffsets(); - - size_t chars_t_size = 0; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::BinaryArray & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); - const size_t chunk_length = chunk.length(); - - chars_t_size += chunk.value_offset(chunk_length - 1) + chunk.value_length(chunk_length - 1); - chars_t_size += chunk_length; /// additional space for null bytes + ParquetBlockInputFormat::ParquetBlockInputFormat(ReadBuffer &in_, Block header_, const Context &context_) + : IInputFormat(std::move(header_), in_), context{context_} { } - column_chars_t.reserve(chars_t_size); - column_offsets.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) + Chunk ParquetBlockInputFormat::generate() { - arrow::BinaryArray & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); - std::shared_ptr buffer = chunk.value_data(); - const size_t chunk_length = chunk.length(); + Chunk res; + auto &header = getPort().getHeader(); - for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i) + if (!in.eof()) { - if (!chunk.IsNull(offset_i) && buffer) + /* + First we load whole stream into string (its very bad and limiting .parquet file size to half? of RAM) + Then producing blocks for every row_group (dont load big .parquet files with one row_group - it can eat x10+ RAM from .parquet file size) + */ + + if (row_group_current < row_group_total) + throw Exception{"Got new data, but data from previous chunks was not read " + + std::to_string(row_group_current) + "/" + std::to_string(row_group_total), + ErrorCodes::CANNOT_READ_ALL_DATA}; + + file_data.clear(); { - const UInt8 * raw_data = buffer->data() + chunk.value_offset(offset_i); - column_chars_t.insert_assume_reserved(raw_data, raw_data + chunk.value_length(offset_i)); - } - column_chars_t.emplace_back('\0'); - - column_offsets.emplace_back(column_chars_t.size()); - } - } -} - -static void fillColumnWithBooleanData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) -{ - auto & column_data = static_cast &>(*internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::BooleanArray & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); - /// buffers[0] is a null bitmap and buffers[1] are actual values - std::shared_ptr buffer = chunk.data()->buffers[1]; - - for (size_t bool_i = 0; bool_i != static_cast(chunk.length()); ++bool_i) - column_data.emplace_back(chunk.Value(bool_i)); - } -} - -/// Arrow stores Parquet::DATE in Int32, while ClickHouse stores Date in UInt16. Therefore, it should be checked before saving -static void fillColumnWithDate32Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) -{ - PaddedPODArray & column_data = static_cast &>(*internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::Date32Array & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); - - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - UInt32 days_num = static_cast(chunk.Value(value_i)); - if (days_num > DATE_LUT_MAX_DAY_NUM) - { - // TODO: will it rollback correctly? - throw Exception{"Input value " + std::to_string(days_num) + " of a column \"" + arrow_column->name() - + "\" is greater than " - "max allowed Date value, which is " - + std::to_string(DATE_LUT_MAX_DAY_NUM), - ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE}; + WriteBufferFromString file_buffer(file_data); + copyData(in, file_buffer); } - column_data.emplace_back(days_num); + buffer = std::make_unique(file_data); + // TODO: maybe use parquet::RandomAccessSource? + auto reader = parquet::ParquetFileReader::Open(std::make_shared<::arrow::io::BufferReader>(*buffer)); + file_reader = std::make_unique(::arrow::default_memory_pool(), + std::move(reader)); + row_group_total = file_reader->num_row_groups(); + row_group_current = 0; } - } -} + //DUMP(row_group_current, row_group_total); + if (row_group_current >= row_group_total) + return res; -/// Arrow stores Parquet::DATETIME in Int64, while ClickHouse stores DateTime in UInt32. Therefore, it should be checked before saving -static void fillColumnWithDate64Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) -{ - auto & column_data = static_cast &>(*internal_column).getData(); - column_data.reserve(arrow_column->length()); + // TODO: also catch a ParquetException thrown by filereader? + //arrow::Status read_status = filereader.ReadTable(&table); + std::shared_ptr table; + arrow::Status read_status = file_reader->ReadRowGroup(row_group_current, &table); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - auto & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - auto timestamp = static_cast(chunk.Value(value_i) / 1000); // Always? in ms - column_data.emplace_back(timestamp); - } - } -} - -static void fillColumnWithTimestampData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) -{ - auto & column_data = static_cast &>(*internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - auto & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); - const auto & type = static_cast(*chunk.type()); - - UInt32 divide = 1; - const auto unit = type.unit(); - switch (unit) - { - case arrow::TimeUnit::SECOND: - divide = 1; - break; - case arrow::TimeUnit::MILLI: - divide = 1000; - break; - case arrow::TimeUnit::MICRO: - divide = 1000000; - break; - case arrow::TimeUnit::NANO: - divide = 1000000000; - break; - } - - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - auto timestamp = static_cast(chunk.Value(value_i) / divide); // ms! TODO: check other 's' 'ns' ... - column_data.emplace_back(timestamp); - } - } -} - -static void fillColumnWithDecimalData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) -{ - auto & column = static_cast &>(*internal_column); - auto & column_data = column.getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - auto & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - column_data.emplace_back(chunk.IsNull(value_i) ? Decimal128(0) : *reinterpret_cast(chunk.Value(value_i))); // TODO: copy column - } - } -} - -/// Creates a null bytemap from arrow's null bitmap -static void fillByteMapFromArrowColumn(std::shared_ptr & arrow_column, MutableColumnPtr & bytemap) -{ - PaddedPODArray & bytemap_data = static_cast &>(*bytemap).getData(); - bytemap_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->data()->num_chunks()); ++chunk_i) - { - std::shared_ptr chunk = arrow_column->data()->chunk(chunk_i); - - for (size_t value_i = 0; value_i != static_cast(chunk->length()); ++value_i) - bytemap_data.emplace_back(chunk->IsNull(value_i)); - } -} - -# define FOR_ARROW_NUMERIC_TYPES(M) \ - M(arrow::Type::UINT8, UInt8) \ - M(arrow::Type::INT8, Int8) \ - M(arrow::Type::UINT16, UInt16) \ - M(arrow::Type::INT16, Int16) \ - M(arrow::Type::UINT32, UInt32) \ - M(arrow::Type::INT32, Int32) \ - M(arrow::Type::UINT64, UInt64) \ - M(arrow::Type::INT64, Int64) \ - M(arrow::Type::FLOAT, Float32) \ - M(arrow::Type::DOUBLE, Float64) -//M(arrow::Type::HALF_FLOAT, Float32) // TODO - - -using NameToColumnPtr = std::unordered_map>; - -const std::unordered_map> arrow_type_to_internal_type = { - //{arrow::Type::DECIMAL, std::make_shared()}, - {arrow::Type::UINT8, std::make_shared()}, - {arrow::Type::INT8, std::make_shared()}, - {arrow::Type::UINT16, std::make_shared()}, - {arrow::Type::INT16, std::make_shared()}, - {arrow::Type::UINT32, std::make_shared()}, - {arrow::Type::INT32, std::make_shared()}, - {arrow::Type::UINT64, std::make_shared()}, - {arrow::Type::INT64, std::make_shared()}, - {arrow::Type::HALF_FLOAT, std::make_shared()}, - {arrow::Type::FLOAT, std::make_shared()}, - {arrow::Type::DOUBLE, std::make_shared()}, - - {arrow::Type::BOOL, std::make_shared()}, - //{arrow::Type::DATE32, std::make_shared()}, - {arrow::Type::DATE32, std::make_shared()}, - //{arrow::Type::DATE32, std::make_shared()}, - {arrow::Type::DATE64, std::make_shared()}, - {arrow::Type::TIMESTAMP, std::make_shared()}, - //{arrow::Type::TIME32, std::make_shared()}, - - - {arrow::Type::STRING, std::make_shared()}, - {arrow::Type::BINARY, std::make_shared()}, - //{arrow::Type::FIXED_SIZE_BINARY, std::make_shared()}, - //{arrow::Type::UUID, std::make_shared()}, - - - // TODO: add other types that are convertable to internal ones: - // 0. ENUM? - // 1. UUID -> String - // 2. JSON -> String - // Full list of types: contrib/arrow/cpp/src/arrow/type.h -}; - - -Chunk ParquetBlockInputFormat::generate() -{ - Chunk res; - Columns columns_list; - UInt64 num_rows = 0; - auto & header = getPort().getHeader(); - columns_list.reserve(header.rows()); - - if (!in.eof()) - { - /* - First we load whole stream into string (its very bad and limiting .parquet file size to half? of RAM) - Then producing blocks for every row_group (dont load big .parquet files with one row_group - it can eat x10+ RAM from .parquet file size) - */ - - if (row_group_current < row_group_total) - throw Exception{"Got new data, but data from previous chunks not readed " + std::to_string(row_group_current) + "/" + std::to_string(row_group_total), ErrorCodes::CANNOT_READ_ALL_DATA}; - - file_data.clear(); - { - WriteBufferFromString file_buffer(file_data); - copyData(in, file_buffer); - } - - buffer = std::make_unique(file_data); - // TODO: maybe use parquet::RandomAccessSource? - auto reader = parquet::ParquetFileReader::Open(std::make_shared<::arrow::io::BufferReader>(*buffer)); - file_reader = std::make_unique(::arrow::default_memory_pool(), std::move(reader)); - row_group_total = file_reader->num_row_groups(); - row_group_current = 0; - } - //DUMP(row_group_current, row_group_total); - if (row_group_current >= row_group_total) + ArrowColumnToCHColumn::arrowTableToCHChunk(res, table, read_status, header, row_group_current, context); return res; - - // TODO: also catch a ParquetException thrown by filereader? - //arrow::Status read_status = filereader.ReadTable(&table); - std::shared_ptr table; - arrow::Status read_status = file_reader->ReadRowGroup(row_group_current, &table); - - if (!read_status.ok()) - throw Exception{"Error while reading parquet data: " + read_status.ToString(), ErrorCodes::CANNOT_READ_ALL_DATA}; - - if (0 == table->num_rows()) - throw Exception{"Empty table in input data", ErrorCodes::EMPTY_DATA_PASSED}; - - if (header.columns() > static_cast(table->num_columns())) - // TODO: What if some columns were not presented? Insert NULLs? What if a column is not nullable? - throw Exception{"Number of columns is less than the table has", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH}; - - ++row_group_current; - - NameToColumnPtr name_to_column_ptr; - for (size_t i = 0, num_columns = static_cast(table->num_columns()); i < num_columns; ++i) - { - std::shared_ptr arrow_column = table->column(i); - name_to_column_ptr[arrow_column->name()] = arrow_column; } - for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) + void registerInputFormatProcessorParquet(FormatFactory &factory) { - ColumnWithTypeAndName header_column = header.getByPosition(column_i); - - if (name_to_column_ptr.find(header_column.name) == name_to_column_ptr.end()) - // TODO: What if some columns were not presented? Insert NULLs? What if a column is not nullable? - throw Exception{"Column \"" + header_column.name + "\" is not presented in input data", ErrorCodes::THERE_IS_NO_COLUMN}; - - std::shared_ptr arrow_column = name_to_column_ptr[header_column.name]; - arrow::Type::type arrow_type = arrow_column->type()->id(); - - // TODO: check if a column is const? - if (!header_column.type->isNullable() && arrow_column->null_count()) - { - throw Exception{"Can not insert NULL data into non-nullable column \"" + header_column.name + "\"", - ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN}; - } - - const bool target_column_is_nullable = header_column.type->isNullable() || arrow_column->null_count(); - - DataTypePtr internal_nested_type; - - if (arrow_type == arrow::Type::DECIMAL) - { - const auto decimal_type = static_cast(arrow_column->type().get()); - internal_nested_type = std::make_shared>(decimal_type->precision(), decimal_type->scale()); - } - else if (arrow_type_to_internal_type.find(arrow_type) != arrow_type_to_internal_type.end()) - { - internal_nested_type = arrow_type_to_internal_type.at(arrow_type); - } - else - { - throw Exception{"The type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name() - + "\" is not supported for conversion from a Parquet data format", - ErrorCodes::CANNOT_CONVERT_TYPE}; - } - - const DataTypePtr internal_type = target_column_is_nullable ? makeNullable(internal_nested_type) : internal_nested_type; - const std::string internal_nested_type_name = internal_nested_type->getName(); - - const DataTypePtr column_nested_type = header_column.type->isNullable() - ? static_cast(header_column.type.get())->getNestedType() - : header_column.type; - - const DataTypePtr column_type = header_column.type; - - const std::string column_nested_type_name = column_nested_type->getName(); - - ColumnWithTypeAndName column; - column.name = header_column.name; - column.type = internal_type; - - /// Data - MutableColumnPtr read_column = internal_nested_type->createColumn(); - switch (arrow_type) - { - case arrow::Type::STRING: - case arrow::Type::BINARY: - //case arrow::Type::FIXED_SIZE_BINARY: - fillColumnWithStringData(arrow_column, read_column); - break; - case arrow::Type::BOOL: - fillColumnWithBooleanData(arrow_column, read_column); - break; - case arrow::Type::DATE32: - fillColumnWithDate32Data(arrow_column, read_column); - break; - case arrow::Type::DATE64: - fillColumnWithDate64Data(arrow_column, read_column); - break; - case arrow::Type::TIMESTAMP: - fillColumnWithTimestampData(arrow_column, read_column); - break; - case arrow::Type::DECIMAL: - //fillColumnWithNumericData>(arrow_column, read_column); // Have problems with trash values under NULL, but faster - fillColumnWithDecimalData(arrow_column, read_column /*, internal_nested_type*/); - break; -# define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ - case ARROW_NUMERIC_TYPE: \ - fillColumnWithNumericData(arrow_column, read_column); \ - break; - - FOR_ARROW_NUMERIC_TYPES(DISPATCH) -# undef DISPATCH - // TODO: support TIMESTAMP_MICROS and TIMESTAMP_MILLIS with truncated micro- and milliseconds? - // TODO: read JSON as a string? - // TODO: read UUID as a string? - default: - throw Exception{"Unsupported parquet type \"" + arrow_column->type()->name() + "\" of an input column \"" - + arrow_column->name() + "\"", - ErrorCodes::UNKNOWN_TYPE}; - } - - if (column.type->isNullable()) - { - MutableColumnPtr null_bytemap = DataTypeUInt8().createColumn(); - fillByteMapFromArrowColumn(arrow_column, null_bytemap); - column.column = ColumnNullable::create(std::move(read_column), std::move(null_bytemap)); - } - else - { - column.column = std::move(read_column); - } - - column.column = castColumn(column, column_type, context); - column.type = column_type; - num_rows = column.column->size(); - columns_list.push_back(std::move(column.column)); + factory.registerInputFormatProcessor( + "Parquet", + [](ReadBuffer &buf, + const Block &sample, + const Context &context, + const RowInputFormatParams &, + const FormatSettings & /* settings */) + { + return std::make_shared(buf, sample, context); + }); } - res.setColumns(columns_list, num_rows); - return res; -} - -void registerInputFormatProcessorParquet(FormatFactory & factory) -{ - factory.registerInputFormatProcessor( - "Parquet", - [](ReadBuffer & buf, - const Block & sample, - const Context & context, - const RowInputFormatParams &, - const FormatSettings & /* settings */){ return std::make_shared(buf, sample, context); }); -} - } #else From ae81fb5acedcd8a3ed45901d686cb40e512d38b4 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 21 Aug 2019 17:30:12 +0300 Subject: [PATCH 325/509] DOCAPI-7103: RU translation for CollapsingMergeTree update. (#6593) --- .../table_engines/collapsingmergetree.md | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/docs/ru/operations/table_engines/collapsingmergetree.md b/docs/ru/operations/table_engines/collapsingmergetree.md index f09d84cf8e8..59fdf66324f 100644 --- a/docs/ru/operations/table_engines/collapsingmergetree.md +++ b/docs/ru/operations/table_engines/collapsingmergetree.md @@ -2,7 +2,7 @@ Движок наследует функциональность от [MergeTree](mergetree.md) и добавляет в алгоритм слияния кусков данных логику сворачивания (удаления) строк. -`CollapsingMergeTree` асинхронно удаляет (сворачивает) пары строк, если все поля в строке эквивалентны, за исключением специального поля `Sign`, которое может принимать значения `1` и `-1`. Строки без пары сохраняются. Подробнее смотрите раздел [Сворачивание (удаление) строк](#table_engine-collapsingmergetree-collapsing). +`CollapsingMergeTree` асинхронно удаляет (сворачивает) пары строк, если все поля в ключе сортировки (`ORDER BY`) эквивалентны, за исключением специального поля `Sign`, которое может принимать значения `1` и `-1`. Строки без пары сохраняются. Подробнее смотрите в разделе [Сворачивание (удаление) строк](#table_engine-collapsingmergetree-collapsing). Движок может значительно уменьшить объем хранения и, как следствие, повысить эффективность запросов `SELECT`. @@ -30,7 +30,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] Тип данных столбца — `Int8`. **Секции запроса** -При создании таблицы `CollapsingMergeTree` используются те же [секции](mergetree.md#table_engine-mergetree-creating-a-table) запроса, что и при создании таблицы `MergeTree`. + +При создании таблицы с движком `CollapsingMergeTree` используются те же [секции запроса](mergetree.md#table_engine-mergetree-creating-a-table) что и при создании таблицы с движком `MergeTree`.
Устаревший способ создания таблицы @@ -79,7 +80,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] └─────────────────────┴───────────┴──────────┴──────┘ ``` -Первая строка отменяет предыдущее состояние объекта (пользователя). Она должен повторять все поля отменённого состояния за исключением `Sign`. +Первая строка отменяет предыдущее состояние объекта (пользователя). Она должен повторять все поля из ключа сортировки для отменённого состояния за исключением `Sign`. Вторая строка содержит текущее состояние. @@ -94,34 +95,34 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] можно удалить, сворачивая (удаляя) устаревшее состояние объекта. `CollapsingMergeTree` выполняет это при слиянии кусков данных. -Зачем нужны 2 строки для каждого изменения, читайте в параграфе [Алгоритм](#table_engine-collapsingmergetree-collapsing-algorithm). +Зачем нужны две строки для каждого изменения описано в разделе [Алгоритм](#table_engine-collapsingmergetree-collapsing-algorithm). **Особенности подхода** -1. Программа, которая записывает данные, должна помнить состояние объекта, чтобы иметь возможность отменить его. Строка отмены состояния должна быть копией предыдущей строки состояния с противоположным значением `Sign`. Это увеличивает начальный размер хранилища, но позволяет быстро записывать данные. +1. Программа, которая записывает данные, должна помнить состояние объекта, чтобы иметь возможность отменить его. Строка отмены состояния должна содержать копию полей сортировочного ключа предыдущей строки состояния с противоположным значением `Sign`. Это увеличивает начальный размер хранилища, но позволяет быстро записывать данные. 2. Длинные растущие массивы в Столбцах снижают эффективность работы движка за счёт нагрузки на запись. Чем проще данные, тем выше эффективность. 3. Результаты запроса `SELECT` сильно зависят от согласованности истории изменений объекта. Будьте точны при подготовке данных для вставки. Можно получить непредсказуемые результаты для несогласованных данных, например отрицательные значения для неотрицательных метрик, таких как глубина сеанса. ### Алгоритм {#table_engine-collapsingmergetree-collapsing-algorithm} -Когда ClickHouse объединяет куски данных, каждая группа последовательных строк с одним и тем же первичным ключом уменьшается до не более чем двух строк, одна из которых имеет `Sign = 1` (строка состояния), а другая строка с `Sign = -1` (строка отмены состояния). Другими словами, записи сворачиваются. +Во время объединения кусков данных, каждая группа последовательных строк с одинаковым сортировочным ключом (`ORDER BY`) уменьшается до не более чем двух строк, одна из которых имеет `Sign = 1` (строка состояния), а другая строка с `Sign = -1` (строка отмены состояния). Другими словами, записи сворачиваются. Для каждого результирующего куска данных ClickHouse сохраняет: 1. Первую строку отмены состояния и последнюю строку состояния, если количество строк обоих видов совпадает. - 1. Последнюю строку состояния, если строк состояния на одну больше, чем строк отмены состояния. + 2. Последнюю строку состояния, если строк состояния на одну больше, чем строк отмены состояния. - 1. Первую строку отмены состояния, если их на одну больше, чем строк состояния. + 3. Первую строку отмены состояния, если их на одну больше, чем строк состояния. - 1. Ни в одну из строк во всех остальных случаях. + 4. Ни в одну из строк во всех остальных случаях. Слияние продолжается, но ClickHouse рассматривает эту ситуацию как логическую ошибку и записывает её в журнал сервера. Эта ошибка может возникать, если одни и те же данные вставлялись несколько раз. Как видно, от сворачивания не должны меняться результаты расчётов статистик. Изменения постепенно сворачиваются так, что остаются лишь последнее состояние почти каждого объекта. -Столбец `Sign` необходим, поскольку алгоритм слияния не гарантирует, что все строки с одинаковым первичным ключом будут находиться в одном результирующем куске данных и даже на одном физическом сервере. ClickHouse выполняет запросы `SELECT` несколькими потоками, и он не может предсказать порядок строк в результате. Если необходимо получить полностью свёрнутые данные из таблицы `CollapsingMergeTree`, то необходимо агрегирование. +Столбец `Sign` необходим, поскольку алгоритм слияния не гарантирует, что все строки с одинаковым ключом сортировки будут находиться в одном результирующем куске данных и даже на одном физическом сервере. ClickHouse выполняет запросы `SELECT` несколькими потоками, и он не может предсказать порядок строк в результате. Если необходимо получить полностью свёрнутые данные из таблицы `CollapsingMergeTree`, то необходимо агрегирование. Для завершения свертывания добавьте в запрос секцию`GROUP BY` и агрегатные функции, которые учитывают знак. Например, для расчета количества используйте `sum(Sign)` вместо`count()`. Чтобы вычислить сумму чего-либо, используйте `sum(Sign * x)` вместо`sum(х)`, и так далее, а также добавьте `HAVING sum(Sign) > 0` . @@ -131,7 +132,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ## Пример использования -Example data: +Исходные данные: ``` ┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ @@ -220,3 +221,4 @@ SELECT * FROM UAct FINAL Такой способ выбора данных очень неэффективен. Не используйте его для больших таблиц. [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/collapsingmergetree/) + From e68775e3d0f3a8201d8f4d1cc319dcd99295099c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Aug 2019 17:51:32 +0300 Subject: [PATCH 326/509] Fix TotalsHavingTransform. --- dbms/src/Processors/Transforms/TotalsHavingTransform.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Processors/Transforms/TotalsHavingTransform.cpp b/dbms/src/Processors/Transforms/TotalsHavingTransform.cpp index cce091b1999..b6931f2c8e1 100644 --- a/dbms/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/dbms/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -161,6 +161,8 @@ void TotalsHavingTransform::transform(Chunk & chunk) if (const_filter_description.always_true) { addToTotals(chunk, nullptr); + auto num_rows = columns.front()->size(); + chunk.setColumns(std::move(columns), num_rows); return; } From 51f6d9751156a506807aba97962b7ca52bbf425e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 21 Aug 2019 18:20:42 +0300 Subject: [PATCH 327/509] Update formats.md --- docs/en/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index c5dbbce674d..2f409706c61 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -146,7 +146,7 @@ SELECT * FROM t_null FORMAT TSKV x=1 y=\N ``` -When there is a large number of small columns, this format is ineffective, and there is generally no reason to use it. It is used in some departments of Yandex. +When there is a large number of small columns, this format is ineffective, and there is generally no reason to use it. Nevertheless, it is no worse than JSONEachRow in terms of efficiency. Both data output and parsing are supported in this format. For parsing, any order is supported for the values of different columns. It is acceptable for some values to be omitted – they are treated as equal to their default values. In this case, zeros and blank rows are used as default values. Complex values that could be specified in the table are not supported as defaults. From a1560448d632ea5dbef1e9bbe2b66c03285598f5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 21 Aug 2019 18:21:36 +0300 Subject: [PATCH 328/509] Update formats.md --- docs/ru/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 20c919665d5..130a32d63fa 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -145,7 +145,7 @@ SELECT * FROM t_null FORMAT TSKV x=1 y=\N ``` -При большом количестве маленьких столбцов, этот формат существенно неэффективен, и обычно нет причин его использовать. Он реализован, так как используется в некоторых отделах Яндекса. +При большом количестве маленьких столбцов, этот формат существенно неэффективен, и обычно нет причин его использовать. Впрочем, он не хуже формата JSONEachRow по производительности. Поддерживается как вывод, так и парсинг данных в этом формате. При парсинге, поддерживается расположение значений разных столбцов в произвольном порядке. Допустимо отсутствие некоторых значений - тогда они воспринимаются как равные значениям по умолчанию. В этом случае в качестве значений по умолчанию используются нули и пустые строки. Сложные значения, которые могут быть заданы в таблице не поддерживаются как значения по умолчанию. From 2e03ac09ecdc103ee0fb029aa6445c0003df595b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Aug 2019 19:38:27 +0300 Subject: [PATCH 329/509] Fix MergingSortedTransform. --- dbms/src/Processors/Transforms/MergingSortedTransform.cpp | 3 ++- dbms/src/Processors/Transforms/MergingSortedTransform.h | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Transforms/MergingSortedTransform.cpp b/dbms/src/Processors/Transforms/MergingSortedTransform.cpp index 8857ec876d7..e37eae82de1 100644 --- a/dbms/src/Processors/Transforms/MergingSortedTransform.cpp +++ b/dbms/src/Processors/Transforms/MergingSortedTransform.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -13,7 +14,7 @@ MergingSortedTransform::MergingSortedTransform( UInt64 limit_, bool quiet_, bool have_all_inputs_) - : IProcessor(InputPorts(num_inputs, header), {header}) + : IProcessor(InputPorts(num_inputs, header), {materializeBlock(header)}) , description(description_), max_block_size(max_block_size_), limit(limit_), quiet(quiet_) , have_all_inputs(have_all_inputs_) , merged_data(header), source_chunks(num_inputs), cursors(num_inputs) diff --git a/dbms/src/Processors/Transforms/MergingSortedTransform.h b/dbms/src/Processors/Transforms/MergingSortedTransform.h index 5a1f417fdb6..223d5253e62 100644 --- a/dbms/src/Processors/Transforms/MergingSortedTransform.h +++ b/dbms/src/Processors/Transforms/MergingSortedTransform.h @@ -93,7 +93,7 @@ protected: columns = chunk.mutateColumns(); if (limit_rows && num_rows > limit_rows) for (auto & column : columns) - column = (*column->cut(0, limit_rows)).mutate(); + column = (*column->cut(0, limit_rows)->convertToFullColumnIfConst()).mutate(); total_merged_rows += num_rows; merged_rows = num_rows; From 321233ddaa5934985686c99e290b66621369575d Mon Sep 17 00:00:00 2001 From: akonyaev Date: Wed, 21 Aug 2019 19:39:20 +0300 Subject: [PATCH 330/509] ADQM-34 fix hardcode for,at name in ArrowColumnToCHColumn class --- .../src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 8 ++++---- dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h | 2 +- dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp | 2 +- .../Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 24a144b10b5..fc1277c4a88 100644 --- a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -268,7 +268,7 @@ namespace DB void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk &res, std::shared_ptr &table, arrow::Status &read_status, const Block &header, - int &row_group_current, const Context &context) + int &row_group_current, const Context &context, std::string format_name) { Columns columns_list; UInt64 num_rows = 0; @@ -277,7 +277,7 @@ namespace DB using NameToColumnPtr = std::unordered_map>; if (!read_status.ok()) - throw Exception{"Error while reading ORC data: " + read_status.ToString(), + throw Exception{"Error while reading " + format_name + " data: " + read_status.ToString(), ErrorCodes::CANNOT_READ_ALL_DATA}; if (0 == table->num_rows()) @@ -333,7 +333,7 @@ namespace DB throw Exception { "The type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name() - + "\" is not supported for conversion from a ORC data format", + + "\" is not supported for conversion from a " + format_name + " data format", ErrorCodes::CANNOT_CONVERT_TYPE}; } @@ -392,7 +392,7 @@ namespace DB default: throw Exception { - "Unsupported ORC type \"" + arrow_column->type()->name() + "\" of an input column \"" + "Unsupported " + format_name + " type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name() + "\"", ErrorCodes::UNKNOWN_TYPE }; diff --git a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index cf4f021f8c0..b5f4732d107 100644 --- a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -39,7 +39,7 @@ namespace DB static void arrowTableToCHChunk(Chunk &res, std::shared_ptr &table, arrow::Status &read_status, const Block &header, - int &row_group_current, const Context &context); + int &row_group_current, const Context &context, std::string format_name); }; } #endif diff --git a/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 4da6b10f3ab..2069c3f3cbf 100644 --- a/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -57,7 +57,7 @@ namespace DB arrow::Status read_status = file_reader->Read(&table); - ArrowColumnToCHColumn::arrowTableToCHChunk(res, table, read_status, header, row_group_current, context); + ArrowColumnToCHColumn::arrowTableToCHChunk(res, table, read_status, header, row_group_current, context, "ORC"); return res; } diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 32a55c70e55..54bab6d7467 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -59,7 +59,7 @@ namespace DB std::shared_ptr table; arrow::Status read_status = file_reader->ReadRowGroup(row_group_current, &table); - ArrowColumnToCHColumn::arrowTableToCHChunk(res, table, read_status, header, row_group_current, context); + ArrowColumnToCHColumn::arrowTableToCHChunk(res, table, read_status, header, row_group_current, context, "Parquet"); return res; } From 6d81087bdc4c777cbdb317a3c0e5d1885fb2264c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Aug 2019 19:53:40 +0300 Subject: [PATCH 331/509] Remove _dummy column if it is not needed. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 27be35ad57f..2ccfc762087 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1550,7 +1550,11 @@ void InterpreterSelectQuery::executeFetchColumns( { /// Unify streams in case they have different headers. auto first_header = streams.at(0)->getHeader(); - for (size_t i = 1; i < streams.size(); ++i) + + if (first_header.columns() > 1 && first_header.has("_dummy")) + first_header.erase("_dummy"); + + for (size_t i = 0; i < streams.size(); ++i) { auto & stream = streams[i]; auto header = stream->getHeader(); From d573c4ec3e1263a590d19b5de751fb7e6e8fd1b1 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Wed, 21 Aug 2019 20:42:44 +0300 Subject: [PATCH 332/509] fix wrong check for disabling "read in order" optimization --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 2 +- .../00995_optimize_read_in_order_with_aggregation.reference | 1 + .../00995_optimize_read_in_order_with_aggregation.sql | 6 ++++++ 3 files changed, 8 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/00995_optimize_read_in_order_with_aggregation.reference create mode 100644 dbms/tests/queries/0_stateless/00995_optimize_read_in_order_with_aggregation.sql diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 27be35ad57f..dbc0cf85824 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -884,7 +884,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS } SortingInfoPtr sorting_info; - if (settings.optimize_read_in_order && storage && query.orderBy() && !query.groupBy() && !query.final() && !query.join()) + if (settings.optimize_read_in_order && storage && query.orderBy() && !query_analyzer->hasAggregation() && !query.final() && !query.join()) { if (const MergeTreeData * merge_tree_data = dynamic_cast(storage.get())) sorting_info = optimizeReadInOrder(*merge_tree_data, query, context, syntax_analyzer_result); diff --git a/dbms/tests/queries/0_stateless/00995_optimize_read_in_order_with_aggregation.reference b/dbms/tests/queries/0_stateless/00995_optimize_read_in_order_with_aggregation.reference new file mode 100644 index 00000000000..d567f8a0b01 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00995_optimize_read_in_order_with_aggregation.reference @@ -0,0 +1 @@ +4950 diff --git a/dbms/tests/queries/0_stateless/00995_optimize_read_in_order_with_aggregation.sql b/dbms/tests/queries/0_stateless/00995_optimize_read_in_order_with_aggregation.sql new file mode 100644 index 00000000000..93c907811a5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00995_optimize_read_in_order_with_aggregation.sql @@ -0,0 +1,6 @@ +SET optimize_read_in_order = 1; +DROP TABLE IF EXISTS order_with_aggr; +CREATE TABLE order_with_aggr(a Int) ENGINE = MergeTree ORDER BY a; + +INSERT INTO order_with_aggr SELECT * FROM numbers(100); +SELECT sum(a) as s FROM order_with_aggr ORDER BY s; From 93d380ddaad180df67ec150e03f2bded4c5a52d7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 23:32:58 +0300 Subject: [PATCH 333/509] Fixed test --- dbms/src/Functions/array/arrayIntersect.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/array/arrayIntersect.cpp b/dbms/src/Functions/array/arrayIntersect.cpp index e6cc24a49ba..6aec4f94dc8 100644 --- a/dbms/src/Functions/array/arrayIntersect.cpp +++ b/dbms/src/Functions/array/arrayIntersect.cpp @@ -398,7 +398,7 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable all_nullable = false; } - auto & result_data = assert_cast(*result_data_ptr); + auto & result_data = static_cast(*result_data_ptr); auto result_offsets_ptr = ColumnArray::ColumnOffsets::create(rows); auto & result_offsets = assert_cast(*result_offsets_ptr); auto null_map_column = ColumnUInt8::create(); From e573b06373c7215b25b5e2859575c1a3ccfeebb0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2019 23:33:38 +0300 Subject: [PATCH 334/509] Updated comment --- dbms/src/Functions/nullIf.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/nullIf.cpp b/dbms/src/Functions/nullIf.cpp index 91fb311f444..507abc8d3e1 100644 --- a/dbms/src/Functions/nullIf.cpp +++ b/dbms/src/Functions/nullIf.cpp @@ -43,7 +43,7 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { - /// nullIf(col1, col2) == if(col1 = col2, NULL, 1) + /// nullIf(col1, col2) == if(col1 = col2, NULL, col1) Block temp_block = block; From a7fa71aaf1a19ddf2715acd2048a861afffa9e06 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Aug 2019 00:28:05 +0300 Subject: [PATCH 335/509] Fixed flappy test --- .../queries/0_stateless/00704_drop_truncate_memory_table.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh b/dbms/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh index 170259f0e24..1bbb69f4dd7 100755 --- a/dbms/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh +++ b/dbms/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh @@ -1,6 +1,8 @@ #!/usr/bin/env bash set -e +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh From 6bbf3bd6341910844f185a376e7db994dafa2fa0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Aug 2019 04:30:49 +0300 Subject: [PATCH 336/509] Added a metric for the number of files to send in Distributed tables --- dbms/src/Common/CurrentMetrics.cpp | 1 + dbms/src/Storages/Distributed/DirectoryMonitor.cpp | 3 +++ 2 files changed, 4 insertions(+) diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index b8e30f3cccd..6bd99fb8f01 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -49,6 +49,7 @@ M(GlobalThreadActive, "Number of threads in global thread pool running a task.") \ M(LocalThread, "Number of threads in local thread pools. Should be similar to GlobalThreadActive.") \ M(LocalThreadActive, "Number of threads in local thread pools running a task.") \ + M(DistributedFilesToInsert, "Number of pending files to process for asynchronous insertion into Distributed tables. Number of files for every shard is summed.") \ namespace CurrentMetrics diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp index 7eefc68f3a8..13c9cf3050a 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp @@ -23,6 +23,7 @@ namespace CurrentMetrics { extern const Metric DistributedSend; + extern const Metric DistributedFilesToInsert; } namespace DB @@ -209,6 +210,8 @@ bool StorageDistributedDirectoryMonitor::processFiles() if (files.empty()) return false; + CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedFilesToInsert, CurrentMetrics::Value(files.size())}; + if (should_batch_inserts) { processFilesWithBatching(files); From ae7ae6d660361d25d12b0dc5f555ef924e1ffd9a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Aug 2019 05:05:46 +0300 Subject: [PATCH 337/509] Update CHANGELOG.md --- CHANGELOG.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 607f650deeb..eceeb5db0ac 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -31,6 +31,16 @@ * Fix build with external `libcxx` [#6010](https://github.com/yandex/ClickHouse/pull/6010) ([Ivan](https://github.com/abyss7)) * Fix shared build with `rdkafka` library [#6101](https://github.com/yandex/ClickHouse/pull/6101) ([Ivan](https://github.com/abyss7)) +## ClickHouse release 19.11.8.46, 2019-08-22 + +### Bug Fix +* Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [#6543](https://github.com/yandex/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) +* Fix NPE when using IN clause with a subquery with a tuple. [#6125](https://github.com/yandex/ClickHouse/issues/6125) [#6550](https://github.com/yandex/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) +* Fixed an issue that if a stale replica become alive, it may still have data parts that were removed by DROP PARTITION. [#6522](https://github.com/yandex/ClickHouse/issues/6522) [#6523](https://github.com/yandex/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) +* Fixed issue with parsing CSV [#6426](https://github.com/yandex/ClickHouse/issues/6426) [#6559](https://github.com/yandex/ClickHouse/pull/6559) ([tavplubix](https://github.com/tavplubix)) +* Fixed data race in system.parts table and ALTER query. This fixes [#6245](https://github.com/yandex/ClickHouse/issues/6245). [#6513](https://github.com/yandex/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [#6514](https://github.com/yandex/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) + ## ClickHouse release 19.11.7.40, 2019-08-14 ### Bug fix From 19cb429b06d3ca454621f45caa8ac86e9331bcb5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Aug 2019 05:08:12 +0300 Subject: [PATCH 338/509] Update CHANGELOG.md --- CHANGELOG.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index eceeb5db0ac..32eb446ac01 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,13 @@ +## ClickHouse release 19.13.3, 2019-08-22 + +### Bug Fix +* Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [#6543](https://github.com/yandex/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) +* Fix NPE when using IN clause with a subquery with a tuple. [#6125](https://github.com/yandex/ClickHouse/issues/6125) [#6550](https://github.com/yandex/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) +* Fixed an issue that if a stale replica become alive, it may still have data parts that were removed by DROP PARTITION. [#6522](https://github.com/yandex/ClickHouse/issues/6522) [#6523](https://github.com/yandex/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) +* Fixed issue with parsing CSV [#6426](https://github.com/yandex/ClickHouse/issues/6426) [#6559](https://github.com/yandex/ClickHouse/pull/6559) ([tavplubix](https://github.com/tavplubix)) +* Fixed data race in system.parts table and ALTER query. This fixes [#6245](https://github.com/yandex/ClickHouse/issues/6245). [#6513](https://github.com/yandex/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [#6514](https://github.com/yandex/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) + ## ClickHouse release 19.13.2.19, 2019-08-14 ### New Feature From bbf2911d61089d643524ef08672abf67a488677a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Aug 2019 05:14:01 +0300 Subject: [PATCH 339/509] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 32eb446ac01..6263a197f45 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,4 @@ -## ClickHouse release 19.13.3, 2019-08-22 +## ClickHouse release 19.13.3.26, 2019-08-22 ### Bug Fix * Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [#6543](https://github.com/yandex/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) From 833d6d60a67764ba688f4b8139a58475fb151e32 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Aug 2019 05:14:46 +0300 Subject: [PATCH 340/509] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6263a197f45..74012668d1b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,7 +3,7 @@ ### Bug Fix * Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [#6543](https://github.com/yandex/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) * Fix NPE when using IN clause with a subquery with a tuple. [#6125](https://github.com/yandex/ClickHouse/issues/6125) [#6550](https://github.com/yandex/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) -* Fixed an issue that if a stale replica become alive, it may still have data parts that were removed by DROP PARTITION. [#6522](https://github.com/yandex/ClickHouse/issues/6522) [#6523](https://github.com/yandex/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) +* Fixed an issue that if a stale replica becomes alive, it may still have data parts that were removed by DROP PARTITION. [#6522](https://github.com/yandex/ClickHouse/issues/6522) [#6523](https://github.com/yandex/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) * Fixed issue with parsing CSV [#6426](https://github.com/yandex/ClickHouse/issues/6426) [#6559](https://github.com/yandex/ClickHouse/pull/6559) ([tavplubix](https://github.com/tavplubix)) * Fixed data race in system.parts table and ALTER query. This fixes [#6245](https://github.com/yandex/ClickHouse/issues/6245). [#6513](https://github.com/yandex/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [#6514](https://github.com/yandex/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) From ee89ee0218c86613db1c6856fda8fb3d1140b33b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Aug 2019 05:15:05 +0300 Subject: [PATCH 341/509] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 74012668d1b..7d6714b6474 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -46,7 +46,7 @@ ### Bug Fix * Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [#6543](https://github.com/yandex/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) * Fix NPE when using IN clause with a subquery with a tuple. [#6125](https://github.com/yandex/ClickHouse/issues/6125) [#6550](https://github.com/yandex/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) -* Fixed an issue that if a stale replica become alive, it may still have data parts that were removed by DROP PARTITION. [#6522](https://github.com/yandex/ClickHouse/issues/6522) [#6523](https://github.com/yandex/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) +* Fixed an issue that if a stale replica becomes alive, it may still have data parts that were removed by DROP PARTITION. [#6522](https://github.com/yandex/ClickHouse/issues/6522) [#6523](https://github.com/yandex/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) * Fixed issue with parsing CSV [#6426](https://github.com/yandex/ClickHouse/issues/6426) [#6559](https://github.com/yandex/ClickHouse/pull/6559) ([tavplubix](https://github.com/tavplubix)) * Fixed data race in system.parts table and ALTER query. This fixes [#6245](https://github.com/yandex/ClickHouse/issues/6245). [#6513](https://github.com/yandex/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [#6514](https://github.com/yandex/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) From 85d3ba099ff93e8edc696feaa93c579a6967ea86 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Aug 2019 05:31:49 +0300 Subject: [PATCH 342/509] Added a comment --- dbms/src/Common/SymbolIndex.cpp | 43 +++++++++++++++++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/dbms/src/Common/SymbolIndex.cpp b/dbms/src/Common/SymbolIndex.cpp index 05268c4b6f1..7f494f0dd9b 100644 --- a/dbms/src/Common/SymbolIndex.cpp +++ b/dbms/src/Common/SymbolIndex.cpp @@ -10,6 +10,49 @@ //#include #include +/** + +ELF object can contain three different places with symbol names and addresses: + +1. Symbol table in section headers. It is used for static linking and usually left in executable. +It is not loaded in memory and they are not necessary for program to run. +It does not relate to debug info and present regardless to -g flag. +You can use strip to get rid of this symbol table. +If you have this symbol table in your binary, you can manually read it and get symbol names, even for symbols from anonymous namespaces. + +2. Hashes in program headers such as DT_HASH and DT_GNU_HASH. +It is necessary for dynamic object (.so libraries and any dynamically linked executable that depend on .so libraries) +because it is used for dynamic linking that happens in runtime and performed by dynamic loader. +Only exported symbols will be presented in that hash tables. Symbols from anonymous namespaces are not. +This part of executable binary is loaded in memory and accessible via 'dl_iterate_phdr', 'dladdr' and 'backtrace_symbols' functions from libc. +ClickHouse versions prior to 19.13 has used just these symbol names to symbolize stack traces +and stack traces may be incomplete due to lack of symbols with internal linkage. +But because ClickHouse is linked with most of the symbols exported (-rdynamic flag) it can still provide good enough stack traces. + +3. DWARF debug info. It contains the most detailed information about symbols and everything else. +It allows to get source file names and line numbers from addresses. Only available if you use -g option for compiler. +It is also used by default for ClickHouse builds, but because of its weight (about two gigabytes) +it is splitted to separate binary and provided in clickhouse-common-static-dbg package. +This separate binary is placed in /usr/lib/debug/usr/bin/clickhouse and is loaded automatically by tools like gdb, addr2line. +When you build ClickHouse by yourself, debug info is not splitted and present in a single huge binary. + +What ClickHouse is using to provide good stack traces? + +In versions prior to 19.13, only "program headers" (2) was used. + +In version 19.13, ClickHouse will read program headers (2) and cache them, +also it will read itself as ELF binary and extract symbol tables from section headers (1) +to also symbolize functions that are not exported for dynamic linking. +And finally, it will read DWARF info (3) if available to display file names and line numbers. + +What detail can you obtain depending on your binary? + +If you have debug info (you build ClickHouse by yourself or install clickhouse-common-static-dbg package), you will get source file names and line numbers. +Otherwise you will get only symbol names. If your binary contains symbol table in section headers (the default, unless stripped), you will get all symbol names. +Otherwise you will get only exported symbols from program headers. + +*/ + namespace DB { From 78af6d793d472f6f8903d5a6cff2a2eeb8c004a0 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 22 Aug 2019 12:34:06 +0800 Subject: [PATCH 343/509] Fix segfault when decoding symbol table. --- dbms/src/Common/SymbolIndex.cpp | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/dbms/src/Common/SymbolIndex.cpp b/dbms/src/Common/SymbolIndex.cpp index 7f494f0dd9b..5cb424ef48b 100644 --- a/dbms/src/Common/SymbolIndex.cpp +++ b/dbms/src/Common/SymbolIndex.cpp @@ -107,13 +107,14 @@ void collectSymbolsFromProgramHeaders(dl_phdr_info * info, size_t sym_cnt = 0; for (auto it = dyn_begin; it->d_tag != DT_NULL; ++it) { - if (it->d_tag == DT_HASH) - { - const ElfW(Word) * hash = reinterpret_cast(correct_address(info->dlpi_addr, it->d_un.d_ptr)); - sym_cnt = hash[1]; - break; - } - else if (it->d_tag == DT_GNU_HASH) + // TODO: this branch leads to invalid address of the hash table. Need further investigation. + // if (it->d_tag == DT_HASH) + // { + // const ElfW(Word) * hash = reinterpret_cast(correct_address(info->dlpi_addr, it->d_un.d_ptr)); + // sym_cnt = hash[1]; + // break; + // } + if (it->d_tag == DT_GNU_HASH) { /// This code based on Musl-libc. From 8ac4ee3dca4d51380fd3e2235bb842ce18017509 Mon Sep 17 00:00:00 2001 From: sev7e0 Date: Thu, 22 Aug 2019 14:29:21 +0800 Subject: [PATCH 344/509] Fix data type (enum) table name error in docs --- docs/en/data_types/enum.md | 2 +- docs/ru/data_types/enum.md | 2 +- docs/zh/data_types/enum.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/data_types/enum.md b/docs/en/data_types/enum.md index 247ec070190..3fbe5b3708b 100644 --- a/docs/en/data_types/enum.md +++ b/docs/en/data_types/enum.md @@ -94,7 +94,7 @@ ENGINE = TinyLog it can store not only `'hello'` and `'world'`, but `NULL`, as well. ``` -INSERT INTO t_enum_null Values('hello'),('world'),(NULL) +INSERT INTO t_enum_nullable Values('hello'),('world'),(NULL) ``` In RAM, an `Enum` column is stored in the same way as `Int8` or `Int16` of the corresponding numerical values. diff --git a/docs/ru/data_types/enum.md b/docs/ru/data_types/enum.md index 7ed0150e65f..9191dc5d2b0 100644 --- a/docs/ru/data_types/enum.md +++ b/docs/ru/data_types/enum.md @@ -90,7 +90,7 @@ ENGINE = TinyLog , то в ней можно будет хранить не только `'hello'` и `'world'`, но и `NULL`. ``` -INSERT INTO t_enum_null Values('hello'),('world'),(NULL) +INSERT INTO t_enum_nullable Values('hello'),('world'),(NULL) ``` В оперативке столбец типа `Enum` представлен так же, как `Int8` или `Int16` соответствующими числовыми значениями. diff --git a/docs/zh/data_types/enum.md b/docs/zh/data_types/enum.md index ca8488b4345..41133b56d45 100644 --- a/docs/zh/data_types/enum.md +++ b/docs/zh/data_types/enum.md @@ -91,7 +91,7 @@ ENGINE = TinyLog 不仅可以存储 `'hello'` 和 `'world'` ,还可以存储 `NULL`。 ``` -INSERT INTO t_enum_null Values('hello'),('world'),(NULL) +INSERT INTO t_enum_nullable Values('hello'),('world'),(NULL) ``` 在内存中,`Enum` 列的存储方式与相应数值的 `Int8` 或 `Int16` 相同。 From 1683547b2d7ac65f22cda65dd8a0c769e8ec91e9 Mon Sep 17 00:00:00 2001 From: akonyaev Date: Thu, 22 Aug 2019 10:24:04 +0300 Subject: [PATCH 345/509] ADQM-34 fix style --- dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 15d0ad861ba..8fb6ab5a359 100644 --- a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -27,7 +27,6 @@ namespace DB extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; extern const int THERE_IS_NO_COLUMN; } - const std::unordered_map> arrow_type_to_internal_type = { //{arrow::Type::DECIMAL, std::make_shared()}, {arrow::Type::UINT8, std::make_shared()}, @@ -63,7 +62,7 @@ namespace DB // 2. JSON -> String // Full list of types: contrib/arrow/cpp/src/arrow/type.h }; - + /// Inserts numeric data right into internal column data to reduce an overhead template > static void fillColumnWithNumericData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) From c3aaf583a0df53bdbc62286c170df733993fe404 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2019 11:26:50 +0300 Subject: [PATCH 346/509] Fix RollupTransform. --- .../Processors/Transforms/RollupTransform.cpp | 33 ++++++++++++------- .../Processors/Transforms/RollupTransform.h | 10 +++--- 2 files changed, 28 insertions(+), 15 deletions(-) diff --git a/dbms/src/Processors/Transforms/RollupTransform.cpp b/dbms/src/Processors/Transforms/RollupTransform.cpp index 9c8270ce091..1f5c140aafe 100644 --- a/dbms/src/Processors/Transforms/RollupTransform.cpp +++ b/dbms/src/Processors/Transforms/RollupTransform.cpp @@ -5,7 +5,7 @@ namespace DB { RollupTransform::RollupTransform(Block header, AggregatingTransformParamsPtr params_) - : IInflatingTransform(std::move(header), params_->getHeader()) + : IAccumulatingTransform(std::move(header), params_->getHeader()) , params(std::move(params_)) , keys(params->params.keys) { @@ -13,18 +13,33 @@ RollupTransform::RollupTransform(Block header, AggregatingTransformParamsPtr par void RollupTransform::consume(Chunk chunk) { - consumed_chunk = std::move(chunk); - last_removed_key = keys.size(); + consumed_chunks.emplace_back(std::move(chunk)); } -bool RollupTransform::canGenerate() +Chunk RollupTransform::merge(Chunks && chunks, bool final) { - return consumed_chunk; + BlocksList rollup_blocks; + for (auto & chunk : chunks) + rollup_blocks.emplace_back(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); + + auto rollup_block = params->aggregator.mergeBlocks(rollup_blocks, final); + auto num_rows = rollup_block.rows(); + return Chunk(rollup_block.getColumns(), num_rows); } Chunk RollupTransform::generate() { - auto gen_chunk = std::move(consumed_chunk); + if (!consumed_chunks.empty()) + { + if (consumed_chunks.size() > 1) + rollup_chunk = merge(std::move(consumed_chunks), false); + else + rollup_chunk = std::move(consumed_chunks.front()); + + consumed_chunks.clear(); + } + + auto gen_chunk = std::move(rollup_chunk); if (last_removed_key) { @@ -35,11 +50,7 @@ Chunk RollupTransform::generate() auto columns = gen_chunk.getColumns(); columns[key] = columns[key]->cloneEmpty()->cloneResized(num_rows); - BlocksList rollup_blocks = { getInputPort().getHeader().cloneWithColumns(columns) }; - auto rollup_block = params->aggregator.mergeBlocks(rollup_blocks, false); - - num_rows = rollup_block.rows(); - consumed_chunk = Chunk(rollup_block.getColumns(), num_rows); + rollup_chunk = merge({Chunk(std::move(columns), num_rows)}, false); } finalizeChunk(gen_chunk); diff --git a/dbms/src/Processors/Transforms/RollupTransform.h b/dbms/src/Processors/Transforms/RollupTransform.h index 754e0237357..fd435740a63 100644 --- a/dbms/src/Processors/Transforms/RollupTransform.h +++ b/dbms/src/Processors/Transforms/RollupTransform.h @@ -1,5 +1,5 @@ #pragma once -#include +#include #include namespace DB @@ -7,7 +7,7 @@ namespace DB /// Takes blocks after grouping, with non-finalized aggregate functions. /// Calculates subtotals and grand totals values for a set of columns. -class RollupTransform : public IInflatingTransform +class RollupTransform : public IAccumulatingTransform { public: RollupTransform(Block header, AggregatingTransformParamsPtr params); @@ -15,14 +15,16 @@ public: protected: void consume(Chunk chunk) override; - bool canGenerate() override; Chunk generate() override; private: AggregatingTransformParamsPtr params; ColumnNumbers keys; - Chunk consumed_chunk; + Chunks consumed_chunks; + Chunk rollup_chunk; size_t last_removed_key = 0; + + Chunk merge(Chunks && chunks, bool final); }; } From 705e2c7437a7ee937bd47e8966c18fa04071ce7c Mon Sep 17 00:00:00 2001 From: akonyaev Date: Thu, 22 Aug 2019 11:28:32 +0300 Subject: [PATCH 347/509] ADQM-34 fix parquet tests --- .../queries/0_stateless/00900_parquet_load.reference | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00900_parquet_load.reference b/dbms/tests/queries/0_stateless/00900_parquet_load.reference index 83d0e8efde9..4e3977e0e96 100644 --- a/dbms/tests/queries/0_stateless/00900_parquet_load.reference +++ b/dbms/tests/queries/0_stateless/00900_parquet_load.reference @@ -39,7 +39,7 @@ 23.0 24.0 === Try load data from datapage_v2.snappy.parquet -Code: 33. DB::Ex---tion: Error while reading parquet data: IOError: Arrow error: IOError: Corrupt snappy compressed data. +Code: 33. DB::Ex---tion: Error while reading Parquet data: IOError: Arrow error: IOError: Corrupt snappy compressed data. === Try load data from fixed_length_decimal_1.parquet 1.0 @@ -171,19 +171,19 @@ Code: 33. DB::Ex---tion: Error while reading parquet data: IOError: Arrow error: Code: 8. DB::Ex---tion: Column "element" is not presented in input data === Try load data from nested_maps.snappy.parquet -Code: 33. DB::Ex---tion: Error while reading parquet data: NotImplemented: Currently only nesting with Lists is supported. +Code: 33. DB::Ex---tion: Error while reading Parquet data: NotImplemented: Currently only nesting with Lists is supported. === Try load data from nonnullable.impala.parquet -Code: 33. DB::Ex---tion: Error while reading parquet data: NotImplemented: Currently only nesting with Lists is supported. +Code: 33. DB::Ex---tion: Error while reading Parquet data: NotImplemented: Currently only nesting with Lists is supported. === Try load data from nullable.impala.parquet -Code: 33. DB::Ex---tion: Error while reading parquet data: NotImplemented: Currently only nesting with Lists is supported. +Code: 33. DB::Ex---tion: Error while reading Parquet data: NotImplemented: Currently only nesting with Lists is supported. === Try load data from nulls.snappy.parquet Code: 8. DB::Ex---tion: Column "b_c_int" is not presented in input data === Try load data from repeated_no_annotation.parquet -Code: 33. DB::Ex---tion: Error while reading parquet data: NotImplemented: Currently only nesting with Lists is supported. +Code: 33. DB::Ex---tion: Error while reading Parquet data: NotImplemented: Currently only nesting with Lists is supported. === Try load data from userdata1.parquet 1454486129 1 Amanda Jordan ajordan0@com.com Female 1.197.201.2 6759521864920116 Indonesia 3/8/1971 49756.53 Internal Auditor 1E+02 From 8228871821b60f0ab4af7f9aba6118ce26c129cb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2019 11:29:29 +0300 Subject: [PATCH 348/509] Fix RollupTransform. --- dbms/src/Processors/Transforms/RollupTransform.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Processors/Transforms/RollupTransform.cpp b/dbms/src/Processors/Transforms/RollupTransform.cpp index 1f5c140aafe..d60007b102e 100644 --- a/dbms/src/Processors/Transforms/RollupTransform.cpp +++ b/dbms/src/Processors/Transforms/RollupTransform.cpp @@ -50,7 +50,9 @@ Chunk RollupTransform::generate() auto columns = gen_chunk.getColumns(); columns[key] = columns[key]->cloneEmpty()->cloneResized(num_rows); - rollup_chunk = merge({Chunk(std::move(columns), num_rows)}, false); + Chunks chunks; + chunks.emplace_back(std::move(columns), num_rows); + rollup_chunk = merge(std::move(chunks), false); } finalizeChunk(gen_chunk); From 4f38b08481f572207e0263b701d15cbeb9f8f5bc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2019 11:53:34 +0300 Subject: [PATCH 349/509] Fix RollupTransform. --- dbms/src/Processors/Transforms/RollupTransform.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Processors/Transforms/RollupTransform.cpp b/dbms/src/Processors/Transforms/RollupTransform.cpp index d60007b102e..fb51b5f6b45 100644 --- a/dbms/src/Processors/Transforms/RollupTransform.cpp +++ b/dbms/src/Processors/Transforms/RollupTransform.cpp @@ -37,6 +37,7 @@ Chunk RollupTransform::generate() rollup_chunk = std::move(consumed_chunks.front()); consumed_chunks.clear(); + last_removed_key = keys.size(); } auto gen_chunk = std::move(rollup_chunk); From 11f8ec76df2086c112522ab4668d8e61fb5285ab Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2019 12:28:23 +0300 Subject: [PATCH 350/509] Fix CubeTransform. --- .../Processors/Transforms/CubeTransform.cpp | 53 ++++++++++++------- .../src/Processors/Transforms/CubeTransform.h | 10 ++-- 2 files changed, 39 insertions(+), 24 deletions(-) diff --git a/dbms/src/Processors/Transforms/CubeTransform.cpp b/dbms/src/Processors/Transforms/CubeTransform.cpp index 5809a480d09..106bd0fef51 100644 --- a/dbms/src/Processors/Transforms/CubeTransform.cpp +++ b/dbms/src/Processors/Transforms/CubeTransform.cpp @@ -5,7 +5,7 @@ namespace DB { CubeTransform::CubeTransform(Block header, AggregatingTransformParamsPtr params_) - : IInflatingTransform(std::move(header), params_->getHeader()) + : IAccumulatingTransform(std::move(header), params_->getHeader()) , params(std::move(params_)) , keys(params->params.keys) { @@ -13,28 +13,45 @@ CubeTransform::CubeTransform(Block header, AggregatingTransformParamsPtr params_ throw Exception("Too many keys are used for CubeTransform.", ErrorCodes::LOGICAL_ERROR); } -void CubeTransform::consume(Chunk chunk) +Chunk CubeTransform::merge(Chunks && chunks, bool final) { - consumed_chunk = std::move(chunk); - auto num_rows = consumed_chunk.getNumRows(); - mask = (UInt64(1) << keys.size()) - 1; + BlocksList rollup_blocks; + for (auto & chunk : chunks) + rollup_blocks.emplace_back(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); - current_columns = consumed_chunk.getColumns(); - current_zero_columns.clear(); - current_zero_columns.reserve(keys.size()); - - for (auto key : keys) - current_zero_columns.emplace_back(current_columns[key]->cloneEmpty()->cloneResized(num_rows)); + auto rollup_block = params->aggregator.mergeBlocks(rollup_blocks, final); + auto num_rows = rollup_block.rows(); + return Chunk(rollup_block.getColumns(), num_rows); } -bool CubeTransform::canGenerate() +void CubeTransform::consume(Chunk chunk) { - return consumed_chunk; + consumed_chunks.emplace_back(std::move(chunk)); } Chunk CubeTransform::generate() { - auto gen_chunk = std::move(consumed_chunk); + if (!consumed_chunks.empty()) + { + if (consumed_chunks.size() > 1) + cube_chunk = merge(std::move(consumed_chunks), false); + else + cube_chunk = std::move(consumed_chunks.front()); + + consumed_chunks.clear(); + + auto num_rows = cube_chunk.getNumRows(); + mask = (UInt64(1) << keys.size()) - 1; + + current_columns = cube_chunk.getColumns(); + current_zero_columns.clear(); + current_zero_columns.reserve(keys.size()); + + for (auto key : keys) + current_zero_columns.emplace_back(current_columns[key]->cloneEmpty()->cloneResized(num_rows)); + } + + auto gen_chunk = std::move(cube_chunk); if (mask) { @@ -47,11 +64,9 @@ Chunk CubeTransform::generate() if ((mask & (UInt64(1) << (size - i - 1))) == 0) columns[keys[i]] = current_zero_columns[i]; - BlocksList cube_blocks = { getInputPort().getHeader().cloneWithColumns(columns) }; - auto cube_block = params->aggregator.mergeBlocks(cube_blocks, false); - - auto num_rows = cube_block.rows(); - consumed_chunk = Chunk(cube_block.getColumns(), num_rows); + Chunks chunks; + chunks.emplace_back(std::move(columns), current_columns.front()->size()); + cube_chunk = merge(std::move(chunks), false); } finalizeChunk(gen_chunk); diff --git a/dbms/src/Processors/Transforms/CubeTransform.h b/dbms/src/Processors/Transforms/CubeTransform.h index 60259832e40..6d0e2338174 100644 --- a/dbms/src/Processors/Transforms/CubeTransform.h +++ b/dbms/src/Processors/Transforms/CubeTransform.h @@ -8,7 +8,7 @@ namespace DB /// Takes blocks after grouping, with non-finalized aggregate functions. /// Calculates all subsets of columns and aggregates over them. -class CubeTransform : public IInflatingTransform +class CubeTransform : public IAccumulatingTransform { public: CubeTransform(Block header, AggregatingTransformParamsPtr params); @@ -16,20 +16,20 @@ public: protected: void consume(Chunk chunk) override; - - bool canGenerate() override; - Chunk generate() override; private: AggregatingTransformParamsPtr params; ColumnNumbers keys; - Chunk consumed_chunk; + Chunks consumed_chunks; + Chunk cube_chunk; Columns current_columns; Columns current_zero_columns; UInt64 mask = 0; + + Chunk merge(Chunks && chunks, bool final); }; } From b93ffdd0387ea998be71bac361d6eda85a9c8c66 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2019 13:16:56 +0300 Subject: [PATCH 351/509] Fix MergingSortedTransform. --- dbms/src/Processors/Transforms/MergingSortedTransform.h | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/dbms/src/Processors/Transforms/MergingSortedTransform.h b/dbms/src/Processors/Transforms/MergingSortedTransform.h index 223d5253e62..f1175c8d347 100644 --- a/dbms/src/Processors/Transforms/MergingSortedTransform.h +++ b/dbms/src/Processors/Transforms/MergingSortedTransform.h @@ -165,6 +165,13 @@ private: void updateCursor(Chunk chunk, size_t source_num) { + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + + chunk.setColumns(std::move(columns), num_rows); + auto & shared_chunk_ptr = source_chunks[source_num]; if (!shared_chunk_ptr) From 1f9087dbb22cf648a580aa7c286c45cc6a16e5c6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2019 13:46:23 +0300 Subject: [PATCH 352/509] Increase memory limit in 00284_external_aggregation. --- dbms/tests/queries/0_stateless/00284_external_aggregation.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00284_external_aggregation.sql b/dbms/tests/queries/0_stateless/00284_external_aggregation.sql index 75d2c0b9bc2..03403b90b6c 100644 --- a/dbms/tests/queries/0_stateless/00284_external_aggregation.sql +++ b/dbms/tests/queries/0_stateless/00284_external_aggregation.sql @@ -1,5 +1,5 @@ SET max_bytes_before_external_group_by = 100000000; -SET max_memory_usage = 201000000; +SET max_memory_usage = 301000000; SELECT sum(k), sum(c) FROM (SELECT number AS k, count() AS c FROM (SELECT * FROM system.numbers LIMIT 10000000) GROUP BY k); SELECT sum(k), sum(c), max(u) FROM (SELECT number AS k, count() AS c, uniqArray(range(number % 16)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000000) GROUP BY k); From fb534bd1ce0b290ba9380d47cbb04f75ccfdb1f0 Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Thu, 22 Aug 2019 14:03:01 +0300 Subject: [PATCH 353/509] Added symlink to clickhouse-client package --- debian/clickhouse-client.install | 1 + 1 file changed, 1 insertion(+) diff --git a/debian/clickhouse-client.install b/debian/clickhouse-client.install index 7331f3eb5ed..5e730db669f 100644 --- a/debian/clickhouse-client.install +++ b/debian/clickhouse-client.install @@ -3,5 +3,6 @@ usr/bin/clickhouse-local usr/bin/clickhouse-compressor usr/bin/clickhouse-benchmark usr/bin/clickhouse-format +usr/bin/clickhouse-obfuscator etc/clickhouse-client/config.xml usr/bin/clickhouse-extract-from-config From 63411b4d4a568bdebec9a463c607e74404a8ae9d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2019 15:51:33 +0300 Subject: [PATCH 354/509] Try to fix 00093_union_race_conditions_4. --- dbms/tests/queries/0_stateless/00093_union_race_conditions_4.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00093_union_race_conditions_4.sh b/dbms/tests/queries/0_stateless/00093_union_race_conditions_4.sh index fcdbe4cbcdd..2d255a0c2f6 100755 --- a/dbms/tests/queries/0_stateless/00093_union_race_conditions_4.sh +++ b/dbms/tests/queries/0_stateless/00093_union_race_conditions_4.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -o errexit set -o pipefail -for i in {1..10}; do seq 1 10 | sed 's/.*/SELECT 1 % ((number + 500) % 1000) FROM system.numbers_mt LIMIT 1000;/' | $CLICKHOUSE_CLIENT -n --receive_timeout=1 --max_block_size=1 >/dev/null 2>&1 && echo 'Fail!' && break; echo -n '.'; done; echo +for i in {1..10}; do seq 1 10 | sed 's/.*/SELECT 1 % ((number + 500) % 1000) FROM numbers_mt(1000);/' | $CLICKHOUSE_CLIENT -n --receive_timeout=1 --max_block_size=1 >/dev/null 2>&1 && echo 'Fail!' && break; echo -n '.'; done; echo From 095124dcba3df720a1bc79adc2277c1d53884418 Mon Sep 17 00:00:00 2001 From: Dmitry Rubashkin Date: Thu, 22 Aug 2019 17:03:37 +0300 Subject: [PATCH 355/509] setOptionsDescription() function added --- dbms/programs/client/Client.cpp | 14 +++----- .../performance-test/PerformanceTestSuite.cpp | 18 ++-------- dbms/src/Common/SetOptionsDescription.h | 35 +++++++++++++++++++ 3 files changed, 41 insertions(+), 26 deletions(-) create mode 100644 dbms/src/Common/SetOptionsDescription.h diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index cf72d7a87c3..fa9431cb9e9 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -67,6 +67,7 @@ #include #include #include +#include #if USE_READLINE #include "Suggest.h" @@ -1641,23 +1642,16 @@ public: } stdin_is_not_tty = !isatty(STDIN_FILENO); - - namespace po = boost::program_options; - - unsigned line_length = po::options_description::m_default_line_length; - unsigned min_description_length = line_length / 2; if (!stdin_is_not_tty) { if (ioctl(STDIN_FILENO, TIOCGWINSZ, &terminal_size)) throwFromErrno("Cannot obtain terminal window size (ioctl TIOCGWINSZ)", ErrorCodes::SYSTEM_ERROR); - line_length = std::max( - static_cast(strlen("--http_native_compression_disable_checksumming_on_decompress ")), - static_cast(terminal_size.ws_col)); - min_description_length = std::min(min_description_length, line_length - 2); } + namespace po = boost::program_options; + /// Main commandline options related to client functionality and all parameters from Settings. - po::options_description main_description("Main options", line_length, min_description_length); + po::options_description main_description = setOptionsDescription("Main options"); main_description.add_options() ("help", "produce help message") ("config-file,C", po::value(), "config-file path") diff --git a/dbms/programs/performance-test/PerformanceTestSuite.cpp b/dbms/programs/performance-test/PerformanceTestSuite.cpp index 943ee4fe6c2..d7b2d10e90f 100644 --- a/dbms/programs/performance-test/PerformanceTestSuite.cpp +++ b/dbms/programs/performance-test/PerformanceTestSuite.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include "TestStopConditions.h" #include "TestStats.h" @@ -325,22 +326,7 @@ try using namespace DB; using po::value; - unsigned line_length = po::options_description::m_default_line_length; - unsigned min_description_length = line_length / 2; - winsize terminal_size {}; - - if (isatty(STDIN_FILENO)) - { - if (ioctl(STDIN_FILENO, TIOCGWINSZ, &terminal_size)) - throwFromErrno("Cannot obtain terminal window size (ioctl TIOCGWINSZ)", ErrorCodes::SYSTEM_ERROR); - - line_length = std::max( - static_cast(strlen("--http_native_compression_disable_checksumming_on_decompress ")), - static_cast(terminal_size.ws_col)); - min_description_length = std::min(min_description_length, line_length - 2); - } - - po::options_description desc("Allowed options", line_length, min_description_length); + po::options_description desc = setOptionsDescription("Allowed options"); desc.add_options() ("help", "produce help message") ("lite", "use lite version of output") diff --git a/dbms/src/Common/SetOptionsDescription.h b/dbms/src/Common/SetOptionsDescription.h new file mode 100644 index 00000000000..a9c4031b064 --- /dev/null +++ b/dbms/src/Common/SetOptionsDescription.h @@ -0,0 +1,35 @@ +#include +#include +#include +#include + + +namespace po = boost::program_options; + + +namespace DB::ErrorCodes +{ + extern const int SYSTEM_ERROR; +} + +static po::options_description setOptionsDescription(const std::string & caption) +{ + unsigned line_length = po::options_description::m_default_line_length; + unsigned min_description_length = line_length / 2; + + if (isatty(STDIN_FILENO)) + { + winsize terminal_size {}; + + if (ioctl(STDIN_FILENO, TIOCGWINSZ, &terminal_size)) + DB::throwFromErrno("Cannot obtain terminal window size (ioctl TIOCGWINSZ)", DB::ErrorCodes::SYSTEM_ERROR); + + std::string longest_option_desc = "--http_native_compression_disable_checksumming_on_decompress"; + + line_length = std::max(static_cast(longest_option_desc.size()), terminal_size.ws_col); + + min_description_length = std::min(min_description_length, line_length - 2); + } + + return po::options_description(caption, line_length, min_description_length); +} From e9d813bff79f3af4784bb905ca61054f6fec79e1 Mon Sep 17 00:00:00 2001 From: Dmitry Rubashkin Date: Thu, 22 Aug 2019 17:30:18 +0300 Subject: [PATCH 356/509] Minor fixes --- dbms/programs/client/Client.cpp | 2 +- dbms/programs/performance-test/PerformanceTestSuite.cpp | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index fa9431cb9e9..726a607812c 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -1666,7 +1666,7 @@ public: * the "\n" is used to distinguish this case because there is hardly a chance an user would use "\n" * as the password. */ - ("password", po::value()->implicit_value("\n"), "password") + ("password", po::value()->implicit_value("\n","\\n"), "password") ("ask-password", "ask-password") ("query_id", po::value(), "query_id") ("query,q", po::value(), "query") diff --git a/dbms/programs/performance-test/PerformanceTestSuite.cpp b/dbms/programs/performance-test/PerformanceTestSuite.cpp index d7b2d10e90f..99408c72f37 100644 --- a/dbms/programs/performance-test/PerformanceTestSuite.cpp +++ b/dbms/programs/performance-test/PerformanceTestSuite.cpp @@ -46,7 +46,6 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int FILE_DOESNT_EXIST; - extern const int SYSTEM_ERROR; } /** Tests launcher for ClickHouse. @@ -323,8 +322,8 @@ std::unordered_map> getTestQueryIndexes(co int mainEntryClickHousePerformanceTest(int argc, char ** argv) try { - using namespace DB; using po::value; + using Strings = DB::Strings; po::options_description desc = setOptionsDescription("Allowed options"); desc.add_options() From b3f91a717cabca06b03aec29851d0776f45f06bc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2019 17:38:44 +0300 Subject: [PATCH 357/509] Use ThreadFromGlobalPool in PipelineExecutor instead of ThreadPool. --- .../Processors/Executors/PipelineExecutor.cpp | 50 ++++++++++++------- 1 file changed, 33 insertions(+), 17 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 31cca4e1a48..6aad6f96b5c 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -590,32 +590,45 @@ void PipelineExecutor::executeImpl(size_t num_threads) for (size_t i = 0; i < num_threads; ++i) executor_contexts.emplace_back(std::make_unique()); + auto thread_group = CurrentThread::getGroup(); + + using ThreadsData = std::vector; + ThreadsData threads; + threads.reserve(num_threads); + + bool finished_flag = false; + + SCOPE_EXIT( + if (!finished_flag) + { + finish(); + + for (auto & thread : threads) + thread.join(); + } + ); + addChildlessProcessorsToStack(stack); - while (!stack.empty()) { - UInt64 proc = stack.top(); - stack.pop(); + std::lock_guard lock(task_queue_mutex); - if (prepareProcessor(proc, stack, stack, 0, false)) + while (!stack.empty()) { - auto cur_state = graph[proc].execution_state.get(); - task_queue.push(cur_state); + UInt64 proc = stack.top(); + stack.pop(); + + if (prepareProcessor(proc, stack, stack, 0, false)) + { + auto cur_state = graph[proc].execution_state.get(); + task_queue.push(cur_state); + } } } - ThreadPool pool(num_threads); - - SCOPE_EXIT( - finish(); - pool.wait() - ); - - auto thread_group = CurrentThread::getGroup(); - for (size_t i = 0; i < num_threads; ++i) { - pool.schedule([this, thread_group, thread_num = i, num_threads] + threads.emplace_back([this, thread_group, thread_num = i, num_threads] { /// ThreadStatus thread_status; @@ -631,7 +644,10 @@ void PipelineExecutor::executeImpl(size_t num_threads) }); } - pool.wait(); + for (auto & thread : threads) + thread.join(); + + finished_flag = true; } String PipelineExecutor::dumpPipeline() const From 6d78e3be94b56f5e844238af7a688f0125d7e274 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Wed, 21 Aug 2019 11:12:39 +0300 Subject: [PATCH 358/509] hasToken function implementation * Function to check if given token is present in a string; * Special case for hasToken to 'tokenbf_v1' index; * Test cases for hasToken() * Test case for hasToken() + 'tokenbf_v1' integration --- dbms/src/Common/StringSearcher.h | 79 +++++++++- dbms/src/Common/Volnitsky.h | 14 +- dbms/src/Functions/FunctionsStringSearch.cpp | 76 ++++++++++ .../MergeTree/MergeTreeIndexFullText.cpp | 13 ++ .../queries/0_stateless/00990_hasToken.python | 124 ++++++++++++++++ .../0_stateless/00990_hasToken.reference | 139 ++++++++++++++++++ .../queries/0_stateless/00990_hasToken.sh | 8 + .../00990_hasToken_and_tokenbf.reference | 3 + .../00990_hasToken_and_tokenbf.sql | 33 +++++ 9 files changed, 481 insertions(+), 8 deletions(-) create mode 100755 dbms/tests/queries/0_stateless/00990_hasToken.python create mode 100644 dbms/tests/queries/0_stateless/00990_hasToken.reference create mode 100755 dbms/tests/queries/0_stateless/00990_hasToken.sh create mode 100644 dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.reference create mode 100644 dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql diff --git a/dbms/src/Common/StringSearcher.h b/dbms/src/Common/StringSearcher.h index 5e78ff23df1..fecf1a7ca81 100644 --- a/dbms/src/Common/StringSearcher.h +++ b/dbms/src/Common/StringSearcher.h @@ -1,5 +1,7 @@ #pragma once +#include +#include #include #include #include @@ -23,6 +25,7 @@ namespace DB namespace ErrorCodes { extern const int UNSUPPORTED_PARAMETER; + extern const int BAD_ARGUMENTS; } @@ -157,7 +160,7 @@ public: #endif } - ALWAYS_INLINE bool compare(const UInt8 * pos) const + ALWAYS_INLINE bool compare(const UInt8 * /*haystack*/, const UInt8 * /*haystack_end*/, const UInt8 * pos) const { static const Poco::UTF8Encoding utf8; @@ -374,7 +377,7 @@ public: #endif } - ALWAYS_INLINE bool compare(const UInt8 * pos) const + ALWAYS_INLINE bool compare(const UInt8 * /*haystack*/, const UInt8 * /*haystack_end*/, const UInt8 * pos) const { #ifdef __SSE4_1__ if (pageSafe(pos)) @@ -567,7 +570,7 @@ public: #endif } - ALWAYS_INLINE bool compare(const UInt8 * pos) const + ALWAYS_INLINE bool compare(const UInt8 * /*haystack*/, const UInt8 * /*haystack_end*/, const UInt8 * pos) const { #ifdef __SSE4_1__ if (pageSafe(pos)) @@ -697,11 +700,81 @@ public: } }; +// Searches for needle surrounded by token-separators. +// Separators are anything inside ASCII (0-128) and not alphanum. +// Any value outside of basic ASCII (>=128) is considered a non-separator symbol, hence UTF-8 strings +// should work just fine. But any Unicode whitespace is not considered a token separtor. +template +class TokenSearcher +{ + StringSearcher searcher; + size_t needle_size; + +public: + TokenSearcher(const char * const needle_, const size_t needle_size_) + : searcher{needle_, needle_size_}, + needle_size(needle_size_) + { + if (std::any_of(reinterpret_cast(needle_), reinterpret_cast(needle_) + needle_size_, isTokenSeparator)) + { + throw Exception{"needle must not contain whitespace characters", ErrorCodes::BAD_ARGUMENTS}; + } + + } + + ALWAYS_INLINE bool compare(const UInt8 * haystack, const UInt8 * haystack_end, const UInt8 * pos) const + { + // use searcher only if pos is in the beginning of token and pos + searcher.needle_size is end of token. + if (isToken(haystack, haystack_end, pos)) + return searcher.compare(haystack, haystack_end, pos); + + return false; + } + + const UInt8 * search(const UInt8 * haystack, const UInt8 * const haystack_end) const + { + // use searcher.search(), then verify that returned value is a token + // if it is not, skip it and re-run + + const UInt8 * pos = haystack; + while (pos < haystack_end) + { + pos = searcher.search(pos, haystack_end); + if (pos == haystack_end || isToken(haystack, haystack_end, pos)) + return pos; + + // assuming that heendle does not contain any token separators. + pos += needle_size; + } + return haystack_end; + } + + const UInt8 * search(const UInt8 * haystack, const size_t haystack_size) const + { + return search(haystack, haystack + haystack_size); + } + + ALWAYS_INLINE bool isToken(const UInt8 * haystack, const UInt8 * const haystack_end, const UInt8* p) const + { + return (p == haystack || isTokenSeparator(*(p - 1))) + && (p + needle_size >= haystack_end || isTokenSeparator(*(p + needle_size))); + } + + ALWAYS_INLINE static bool isTokenSeparator(const UInt8 c) + { + if (isAlphaNumericASCII(c) || !isASCII(c)) + return false; + + return true; + } +}; + using ASCIICaseSensitiveStringSearcher = StringSearcher; using ASCIICaseInsensitiveStringSearcher = StringSearcher; using UTF8CaseSensitiveStringSearcher = StringSearcher; using UTF8CaseInsensitiveStringSearcher = StringSearcher; +using ASCIICaseSensitiveTokenSearcher = TokenSearcher; /** Uses functions from libc. diff --git a/dbms/src/Common/Volnitsky.h b/dbms/src/Common/Volnitsky.h index 748cbe09138..c87bdd79dab 100644 --- a/dbms/src/Common/Volnitsky.h +++ b/dbms/src/Common/Volnitsky.h @@ -327,6 +327,8 @@ protected: FallbackSearcher fallback_searcher; public: + using Searcher = FallbackSearcher; + /** haystack_size_hint - the expected total size of the haystack for `search` calls. Optional (zero means unspecified). * If you specify it small enough, the fallback algorithm will be used, * since it is considered that it's useless to waste time initializing the hash table. @@ -373,7 +375,7 @@ public: const auto res = pos - (hash[cell_num] - 1); /// pointer in the code is always padded array so we can use pagesafe semantics - if (fallback_searcher.compare(res)) + if (fallback_searcher.compare(haystack, haystack_end, res)) return res; } } @@ -520,7 +522,7 @@ public: { const auto res = pos - (hash[cell_num].off - 1); const size_t ind = hash[cell_num].id; - if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(res)) + if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(haystack, haystack_end, res)) return true; } } @@ -552,7 +554,7 @@ public: { const auto res = pos - (hash[cell_num].off - 1); const size_t ind = hash[cell_num].id; - if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(res)) + if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(haystack, haystack_end, res)) ans = std::min(ans, ind); } } @@ -590,7 +592,7 @@ public: { const auto res = pos - (hash[cell_num].off - 1); const size_t ind = hash[cell_num].id; - if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(res)) + if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(haystack, haystack_end, res)) ans = std::min(ans, res - haystack); } } @@ -625,7 +627,7 @@ public: { const auto * res = pos - (hash[cell_num].off - 1); const size_t ind = hash[cell_num].id; - if (ans[ind] == 0 && res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(res)) + if (ans[ind] == 0 && res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(haystack, haystack_end, res)) ans[ind] = count_chars(haystack, res); } } @@ -650,6 +652,8 @@ using VolnitskyUTF8 = VolnitskyBase; /// ignores non-ASCII bytes using VolnitskyCaseInsensitiveUTF8 = VolnitskyBase; +using VolnitskyToken = VolnitskyBase; + using MultiVolnitsky = MultiVolnitskyBase; using MultiVolnitskyUTF8 = MultiVolnitskyBase; using MultiVolnitskyCaseInsensitive = MultiVolnitskyBase; diff --git a/dbms/src/Functions/FunctionsStringSearch.cpp b/dbms/src/Functions/FunctionsStringSearch.cpp index 726eb8738af..5d688232bd4 100644 --- a/dbms/src/Functions/FunctionsStringSearch.cpp +++ b/dbms/src/Functions/FunctionsStringSearch.cpp @@ -434,6 +434,74 @@ struct MultiSearchFirstIndexImpl } }; +/** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation. + */ +template +struct HasTokenImpl +{ + using ResultType = UInt8; + + static void vector_constant( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray & res) + { + if (offsets.empty()) + return; + + const UInt8 * begin = data.data(); + const UInt8 * pos = begin; + const UInt8 * end = pos + data.size(); + + /// The current index in the array of strings. + size_t i = 0; + + VolnitskyToken searcher(pattern.data(), pattern.size(), end - pos); + + /// We will search for the next occurrence in all rows at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) + { + /// Let's determine which index it refers to. + while (begin + offsets[i] <= pos) + { + res[i] = negate_result; + ++i; + } + + /// We check that the entry does not pass through the boundaries of strings. + if (pos + pattern.size() < begin + offsets[i]) + res[i] = !negate_result; + else + res[i] = negate_result; + + pos = begin + offsets[i]; + ++i; + } + + /// Tail, in which there can be no substring. + if (i < res.size()) + memset(&res[i], negate_result, (res.size() - i) * sizeof(res[0])); + } + + static void constant_constant(const std::string & data, const std::string & pattern, UInt8 & res) + { + VolnitskyToken searcher(pattern.data(), pattern.size(), data.size()); + const auto found = searcher.search(data.c_str(), data.size()) != data.end().base(); + res = negate_result ^ found; + } + + template + static void vector_vector(Args &&...) + { + throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + } + + /// Search different needles in single haystack. + template + static void constant_vector(Args &&...) + { + throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + } +}; + struct NamePosition { @@ -516,6 +584,11 @@ struct NameMultiSearchFirstPositionCaseInsensitiveUTF8 static constexpr auto name = "multiSearchFirstPositionCaseInsensitiveUTF8"; }; +struct NameHasToken +{ + static constexpr auto name = "hasToken"; +}; + using FunctionPosition = FunctionsStringSearch, NamePosition>; using FunctionPositionUTF8 = FunctionsStringSearch, NamePositionUTF8>; @@ -542,6 +615,7 @@ using FunctionMultiSearchFirstPositionUTF8 = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitive>; using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitiveUTF8>; +using FunctionHasToken = FunctionsStringSearch, NameHasToken>; void registerFunctionsStringSearch(FunctionFactory & factory) { @@ -570,6 +644,8 @@ void registerFunctionsStringSearch(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive); } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 3625c6f1aa5..246ad6784b2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -168,6 +168,19 @@ const MergeTreeConditionFullText::AtomMap MergeTreeConditionFullText::atom_map return true; } }, + { + "hasToken", + [] (RPNElement & out, const Field & value, const MergeTreeIndexFullText & idx) + { + out.function = RPNElement::FUNCTION_EQUALS; + out.bloom_filter = std::make_unique( + idx.bloom_filter_size, idx.bloom_filter_hashes, idx.seed); + + const auto & str = value.get(); + stringToBloomFilter(str.c_str(), str.size(), idx.token_extractor_func, *out.bloom_filter); + return true; + } + }, { "startsWith", [] (RPNElement & out, const Field & value, const MergeTreeIndexFullText & idx) diff --git a/dbms/tests/queries/0_stateless/00990_hasToken.python b/dbms/tests/queries/0_stateless/00990_hasToken.python new file mode 100755 index 00000000000..217d96dfe52 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_hasToken.python @@ -0,0 +1,124 @@ +#!/usr/bin/env python +# encoding: utf-8 + +import re + +HAYSTACKS = [ + "hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay needle", + "needle hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay", + "hay hay hay hay hay hay hay hay hay needle hay hay hay hay hay hay hay hay hay", +] + +NEEDLE = "needle" + +HAY_RE = re.compile(r'\bhay\b', re.IGNORECASE) +NEEDLE_RE = re.compile(r'\bneedle\b', re.IGNORECASE) + +def replace_follow_case(replacement): + def func(match): + g = match.group() + if g.islower(): return replacement.lower() + if g.istitle(): return replacement.title() + if g.isupper(): return replacement.upper() + return replacement + return func + +def replace_separators(query, new_sep): + SEP_RE = re.compile('\\s+') + result = SEP_RE.sub(new_sep, query) + return result + +def enlarge_haystack(query, times, separator=''): + return HAY_RE.sub(replace_follow_case(('hay' + separator) * times), query) + +def small_needle(query): + return NEEDLE_RE.sub(replace_follow_case('n'), query) + +def remove_needle(query): + return NEEDLE_RE.sub('', query) + +def replace_needle(query, new_needle): + return NEEDLE_RE.sub(new_needle, query) + +# with str.lower, str.uppert, str.title and such +def transform_needle(query, string_transformation_func): + def replace_with_transformation(match): + g = match.group() + return string_transformation_func(g) + + return NEEDLE_RE.sub(replace_with_transformation, query) + + +def create_cases(table_row_template, table_query_template, const_query_template): + const_queries = [] + table_rows = [] + table_queries = set() + + def add_case(haystack, needle, match): + match = int(match) + const_queries.append(const_query_template.format(haystack=haystack, needle=needle, match=match)) + table_queries.add(table_query_template.format(haystack=haystack, needle=needle, match=match)) + table_rows.append(table_row_template.format(haystack=haystack, needle=needle, match=match)) + + # Negative cases + add_case(remove_needle(HAYSTACKS[0]), NEEDLE, False) + for haystack in HAYSTACKS: + add_case(transform_needle(haystack, str.title), NEEDLE, False) + sep = '' + h = replace_separators(haystack, sep) + add_case(h, NEEDLE, False) + add_case(small_needle(h), small_needle(NEEDLE), False) + add_case(enlarge_haystack(h, 10, sep), NEEDLE, False) + + # positive cases + for haystack in HAYSTACKS: + add_case(transform_needle(haystack, str.title), transform_needle(NEEDLE, str.title), True) + add_case(transform_needle(haystack, str.upper), transform_needle(NEEDLE, str.upper), True) + + # Not checking all separators since some (like ' and \n) cause issues when coupled with + # re-based replacement and quoting in query + # other are rare in practice and checking all separators makes this test too lengthy. + + # r'\\\\' turns into a single '\' in query + #separators = list(''' \t`~!@#$%^&*()-=+|]}[{";:/?.>,<''') + [r'\\\\'] + separators = list(''' \t;:?.,''') + [r'\\\\'] + for sep in separators: + h = replace_separators(haystack, sep) + add_case(h, NEEDLE, True) + add_case(small_needle(h), small_needle(NEEDLE), True) + add_case(enlarge_haystack(h, 200, sep), NEEDLE, True) + add_case(replace_needle(h, 'иголка'), replace_needle(NEEDLE, 'иголка'), True) + add_case(replace_needle(h, '指针'), replace_needle(NEEDLE, '指针'), True) + + return table_rows, table_queries, const_queries + +def main(): + + def query(x): + print x + + CONST_QUERY = """SELECT hasToken('{haystack}', '{needle}'), ' expecting ', {match};""" + #SELECT hasToken(haystack, '{needle}') FROM ht WHERE needle = '{needle}' AND match = {match};""" + TABLE_QUERY = """WITH '{needle}' as n SELECT haystack, needle, hasToken(haystack, n) as result FROM ht WHERE needle = n AND result != match;""" + TABLE_ROW = """('{haystack}', '{needle}', {match})""" + + rows, table_queries, const_queries = create_cases(TABLE_ROW, TABLE_QUERY, CONST_QUERY) + for q in const_queries: + query(q) + + query("""DROP TABLE IF EXISTS ht; + CREATE TABLE IF NOT EXISTS + ht +( + haystack String, + needle String, + match UInt8 +) +ENGINE MergeTree() +ORDER BY haystack; +INSERT INTO ht VALUES {values};""".format(values=", ".join(rows))) + for q in sorted(table_queries): + query(q) + +if __name__ == '__main__': + main() diff --git a/dbms/tests/queries/0_stateless/00990_hasToken.reference b/dbms/tests/queries/0_stateless/00990_hasToken.reference new file mode 100644 index 00000000000..867c0c1c691 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_hasToken.reference @@ -0,0 +1,139 @@ +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 diff --git a/dbms/tests/queries/0_stateless/00990_hasToken.sh b/dbms/tests/queries/0_stateless/00990_hasToken.sh new file mode 100755 index 00000000000..4ccb77b8ecc --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_hasToken.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +# We should have correct env vars from shell_config.sh to run this test + +python $CURDIR/00990_hasToken.python | ${CLICKHOUSE_CLIENT} -nm diff --git a/dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.reference b/dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.reference new file mode 100644 index 00000000000..10e8f0d2c59 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.reference @@ -0,0 +1,3 @@ +2007 +2007 +2007 diff --git a/dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql b/dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql new file mode 100644 index 00000000000..60e4d959417 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql @@ -0,0 +1,33 @@ +SET allow_experimental_data_skipping_indices = 1; + +DROP TABLE IF EXISTS bloom_filter; + +CREATE TABLE bloom_filter +( + id UInt64, + s String, + INDEX tok_bf (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1 +) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 8; + +insert into bloom_filter select number, 'yyy,uuu' from numbers(1024); +insert into bloom_filter select number+2000, 'abc,def,zzz' from numbers(8); +insert into bloom_filter select number+3000, 'yyy,uuu' from numbers(1024); +insert into bloom_filter select number+3000, 'abcdefzzz' from numbers(1024); + +set max_rows_to_read = 16; + +SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'abc'); +SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'def'); +SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'zzz'); + +-- invert result +-- this does not work as expected, reading more rows that it should +-- SELECT max(id) FROM bloom_filter WHERE NOT hasToken(s, 'yyy'); + +-- accessing to many rows +SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'yyy'); -- { serverError 158 } + +-- this syntax is not supported by tokenbf +SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'zzz') == 1; -- { serverError 158 } + +DROP TABLE bloom_filter; \ No newline at end of file From a50d6e713299f8b54c7d1b81cc9742bf4a6a211e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2019 18:07:07 +0300 Subject: [PATCH 359/509] Update 00416_pocopatch_progress_in_http_headers. --- ...copatch_progress_in_http_headers.reference | 23 ++++++++++--------- ...0416_pocopatch_progress_in_http_headers.sh | 5 ++-- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.reference b/dbms/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.reference index e838f583cdf..a2c79e66928 100644 --- a/dbms/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.reference +++ b/dbms/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.reference @@ -3,17 +3,18 @@ < X-ClickHouse-Progress: {"read_rows":"10","read_bytes":"80","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} < X-ClickHouse-Progress: {"read_rows":"10","read_bytes":"80","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} 9 -< X-ClickHouse-Progress: {"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< X-ClickHouse-Progress: {"read_rows":"2","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< X-ClickHouse-Progress: {"read_rows":"3","read_bytes":"24","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< X-ClickHouse-Progress: {"read_rows":"4","read_bytes":"32","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< X-ClickHouse-Progress: {"read_rows":"5","read_bytes":"40","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< X-ClickHouse-Progress: {"read_rows":"6","read_bytes":"48","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< X-ClickHouse-Progress: {"read_rows":"7","read_bytes":"56","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< X-ClickHouse-Progress: {"read_rows":"8","read_bytes":"64","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< X-ClickHouse-Progress: {"read_rows":"9","read_bytes":"72","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< X-ClickHouse-Progress: {"read_rows":"10","read_bytes":"80","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< X-ClickHouse-Progress: {"read_rows":"10","read_bytes":"80","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Progress: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} +< X-ClickHouse-Progress: {"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} +< X-ClickHouse-Progress: {"read_rows":"2","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} +< X-ClickHouse-Progress: {"read_rows":"3","read_bytes":"24","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} +< X-ClickHouse-Progress: {"read_rows":"4","read_bytes":"32","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} +< X-ClickHouse-Progress: {"read_rows":"5","read_bytes":"40","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} +< X-ClickHouse-Progress: {"read_rows":"6","read_bytes":"48","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} +< X-ClickHouse-Progress: {"read_rows":"7","read_bytes":"56","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} +< X-ClickHouse-Progress: {"read_rows":"8","read_bytes":"64","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} +< X-ClickHouse-Progress: {"read_rows":"9","read_bytes":"72","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} +< X-ClickHouse-Progress: {"read_rows":"10","read_bytes":"80","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} +< X-ClickHouse-Progress: {"read_rows":"10","read_bytes":"80","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"} 0 1 2 diff --git a/dbms/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh b/dbms/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh index d95798bc95c..c86154a8402 100755 --- a/dbms/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh +++ b/dbms/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh @@ -6,9 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?max_block_size=5&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT max(number) FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]' # This test will fail with external poco (progress not supported) -# "grep -v 11" in order to skip extra progress header for 11-th row (for processors pipeline) -${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&experimental_use_processors=0" -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]' | grep -v 11 -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&enable_http_compression=1&experimental_use_processors=0" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d +${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT number FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]' +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&enable_http_compression=1" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d # 'send_progress_in_http_headers' is false by default ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?max_block_size=1&http_headers_progress_interval_ms=0" -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep -q 'X-ClickHouse-Progress' && echo 'Fail' || true From b6c8a492445e99d1d4ca318c6cd580c6362fb634 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2019 20:08:10 +0300 Subject: [PATCH 360/509] Update 00284_external_aggregation --- dbms/tests/queries/0_stateless/00284_external_aggregation.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00284_external_aggregation.sql b/dbms/tests/queries/0_stateless/00284_external_aggregation.sql index 03403b90b6c..cd9abec59a8 100644 --- a/dbms/tests/queries/0_stateless/00284_external_aggregation.sql +++ b/dbms/tests/queries/0_stateless/00284_external_aggregation.sql @@ -1,5 +1,5 @@ SET max_bytes_before_external_group_by = 100000000; -SET max_memory_usage = 301000000; +SET max_memory_usage = 351000000; SELECT sum(k), sum(c) FROM (SELECT number AS k, count() AS c FROM (SELECT * FROM system.numbers LIMIT 10000000) GROUP BY k); SELECT sum(k), sum(c), max(u) FROM (SELECT number AS k, count() AS c, uniqArray(range(number % 16)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000000) GROUP BY k); From acce56095fc78c7d4fbe6e80746c4241fe698d89 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 22 Aug 2019 22:35:46 +0300 Subject: [PATCH 361/509] limit number of background threads for mutations --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 9 ++++++++- .../src/Storages/MergeTree/MergeTreeSettings.h | 1 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 18 +++++++++++------- dbms/src/Storages/StorageMergeTree.cpp | 7 ++----- .../Storages/StorageReplicatedMergeTree.cpp | 3 +-- 5 files changed, 23 insertions(+), 15 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 74193fa7156..5a9affaacd4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -157,7 +157,14 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() { - return static_cast(DiskSpaceMonitor::getUnreservedFreeSpace(data.full_path) / DISK_USAGE_COEFFICIENT_TO_RESERVE); + size_t total_threads_in_pool = pool.getNumberOfThreads(); + size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); + + /// Allow mutations only if there are enough threads, leave free threads for merges else + if (total_threads_in_pool - busy_threads_in_pool >= data.settings.number_of_free_entries_in_pool_to_execute_mutation) + return static_cast(DiskSpaceMonitor::getUnreservedFreeSpace(data.full_path) / DISK_USAGE_COEFFICIENT_TO_RESERVE); + + return 0; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index 36e82b96961..afd0772a937 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,6 +33,7 @@ struct MergeTreeSettings : public SettingsCollection M(SettingUInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.") \ M(SettingUInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.") \ M(SettingUInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.") \ + M(SettingUInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"") \ M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.") \ M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.") \ \ diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index e6251502576..fd65f14fedb 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -956,15 +956,19 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( return false; } - /** Execute merge only if there are enough free threads in background pool to do merges of that size. - * But if all threads are free (maximal size of merge is allowed) then execute any merge, - * (because it may be ordered by OPTIMIZE or early with differrent settings). + UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge() + : merger_mutator.getMaxSourcePartSizeForMutation(); + /** If there are enough free threads in background pool to do large merges (maximal size of merge is allowed), + * then ignore value returned by getMaxSourcePartsSizeForMerge() and execute merge of any size, + * because it may be ordered by OPTIMIZE or early with different settings. + * Setting max_bytes_to_merge_at_max_space_in_pool still working for regular merges, + * because the leader replica does not assign merges of greater size (except OPTIMIZE PARTITION and OPTIMIZE FINAL). */ - UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(); - if (max_source_parts_size != data.settings.max_bytes_to_merge_at_max_space_in_pool - && sum_parts_size_in_bytes > max_source_parts_size) + bool ignore_max_size = (entry.type == LogEntry::MERGE_PARTS) && (max_source_parts_size == data.settings.max_bytes_to_merge_at_max_space_in_pool); + + if (!ignore_max_size && sum_parts_size_in_bytes > max_source_parts_size) { - String reason = "Not executing log entry for part " + entry.new_part_name + String reason = "Not executing log entry " + entry.typeToString() + " for part " + entry.new_part_name + " because source parts size (" + formatReadableSizeWithBinarySuffix(sum_parts_size_in_bytes) + ") is greater than the current maximum (" + formatReadableSizeWithBinarySuffix(max_source_parts_size) + ")."; LOG_DEBUG(log, reason); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index d062bb197ca..779efe95a8c 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -624,8 +624,6 @@ bool StorageMergeTree::tryMutatePart() /// You must call destructor with unlocked `currently_merging_mutex`. std::optional tagger; { - auto disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path); - std::lock_guard lock(currently_merging_mutex); if (current_mutations_by_version.empty()) @@ -641,8 +639,7 @@ bool StorageMergeTree::tryMutatePart() if (mutations_begin_it == mutations_end_it) continue; - auto estimated_needed_space = MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}); - if (estimated_needed_space > disk_space) + if (merger_mutator.getMaxSourcePartSizeForMutation() < part->bytes_on_disk) continue; for (auto it = mutations_begin_it; it != mutations_end_it; ++it) @@ -655,7 +652,7 @@ bool StorageMergeTree::tryMutatePart() future_part.part_info = new_part_info; future_part.name = part->getNewName(new_part_info); - tagger.emplace(future_part, estimated_needed_space, *this); + tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this); break; } } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 028235d9eef..7a946400658 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2208,14 +2208,13 @@ void StorageReplicatedMergeTree::mergeSelectingTask() UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); FutureMergedMutatedPart future_merged_part; - - /// If there are many mutations in queue it may happen, that we cannot enqueue enough merges to merge all new parts if (max_source_parts_size_for_merge > 0 && merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred)) { success = createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, deduplicate, force_ttl); } + /// If there are many mutations in queue it may happen, that we cannot enqueue enough merges to merge all new parts else if (max_source_part_size_for_mutation > 0 && queue.countMutations() > 0 && merges_and_mutations_queued.second < settings.max_replicated_mutations_in_queue) { From 9c054419323fb8db91b586f6c19208507c7452c5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 22 Aug 2019 22:54:42 +0300 Subject: [PATCH 362/509] better test --- .../configs/merge_tree_max_parts.xml | 6 +++ .../configs/merge_tree_queue.xml | 7 --- .../test_replicated_mutations/test.py | 50 ++++++++++++------- 3 files changed, 39 insertions(+), 24 deletions(-) create mode 100644 dbms/tests/integration/test_replicated_mutations/configs/merge_tree_max_parts.xml delete mode 100644 dbms/tests/integration/test_replicated_mutations/configs/merge_tree_queue.xml diff --git a/dbms/tests/integration/test_replicated_mutations/configs/merge_tree_max_parts.xml b/dbms/tests/integration/test_replicated_mutations/configs/merge_tree_max_parts.xml new file mode 100644 index 00000000000..60047dcab2c --- /dev/null +++ b/dbms/tests/integration/test_replicated_mutations/configs/merge_tree_max_parts.xml @@ -0,0 +1,6 @@ + + + 50 + 50 + + \ No newline at end of file diff --git a/dbms/tests/integration/test_replicated_mutations/configs/merge_tree_queue.xml b/dbms/tests/integration/test_replicated_mutations/configs/merge_tree_queue.xml deleted file mode 100644 index ccc63f2eaec..00000000000 --- a/dbms/tests/integration/test_replicated_mutations/configs/merge_tree_queue.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - 50 - 8 - 4 - - \ No newline at end of file diff --git a/dbms/tests/integration/test_replicated_mutations/test.py b/dbms/tests/integration/test_replicated_mutations/test.py index dd42a70e280..0347ba4782c 100644 --- a/dbms/tests/integration/test_replicated_mutations/test.py +++ b/dbms/tests/integration/test_replicated_mutations/test.py @@ -14,10 +14,12 @@ node1 = cluster.add_instance('node1', macros={'cluster': 'test1'}, with_zookeepe # Check, that limits on max part size for merges doesn`t affect mutations node2 = cluster.add_instance('node2', macros={'cluster': 'test1'}, main_configs=["configs/merge_tree.xml"], with_zookeeper=True) -node3 = cluster.add_instance('node3', macros={'cluster': 'test2'}, main_configs=["configs/merge_tree_queue.xml"], with_zookeeper=True) -node4 = cluster.add_instance('node4', macros={'cluster': 'test2'}, main_configs=["configs/merge_tree_queue.xml"], with_zookeeper=True) +node3 = cluster.add_instance('node3', macros={'cluster': 'test2'}, main_configs=["configs/merge_tree_max_parts.xml"], with_zookeeper=True) +node4 = cluster.add_instance('node4', macros={'cluster': 'test2'}, main_configs=["configs/merge_tree_max_parts.xml"], with_zookeeper=True) -all_nodes = [node1, node2, node3, node4] +node5 = cluster.add_instance('node5', macros={'cluster': 'test3'}, main_configs=["configs/merge_tree_max_parts.xml"]) + +all_nodes = [node1, node2, node3, node4, node5] @pytest.fixture(scope="module") def started_cluster(): @@ -27,9 +29,11 @@ def started_cluster(): for node in all_nodes: node.query("DROP TABLE IF EXISTS test_mutations") - for node in all_nodes: + for node in [node1, node2, node3, node4]: node.query("CREATE TABLE test_mutations(d Date, x UInt32, i UInt32) ENGINE ReplicatedMergeTree('/clickhouse/{cluster}/tables/test/test_mutations', '{instance}') ORDER BY x PARTITION BY toYYYYMM(d)") + node5.query("CREATE TABLE test_mutations(d Date, x UInt32, i UInt32) ENGINE MergeTree() ORDER BY x PARTITION BY toYYYYMM(d)") + yield cluster finally: @@ -56,7 +60,7 @@ class Runner: self.exceptions = [] - def do_insert(self, thread_num): + def do_insert(self, thread_num, partitions_num): self.stop_ev.wait(random.random()) # Each thread inserts a small random number of rows with random year, month 01 and day determined @@ -74,7 +78,7 @@ class Runner: for x in xs: self.currently_inserting_xs[x] += 1 - year = 2000 + random.randint(0, 10) + year = 2000 + random.randint(0, partitions_num) date_str = '{year}-{month}-{day}'.format(year=year, month=month, day=day) payload = '' for x in xs: @@ -158,7 +162,7 @@ def test_mutations(started_cluster): threads = [] for thread_num in range(5): - threads.append(threading.Thread(target=runner.do_insert, args=(thread_num, ))) + threads.append(threading.Thread(target=runner.do_insert, args=(thread_num, 10))) for thread_num in (11, 12, 13): threads.append(threading.Thread(target=runner.do_delete, args=(thread_num,))) @@ -178,7 +182,9 @@ def test_mutations(started_cluster): all_done = wait_for_mutations(nodes, runner.total_mutations) - print node1.query("SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations' FORMAT TSVWithNames") + print "Total mutations: ", runner.total_mutations + for node in nodes: + print node.query("SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations' FORMAT TSVWithNames") assert all_done expected_sum = runner.total_inserted_xs - runner.total_deleted_xs @@ -188,24 +194,30 @@ def test_mutations(started_cluster): assert actual_sums[i] == expected_sum -def test_mutations_dont_prevent_merges(started_cluster): - nodes = [node3, node4] - for year in range(2000, 2008): +@pytest.mark.parametrize( + ('nodes', ), + [ + ([node5, ], ), # MergeTree + ([node3, node4], ), # ReplicatedMergeTree + ] +) +def test_mutations_dont_prevent_merges(started_cluster, nodes): + for year in range(2000, 2016): rows = '' date_str = '{}-01-{}'.format(year, random.randint(1, 10)) for i in range(10): rows += '{} {} {}\n'.format(date_str, random.randint(1, 10), i) - node3.query("INSERT INTO test_mutations FORMAT TSV", rows) + nodes[0].query("INSERT INTO test_mutations FORMAT TSV", rows) - # will run mutations of 8 parts in parallel, mutations will sleep for about 20 seconds - node3.query("ALTER TABLE test_mutations UPDATE i = sleepEachRow(2) WHERE 1") + # will run mutations of 16 parts in parallel, mutations will sleep for about 20 seconds + nodes[0].query("ALTER TABLE test_mutations UPDATE i = sleepEachRow(2) WHERE 1") runner = Runner(nodes) threads = [] - for thread_num in range(10): - threads.append(threading.Thread(target=runner.do_insert, args=(thread_num, ))) + for thread_num in range(2): + threads.append(threading.Thread(target=runner.do_insert, args=(thread_num, 0))) - # will insert approx 4-5 new parts per 1 second into each partition + # will insert approx 8-10 new parts per 1 second into one partition for t in threads: t.start() @@ -215,5 +227,9 @@ def test_mutations_dont_prevent_merges(started_cluster): for t in threads: t.join() + for node in nodes: + print node.query("SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations' FORMAT TSVWithNames") + print node.query("SELECT partition, count(name), sum(active), sum(active*rows) FROM system.parts WHERE table ='test_mutations' GROUP BY partition FORMAT TSVWithNames") + assert all_done assert all([str(e).find("Too many parts") < 0 for e in runner.exceptions]) From e8bc2189840613f6cbe1caaca987c293adff1b16 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 00:35:16 +0300 Subject: [PATCH 363/509] Rewrite flappy test --- .../00600_replace_running_query.reference | 8 ++++--- .../00600_replace_running_query.sh | 22 ++++++++++++------- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00600_replace_running_query.reference b/dbms/tests/queries/0_stateless/00600_replace_running_query.reference index 237dd6b5309..804267a1c11 100644 --- a/dbms/tests/queries/0_stateless/00600_replace_running_query.reference +++ b/dbms/tests/queries/0_stateless/00600_replace_running_query.reference @@ -1,5 +1,7 @@ 0 -1 0 -3 0 -2 0 +1 +1 +1 +finished 42 readonly SELECT 2, count() FROM system.numbers +1 44 diff --git a/dbms/tests/queries/0_stateless/00600_replace_running_query.sh b/dbms/tests/queries/0_stateless/00600_replace_running_query.sh index 8e4677bb1d5..513f6d8440e 100755 --- a/dbms/tests/queries/0_stateless/00600_replace_running_query.sh +++ b/dbms/tests/queries/0_stateless/00600_replace_running_query.sh @@ -1,17 +1,18 @@ #!/usr/bin/env bash +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -set -e -o pipefail function wait_for_query_to_start() { while [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.1; done } -$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL?query_id=hello&replace_running_query=1" -d 'SELECT sum(ignore(*)) FROM (SELECT number % 1000 AS k, groupArray(number) FROM numbers(100000000) GROUP BY k)' 2>&1 > /dev/null & +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL?query_id=hello&replace_running_query=1" -d 'SELECT 1, count() FROM system.numbers' 2>&1 > /dev/null & wait_for_query_to_start 'hello' # Replace it @@ -20,15 +21,20 @@ $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL?query_id=hello&replace_running_query=1" -d # Wait for it to be replaced wait -${CLICKHOUSE_CLIENT} --user=readonly --query_id=42 --query='SELECT 1, sleep(1)' & +${CLICKHOUSE_CLIENT} --user=readonly --query_id=42 --query='SELECT 2, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & wait_for_query_to_start '42' -( ${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 43' ||: ) 2>&1 | grep -F 'is already running by user' > /dev/null + +# Trying to run another query with the same query_id +${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 43' 2>&1 | grep -cF 'is already running by user' + +# Trying to replace query of a different user +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL?query_id=42&replace_running_query=1" -d 'SELECT 1' | grep -cF 'is already running by user' + +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = '42' SYNC" wait -${CLICKHOUSE_CLIENT} --query='SELECT 3, sleep(1)' & -sleep 0.1 -${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 2, sleep(1)' & +${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 3, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & wait_for_query_to_start '42' -( ${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --queue_max_wait_ms=500 --query='SELECT 43' ||: ) 2>&1 | grep -F "can't be stopped" > /dev/null +${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --queue_max_wait_ms=500 --query='SELECT 43' 2>&1 | grep -F "can't be stopped" > /dev/null ${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --query='SELECT 44' wait From dce975321599838a03ed22ab3a52a493a1f01f5d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 00:44:40 +0300 Subject: [PATCH 364/509] Fix splitted build. --- contrib/arrow-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 7b94acc9031..bc229deeced 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -219,7 +219,7 @@ endif() add_library(${ARROW_LIBRARY} ${ARROW_SRCS}) add_dependencies(${ARROW_LIBRARY} protoc) target_include_directories(${ARROW_LIBRARY} SYSTEM PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}/cpp/src ${Boost_INCLUDE_DIRS}) -target_link_libraries(${ARROW_LIBRARY} PRIVATE ${DOUBLE_CONVERSION_LIBRARIES} Threads::Threads) +target_link_libraries(${ARROW_LIBRARY} PRIVATE ${DOUBLE_CONVERSION_LIBRARIES} ${PROTOBUF_LIBRARIES} Threads::Threads) if (ARROW_WITH_LZ4) target_link_libraries(${ARROW_LIBRARY} PRIVATE ${LZ4_LIBRARY}) endif() From 859736d935275ef9d66e1f84c853f7f69f780394 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 01:41:30 +0300 Subject: [PATCH 365/509] Basic code quality of Live View --- .../PushingToViewsBlockOutputStream.cpp | 3 +- dbms/src/Storages/StorageLiveView.cpp | 217 ++++++++++++++++-- dbms/src/Storages/StorageLiveView.h | 184 +-------------- 3 files changed, 207 insertions(+), 197 deletions(-) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 840d3479ab9..d16d68bf72b 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -106,8 +106,7 @@ void PushingToViewsBlockOutputStream::write(const Block & block) if (auto * live_view = dynamic_cast(storage.get())) { - BlockOutputStreamPtr output_ = std::make_shared(*live_view); - StorageLiveView::writeIntoLiveView(*live_view, block, context, output_); + StorageLiveView::writeIntoLiveView(*live_view, block, context); } else { diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp index 6e42b9780e1..fdfa6a3a80d 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -9,6 +9,7 @@ distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ + #include #include #include @@ -17,10 +18,17 @@ limitations under the License. */ #include #include #include +#include #include #include #include #include +#include +#include +#include +#include +#include +#include #include #include @@ -98,6 +106,174 @@ static void checkAllowedQueries(const ASTSelectQuery & query) } } + +class LiveViewBlockOutputStream : public IBlockOutputStream +{ +public: + explicit LiveViewBlockOutputStream(StorageLiveView & storage_) : storage(storage_) {} + + void writePrefix() override + { + new_blocks = std::make_shared(); + new_blocks_metadata = std::make_shared(); + new_hash = std::make_shared(); + } + + void writeSuffix() override + { + UInt128 key; + String key_str; + + new_hash->get128(key.low, key.high); + key_str = key.toHexString(); + + Poco::FastMutex::ScopedLock lock(storage.mutex); + + if (storage.getBlocksHashKey() != key_str) + { + new_blocks_metadata->hash = key_str; + new_blocks_metadata->version = storage.getBlocksVersion() + 1; + + for (auto & block : *new_blocks) + { + block.insert({DataTypeUInt64().createColumnConst( + block.rows(), new_blocks_metadata->version)->convertToFullColumnIfConst(), + std::make_shared(), + "_version"}); + } + + (*storage.blocks_ptr) = new_blocks; + (*storage.blocks_metadata_ptr) = new_blocks_metadata; + + storage.condition.broadcast(); + } + + new_blocks.reset(); + new_blocks_metadata.reset(); + new_hash.reset(); + } + + void write(const Block & block) override + { + new_blocks->push_back(block); + block.updateHash(*new_hash); + } + + Block getHeader() const override { return storage.getHeader(); } + +private: + using SipHashPtr = std::shared_ptr; + + BlocksPtr new_blocks; + BlocksMetadataPtr new_blocks_metadata; + SipHashPtr new_hash; + StorageLiveView & storage; +}; + + +void StorageLiveView::writeIntoLiveView( + StorageLiveView & live_view, + const Block & block, + const Context & context) +{ + BlockOutputStreamPtr output = std::make_shared(live_view); + + /// Check if live view has any readers if not + /// just reset blocks to empty and do nothing else + /// When first reader comes the blocks will be read. + { + Poco::FastMutex::ScopedLock lock(live_view.mutex); + if (!live_view.hasActiveUsers()) + { + live_view.reset(); + return; + } + } + + bool is_block_processed = false; + BlockInputStreams from; + BlocksPtrs mergeable_blocks; + BlocksPtr new_mergeable_blocks = std::make_shared(); + + { + Poco::FastMutex::ScopedLock lock(live_view.mutex); + + mergeable_blocks = live_view.getMergeableBlocks(); + if (!mergeable_blocks || mergeable_blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh) + { + mergeable_blocks = std::make_shared>(); + BlocksPtr base_mergeable_blocks = std::make_shared(); + InterpreterSelectQuery interpreter(live_view.getInnerQuery(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); + auto view_mergeable_stream = std::make_shared( + interpreter.execute().in); + while (Block this_block = view_mergeable_stream->read()) + base_mergeable_blocks->push_back(this_block); + mergeable_blocks->push_back(base_mergeable_blocks); + live_view.setMergeableBlocks(mergeable_blocks); + + /// Create from streams + for (auto & blocks_ : *mergeable_blocks) + { + if (blocks_->empty()) + continue; + auto sample_block = blocks_->front().cloneEmpty(); + BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); + from.push_back(std::move(stream)); + } + + is_block_processed = true; + } + } + + if (!is_block_processed) + { + auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); + BlockInputStreams streams = {std::make_shared(block)}; + auto proxy_storage = std::make_shared(parent_storage, std::move(streams), QueryProcessingStage::FetchColumns); + InterpreterSelectQuery select_block(live_view.getInnerQuery(), + context, proxy_storage, + QueryProcessingStage::WithMergeableState); + auto data_mergeable_stream = std::make_shared( + select_block.execute().in); + while (Block this_block = data_mergeable_stream->read()) + new_mergeable_blocks->push_back(this_block); + + if (new_mergeable_blocks->empty()) + return; + + { + Poco::FastMutex::ScopedLock lock(live_view.mutex); + + mergeable_blocks = live_view.getMergeableBlocks(); + mergeable_blocks->push_back(new_mergeable_blocks); + + /// Create from streams + for (auto & blocks_ : *mergeable_blocks) + { + if (blocks_->empty()) + continue; + auto sample_block = blocks_->front().cloneEmpty(); + BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); + from.push_back(std::move(stream)); + } + } + } + + auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); + auto proxy_storage = std::make_shared(parent_storage, std::move(from), QueryProcessingStage::WithMergeableState); + InterpreterSelectQuery select(live_view.getInnerQuery(), context, proxy_storage, QueryProcessingStage::Complete); + BlockInputStreamPtr data = std::make_shared(select.execute().in); + + /// Squashing is needed here because the view query can generate a lot of blocks + /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY + /// and two-level aggregation is triggered). + data = std::make_shared( + data, context.getGlobalContext().getSettingsRef().min_insert_block_size_rows, context.getGlobalContext().getSettingsRef().min_insert_block_size_bytes); + + copyData(*data, *output); +} + + StorageLiveView::StorageLiveView( const String & table_name_, const String & database_name_, @@ -259,11 +435,10 @@ void StorageLiveView::noUsersThread(const UInt64 & timeout) { while (1) { - Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); - if (!noUsersThreadWakeUp && !noUsersThreadCondition.tryWait(noUsersThreadMutex, - timeout * 1000)) + Poco::FastMutex::ScopedLock lock(no_users_thread_mutex); + if (!no_users_thread_wakeup && !no_users_thread_condition.tryWait(no_users_thread_mutex, timeout * 1000)) { - noUsersThreadWakeUp = false; + no_users_thread_wakeup = false; if (shutdown_called) return; if (hasUsers()) @@ -301,7 +476,7 @@ void StorageLiveView::noUsersThread(const UInt64 & timeout) void StorageLiveView::startNoUsersThread(const UInt64 & timeout) { bool expected = false; - if (!startnousersthread_called.compare_exchange_strong(expected, true)) + if (!start_no_users_thread_called.compare_exchange_strong(expected, true)) return; if (is_dropped) @@ -312,20 +487,20 @@ void StorageLiveView::startNoUsersThread(const UInt64 & timeout) if (no_users_thread.joinable()) { { - Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); - noUsersThreadWakeUp = true; - noUsersThreadCondition.signal(); + Poco::FastMutex::ScopedLock lock(no_users_thread_mutex); + no_users_thread_wakeup = true; + no_users_thread_condition.signal(); } no_users_thread.join(); } { - Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); - noUsersThreadWakeUp = false; + Poco::FastMutex::ScopedLock lock(no_users_thread_mutex); + no_users_thread_wakeup = false; } if (!is_dropped) no_users_thread = std::thread(&StorageLiveView::noUsersThread, this, timeout); } - startnousersthread_called = false; + start_no_users_thread_called = false; } void StorageLiveView::startup() @@ -341,13 +516,13 @@ void StorageLiveView::shutdown() if (no_users_thread.joinable()) { - Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); - noUsersThreadWakeUp = true; - noUsersThreadCondition.signal(); + Poco::FastMutex::ScopedLock lock(no_users_thread_mutex); + no_users_thread_wakeup = true; + no_users_thread_condition.signal(); /// Must detach the no users thread /// as we can't join it as it will result /// in a deadlock - no_users_thread.detach(); + no_users_thread.detach(); /// TODO Not viable at all. } } @@ -423,9 +598,9 @@ BlockInputStreams StorageLiveView::watch( if (no_users_thread.joinable()) { - Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); - noUsersThreadWakeUp = true; - noUsersThreadCondition.signal(); + Poco::FastMutex::ScopedLock lock(no_users_thread_mutex); + no_users_thread_wakeup = true; + no_users_thread_condition.signal(); } { @@ -448,9 +623,9 @@ BlockInputStreams StorageLiveView::watch( if (no_users_thread.joinable()) { - Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); - noUsersThreadWakeUp = true; - noUsersThreadCondition.signal(); + Poco::FastMutex::ScopedLock lock(no_users_thread_mutex); + no_users_thread_wakeup = true; + no_users_thread_condition.signal(); } { diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h index 6bff9c2dc85..3b2398a421f 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/StorageLiveView.h @@ -12,33 +12,25 @@ limitations under the License. */ #pragma once #include -#include -#include -#include -#include -#include -#include -#include #include #include #include #include #include + namespace DB { -class IAST; - struct BlocksMetadata { String hash; UInt64 version; }; +class IAST; using ASTPtr = std::shared_ptr; using BlocksMetadataPtr = std::shared_ptr; -using SipHashPtr = std::shared_ptr; class StorageLiveView : public ext::shared_ptr_helper, public IStorage { @@ -87,9 +79,9 @@ public: /// Background thread for temporary tables /// which drops this table if there are no users void startNoUsersThread(const UInt64 & timeout); - Poco::FastMutex noUsersThreadMutex; - bool noUsersThreadWakeUp{false}; - Poco::Condition noUsersThreadCondition; + Poco::FastMutex no_users_thread_mutex; + bool no_users_thread_wakeup{false}; + Poco::Condition no_users_thread_condition; /// Get blocks hash /// must be called with mutex locked String getBlocksHashKey() @@ -150,105 +142,10 @@ public: Block getHeader() const; - static void writeIntoLiveView(StorageLiveView & live_view, - const Block & block, - const Context & context, - BlockOutputStreamPtr & output) - { - /// Check if live view has any readers if not - /// just reset blocks to empty and do nothing else - /// When first reader comes the blocks will be read. - { - Poco::FastMutex::ScopedLock lock(live_view.mutex); - if (!live_view.hasActiveUsers()) - { - live_view.reset(); - return; - } - } - - bool is_block_processed = false; - BlockInputStreams from; - BlocksPtrs mergeable_blocks; - BlocksPtr new_mergeable_blocks = std::make_shared(); - - { - Poco::FastMutex::ScopedLock lock(live_view.mutex); - - mergeable_blocks = live_view.getMergeableBlocks(); - if (!mergeable_blocks || mergeable_blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh) - { - mergeable_blocks = std::make_shared>(); - BlocksPtr base_mergeable_blocks = std::make_shared(); - InterpreterSelectQuery interpreter(live_view.getInnerQuery(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); - auto view_mergeable_stream = std::make_shared( - interpreter.execute().in); - while (Block this_block = view_mergeable_stream->read()) - base_mergeable_blocks->push_back(this_block); - mergeable_blocks->push_back(base_mergeable_blocks); - live_view.setMergeableBlocks(mergeable_blocks); - - /// Create from streams - for (auto & blocks_ : *mergeable_blocks) - { - if (blocks_->empty()) - continue; - auto sample_block = blocks_->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); - from.push_back(std::move(stream)); - } - - is_block_processed = true; - } - } - - if (!is_block_processed) - { - auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); - BlockInputStreams streams = {std::make_shared(block)}; - auto proxy_storage = std::make_shared(parent_storage, std::move(streams), QueryProcessingStage::FetchColumns); - InterpreterSelectQuery select_block(live_view.getInnerQuery(), - context, proxy_storage, - QueryProcessingStage::WithMergeableState); - auto data_mergeable_stream = std::make_shared( - select_block.execute().in); - while (Block this_block = data_mergeable_stream->read()) - new_mergeable_blocks->push_back(this_block); - - if (new_mergeable_blocks->empty()) - return; - - { - Poco::FastMutex::ScopedLock lock(live_view.mutex); - - mergeable_blocks = live_view.getMergeableBlocks(); - mergeable_blocks->push_back(new_mergeable_blocks); - - /// Create from streams - for (auto & blocks_ : *mergeable_blocks) - { - if (blocks_->empty()) - continue; - auto sample_block = blocks_->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); - from.push_back(std::move(stream)); - } - } - } - - auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); - auto proxy_storage = std::make_shared(parent_storage, std::move(from), QueryProcessingStage::WithMergeableState); - InterpreterSelectQuery select(live_view.getInnerQuery(), context, proxy_storage, QueryProcessingStage::Complete); - BlockInputStreamPtr data = std::make_shared(select.execute().in); - - /// Squashing is needed here because the view query can generate a lot of blocks - /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY - /// and two-level aggregation is triggered). - data = std::make_shared( - data, context.getGlobalContext().getSettingsRef().min_insert_block_size_rows, context.getGlobalContext().getSettingsRef().min_insert_block_size_bytes); - - copyData(*data, *output); - } + static void writeIntoLiveView( + StorageLiveView & live_view, + const Block & block, + const Context & context); private: String select_database_name; @@ -271,7 +168,7 @@ private: void noUsersThread(const UInt64 & timeout); std::thread no_users_thread; std::atomic shutdown_called{false}; - std::atomic startnousersthread_called{false}; + std::atomic start_no_users_thread_called{false}; UInt64 temporary_live_view_timeout; StorageLiveView( @@ -283,65 +180,4 @@ private: ); }; -class LiveViewBlockOutputStream : public IBlockOutputStream -{ -public: - explicit LiveViewBlockOutputStream(StorageLiveView & storage_) : storage(storage_) {} - - void writePrefix() override - { - new_blocks = std::make_shared(); - new_blocks_metadata = std::make_shared(); - new_hash = std::make_shared(); - } - - void writeSuffix() override - { - UInt128 key; - String key_str; - - new_hash->get128(key.low, key.high); - key_str = key.toHexString(); - - Poco::FastMutex::ScopedLock lock(storage.mutex); - - if (storage.getBlocksHashKey() != key_str) - { - new_blocks_metadata->hash = key_str; - new_blocks_metadata->version = storage.getBlocksVersion() + 1; - - for (auto & block : *new_blocks) - { - block.insert({DataTypeUInt64().createColumnConst( - block.rows(), new_blocks_metadata->version)->convertToFullColumnIfConst(), - std::make_shared(), - "_version"}); - } - - (*storage.blocks_ptr) = new_blocks; - (*storage.blocks_metadata_ptr) = new_blocks_metadata; - - storage.condition.broadcast(); - } - - new_blocks.reset(); - new_blocks_metadata.reset(); - new_hash.reset(); - } - - void write(const Block & block) override - { - new_blocks->push_back(block); - block.updateHash(*new_hash); - } - - Block getHeader() const override { return storage.getHeader(); } - -private: - BlocksPtr new_blocks; - BlocksMetadataPtr new_blocks_metadata; - SipHashPtr new_hash; - StorageLiveView & storage; -}; - } From c4712f1e6e2376232747e839850127238a008061 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 02:22:57 +0300 Subject: [PATCH 366/509] Make the code less bad --- dbms/CMakeLists.txt | 1 + .../PushingToViewsBlockOutputStream.cpp | 2 +- .../PushingToViewsBlockOutputStream.h | 1 - .../Interpreters/InterpreterAlterQuery.cpp | 4 +- .../LiveView}/LiveViewBlockInputStream.h | 38 ++---- .../LiveView/LiveViewBlockOutputStream.h | 74 +++++++++++ .../{ => LiveView}/LiveViewCommands.h | 4 +- .../LiveViewEventsBlockInputStream.h | 27 ++-- .../Storages/{ => LiveView}/ProxyStorage.h | 0 .../{ => LiveView}/StorageLiveView.cpp | 125 +++++------------- .../Storages/{ => LiveView}/StorageLiveView.h | 27 ++-- dbms/src/Storages/StorageFactory.cpp | 14 -- dbms/src/Storages/registerStorages.cpp | 2 - 13 files changed, 152 insertions(+), 167 deletions(-) rename dbms/src/{DataStreams => Storages/LiveView}/LiveViewBlockInputStream.h (81%) create mode 100644 dbms/src/Storages/LiveView/LiveViewBlockOutputStream.h rename dbms/src/Storages/{ => LiveView}/LiveViewCommands.h (97%) rename dbms/src/{DataStreams => Storages/LiveView}/LiveViewEventsBlockInputStream.h (91%) rename dbms/src/Storages/{ => LiveView}/ProxyStorage.h (100%) rename dbms/src/Storages/{ => LiveView}/StorageLiveView.cpp (85%) rename dbms/src/Storages/{ => LiveView}/StorageLiveView.h (94%) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index b589c398238..f011cc21103 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -114,6 +114,7 @@ add_headers_and_sources(dbms src/Columns) add_headers_and_sources(dbms src/Storages) add_headers_and_sources(dbms src/Storages/Distributed) add_headers_and_sources(dbms src/Storages/MergeTree) +add_headers_and_sources(dbms src/Storages/LiveView) add_headers_and_sources(dbms src/Client) add_headers_and_sources(dbms src/Formats) add_headers_and_sources(dbms src/Processors) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index d16d68bf72b..807a9129a75 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h index e3f96241b1d..c9b0538e470 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB { diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index a3e6824c3a5..61d5e011d37 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -8,9 +8,9 @@ #include #include #include -#include +#include +#include #include -#include #include diff --git a/dbms/src/DataStreams/LiveViewBlockInputStream.h b/dbms/src/Storages/LiveView/LiveViewBlockInputStream.h similarity index 81% rename from dbms/src/DataStreams/LiveViewBlockInputStream.h rename to dbms/src/Storages/LiveView/LiveViewBlockInputStream.h index b3756c9ff6d..345fceaf095 100644 --- a/dbms/src/DataStreams/LiveViewBlockInputStream.h +++ b/dbms/src/Storages/LiveView/LiveViewBlockInputStream.h @@ -1,23 +1,6 @@ -/* Copyright (c) 2018 BlackBerry Limited - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at -http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. */ #pragma once -#include - -#include -#include -#include #include -#include namespace DB @@ -61,8 +44,8 @@ public: if (isCancelled() || storage->is_dropped) return; IBlockInputStream::cancel(kill); - Poco::FastMutex::ScopedLock lock(storage->mutex); - storage->condition.broadcast(); + std::lock_guard lock(storage->mutex); + storage->condition.notify_all(); } Block getHeader() const override { return storage->getHeader(); } @@ -92,14 +75,14 @@ public: NonBlockingResult tryRead() { - return tryRead_(false); + return tryReadImpl(false); } protected: Block readImpl() override { /// try reading - return tryRead_(true).first; + return tryReadImpl(true).first; } /** tryRead method attempts to read a block in either blocking @@ -107,7 +90,7 @@ protected: * then method return empty block with flag set to false * to indicate that method would block to get the next block. */ - NonBlockingResult tryRead_(bool blocking) + NonBlockingResult tryReadImpl(bool blocking) { Block res; @@ -118,7 +101,7 @@ protected: /// If blocks were never assigned get blocks if (!blocks) { - Poco::FastMutex::ScopedLock lock(storage->mutex); + std::lock_guard lock(storage->mutex); if (!active) return { Block(), false }; blocks = (*blocks_ptr); @@ -135,7 +118,7 @@ protected: if (it == end) { { - Poco::FastMutex::ScopedLock lock(storage->mutex); + std::unique_lock lock(storage->mutex); if (!active) return { Block(), false }; /// If we are done iterating over our blocks @@ -162,7 +145,10 @@ protected: while (true) { UInt64 timestamp_usec = static_cast(timestamp.epochMicroseconds()); - bool signaled = storage->condition.tryWait(storage->mutex, std::max(static_cast(0), heartbeat_interval_usec - (timestamp_usec - last_event_timestamp_usec)) / 1000); + + /// Or spurious wakeup. + bool signaled = std::cv_status::no_timeout == storage->condition.wait_for(lock, + std::chrono::microseconds(std::max(UInt64(0), heartbeat_interval_usec - (timestamp_usec - last_event_timestamp_usec)))); if (isCancelled() || storage->is_dropped) { @@ -181,7 +167,7 @@ protected: } } } - return tryRead_(blocking); + return tryReadImpl(blocking); } res = *it; diff --git a/dbms/src/Storages/LiveView/LiveViewBlockOutputStream.h b/dbms/src/Storages/LiveView/LiveViewBlockOutputStream.h new file mode 100644 index 00000000000..548bcf1b86a --- /dev/null +++ b/dbms/src/Storages/LiveView/LiveViewBlockOutputStream.h @@ -0,0 +1,74 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class LiveViewBlockOutputStream : public IBlockOutputStream +{ +public: + explicit LiveViewBlockOutputStream(StorageLiveView & storage_) : storage(storage_) {} + + void writePrefix() override + { + new_blocks = std::make_shared(); + new_blocks_metadata = std::make_shared(); + new_hash = std::make_shared(); + } + + void writeSuffix() override + { + UInt128 key; + String key_str; + + new_hash->get128(key.low, key.high); + key_str = key.toHexString(); + + std::lock_guard lock(storage.mutex); + + if (storage.getBlocksHashKey() != key_str) + { + new_blocks_metadata->hash = key_str; + new_blocks_metadata->version = storage.getBlocksVersion() + 1; + + for (auto & block : *new_blocks) + { + block.insert({DataTypeUInt64().createColumnConst( + block.rows(), new_blocks_metadata->version)->convertToFullColumnIfConst(), + std::make_shared(), + "_version"}); + } + + (*storage.blocks_ptr) = new_blocks; + (*storage.blocks_metadata_ptr) = new_blocks_metadata; + + storage.condition.notify_all(); + } + + new_blocks.reset(); + new_blocks_metadata.reset(); + new_hash.reset(); + } + + void write(const Block & block) override + { + new_blocks->push_back(block); + block.updateHash(*new_hash); + } + + Block getHeader() const override { return storage.getHeader(); } + +private: + using SipHashPtr = std::shared_ptr; + + BlocksPtr new_blocks; + BlocksMetadataPtr new_blocks_metadata; + SipHashPtr new_hash; + StorageLiveView & storage; +}; + +} diff --git a/dbms/src/Storages/LiveViewCommands.h b/dbms/src/Storages/LiveView/LiveViewCommands.h similarity index 97% rename from dbms/src/Storages/LiveViewCommands.h rename to dbms/src/Storages/LiveView/LiveViewCommands.h index 35015a7e5aa..54048c28a5f 100644 --- a/dbms/src/Storages/LiveViewCommands.h +++ b/dbms/src/Storages/LiveView/LiveViewCommands.h @@ -12,9 +12,9 @@ limitations under the License. */ #pragma once -#include #include -#include +#include +#include namespace DB { diff --git a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h b/dbms/src/Storages/LiveView/LiveViewEventsBlockInputStream.h similarity index 91% rename from dbms/src/DataStreams/LiveViewEventsBlockInputStream.h rename to dbms/src/Storages/LiveView/LiveViewEventsBlockInputStream.h index 93fb6a76372..120d0098536 100644 --- a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h +++ b/dbms/src/Storages/LiveView/LiveViewEventsBlockInputStream.h @@ -9,11 +9,9 @@ distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ + #pragma once -#include - -#include #include #include #include @@ -21,7 +19,7 @@ limitations under the License. */ #include #include #include -#include +#include namespace DB @@ -66,8 +64,8 @@ public: if (isCancelled() || storage->is_dropped) return; IBlockInputStream::cancel(kill); - Poco::FastMutex::ScopedLock lock(storage->mutex); - storage->condition.broadcast(); + std::lock_guard lock(storage->mutex); + storage->condition.notify_all(); } Block getHeader() const override @@ -103,7 +101,7 @@ public: NonBlockingResult tryRead() { - return tryRead_(false); + return tryReadImpl(false); } Block getEventBlock() @@ -120,7 +118,7 @@ protected: Block readImpl() override { /// try reading - return tryRead_(true).first; + return tryReadImpl(true).first; } /** tryRead method attempts to read a block in either blocking @@ -128,7 +126,7 @@ protected: * then method return empty block with flag set to false * to indicate that method would block to get the next block. */ - NonBlockingResult tryRead_(bool blocking) + NonBlockingResult tryReadImpl(bool blocking) { if (has_limit && num_updates == static_cast(limit)) { @@ -137,7 +135,7 @@ protected: /// If blocks were never assigned get blocks if (!blocks) { - Poco::FastMutex::ScopedLock lock(storage->mutex); + std::lock_guard lock(storage->mutex); if (!active) return { Block(), false }; blocks = (*blocks_ptr); @@ -155,7 +153,7 @@ protected: if (it == end) { { - Poco::FastMutex::ScopedLock lock(storage->mutex); + std::unique_lock lock(storage->mutex); if (!active) return { Block(), false }; /// If we are done iterating over our blocks @@ -183,7 +181,10 @@ protected: while (true) { UInt64 timestamp_usec = static_cast(timestamp.epochMicroseconds()); - bool signaled = storage->condition.tryWait(storage->mutex, std::max(static_cast(0), heartbeat_interval_usec - (timestamp_usec - last_event_timestamp_usec)) / 1000); + + /// Or spurious wakeup. + bool signaled = std::cv_status::no_timeout == storage->condition.wait_for(lock, + std::chrono::microseconds(std::max(UInt64(0), heartbeat_interval_usec - (timestamp_usec - last_event_timestamp_usec)))); if (isCancelled() || storage->is_dropped) { @@ -202,7 +203,7 @@ protected: } } } - return tryRead_(blocking); + return tryReadImpl(blocking); } // move right to the end diff --git a/dbms/src/Storages/ProxyStorage.h b/dbms/src/Storages/LiveView/ProxyStorage.h similarity index 100% rename from dbms/src/Storages/ProxyStorage.h rename to dbms/src/Storages/LiveView/ProxyStorage.h diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp similarity index 85% rename from dbms/src/Storages/StorageLiveView.cpp rename to dbms/src/Storages/LiveView/StorageLiveView.cpp index fdfa6a3a80d..98d48392e09 100644 --- a/dbms/src/Storages/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -20,9 +20,6 @@ limitations under the License. */ #include #include #include -#include -#include -#include #include #include #include @@ -30,10 +27,15 @@ limitations under the License. */ #include #include #include +#include + +#include +#include +#include +#include +#include -#include #include -#include #include #include #include @@ -107,70 +109,6 @@ static void checkAllowedQueries(const ASTSelectQuery & query) } -class LiveViewBlockOutputStream : public IBlockOutputStream -{ -public: - explicit LiveViewBlockOutputStream(StorageLiveView & storage_) : storage(storage_) {} - - void writePrefix() override - { - new_blocks = std::make_shared(); - new_blocks_metadata = std::make_shared(); - new_hash = std::make_shared(); - } - - void writeSuffix() override - { - UInt128 key; - String key_str; - - new_hash->get128(key.low, key.high); - key_str = key.toHexString(); - - Poco::FastMutex::ScopedLock lock(storage.mutex); - - if (storage.getBlocksHashKey() != key_str) - { - new_blocks_metadata->hash = key_str; - new_blocks_metadata->version = storage.getBlocksVersion() + 1; - - for (auto & block : *new_blocks) - { - block.insert({DataTypeUInt64().createColumnConst( - block.rows(), new_blocks_metadata->version)->convertToFullColumnIfConst(), - std::make_shared(), - "_version"}); - } - - (*storage.blocks_ptr) = new_blocks; - (*storage.blocks_metadata_ptr) = new_blocks_metadata; - - storage.condition.broadcast(); - } - - new_blocks.reset(); - new_blocks_metadata.reset(); - new_hash.reset(); - } - - void write(const Block & block) override - { - new_blocks->push_back(block); - block.updateHash(*new_hash); - } - - Block getHeader() const override { return storage.getHeader(); } - -private: - using SipHashPtr = std::shared_ptr; - - BlocksPtr new_blocks; - BlocksMetadataPtr new_blocks_metadata; - SipHashPtr new_hash; - StorageLiveView & storage; -}; - - void StorageLiveView::writeIntoLiveView( StorageLiveView & live_view, const Block & block, @@ -182,7 +120,7 @@ void StorageLiveView::writeIntoLiveView( /// just reset blocks to empty and do nothing else /// When first reader comes the blocks will be read. { - Poco::FastMutex::ScopedLock lock(live_view.mutex); + std::lock_guard lock(live_view.mutex); if (!live_view.hasActiveUsers()) { live_view.reset(); @@ -196,7 +134,7 @@ void StorageLiveView::writeIntoLiveView( BlocksPtr new_mergeable_blocks = std::make_shared(); { - Poco::FastMutex::ScopedLock lock(live_view.mutex); + std::lock_guard lock(live_view.mutex); mergeable_blocks = live_view.getMergeableBlocks(); if (!mergeable_blocks || mergeable_blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh) @@ -242,7 +180,7 @@ void StorageLiveView::writeIntoLiveView( return; { - Poco::FastMutex::ScopedLock lock(live_view.mutex); + std::lock_guard lock(live_view.mutex); mergeable_blocks = live_view.getMergeableBlocks(); mergeable_blocks->push_back(new_mergeable_blocks); @@ -435,8 +373,8 @@ void StorageLiveView::noUsersThread(const UInt64 & timeout) { while (1) { - Poco::FastMutex::ScopedLock lock(no_users_thread_mutex); - if (!no_users_thread_wakeup && !no_users_thread_condition.tryWait(no_users_thread_mutex, timeout * 1000)) + std::unique_lock lock(no_users_thread_mutex); + if (!no_users_thread_condition.wait_for(lock, std::chrono::seconds(timeout), [&] { return no_users_thread_wakeup; })) { no_users_thread_wakeup = false; if (shutdown_called) @@ -487,14 +425,14 @@ void StorageLiveView::startNoUsersThread(const UInt64 & timeout) if (no_users_thread.joinable()) { { - Poco::FastMutex::ScopedLock lock(no_users_thread_mutex); + std::lock_guard lock(no_users_thread_mutex); no_users_thread_wakeup = true; - no_users_thread_condition.signal(); + no_users_thread_condition.notify_one(); } no_users_thread.join(); } { - Poco::FastMutex::ScopedLock lock(no_users_thread_mutex); + std::lock_guard lock(no_users_thread_mutex); no_users_thread_wakeup = false; } if (!is_dropped) @@ -516,9 +454,9 @@ void StorageLiveView::shutdown() if (no_users_thread.joinable()) { - Poco::FastMutex::ScopedLock lock(no_users_thread_mutex); + std::lock_guard lock(no_users_thread_mutex); no_users_thread_wakeup = true; - no_users_thread_condition.signal(); + no_users_thread_condition.notify_one(); /// Must detach the no users thread /// as we can't join it as it will result /// in a deadlock @@ -536,18 +474,19 @@ void StorageLiveView::drop() global_context.removeDependency( DatabaseAndTableName(select_database_name, select_table_name), DatabaseAndTableName(database_name, table_name)); - Poco::FastMutex::ScopedLock lock(mutex); + + std::lock_guard lock(mutex); is_dropped = true; - condition.broadcast(); + condition.notify_all(); } void StorageLiveView::refresh(const Context & context) { auto alter_lock = lockAlterIntention(context.getCurrentQueryId()); { - Poco::FastMutex::ScopedLock lock(mutex); + std::lock_guard lock(mutex); if (getNewBlocks()) - condition.broadcast(); + condition.notify_all(); } } @@ -562,11 +501,11 @@ BlockInputStreams StorageLiveView::read( /// add user to the blocks_ptr std::shared_ptr stream_blocks_ptr = blocks_ptr; { - Poco::FastMutex::ScopedLock lock(mutex); + std::lock_guard lock(mutex); if (!(*blocks_ptr)) { if (getNewBlocks()) - condition.broadcast(); + condition.notify_all(); } } return { std::make_shared(stream_blocks_ptr, getHeader()) }; @@ -598,17 +537,17 @@ BlockInputStreams StorageLiveView::watch( if (no_users_thread.joinable()) { - Poco::FastMutex::ScopedLock lock(no_users_thread_mutex); + std::lock_guard lock(no_users_thread_mutex); no_users_thread_wakeup = true; - no_users_thread_condition.signal(); + no_users_thread_condition.notify_one(); } { - Poco::FastMutex::ScopedLock lock(mutex); + std::lock_guard lock(mutex); if (!(*blocks_ptr)) { if (getNewBlocks()) - condition.broadcast(); + condition.notify_all(); } } @@ -623,17 +562,17 @@ BlockInputStreams StorageLiveView::watch( if (no_users_thread.joinable()) { - Poco::FastMutex::ScopedLock lock(no_users_thread_mutex); + std::lock_guard lock(no_users_thread_mutex); no_users_thread_wakeup = true; - no_users_thread_condition.signal(); + no_users_thread_condition.notify_one(); } { - Poco::FastMutex::ScopedLock lock(mutex); + std::lock_guard lock(mutex); if (!(*blocks_ptr)) { if (getNewBlocks()) - condition.broadcast(); + condition.notify_all(); } } diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h similarity index 94% rename from dbms/src/Storages/StorageLiveView.h rename to dbms/src/Storages/LiveView/StorageLiveView.h index 3b2398a421f..9930d8d6154 100644 --- a/dbms/src/Storages/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -11,12 +11,11 @@ See the License for the specific language governing permissions and limitations under the License. */ #pragma once -#include -#include #include -#include #include -#include + +#include +#include namespace DB @@ -35,6 +34,8 @@ using BlocksMetadataPtr = std::shared_ptr; class StorageLiveView : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; +friend class LiveViewBlockInputStream; +friend class LiveViewEventsBlockInputStream; friend class LiveViewBlockOutputStream; public: @@ -55,12 +56,6 @@ public: bool supportsSampling() const override { return true; } bool supportsFinal() const override { return true; } - /// Mutex for the blocks and ready condition - Poco::FastMutex mutex; - /// New blocks ready condition to broadcast to readers - /// that new blocks are available - Poco::Condition condition; - bool isTemporary() { return is_temporary; } /// Check if we have any readers @@ -79,16 +74,16 @@ public: /// Background thread for temporary tables /// which drops this table if there are no users void startNoUsersThread(const UInt64 & timeout); - Poco::FastMutex no_users_thread_mutex; + std::mutex no_users_thread_mutex; bool no_users_thread_wakeup{false}; - Poco::Condition no_users_thread_condition; + std::condition_variable no_users_thread_condition; /// Get blocks hash /// must be called with mutex locked String getBlocksHashKey() { if (*blocks_metadata_ptr) return (*blocks_metadata_ptr)->hash; - return ""; + return {}; } /// Get blocks version /// must be called with mutex locked @@ -157,6 +152,12 @@ private: bool is_temporary {false}; mutable Block sample_block; + /// Mutex for the blocks and ready condition + std::mutex mutex; + /// New blocks ready condition to broadcast to readers + /// that new blocks are available + std::condition_variable condition; + /// Active users std::shared_ptr active_ptr; /// Current data blocks that store query result diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index 7d92ce0ea2c..862f76bc3ce 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -68,14 +68,6 @@ StoragePtr StorageFactory::get( name = "LiveView"; } - else if (query.is_live_channel) - { - - if (query.storage) - throw Exception("Specifying ENGINE is not allowed for a LiveChannel", ErrorCodes::INCORRECT_QUERY); - - name = "LiveChannel"; - } else { /// Check for some special types, that are not allowed to be stored in tables. Example: NULL data type. @@ -137,12 +129,6 @@ StoragePtr StorageFactory::get( "Direct creation of tables with ENGINE LiveView is not supported, use CREATE LIVE VIEW statement", ErrorCodes::INCORRECT_QUERY); } - else if (name == "LiveChannel") - { - throw Exception( - "Direct creation of tables with ENGINE LiveChannel is not supported, use CREATE LIVE CHANNEL statement", - ErrorCodes::INCORRECT_QUERY); - } } } diff --git a/dbms/src/Storages/registerStorages.cpp b/dbms/src/Storages/registerStorages.cpp index 811a031c7a3..c21156ea44d 100644 --- a/dbms/src/Storages/registerStorages.cpp +++ b/dbms/src/Storages/registerStorages.cpp @@ -25,7 +25,6 @@ void registerStorageJoin(StorageFactory & factory); void registerStorageView(StorageFactory & factory); void registerStorageMaterializedView(StorageFactory & factory); void registerStorageLiveView(StorageFactory & factory); -//void registerStorageLiveChannel(StorageFactory & factory); #if USE_HDFS void registerStorageHDFS(StorageFactory & factory); @@ -67,7 +66,6 @@ void registerStorages() registerStorageView(factory); registerStorageMaterializedView(factory); registerStorageLiveView(factory); - //registerStorageLiveChannel(factory); #if USE_HDFS registerStorageHDFS(factory); From 78c3be175ca2ca52b04e18c5e2c690242a6eeb44 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 02:46:40 +0300 Subject: [PATCH 367/509] Added experimental setting for LIVE VIEWs --- dbms/src/Core/Settings.h | 2 ++ dbms/src/Interpreters/InterpreterWatchQuery.cpp | 4 ++++ dbms/src/Storages/LiveView/StorageLiveView.cpp | 5 ++++- 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 22b0b5c8d03..6a22869c8dc 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -342,6 +342,8 @@ struct Settings : public SettingsCollection M(SettingUInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.") \ M(SettingBool, check_query_single_value_result, true, "Return check query result as single 1/0 value") \ \ + M(SettingBool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.") \ + \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") \ diff --git a/dbms/src/Interpreters/InterpreterWatchQuery.cpp b/dbms/src/Interpreters/InterpreterWatchQuery.cpp index 3ba8e2eadaa..da09022e252 100644 --- a/dbms/src/Interpreters/InterpreterWatchQuery.cpp +++ b/dbms/src/Interpreters/InterpreterWatchQuery.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes extern const int UNKNOWN_STORAGE; extern const int UNKNOWN_TABLE; extern const int TOO_MANY_COLUMNS; + extern const int SUPPORT_IS_DISABLED; } BlockInputStreamPtr InterpreterWatchQuery::executeImpl() @@ -34,6 +35,9 @@ BlockInputStreamPtr InterpreterWatchQuery::executeImpl() BlockIO InterpreterWatchQuery::execute() { + if (!context.getSettingsRef().allow_experimental_live_view) + throw Exception("Experimental LIVE VIEW feature is not enabled (the setting 'allow_experimental_live_view')", ErrorCodes::SUPPORT_IS_DISABLED); + BlockIO res; const ASTWatchQuery & query = typeid_cast(*query_ptr); String database; diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 98d48392e09..3c0d205fa3f 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -19,7 +19,6 @@ limitations under the License. */ #include #include #include -#include #include #include #include @@ -50,6 +49,7 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; extern const int TABLE_WAS_NOT_DROPPED; extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW; + extern const int SUPPORT_IS_DISABLED; } static void extractDependentTable(ASTSelectQuery & query, String & select_database_name, String & select_table_name) @@ -586,6 +586,9 @@ void registerStorageLiveView(StorageFactory & factory) { factory.registerStorage("LiveView", [](const StorageFactory::Arguments & args) { + if (!args.local_context.getSettingsRef().allow_experimental_live_view) + throw Exception("Experimental LIVE VIEW feature is not enabled (the setting 'allow_experimental_live_view')", ErrorCodes::SUPPORT_IS_DISABLED); + return StorageLiveView::create(args.table_name, args.database_name, args.local_context, args.query, args.columns); }); } From 96869d405fc132a7443cfbc13d1e583079503db8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 02:48:52 +0300 Subject: [PATCH 368/509] Temporarily disable all LIVE VIEW tests because this feature has subtle bugs that manifestate under TSan --- .../00960_live_view_watch_events_live.py | 42 ---- ...0960_live_view_watch_events_live.reference | 0 .../00961_temporary_live_view_watch.reference | 3 - .../00961_temporary_live_view_watch.sql | 18 -- .../00962_temporary_live_view_watch_live.py | 42 ---- ...2_temporary_live_view_watch_live.reference | 0 ...y_live_view_watch_live_timeout.py.disabled | 49 ----- ...ary_live_view_watch_live_timeout.reference | 0 .../00964_live_view_watch_events_heartbeat.py | 44 ---- ...live_view_watch_events_heartbeat.reference | 0 .../00965_live_view_watch_heartbeat.py | 45 ---- .../00965_live_view_watch_heartbeat.reference | 0 .../00966_live_view_watch_events_http.py | 37 ---- ...0966_live_view_watch_events_http.reference | 0 .../0_stateless/00967_live_view_watch_http.py | 37 ---- .../00967_live_view_watch_http.reference | 0 ...t_format_jsoneachrowwithprogress.reference | 4 - ..._select_format_jsoneachrowwithprogress.sql | 12 - ...h_format_jsoneachrowwithprogress.reference | 6 - ...w_watch_format_jsoneachrowwithprogress.sql | 18 -- ...0_live_view_watch_events_http_heartbeat.py | 43 ---- ...view_watch_events_http_heartbeat.reference | 0 .../00971_live_view_watch_http_heartbeat.py | 43 ---- ...1_live_view_watch_http_heartbeat.reference | 0 .../00972_live_view_select_1.reference | 1 - .../0_stateless/00972_live_view_select_1.sql | 7 - .../00973_live_view_select.reference | 4 - .../0_stateless/00973_live_view_select.sql | 18 -- ...ive_view_select_with_aggregation.reference | 2 - ...0974_live_view_select_with_aggregation.sql | 16 -- .../00975_live_view_create.reference | 0 .../0_stateless/00975_live_view_create.sql | 7 - .../00976_live_view_select_version.reference | 3 - .../00976_live_view_select_version.sql | 12 - .../00977_live_view_watch_events.reference | 3 - .../00977_live_view_watch_events.sql | 18 -- .../00978_live_view_watch.reference | 3 - .../0_stateless/00978_live_view_watch.sql | 18 -- .../0_stateless/00979_live_view_watch_live.py | 48 ---- .../00979_live_view_watch_live.reference | 0 ...00980_create_temporary_live_view.reference | 3 - .../00980_create_temporary_live_view.sql | 15 -- .../00991_live_view_watch_event_live.python | 81 ------- ...00991_live_view_watch_event_live.reference | 7 - ...991_live_view_watch_event_live.sh.disabled | 6 - .../00991_live_view_watch_http.python | 63 ------ .../00991_live_view_watch_http.reference | 4 - .../00991_live_view_watch_http.sh.disabled | 6 - ...ry_live_view_watch_events_heartbeat.python | 83 ------- ...live_view_watch_events_heartbeat.reference | 0 ...ve_view_watch_events_heartbeat.sh.disabled | 6 - ...0991_temporary_live_view_watch_live.python | 81 ------- ...1_temporary_live_view_watch_live.reference | 7 - ...temporary_live_view_watch_live.sh.disabled | 6 - .../queries/0_stateless/helpers/client.py | 36 --- .../queries/0_stateless/helpers/httpclient.py | 14 -- .../queries/0_stateless/helpers/httpexpect.py | 73 ------- .../queries/0_stateless/helpers/uexpect.py | 206 ------------------ 58 files changed, 1300 deletions(-) delete mode 100755 dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py delete mode 100644 dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.reference delete mode 100644 dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference delete mode 100644 dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql delete mode 100755 dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py delete mode 100644 dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.reference delete mode 100755 dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled delete mode 100644 dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.reference delete mode 100755 dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py delete mode 100644 dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.reference delete mode 100755 dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py delete mode 100644 dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.reference delete mode 100755 dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py delete mode 100644 dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.reference delete mode 100755 dbms/tests/queries/0_stateless/00967_live_view_watch_http.py delete mode 100644 dbms/tests/queries/0_stateless/00967_live_view_watch_http.reference delete mode 100644 dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference delete mode 100644 dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql delete mode 100644 dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference delete mode 100644 dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql delete mode 100755 dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py delete mode 100644 dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.reference delete mode 100755 dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py delete mode 100644 dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.reference delete mode 100644 dbms/tests/queries/0_stateless/00972_live_view_select_1.reference delete mode 100644 dbms/tests/queries/0_stateless/00972_live_view_select_1.sql delete mode 100644 dbms/tests/queries/0_stateless/00973_live_view_select.reference delete mode 100644 dbms/tests/queries/0_stateless/00973_live_view_select.sql delete mode 100644 dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.reference delete mode 100644 dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql delete mode 100644 dbms/tests/queries/0_stateless/00975_live_view_create.reference delete mode 100644 dbms/tests/queries/0_stateless/00975_live_view_create.sql delete mode 100644 dbms/tests/queries/0_stateless/00976_live_view_select_version.reference delete mode 100644 dbms/tests/queries/0_stateless/00976_live_view_select_version.sql delete mode 100644 dbms/tests/queries/0_stateless/00977_live_view_watch_events.reference delete mode 100644 dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql delete mode 100644 dbms/tests/queries/0_stateless/00978_live_view_watch.reference delete mode 100644 dbms/tests/queries/0_stateless/00978_live_view_watch.sql delete mode 100755 dbms/tests/queries/0_stateless/00979_live_view_watch_live.py delete mode 100644 dbms/tests/queries/0_stateless/00979_live_view_watch_live.reference delete mode 100644 dbms/tests/queries/0_stateless/00980_create_temporary_live_view.reference delete mode 100644 dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql delete mode 100644 dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python delete mode 100644 dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference delete mode 100755 dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled delete mode 100755 dbms/tests/queries/0_stateless/00991_live_view_watch_http.python delete mode 100644 dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference delete mode 100755 dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled delete mode 100644 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python delete mode 100644 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.reference delete mode 100755 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled delete mode 100644 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python delete mode 100644 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference delete mode 100755 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled delete mode 100644 dbms/tests/queries/0_stateless/helpers/client.py delete mode 100644 dbms/tests/queries/0_stateless/helpers/httpclient.py delete mode 100644 dbms/tests/queries/0_stateless/helpers/httpexpect.py delete mode 100644 dbms/tests/queries/0_stateless/helpers/uexpect.py diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py deleted file mode 100755 index b7fc3f4e3a6..00000000000 --- a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py +++ /dev/null @@ -1,42 +0,0 @@ -#!/usr/bin/env python -import os -import sys -import signal - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block - -log = None -# uncomment the line below for debugging -#log=sys.stdout - -with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: - client1.expect(prompt) - client2.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.lv') - client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') - client1.expect(prompt) - client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') - client1.expect(prompt) - client1.send('WATCH test.lv EVENTS') - client1.expect('1.*' + end_of_block) - client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect('2.*' + end_of_block) - client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') - client1.expect('3.*' + end_of_block) - # send Ctrl-C - client1.send('\x03', eol='') - match = client1.expect('(%s)|([#\$] )' % prompt) - if match.groups()[1]: - client1.send(client1.command) - client1.expect(prompt) - client1.send('DROP TABLE test.lv') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.reference b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference deleted file mode 100644 index 6fbbedf1b21..00000000000 --- a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference +++ /dev/null @@ -1,3 +0,0 @@ -0 1 -6 2 -21 3 diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql deleted file mode 100644 index c3e2ab8d102..00000000000 --- a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql +++ /dev/null @@ -1,18 +0,0 @@ -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; - -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; - -WATCH test.lv LIMIT 0; - -INSERT INTO test.mt VALUES (1),(2),(3); - -WATCH test.lv LIMIT 0; - -INSERT INTO test.mt VALUES (4),(5),(6); - -WATCH test.lv LIMIT 0; - -DROP TABLE test.lv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py deleted file mode 100755 index f27b1213c70..00000000000 --- a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py +++ /dev/null @@ -1,42 +0,0 @@ -#!/usr/bin/env python -import os -import sys -import signal - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block - -log = None -# uncomment the line below for debugging -#log=sys.stdout - -with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: - client1.expect(prompt) - client2.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.lv') - client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') - client1.expect(prompt) - client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') - client1.expect(prompt) - client1.send('WATCH test.lv') - client1.expect(r'0.*1' + end_of_block) - client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect(r'6.*2' + end_of_block) - client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') - client1.expect(r'21.*3' + end_of_block) - # send Ctrl-C - client1.send('\x03', eol='') - match = client1.expect('(%s)|([#\$] )' % prompt) - if match.groups()[1]: - client1.send(client1.command) - client1.expect(prompt) - client1.send('DROP TABLE test.lv') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled deleted file mode 100755 index df627c84e49..00000000000 --- a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled +++ /dev/null @@ -1,49 +0,0 @@ -#!/usr/bin/env python -import os -import sys -import signal - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block - -log = None -# uncomment the line below for debugging -#log=sys.stdout - -with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: - client1.expect(prompt) - client2.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.lv') - client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('SET temporary_live_view_timeout=1') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') - client1.expect(prompt) - client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') - client1.expect(prompt) - client1.send('WATCH test.lv') - client1.expect(r'0.*1' + end_of_block) - client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client2.expect(prompt) - client1.expect(r'6.*2' + end_of_block) - client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') - client2.expect(prompt) - client1.expect(r'21.*3' + end_of_block) - # send Ctrl-C - client1.send('\x03', eol='') - match = client1.expect('(%s)|([#\$] )' % prompt) - if match.groups()[1]: - client1.send(client1.command) - client1.expect(prompt) - client1.send('SELECT sleep(1)') - client1.expect(prompt) - client1.send('DROP TABLE test.lv') - client1.expect('Table test.lv doesn\'t exist') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.reference b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py deleted file mode 100755 index 5664c0e6c6d..00000000000 --- a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py +++ /dev/null @@ -1,44 +0,0 @@ -#!/usr/bin/env python -import os -import sys -import signal - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block - -log = None -# uncomment the line below for debugging -#log=sys.stdout - -with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: - client1.expect(prompt) - client2.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.lv') - client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('SET live_view_heartbeat_interval=1') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') - client1.expect(prompt) - client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') - client1.expect(prompt) - client1.send('WATCH test.lv EVENTS') - client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect('2.*' + end_of_block) - client1.expect('Progress: 2.00 rows.*\)') - # wait for heartbeat - client1.expect('Progress: 2.00 rows.*\)') - # send Ctrl-C - client1.send('\x03', eol='') - match = client1.expect('(%s)|([#\$] )' % prompt) - if match.groups()[1]: - client1.send(client1.command) - client1.expect(prompt) - client1.send('DROP TABLE test.lv') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.reference b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py deleted file mode 100755 index 03e22175dff..00000000000 --- a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py +++ /dev/null @@ -1,45 +0,0 @@ -#!/usr/bin/env python -import os -import sys -import signal - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block - -log = None -# uncomment the line below for debugging -#log=sys.stdout - -with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: - client1.expect(prompt) - client2.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.lv') - client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('SET live_view_heartbeat_interval=1') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') - client1.expect(prompt) - client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') - client1.expect(prompt) - client1.send('WATCH test.lv') - client1.expect(r'0.*1' + end_of_block) - client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect(r'6.*2' + end_of_block) - client1.expect('Progress: 2.00 rows.*\)') - # wait for heartbeat - client1.expect('Progress: 2.00 rows.*\)') - # send Ctrl-C - client1.send('\x03', eol='') - match = client1.expect('(%s)|([#\$] )' % prompt) - if match.groups()[1]: - client1.send(client1.command) - client1.expect(prompt) - client1.send('DROP TABLE test.lv') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.reference b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py deleted file mode 100755 index bb9d6152200..00000000000 --- a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py +++ /dev/null @@ -1,37 +0,0 @@ -#!/usr/bin/env python -import os -import sys - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block -from httpclient import client as http_client - -log = None -# uncomment the line below for debugging -#log=sys.stdout - -with client(name='client1>', log=log) as client1: - client1.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.lv') - client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') - client1.expect(prompt) - client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') - client1.expect(prompt) - - - with http_client({'method':'GET', 'url': '/?query=WATCH%20test.lv%20EVENTS'}, name='client2>', log=log) as client2: - client2.expect('.*1\n') - client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect(prompt) - client2.expect('.*2\n') - - client1.send('DROP TABLE test.lv') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.reference b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py deleted file mode 100755 index d3439431eb3..00000000000 --- a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py +++ /dev/null @@ -1,37 +0,0 @@ -#!/usr/bin/env python -import os -import sys - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block -from httpclient import client as http_client - -log = None -# uncomment the line below for debugging -#log=sys.stdout - -with client(name='client1>', log=log) as client1: - client1.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.lv') - client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') - client1.expect(prompt) - client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') - client1.expect(prompt) - - - with http_client({'method':'GET', 'url':'/?query=WATCH%20test.lv'}, name='client2>', log=log) as client2: - client2.expect('.*0\t1\n') - client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect(prompt) - client2.expect('.*6\t2\n') - - client1.send('DROP TABLE test.lv') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.reference b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference deleted file mode 100644 index 5ae423d90d1..00000000000 --- a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference +++ /dev/null @@ -1,4 +0,0 @@ -{"row":{"a":1}} -{"row":{"a":2}} -{"row":{"a":3}} -{"progress":{"read_rows":"3","read_bytes":"36","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} diff --git a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql deleted file mode 100644 index 8c6f4197d54..00000000000 --- a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql +++ /dev/null @@ -1,12 +0,0 @@ -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; - -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; - -INSERT INTO test.mt VALUES (1),(2),(3); - -SELECT * FROM test.lv FORMAT JSONEachRowWithProgress; - -DROP TABLE test.lv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference deleted file mode 100644 index 287a1ced92d..00000000000 --- a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference +++ /dev/null @@ -1,6 +0,0 @@ -{"row":{"sum(a)":"0","_version":"1"}} -{"progress":{"read_rows":"1","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} -{"row":{"sum(a)":"6","_version":"2"}} -{"progress":{"read_rows":"1","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} -{"row":{"sum(a)":"21","_version":"3"}} -{"progress":{"read_rows":"1","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} diff --git a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql deleted file mode 100644 index 725a4ad00ed..00000000000 --- a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql +++ /dev/null @@ -1,18 +0,0 @@ -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; - -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; - -WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; - -INSERT INTO test.mt VALUES (1),(2),(3); - -WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; - -INSERT INTO test.mt VALUES (4),(5),(6); - -WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; - -DROP TABLE test.lv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py deleted file mode 100755 index 63628c4a76f..00000000000 --- a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py +++ /dev/null @@ -1,43 +0,0 @@ -#!/usr/bin/env python -import os -import sys - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block -from httpclient import client as http_client - -log = None -# uncomment the line below for debugging -#log=sys.stdout - -with client(name='client1>', log=log) as client1: - client1.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.lv') - client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') - client1.expect(prompt) - client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') - client1.expect(prompt) - - - with http_client({'method':'GET', 'url': '/?live_view_heartbeat_interval=1&query=WATCH%20test.lv%20EVENTS%20FORMAT%20JSONEachRowWithProgress'}, name='client2>', log=log) as client2: - client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}\n', escape=True) - client2.expect('{"row":{"version":"1"}', escape=True) - client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) - # heartbeat is provided by progress message - client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) - - client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect(prompt) - - client2.expect('{"row":{"version":"2"}}\n', escape=True) - - client1.send('DROP TABLE test.lv') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.reference b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py deleted file mode 100755 index 7bdb47b7caa..00000000000 --- a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py +++ /dev/null @@ -1,43 +0,0 @@ -#!/usr/bin/env python -import os -import sys - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block -from httpclient import client as http_client - -log = None -# uncomment the line below for debugging -#log=sys.stdout - -with client(name='client1>', log=log) as client1: - client1.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.lv') - client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') - client1.expect(prompt) - client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') - client1.expect(prompt) - - with http_client({'method':'GET', 'url':'/?live_view_heartbeat_interval=1&query=WATCH%20test.lv%20FORMAT%20JSONEachRowWithProgress'}, name='client2>', log=log) as client2: - client2.expect('"progress".*',) - client2.expect('{"row":{"sum(a)":"0","_version":"1"}}\n', escape=True) - client2.expect('"progress".*\n') - # heartbeat is provided by progress message - client2.expect('"progress".*\n') - - client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect(prompt) - - client2.expect('"progress".*"read_rows":"2".*\n') - client2.expect('{"row":{"sum(a)":"6","_version":"2"}}\n', escape=True) - - client1.send('DROP TABLE test.lv') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.reference b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference b/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql b/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql deleted file mode 100644 index 661080b577b..00000000000 --- a/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql +++ /dev/null @@ -1,7 +0,0 @@ -DROP TABLE IF EXISTS test.lv; - -CREATE LIVE VIEW test.lv AS SELECT 1; - -SELECT * FROM test.lv; - -DROP TABLE test.lv; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_select.reference b/dbms/tests/queries/0_stateless/00973_live_view_select.reference deleted file mode 100644 index 75236c0daf7..00000000000 --- a/dbms/tests/queries/0_stateless/00973_live_view_select.reference +++ /dev/null @@ -1,4 +0,0 @@ -6 1 -6 1 -12 2 -12 2 diff --git a/dbms/tests/queries/0_stateless/00973_live_view_select.sql b/dbms/tests/queries/0_stateless/00973_live_view_select.sql deleted file mode 100644 index ff4a45ffcc1..00000000000 --- a/dbms/tests/queries/0_stateless/00973_live_view_select.sql +++ /dev/null @@ -1,18 +0,0 @@ -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; - -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; - -INSERT INTO test.mt VALUES (1),(2),(3); - -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; - -INSERT INTO test.mt VALUES (1),(2),(3); - -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; - -DROP TABLE test.lv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.reference b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.reference deleted file mode 100644 index 6d50f0e9c3a..00000000000 --- a/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.reference +++ /dev/null @@ -1,2 +0,0 @@ -6 -21 diff --git a/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql deleted file mode 100644 index 3c11f855c9d..00000000000 --- a/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql +++ /dev/null @@ -1,16 +0,0 @@ -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; - -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; - -INSERT INTO test.mt VALUES (1),(2),(3); - -SELECT sum(a) FROM test.lv; - -INSERT INTO test.mt VALUES (4),(5),(6); - -SELECT sum(a) FROM test.lv; - -DROP TABLE test.lv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00975_live_view_create.reference b/dbms/tests/queries/0_stateless/00975_live_view_create.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/00975_live_view_create.sql b/dbms/tests/queries/0_stateless/00975_live_view_create.sql deleted file mode 100644 index 1c929b15b00..00000000000 --- a/dbms/tests/queries/0_stateless/00975_live_view_create.sql +++ /dev/null @@ -1,7 +0,0 @@ -DROP TABLE IF EXISTS test.mt; - -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; - -DROP TABLE test.lv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00976_live_view_select_version.reference b/dbms/tests/queries/0_stateless/00976_live_view_select_version.reference deleted file mode 100644 index 453bd800469..00000000000 --- a/dbms/tests/queries/0_stateless/00976_live_view_select_version.reference +++ /dev/null @@ -1,3 +0,0 @@ -1 1 -2 1 -3 1 diff --git a/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql b/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql deleted file mode 100644 index 5f3ab1f7546..00000000000 --- a/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql +++ /dev/null @@ -1,12 +0,0 @@ -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; - -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; - -INSERT INTO test.mt VALUES (1),(2),(3); - -SELECT *,_version FROM test.lv; - -DROP TABLE test.lv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00977_live_view_watch_events.reference b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.reference deleted file mode 100644 index 01e79c32a8c..00000000000 --- a/dbms/tests/queries/0_stateless/00977_live_view_watch_events.reference +++ /dev/null @@ -1,3 +0,0 @@ -1 -2 -3 diff --git a/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql deleted file mode 100644 index a3b84e8d4c1..00000000000 --- a/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql +++ /dev/null @@ -1,18 +0,0 @@ -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; - -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; - -WATCH test.lv EVENTS LIMIT 0; - -INSERT INTO test.mt VALUES (1),(2),(3); - -WATCH test.lv EVENTS LIMIT 0; - -INSERT INTO test.mt VALUES (4),(5),(6); - -WATCH test.lv EVENTS LIMIT 0; - -DROP TABLE test.lv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00978_live_view_watch.reference b/dbms/tests/queries/0_stateless/00978_live_view_watch.reference deleted file mode 100644 index 6fbbedf1b21..00000000000 --- a/dbms/tests/queries/0_stateless/00978_live_view_watch.reference +++ /dev/null @@ -1,3 +0,0 @@ -0 1 -6 2 -21 3 diff --git a/dbms/tests/queries/0_stateless/00978_live_view_watch.sql b/dbms/tests/queries/0_stateless/00978_live_view_watch.sql deleted file mode 100644 index abe4a6c32ae..00000000000 --- a/dbms/tests/queries/0_stateless/00978_live_view_watch.sql +++ /dev/null @@ -1,18 +0,0 @@ -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; - -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; - -WATCH test.lv LIMIT 0; - -INSERT INTO test.mt VALUES (1),(2),(3); - -WATCH test.lv LIMIT 0; - -INSERT INTO test.mt VALUES (4),(5),(6); - -WATCH test.lv LIMIT 0; - -DROP TABLE test.lv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py deleted file mode 100755 index 948e4c93662..00000000000 --- a/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py +++ /dev/null @@ -1,48 +0,0 @@ -#!/usr/bin/env python -import os -import sys -import signal - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block - -log = None -# uncomment the line below for debugging -#log=sys.stdout - -with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: - client1.expect(prompt) - client2.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.lv') - client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') - client1.expect(prompt) - client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') - client1.expect(prompt) - client1.send('WATCH test.lv') - client1.expect(r'0.*1' + end_of_block) - client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') - client1.expect(r'6.*2' + end_of_block) - client2.expect(prompt) - client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') - client1.expect(r'21.*3' + end_of_block) - client2.expect(prompt) - for i in range(1,129): - client2.send('INSERT INTO test.mt VALUES (1)') - client1.expect(r'%d.*%d' % (21+i, 3+i) + end_of_block) - client2.expect(prompt) - # send Ctrl-C - client1.send('\x03', eol='') - match = client1.expect('(%s)|([#\$] )' % prompt) - if match.groups()[1]: - client1.send(client1.command) - client1.expect(prompt) - client1.send('DROP TABLE test.lv') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00979_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.reference b/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.reference deleted file mode 100644 index 7f9fcbb2e9c..00000000000 --- a/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.reference +++ /dev/null @@ -1,3 +0,0 @@ -temporary_live_view_timeout 5 -live_view_heartbeat_interval 15 -0 diff --git a/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql b/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql deleted file mode 100644 index 8cd6ee06ace..00000000000 --- a/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql +++ /dev/null @@ -1,15 +0,0 @@ -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; - -SELECT name, value from system.settings WHERE name = 'temporary_live_view_timeout'; -SELECT name, value from system.settings WHERE name = 'live_view_heartbeat_interval'; - -SET temporary_live_view_timeout=1; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; - -SHOW TABLES LIKE 'lv'; -SELECT sleep(2); -SHOW TABLES LIKE 'lv'; - -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python deleted file mode 100644 index 782671cdfaf..00000000000 --- a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python +++ /dev/null @@ -1,81 +0,0 @@ -#!/usr/bin/env python - -import subprocess -import threading -import Queue as queue -import os -import sys -import signal - - -CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') -CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') -CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') - - -def send_query(query): - cmd = list(CLICKHOUSE_CLIENT.split()) - cmd += ['--query', query] - # print(cmd) - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout - - -def send_query_in_process_group(query): - cmd = list(CLICKHOUSE_CLIENT.split()) - cmd += ['--query', query] - # print(cmd) - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) - - -def read_lines_and_push_to_queue(pipe, queue): - try: - for line in iter(pipe.readline, ''): - line = line.strip() - print(line) - sys.stdout.flush() - queue.put(line) - except KeyboardInterrupt: - pass - - queue.put(None) - - -def test(): - send_query('DROP TABLE IF EXISTS test.lv').read() - send_query('DROP TABLE IF EXISTS test.mt').read() - send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() - send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() - - q = queue.Queue() - p = send_query_in_process_group('WATCH test.lv') - thread = threading.Thread(target=read_lines_and_push_to_queue, args=(p.stdout, q)) - thread.start() - - line = q.get() - print(line) - assert (line == '0\t1') - - send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() - line = q.get() - print(line) - assert (line == '6\t2') - - send_query('INSERT INTO test.mt VALUES (4),(5),(6)').read() - line = q.get() - print(line) - assert (line == '21\t3') - - # Send Ctrl+C to client. - os.killpg(os.getpgid(p.pid), signal.SIGINT) - # This insert shouldn't affect lv. - send_query('INSERT INTO test.mt VALUES (7),(8),(9)').read() - line = q.get() - print(line) - assert (line is None) - - send_query('DROP TABLE if exists test.lv').read() - send_query('DROP TABLE if exists test.lv').read() - - thread.join() - -test() diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference deleted file mode 100644 index 1e94cdade41..00000000000 --- a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference +++ /dev/null @@ -1,7 +0,0 @@ -0 1 -0 1 -6 2 -6 2 -21 3 -21 3 -None diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled deleted file mode 100755 index 10e4e98b2e3..00000000000 --- a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled +++ /dev/null @@ -1,6 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. $CURDIR/../shell_config.sh - -python $CURDIR/00991_live_view_watch_event_live.python diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python deleted file mode 100755 index 938547ca0cb..00000000000 --- a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python +++ /dev/null @@ -1,63 +0,0 @@ -#!/usr/bin/env python - -import subprocess -import threading -import Queue as queue -import os -import sys - - -CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') -CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') -CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') - - -def send_query(query): - cmd = list(CLICKHOUSE_CLIENT.split()) - cmd += ['--query', query] - # print(cmd) - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout - - -def send_http_query(query): - cmd = list(CLICKHOUSE_CURL.split()) # list(['curl', '-sSN', '--max-time', '10']) - cmd += ['-sSN', CLICKHOUSE_URL, '-d', query] - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout - - -def read_lines_and_push_to_queue(pipe, queue): - for line in iter(pipe.readline, ''): - line = line.strip() - print(line) - sys.stdout.flush() - queue.put(line) - - queue.put(None) - - -def test(): - send_query('DROP TABLE IF EXISTS test.lv').read() - send_query('DROP TABLE IF EXISTS test.mt').read() - send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() - send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() - - q = queue.Queue() - pipe = send_http_query('WATCH test.lv') - thread = threading.Thread(target=read_lines_and_push_to_queue, args=(pipe, q)) - thread.start() - - line = q.get() - print(line) - assert (line == '0\t1') - - send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() - line = q.get() - print(line) - assert (line == '6\t2') - - send_query('DROP TABLE if exists test.lv').read() - send_query('DROP TABLE if exists test.lv').read() - - thread.join() - -test() diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference deleted file mode 100644 index 489457d751b..00000000000 --- a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference +++ /dev/null @@ -1,4 +0,0 @@ -0 1 -0 1 -6 2 -6 2 diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled deleted file mode 100755 index 88cce77f595..00000000000 --- a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled +++ /dev/null @@ -1,6 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. $CURDIR/../shell_config.sh - -python $CURDIR/00991_live_view_watch_http.python diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python deleted file mode 100644 index 70063adc6e3..00000000000 --- a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python +++ /dev/null @@ -1,83 +0,0 @@ -#!/usr/bin/env python - -import subprocess -import threading -import Queue as queue -import os -import sys -import signal - - -CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') -CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') -CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') - - -def send_query(query): - cmd = list(CLICKHOUSE_CLIENT.split()) - cmd += ['--query', query] - # print(cmd) - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout - - -def send_query_in_process_group(query): - cmd = list(CLICKHOUSE_CLIENT.split()) - cmd += ['--query', query, '--live_view_heartbeat_interval=1', '--progress'] - # print(cmd) - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) - - -def read_lines_and_push_to_queue(pipe, queue): - try: - for line in iter(pipe.readline, ''): - line = line.strip() - # print(line) - sys.stdout.flush() - queue.put(line) - except KeyboardInterrupt: - pass - - queue.put(None) - - -def test(): - send_query('DROP TABLE IF EXISTS test.lv').read() - send_query('DROP TABLE IF EXISTS test.mt').read() - send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() - send_query('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() - - q = queue.Queue() - p = send_query_in_process_group('WATCH test.lv') - thread = threading.Thread(target=read_lines_and_push_to_queue, args=(p.stdout, q)) - thread.start() - - line = q.get() - # print(line) - assert (line.endswith('0\t1')) - assert ('Progress: 0.00 rows' in line) - - send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() - line = q.get() - assert (line.endswith('6\t2')) - assert ('Progress: 1.00 rows' in line) - - # send_query('INSERT INTO test.mt VALUES (4),(5),(6)').read() - # line = q.get() - # print(line) - # assert (line.endswith('6\t2')) - # assert ('Progress: 1.00 rows' in line) - - # Send Ctrl+C to client. - os.killpg(os.getpgid(p.pid), signal.SIGINT) - # This insert shouldn't affect lv. - send_query('INSERT INTO test.mt VALUES (7),(8),(9)').read() - line = q.get() - # print(line) - # assert (line is None) - - send_query('DROP TABLE if exists test.lv').read() - send_query('DROP TABLE if exists test.lv').read() - - thread.join() - -test() diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.reference b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled deleted file mode 100755 index f7aa13d52b3..00000000000 --- a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled +++ /dev/null @@ -1,6 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. $CURDIR/../shell_config.sh - -python $CURDIR/00991_temporary_live_view_watch_events_heartbeat.python diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python deleted file mode 100644 index d290018a02c..00000000000 --- a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python +++ /dev/null @@ -1,81 +0,0 @@ -#!/usr/bin/env python - -import subprocess -import threading -import Queue as queue -import os -import sys -import signal - - -CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') -CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') -CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') - - -def send_query(query): - cmd = list(CLICKHOUSE_CLIENT.split()) - cmd += ['--query', query] - # print(cmd) - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout - - -def send_query_in_process_group(query): - cmd = list(CLICKHOUSE_CLIENT.split()) - cmd += ['--query', query] - # print(cmd) - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) - - -def read_lines_and_push_to_queue(pipe, queue): - try: - for line in iter(pipe.readline, ''): - line = line.strip() - print(line) - sys.stdout.flush() - queue.put(line) - except KeyboardInterrupt: - pass - - queue.put(None) - - -def test(): - send_query('DROP TABLE IF EXISTS test.lv').read() - send_query('DROP TABLE IF EXISTS test.mt').read() - send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() - send_query('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() - - q = queue.Queue() - p = send_query_in_process_group('WATCH test.lv') - thread = threading.Thread(target=read_lines_and_push_to_queue, args=(p.stdout, q)) - thread.start() - - line = q.get() - print(line) - assert (line == '0\t1') - - send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() - line = q.get() - print(line) - assert (line == '6\t2') - - send_query('INSERT INTO test.mt VALUES (4),(5),(6)').read() - line = q.get() - print(line) - assert (line == '21\t3') - - # Send Ctrl+C to client. - os.killpg(os.getpgid(p.pid), signal.SIGINT) - # This insert shouldn't affect lv. - send_query('INSERT INTO test.mt VALUES (7),(8),(9)').read() - line = q.get() - print(line) - assert (line is None) - - send_query('DROP TABLE if exists test.lv').read() - send_query('DROP TABLE if exists test.lv').read() - - thread.join() - -test() diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference deleted file mode 100644 index 1e94cdade41..00000000000 --- a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference +++ /dev/null @@ -1,7 +0,0 @@ -0 1 -0 1 -6 2 -6 2 -21 3 -21 3 -None diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled deleted file mode 100755 index 4d01d1c3a8e..00000000000 --- a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled +++ /dev/null @@ -1,6 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. $CURDIR/../shell_config.sh - -python $CURDIR/00991_temporary_live_view_watch_live.python diff --git a/dbms/tests/queries/0_stateless/helpers/client.py b/dbms/tests/queries/0_stateless/helpers/client.py deleted file mode 100644 index f3938d3bf63..00000000000 --- a/dbms/tests/queries/0_stateless/helpers/client.py +++ /dev/null @@ -1,36 +0,0 @@ -import os -import sys -import time - -CURDIR = os.path.dirname(os.path.realpath(__file__)) - -sys.path.insert(0, os.path.join(CURDIR)) - -import uexpect - -prompt = ':\) ' -end_of_block = r'.*\r\n.*\r\n' - -class client(object): - def __init__(self, command=None, name='', log=None): - self.client = uexpect.spawn(['/bin/bash','--noediting']) - if command is None: - command = os.environ.get('CLICKHOUSE_BINARY', 'clickhouse') + '-client' - self.client.command = command - self.client.eol('\r') - self.client.logger(log, prefix=name) - self.client.timeout(20) - self.client.expect('[#\$] ', timeout=2) - self.client.send(command) - - def __enter__(self): - return self.client.__enter__() - - def __exit__(self, type, value, traceback): - self.client.reader['kill_event'].set() - # send Ctrl-C - self.client.send('\x03', eol='') - time.sleep(0.3) - self.client.send('quit', eol='\r') - self.client.send('\x03', eol='') - return self.client.__exit__(type, value, traceback) diff --git a/dbms/tests/queries/0_stateless/helpers/httpclient.py b/dbms/tests/queries/0_stateless/helpers/httpclient.py deleted file mode 100644 index a42fad2cbc3..00000000000 --- a/dbms/tests/queries/0_stateless/helpers/httpclient.py +++ /dev/null @@ -1,14 +0,0 @@ -import os -import sys - -CURDIR = os.path.dirname(os.path.realpath(__file__)) - -sys.path.insert(0, os.path.join(CURDIR)) - -import httpexpect - -def client(request, name='', log=None): - client = httpexpect.spawn({'host':'localhost','port':8123}, request) - client.logger(log, prefix=name) - client.timeout(20) - return client diff --git a/dbms/tests/queries/0_stateless/helpers/httpexpect.py b/dbms/tests/queries/0_stateless/helpers/httpexpect.py deleted file mode 100644 index e440dafce4e..00000000000 --- a/dbms/tests/queries/0_stateless/helpers/httpexpect.py +++ /dev/null @@ -1,73 +0,0 @@ -# Copyright (c) 2019 Vitaliy Zakaznikov -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -import os -import sys -import httplib - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, CURDIR) - -import uexpect - -from threading import Thread, Event -from Queue import Queue, Empty - -class IO(uexpect.IO): - def __init__(self, connection, response, queue, reader): - self.connection = connection - self.response = response - super(IO, self).__init__(None, None, queue, reader) - - def write(self, data): - raise NotImplementedError - - def close(self, force=True): - self.reader['kill_event'].set() - self.connection.close() - if self._logger: - self._logger.write('\n') - self._logger.flush() - - -def reader(response, queue, kill_event): - while True: - try: - if kill_event.is_set(): - break - data = response.read(1) - queue.put(data) - except Exception, e: - if kill_event.is_set(): - break - raise - -def spawn(connection, request): - connection = httplib.HTTPConnection(**connection) - connection.request(**request) - response = connection.getresponse() - - queue = Queue() - reader_kill_event = Event() - thread = Thread(target=reader, args=(response, queue, reader_kill_event)) - thread.daemon = True - thread.start() - - return IO(connection, response, queue, reader={'thread':thread, 'kill_event':reader_kill_event}) - -if __name__ == '__main__': - with http({'host':'localhost','port':8123},{'method':'GET', 'url':'?query=SELECT%201'}) as client: - client.logger(sys.stdout) - client.timeout(2) - print client.response.status, client.response.reason - client.expect('1\n') diff --git a/dbms/tests/queries/0_stateless/helpers/uexpect.py b/dbms/tests/queries/0_stateless/helpers/uexpect.py deleted file mode 100644 index f71b32a53e1..00000000000 --- a/dbms/tests/queries/0_stateless/helpers/uexpect.py +++ /dev/null @@ -1,206 +0,0 @@ -# Copyright (c) 2019 Vitaliy Zakaznikov -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -import os -import pty -import time -import sys -import re - -from threading import Thread, Event -from subprocess import Popen -from Queue import Queue, Empty - -class TimeoutError(Exception): - def __init__(self, timeout): - self.timeout = timeout - - def __str__(self): - return 'Timeout %.3fs' % float(self.timeout) - -class ExpectTimeoutError(Exception): - def __init__(self, pattern, timeout, buffer): - self.pattern = pattern - self.timeout = timeout - self.buffer = buffer - - def __str__(self): - s = 'Timeout %.3fs ' % float(self.timeout) - if self.pattern: - s += 'for %s ' % repr(self.pattern.pattern) - if self.buffer: - s += 'buffer %s ' % repr(self.buffer[:]) - s += 'or \'%s\'' % ','.join(['%x' % ord(c) for c in self.buffer[:]]) - return s - -class IO(object): - class EOF(object): - pass - - class Timeout(object): - pass - - EOF = EOF - TIMEOUT = Timeout - - class Logger(object): - def __init__(self, logger, prefix=''): - self._logger = logger - self._prefix = prefix - - def write(self, data): - self._logger.write(('\n' + data).replace('\n','\n' + self._prefix)) - - def flush(self): - self._logger.flush() - - def __init__(self, process, master, queue, reader): - self.process = process - self.master = master - self.queue = queue - self.buffer = None - self.before = None - self.after = None - self.match = None - self.pattern = None - self.reader = reader - self._timeout = None - self._logger = None - self._eol = '' - - def __enter__(self): - return self - - def __exit__(self, type, value, traceback): - self.close() - - def logger(self, logger=None, prefix=''): - if logger: - self._logger = self.Logger(logger, prefix=prefix) - return self._logger - - def timeout(self, timeout=None): - if timeout: - self._timeout = timeout - return self._timeout - - def eol(self, eol=None): - if eol: - self._eol = eol - return self._eol - - def close(self, force=True): - self.reader['kill_event'].set() - os.system('pkill -TERM -P %d' % self.process.pid) - if force: - self.process.kill() - else: - self.process.terminate() - os.close(self.master) - if self._logger: - self._logger.write('\n') - self._logger.flush() - - def send(self, data, eol=None): - if eol is None: - eol = self._eol - return self.write(data + eol) - - def write(self, data): - return os.write(self.master, data) - - def expect(self, pattern, timeout=None, escape=False): - self.match = None - self.before = None - self.after = None - if escape: - pattern = re.escape(pattern) - pattern = re.compile(pattern) - if timeout is None: - timeout = self._timeout - timeleft = timeout - while True: - start_time = time.time() - if self.buffer is not None: - self.match = pattern.search(self.buffer, 0) - if self.match is not None: - self.after = self.buffer[self.match.start():self.match.end()] - self.before = self.buffer[:self.match.start()] - self.buffer = self.buffer[self.match.end():] - break - if timeleft < 0: - break - try: - data = self.read(timeout=timeleft, raise_exception=True) - except TimeoutError: - if self._logger: - self._logger.write((self.buffer or '') + '\n') - self._logger.flush() - exception = ExpectTimeoutError(pattern, timeout, self.buffer) - self.buffer = None - raise exception - timeleft -= (time.time() - start_time) - if data: - self.buffer = (self.buffer + data) if self.buffer else data - if self._logger: - self._logger.write((self.before or '') + (self.after or '')) - self._logger.flush() - if self.match is None: - exception = ExpectTimeoutError(pattern, timeout, self.buffer) - self.buffer = None - raise exception - return self.match - - def read(self, timeout=0, raise_exception=False): - data = '' - timeleft = timeout - try: - while timeleft >= 0 : - start_time = time.time() - data += self.queue.get(timeout=timeleft) - if data: - break - timeleft -= (time.time() - start_time) - except Empty: - if data: - return data - if raise_exception: - raise TimeoutError(timeout) - pass - if not data and raise_exception: - raise TimeoutError(timeout) - - return data - -def spawn(command): - master, slave = pty.openpty() - process = Popen(command, preexec_fn=os.setsid, stdout=slave, stdin=slave, stderr=slave, bufsize=1) - os.close(slave) - - queue = Queue() - reader_kill_event = Event() - thread = Thread(target=reader, args=(process, master, queue, reader_kill_event)) - thread.daemon = True - thread.start() - - return IO(process, master, queue, reader={'thread':thread, 'kill_event':reader_kill_event}) - -def reader(process, out, queue, kill_event): - while True: - try: - data = os.read(out, 65536) - queue.put(data) - except: - if kill_event.is_set(): - break - raise From cff8ec43f992ca2daff2141ea15cd964c7401168 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 04:31:04 +0300 Subject: [PATCH 369/509] Rename neighbour -> neighbor --- dbms/src/Functions/{neighbour.cpp => neighbor.cpp} | 14 +++++++------- .../Functions/registerFunctionsMiscellaneous.cpp | 4 ++-- .../en/query_language/functions/other_functions.md | 4 ++-- .../ru/query_language/functions/other_functions.md | 4 ++-- 4 files changed, 13 insertions(+), 13 deletions(-) rename dbms/src/Functions/{neighbour.cpp => neighbor.cpp} (96%) diff --git a/dbms/src/Functions/neighbour.cpp b/dbms/src/Functions/neighbor.cpp similarity index 96% rename from dbms/src/Functions/neighbour.cpp rename to dbms/src/Functions/neighbor.cpp index 02cd4df4996..6ac2c966016 100644 --- a/dbms/src/Functions/neighbour.cpp +++ b/dbms/src/Functions/neighbor.cpp @@ -21,17 +21,17 @@ namespace ErrorCodes // | c1 | // | 10 | // | 20 | -// SELECT c1, neighbour(c1, 1) as c2: +// SELECT c1, neighbor(c1, 1) as c2: // | c1 | c2 | // | 10 | 20 | // | 20 | 0 | -class FunctionNeighbour : public IFunction +class FunctionNeighbor : public IFunction { public: - static constexpr auto name = "neighbour"; - static FunctionPtr create(const Context & context) { return std::make_shared(context); } + static constexpr auto name = "neighbor"; + static FunctionPtr create(const Context & context) { return std::make_shared(context); } - FunctionNeighbour(const Context & context_) : context(context_) {} + FunctionNeighbor(const Context & context_) : context(context_) {} /// Get the name of the function. String getName() const override { return name; } @@ -255,9 +255,9 @@ private: const Context & context; }; -void registerFunctionNeighbour(FunctionFactory & factory) +void registerFunctionNeighbor(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp index e3d4714638e..1d95844ce9b 100644 --- a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp @@ -18,7 +18,7 @@ void registerFunctionBlockSize(FunctionFactory &); void registerFunctionBlockNumber(FunctionFactory &); void registerFunctionRowNumberInBlock(FunctionFactory &); void registerFunctionRowNumberInAllBlocks(FunctionFactory &); -void registerFunctionNeighbour(FunctionFactory &); +void registerFunctionNeighbor(FunctionFactory &); void registerFunctionSleep(FunctionFactory &); void registerFunctionSleepEachRow(FunctionFactory &); void registerFunctionMaterialize(FunctionFactory &); @@ -70,7 +70,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionBlockNumber(factory); registerFunctionRowNumberInBlock(factory); registerFunctionRowNumberInAllBlocks(factory); - registerFunctionNeighbour(factory); + registerFunctionNeighbor(factory); registerFunctionSleep(factory); registerFunctionSleepEachRow(factory); registerFunctionMaterialize(factory); diff --git a/docs/en/query_language/functions/other_functions.md b/docs/en/query_language/functions/other_functions.md index 36c40fc4b02..349397059af 100644 --- a/docs/en/query_language/functions/other_functions.md +++ b/docs/en/query_language/functions/other_functions.md @@ -314,7 +314,7 @@ Returns the ordinal number of the row in the data block. Different data blocks a Returns the ordinal number of the row in the data block. This function only considers the affected data blocks. -## neighbour(column, offset\[, default_value\]) +## neighbor(column, offset\[, default_value\]) Returns value for `column`, in `offset` distance from current row. This function is a partial implementation of [window functions](https://en.wikipedia.org/wiki/SQL_window_function) LEAD() and LAG(). @@ -330,7 +330,7 @@ WITH toDate('2018-01-01') AS start_date SELECT toStartOfMonth(start_date + (number * 32)) AS month, toInt32(month) % 100 AS money, - neighbour(money, -12) AS prev_year, + neighbor(money, -12) AS prev_year, round(prev_year / money, 2) AS year_over_year FROM numbers(16) ``` diff --git a/docs/ru/query_language/functions/other_functions.md b/docs/ru/query_language/functions/other_functions.md index 41cd7c8c63b..fdc46a0d4ee 100644 --- a/docs/ru/query_language/functions/other_functions.md +++ b/docs/ru/query_language/functions/other_functions.md @@ -291,7 +291,7 @@ SELECT ## rowNumberInAllBlocks() Возвращает порядковый номер строки в блоке данных. Функция учитывает только задействованные блоки данных. -## neighbour(column, offset\[, default_value\]) +## neighbor(column, offset\[, default_value\]) Функция позволяет получить доступ к значению в колонке `column`, находящемуся на смещении `offset` относительно текущей строки. Является частичной реализацией [оконных функций](https://en.wikipedia.org/wiki/SQL_window_function) LEAD() и LAG(). @@ -307,7 +307,7 @@ WITH toDate('2018-01-01') AS start_date SELECT toStartOfMonth(start_date + (number * 32)) AS month, toInt32(month) % 100 AS money, - neighbour(money, -12) AS prev_year, + neighbor(money, -12) AS prev_year, round(prev_year / money, 2) AS year_over_year FROM numbers(16) ``` From e3bd572fc75f20d83f421e00981524e6bcd1b8ba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 05:00:35 +0300 Subject: [PATCH 370/509] Removed unused settings --- dbms/src/Core/Settings.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 1c2cab8b860..3cc5d291708 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -350,8 +350,6 @@ struct Settings : public SettingsCollection \ M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.") \ M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.") \ - M(SettingSeconds, temporary_live_channel_timeout, DEFAULT_TEMPORARY_LIVE_CHANNEL_TIMEOUT_SEC, "Timeout after which temporary live channel is deleted.") \ - M(SettingMilliseconds, alter_channel_wait_ms, DEFAULT_ALTER_LIVE_CHANNEL_WAIT_MS, "The wait time for alter channel request.") \ M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.") \ DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) From 99f4c9c8130a742d532294ce5edf2ac6cfd86a8c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 05:01:24 +0300 Subject: [PATCH 371/509] Moved settings that were in a wrong place --- dbms/src/Core/Settings.h | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 3cc5d291708..ff3c8fb5890 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -343,14 +343,13 @@ struct Settings : public SettingsCollection M(SettingBool, check_query_single_value_result, true, "Return check query result as single 1/0 value") \ \ M(SettingBool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.") \ + M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.") \ + M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.") \ + M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.") \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") \ - \ - M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.") \ - M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.") \ - M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.") \ DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) From 1222973cb3fa6e267c6bb7d29c44fef2beafe173 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 05:02:14 +0300 Subject: [PATCH 372/509] Function "neighbor": merging #5925 --- dbms/src/Functions/neighbor.cpp | 50 +++++++++++++++------------------ 1 file changed, 23 insertions(+), 27 deletions(-) diff --git a/dbms/src/Functions/neighbor.cpp b/dbms/src/Functions/neighbor.cpp index 6ac2c966016..02d28028d3b 100644 --- a/dbms/src/Functions/neighbor.cpp +++ b/dbms/src/Functions/neighbor.cpp @@ -66,7 +66,6 @@ public: "Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + " - can not be Nullable", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - // check that default value column has supertype with first argument if (number_of_arguments == 3) { @@ -99,10 +98,10 @@ public: { if (isColumnConst(*default_values_column)) { - Field constant_value = (*default_values_column)[0]; - for (size_t row = 0; row < row_count; row++) + const IColumn & constant_content = assert_cast(*default_values_column).getDataColumn(); + for (size_t row = 0; row < row_count; ++row) { - target->insert(constant_value); + target->insertFrom(constant_content, 0); } } else @@ -112,7 +111,7 @@ public: } else { - for (size_t row = 0; row < row_count; row++) + for (size_t row = 0; row < row_count; ++row) { target->insertDefault(); } @@ -121,37 +120,33 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { - auto offset_structure = block.getByPosition(arguments[1]); + const ColumnWithTypeAndName & source_column_name_and_type = block.getByPosition(arguments[0]); + const DataTypePtr & result_type = block.getByPosition(result).type; - ColumnPtr & offset_column = offset_structure.column; + ColumnPtr source_column = source_column_name_and_type.column; - auto is_constant_offset = isColumnConst(*offset_structure.column); - ColumnPtr default_values_column = nullptr; - if (arguments.size() == 3) - { - default_values_column = block.getByPosition(arguments[2]).column; - } - - ColumnWithTypeAndName & source_column_name_and_type = block.getByPosition(arguments[0]); - DataTypes types = {source_column_name_and_type.type}; - if (default_values_column) - { - types.push_back(block.getByPosition(arguments[2]).type); - } - const DataTypePtr & result_type = getLeastSupertype(types); - auto source_column = source_column_name_and_type.column; - - // adjust source and default values columns to resulting datatype + // adjust source and default values columns to resulting data type if (!source_column_name_and_type.type->equals(*result_type)) { source_column = castColumn(source_column_name_and_type, result_type, context); } - if (default_values_column && !block.getByPosition(arguments[2]).type->equals(*result_type)) + ColumnPtr default_values_column; + + /// Has argument with default value: neighbor(source, offset, default) + if (arguments.size() == 3) { - default_values_column = castColumn(block.getByPosition(arguments[2]), result_type, context); + default_values_column = block.getByPosition(arguments[2]).column; + + if (!block.getByPosition(arguments[2]).type->equals(*result_type)) + default_values_column = castColumn(block.getByPosition(arguments[2]), result_type, context); } + const auto & offset_structure = block.getByPosition(arguments[1]); + ColumnPtr offset_column = offset_structure.column; + + auto is_constant_offset = isColumnConst(*offset_structure.column); + // since we are working with both signed and unsigned - we'll try to use Int64 for handling all of them const DataTypePtr desired_type = std::make_shared(); if (!block.getByPosition(arguments[1]).type->equals(*desired_type)) @@ -161,6 +156,7 @@ public: if (isColumnConst(*source_column)) { + /// NOTE Inconsistency when default_values are specified. auto column = result_type->createColumnConst(input_rows_count, (*source_column)[0]); block.getByPosition(result).column = std::move(column); } @@ -199,7 +195,7 @@ public: else { // with dynamic offset - handle row by row - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { Int64 offset_value = offset_column->getInt(row); if (offset_value == 0) From ff9e92eab9ace56801f032e1bf3990217ee456d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 05:53:09 +0300 Subject: [PATCH 373/509] Renamed function in test --- .../0_stateless/00957_neighbor.reference | 69 +++++++++++++++++++ .../queries/0_stateless/00957_neighbor.sql | 42 +++++++++++ 2 files changed, 111 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00957_neighbor.reference create mode 100644 dbms/tests/queries/0_stateless/00957_neighbor.sql diff --git a/dbms/tests/queries/0_stateless/00957_neighbor.reference b/dbms/tests/queries/0_stateless/00957_neighbor.reference new file mode 100644 index 00000000000..5a523d2c575 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00957_neighbor.reference @@ -0,0 +1,69 @@ +Zero offset +0 0 +1 1 +2 2 +Nullable values +\N 0 \N +\N 1 2 +2 2 \N +Result with different type +0 1 +1 2 +2 -10 +Offset > block +0 0 +1 0 +2 0 +Abs(Offset) > block +0 0 +1 0 +2 0 +Positive offset +0 1 +1 2 +2 0 +Negative offset +0 1 +1 2 +2 0 +Positive offset with defaults +0 2 +1 3 +2 12 +3 13 +Negative offset with defaults +0 10 +1 11 +2 0 +3 1 +Positive offset with const defaults +0 1 +1 2 +2 1000 +Negative offset with const defaults +0 1000 +1 0 +2 1 +Dynamic column and offset, out of bounds +0 0 0 +1 2 3 +2 4 20 +3 6 30 +Dynamic column and offset, negative +0 0 0 +1 -2 10 +2 -4 20 +3 -6 30 +4 -8 40 +5 -10 50 +Dynamic column and offset, without defaults +0 4 4 +1 2 3 +2 0 2 +3 -2 1 +4 -4 0 +5 -6 0 +Constant column +0 1000 +1 1000 +2 1000 diff --git a/dbms/tests/queries/0_stateless/00957_neighbor.sql b/dbms/tests/queries/0_stateless/00957_neighbor.sql new file mode 100644 index 00000000000..2f701d4d5c7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00957_neighbor.sql @@ -0,0 +1,42 @@ +-- no arguments +select neighbor(); -- { serverError 42 } +-- single argument +select neighbor(1); -- { serverError 42 } +-- greater than 3 arguments +select neighbor(1,2,3,4); -- { serverError 42 } +-- bad default value +select neighbor(dummy, 1, 'hello'); -- { serverError 43 } +-- types without common supertype (UInt64 and Int8) +select number, neighbor(number, 1, -10) from numbers(3); -- { serverError 43 } +-- nullable offset is not allowed +select number, if(number > 1, number, null) as offset, neighbor(number, offset) from numbers(3); -- { serverError 43 } +select 'Zero offset'; +select number, neighbor(number, 0) from numbers(3); +select 'Nullable values'; +select if(number > 1, number, null) as value, number as offset, neighbor(value, offset) as neighbor from numbers(3); +select 'Result with different type'; +select toInt32(number) as n, neighbor(n, 1, -10) from numbers(3); +select 'Offset > block'; +select number, neighbor(number, 10) from numbers(3); +select 'Abs(Offset) > block'; +select number, neighbor(number, -10) from numbers(3); +select 'Positive offset'; +select number, neighbor(number, 1) from numbers(3); +select 'Negative offset'; +select number, neighbor(number, 1) from numbers(3); +select 'Positive offset with defaults'; +select number, neighbor(number, 2, number + 10) from numbers(4); +select 'Negative offset with defaults'; +select number, neighbor(number, -2, number + 10) from numbers(4); +select 'Positive offset with const defaults'; +select number, neighbor(number, 1, 1000) from numbers(3); +select 'Negative offset with const defaults'; +select number, neighbor(number, -1, 1000) from numbers(3); +select 'Dynamic column and offset, out of bounds'; +select number, number * 2 as offset, neighbor(number, offset, number * 10) from numbers(4); +select 'Dynamic column and offset, negative'; +select number, -number * 2 as offset, neighbor(number, offset, number * 10) from numbers(6); +select 'Dynamic column and offset, without defaults'; +select number, -(number - 2) * 2 as offset, neighbor(number, offset) from numbers(6); +select 'Constant column'; +select number, neighbor(1000, 10) from numbers(3); \ No newline at end of file From 341e2e4587a18065c2da1ca888c73389f48ce36c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 06:00:06 +0300 Subject: [PATCH 374/509] Step 1: make it correct. --- dbms/src/Functions/neighbor.cpp | 197 ++++-------------- .../0_stateless/00957_neighbor.reference | 6 +- .../queries/0_stateless/00957_neighbor.sql | 4 +- .../0_stateless/00957_neighbour.reference | 69 ------ .../queries/0_stateless/00957_neighbour.sql | 42 ---- 5 files changed, 42 insertions(+), 276 deletions(-) delete mode 100644 dbms/tests/queries/0_stateless/00957_neighbour.reference delete mode 100644 dbms/tests/queries/0_stateless/00957_neighbour.sql diff --git a/dbms/src/Functions/neighbor.cpp b/dbms/src/Functions/neighbor.cpp index 02d28028d3b..1c640ffb76c 100644 --- a/dbms/src/Functions/neighbor.cpp +++ b/dbms/src/Functions/neighbor.cpp @@ -46,6 +46,8 @@ public: bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return false; } + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { size_t number_of_arguments = arguments.size(); @@ -68,183 +70,58 @@ public: // check that default value column has supertype with first argument if (number_of_arguments == 3) - { - DataTypes types = {arguments[0], arguments[2]}; - try - { - return getLeastSupertype(types); - } - catch (const Exception &) - { - throw Exception( - "Illegal types of arguments (" + types[0]->getName() + ", " + types[1]->getName() - + ")" - " of function " - + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - } + return getLeastSupertype({arguments[0], arguments[2]}); return arguments[0]; } - static void insertDefaults(const MutableColumnPtr & target, size_t row_count, ColumnPtr & default_values_column, size_t offset) - { - if (row_count == 0) - { - return; - } - if (default_values_column) - { - if (isColumnConst(*default_values_column)) - { - const IColumn & constant_content = assert_cast(*default_values_column).getDataColumn(); - for (size_t row = 0; row < row_count; ++row) - { - target->insertFrom(constant_content, 0); - } - } - else - { - target->insertRangeFrom(*default_values_column, offset, row_count); - } - } - else - { - for (size_t row = 0; row < row_count; ++row) - { - target->insertDefault(); - } - } - } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { - const ColumnWithTypeAndName & source_column_name_and_type = block.getByPosition(arguments[0]); const DataTypePtr & result_type = block.getByPosition(result).type; - ColumnPtr source_column = source_column_name_and_type.column; + const ColumnWithTypeAndName & source_elem = block.getByPosition(arguments[0]); + const ColumnWithTypeAndName & offset_elem = block.getByPosition(arguments[1]); + bool has_defaults = arguments.size() == 3; - // adjust source and default values columns to resulting data type - if (!source_column_name_and_type.type->equals(*result_type)) + ColumnPtr source_column_casted = castColumn(source_elem, result_type, context); + ColumnPtr offset_column = offset_elem.column; + + ColumnPtr default_column_casted; + if (has_defaults) { - source_column = castColumn(source_column_name_and_type, result_type, context); + const ColumnWithTypeAndName & default_elem = block.getByPosition(arguments[2]); + default_column_casted = castColumn(default_elem, result_type, context); } - ColumnPtr default_values_column; + bool source_is_constant = isColumnConst(*source_column_casted); + bool offset_is_constant = isColumnConst(*offset_column); - /// Has argument with default value: neighbor(source, offset, default) - if (arguments.size() == 3) + bool default_is_constant = false; + if (has_defaults) + default_is_constant = isColumnConst(*default_column_casted); + + if (source_is_constant) + source_column_casted = assert_cast(*source_column_casted).getDataColumnPtr(); + if (offset_is_constant) + offset_column = assert_cast(*offset_column).getDataColumnPtr(); + if (default_is_constant) + default_column_casted = assert_cast(*default_column_casted).getDataColumnPtr(); + + auto column = result_type->createColumn(); + + for (size_t row = 0; row < input_rows_count; ++row) { - default_values_column = block.getByPosition(arguments[2]).column; + Int64 src_idx = row + offset_column->getInt(offset_is_constant ? 0 : row); - if (!block.getByPosition(arguments[2]).type->equals(*result_type)) - default_values_column = castColumn(block.getByPosition(arguments[2]), result_type, context); - } - - const auto & offset_structure = block.getByPosition(arguments[1]); - ColumnPtr offset_column = offset_structure.column; - - auto is_constant_offset = isColumnConst(*offset_structure.column); - - // since we are working with both signed and unsigned - we'll try to use Int64 for handling all of them - const DataTypePtr desired_type = std::make_shared(); - if (!block.getByPosition(arguments[1]).type->equals(*desired_type)) - { - offset_column = castColumn(offset_structure, desired_type, context); - } - - if (isColumnConst(*source_column)) - { - /// NOTE Inconsistency when default_values are specified. - auto column = result_type->createColumnConst(input_rows_count, (*source_column)[0]); - block.getByPosition(result).column = std::move(column); - } - else - { - auto column = result_type->createColumn(); - column->reserve(input_rows_count); - // with constant offset - insertRangeFrom - if (is_constant_offset) - { - Int64 offset_value = offset_column->getInt(0); - - auto offset_value_casted = static_cast(std::abs(offset_value)); - size_t default_value_count = std::min(offset_value_casted, input_rows_count); - if (offset_value > 0) - { - // insert shifted value - if (offset_value_casted <= input_rows_count) - { - column->insertRangeFrom(*source_column, offset_value_casted, input_rows_count - offset_value_casted); - } - insertDefaults(column, default_value_count, default_values_column, input_rows_count - default_value_count); - } - else if (offset_value < 0) - { - // insert defaults up to offset_value - insertDefaults(column, default_value_count, default_values_column, 0); - column->insertRangeFrom(*source_column, 0, input_rows_count - default_value_count); - } - else - { - // populate column with source values, when offset is equal to zero - column->insertRangeFrom(*source_column, 0, input_rows_count); - } - } + if (src_idx >= 0 && src_idx < Int64(input_rows_count)) + column->insertFrom(*source_column_casted, source_is_constant ? 0 : src_idx); + else if (has_defaults) + column->insertFrom(*default_column_casted, default_is_constant ? 0 : row); else - { - // with dynamic offset - handle row by row - for (size_t row = 0; row < input_rows_count; ++row) - { - Int64 offset_value = offset_column->getInt(row); - if (offset_value == 0) - { - column->insertFrom(*source_column, row); - } - else if (offset_value > 0) - { - size_t real_offset = row + offset_value; - if (real_offset > input_rows_count) - { - if (default_values_column) - { - column->insertFrom(*default_values_column, row); - } - else - { - column->insertDefault(); - } - } - else - { - column->insertFrom(*source_column, real_offset); - } - } - else - { - // out of range - auto offset_value_casted = static_cast(std::abs(offset_value)); - if (offset_value_casted > row) - { - if (default_values_column) - { - column->insertFrom(*default_values_column, row); - } - else - { - column->insertDefault(); - } - } - else - { - column->insertFrom(*source_column, row - offset_value_casted); - } - } - } - } - block.getByPosition(result).column = std::move(column); + column->insertDefault(); } + + block.getByPosition(result).column = std::move(column); } private: diff --git a/dbms/tests/queries/0_stateless/00957_neighbor.reference b/dbms/tests/queries/0_stateless/00957_neighbor.reference index 5a523d2c575..d25d727da5d 100644 --- a/dbms/tests/queries/0_stateless/00957_neighbor.reference +++ b/dbms/tests/queries/0_stateless/00957_neighbor.reference @@ -64,6 +64,6 @@ Dynamic column and offset, without defaults 4 -4 0 5 -6 0 Constant column -0 1000 -1 1000 -2 1000 +0 0 +1 0 +2 0 diff --git a/dbms/tests/queries/0_stateless/00957_neighbor.sql b/dbms/tests/queries/0_stateless/00957_neighbor.sql index 2f701d4d5c7..c4b801c80cb 100644 --- a/dbms/tests/queries/0_stateless/00957_neighbor.sql +++ b/dbms/tests/queries/0_stateless/00957_neighbor.sql @@ -5,9 +5,9 @@ select neighbor(1); -- { serverError 42 } -- greater than 3 arguments select neighbor(1,2,3,4); -- { serverError 42 } -- bad default value -select neighbor(dummy, 1, 'hello'); -- { serverError 43 } +select neighbor(dummy, 1, 'hello'); -- { serverError 386 } -- types without common supertype (UInt64 and Int8) -select number, neighbor(number, 1, -10) from numbers(3); -- { serverError 43 } +select number, neighbor(number, 1, -10) from numbers(3); -- { serverError 386 } -- nullable offset is not allowed select number, if(number > 1, number, null) as offset, neighbor(number, offset) from numbers(3); -- { serverError 43 } select 'Zero offset'; diff --git a/dbms/tests/queries/0_stateless/00957_neighbour.reference b/dbms/tests/queries/0_stateless/00957_neighbour.reference deleted file mode 100644 index 5a523d2c575..00000000000 --- a/dbms/tests/queries/0_stateless/00957_neighbour.reference +++ /dev/null @@ -1,69 +0,0 @@ -Zero offset -0 0 -1 1 -2 2 -Nullable values -\N 0 \N -\N 1 2 -2 2 \N -Result with different type -0 1 -1 2 -2 -10 -Offset > block -0 0 -1 0 -2 0 -Abs(Offset) > block -0 0 -1 0 -2 0 -Positive offset -0 1 -1 2 -2 0 -Negative offset -0 1 -1 2 -2 0 -Positive offset with defaults -0 2 -1 3 -2 12 -3 13 -Negative offset with defaults -0 10 -1 11 -2 0 -3 1 -Positive offset with const defaults -0 1 -1 2 -2 1000 -Negative offset with const defaults -0 1000 -1 0 -2 1 -Dynamic column and offset, out of bounds -0 0 0 -1 2 3 -2 4 20 -3 6 30 -Dynamic column and offset, negative -0 0 0 -1 -2 10 -2 -4 20 -3 -6 30 -4 -8 40 -5 -10 50 -Dynamic column and offset, without defaults -0 4 4 -1 2 3 -2 0 2 -3 -2 1 -4 -4 0 -5 -6 0 -Constant column -0 1000 -1 1000 -2 1000 diff --git a/dbms/tests/queries/0_stateless/00957_neighbour.sql b/dbms/tests/queries/0_stateless/00957_neighbour.sql deleted file mode 100644 index b60cc3123b6..00000000000 --- a/dbms/tests/queries/0_stateless/00957_neighbour.sql +++ /dev/null @@ -1,42 +0,0 @@ --- no arguments -select neighbour(); -- { serverError 42 } --- single argument -select neighbour(1); -- { serverError 42 } --- greater than 3 arguments -select neighbour(1,2,3,4); -- { serverError 42 } --- bad default value -select neighbour(dummy, 1, 'hello'); -- { serverError 43 } --- types without common supertype (UInt64 and Int8) -select number, neighbour(number, 1, -10) from numbers(3); -- { serverError 43 } --- nullable offset is not allowed -select number, if(number > 1, number, null) as offset, neighbour(number, offset) from numbers(3); -- { serverError 43 } -select 'Zero offset'; -select number, neighbour(number, 0) from numbers(3); -select 'Nullable values'; -select if(number > 1, number, null) as value, number as offset, neighbour(value, offset) as neighbour from numbers(3); -select 'Result with different type'; -select toInt32(number) as n, neighbour(n, 1, -10) from numbers(3); -select 'Offset > block'; -select number, neighbour(number, 10) from numbers(3); -select 'Abs(Offset) > block'; -select number, neighbour(number, -10) from numbers(3); -select 'Positive offset'; -select number, neighbour(number, 1) from numbers(3); -select 'Negative offset'; -select number, neighbour(number, 1) from numbers(3); -select 'Positive offset with defaults'; -select number, neighbour(number, 2, number + 10) from numbers(4); -select 'Negative offset with defaults'; -select number, neighbour(number, -2, number + 10) from numbers(4); -select 'Positive offset with const defaults'; -select number, neighbour(number, 1, 1000) from numbers(3); -select 'Negative offset with const defaults'; -select number, neighbour(number, -1, 1000) from numbers(3); -select 'Dynamic column and offset, out of bounds'; -select number, number * 2 as offset, neighbour(number, offset, number * 10) from numbers(4); -select 'Dynamic column and offset, negative'; -select number, -number * 2 as offset, neighbour(number, offset, number * 10) from numbers(6); -select 'Dynamic column and offset, without defaults'; -select number, -(number - 2) * 2 as offset, neighbour(number, offset) from numbers(6); -select 'Constant column'; -select number, neighbour(1000, 10) from numbers(3); \ No newline at end of file From 6685365ab8c5b74f9650492c88a012596eb1b0c6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 07:25:32 +0300 Subject: [PATCH 375/509] Added optimized case --- dbms/src/Functions/neighbor.cpp | 78 ++++++++++++++++++++++++++++----- 1 file changed, 68 insertions(+), 10 deletions(-) diff --git a/dbms/src/Functions/neighbor.cpp b/dbms/src/Functions/neighbor.cpp index 1c640ffb76c..994e6311678 100644 --- a/dbms/src/Functions/neighbor.cpp +++ b/dbms/src/Functions/neighbor.cpp @@ -107,21 +107,79 @@ public: if (default_is_constant) default_column_casted = assert_cast(*default_column_casted).getDataColumnPtr(); - auto column = result_type->createColumn(); - - for (size_t row = 0; row < input_rows_count; ++row) + if (offset_is_constant) { - Int64 src_idx = row + offset_column->getInt(offset_is_constant ? 0 : row); + /// Optimization for the case when we can copy many values at once. - if (src_idx >= 0 && src_idx < Int64(input_rows_count)) - column->insertFrom(*source_column_casted, source_is_constant ? 0 : src_idx); - else if (has_defaults) - column->insertFrom(*default_column_casted, default_is_constant ? 0 : row); + Int64 offset = offset_column->getInt(0); + + auto result_column = result_type->createColumn(); + + auto insert_range_from = [&](bool is_const, const ColumnPtr & src, Int64 begin, Int64 size) + { + /// Saturation of bounds. + if (begin < 0) + { + size += begin; + begin = 0; + } + if (size <= 0) + return; + if (size > Int64(input_rows_count)) + size = input_rows_count; + + if (!src) + { + for (Int64 i = 0; i < size; ++i) + result_column->insertDefault(); + } + else if (is_const) + { + for (Int64 i = 0; i < size; ++i) + result_column->insertFrom(*src, 0); + } + else + { + result_column->insertRangeFrom(*src, begin, size); + } + }; + + if (offset == 0) + { + /// Degenerate case, just copy source column as is. + block.getByPosition(result).column = source_column_casted; /// TODO + } + else if (offset > 0) + { + insert_range_from(source_is_constant, source_column_casted, offset, Int64(input_rows_count) - offset); + insert_range_from(default_is_constant, default_column_casted, Int64(input_rows_count) - offset, offset); + block.getByPosition(result).column = std::move(result_column); + } else - column->insertDefault(); + { + insert_range_from(default_is_constant, default_column_casted, 0, -offset); + insert_range_from(source_is_constant, source_column_casted, 0, Int64(input_rows_count) + offset); + block.getByPosition(result).column = std::move(result_column); + } } + else + { + auto result_column = result_type->createColumn(); - block.getByPosition(result).column = std::move(column); + for (size_t row = 0; row < input_rows_count; ++row) + { + Int64 src_idx = row + offset_column->getInt(offset_is_constant ? 0 : row); + + if (src_idx >= 0 && src_idx < Int64(input_rows_count)) + result_column->insertFrom(*source_column_casted, source_is_constant ? 0 : src_idx); + else if (has_defaults) + result_column->insertFrom(*default_column_casted, default_is_constant ? 0 : row); + else + result_column->insertDefault(); + } + + block.getByPosition(result).column = std::move(result_column); + } } private: From 20b9af29f555f6e125c811e95cc19c39ca7857d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 07:34:59 +0300 Subject: [PATCH 376/509] More tests --- dbms/src/Functions/neighbor.cpp | 2 +- .../0_stateless/00996_neigbor.reference | 270 ++++++++++++++++++ .../queries/0_stateless/00996_neigbor.sql | 42 +++ 3 files changed, 313 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/00996_neigbor.reference create mode 100644 dbms/tests/queries/0_stateless/00996_neigbor.sql diff --git a/dbms/src/Functions/neighbor.cpp b/dbms/src/Functions/neighbor.cpp index 994e6311678..078f704c771 100644 --- a/dbms/src/Functions/neighbor.cpp +++ b/dbms/src/Functions/neighbor.cpp @@ -147,7 +147,7 @@ public: if (offset == 0) { /// Degenerate case, just copy source column as is. - block.getByPosition(result).column = source_column_casted; /// TODO + block.getByPosition(result).column = source_is_constant ? ColumnConst::create(source_column_casted, input_rows_count) : source_column_casted; } else if (offset > 0) { diff --git a/dbms/tests/queries/0_stateless/00996_neigbor.reference b/dbms/tests/queries/0_stateless/00996_neigbor.reference new file mode 100644 index 00000000000..ebdb8c9e684 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00996_neigbor.reference @@ -0,0 +1,270 @@ +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +0 5 +1 6 +2 7 +3 8 +4 9 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 0 +6 1 +7 2 +8 3 +9 4 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 5 +1 6 +2 7 +3 8 +4 9 +5 Hello +6 Hello +7 Hello +8 Hello +9 Hello +0 World +1 World +2 World +3 World +4 World +5 0 +6 1 +7 2 +8 3 +9 4 +0 5 +1 6 +2 7 +3 8 +4 9 +5 Hello 5 +6 Hello 6 +7 Hello 7 +8 Hello 8 +9 Hello 9 +0 World 0 +1 World 1 +2 World 2 +3 World 3 +4 World 4 +5 0 +6 1 +7 2 +8 3 +9 4 +0 ClickHouse +1 ClickHouse +2 ClickHouse +3 ClickHouse +4 ClickHouse +5 ClickHouse +6 ClickHouse +7 ClickHouse +8 ClickHouse +9 ClickHouse +0 ClickHouse +1 ClickHouse +2 ClickHouse +3 ClickHouse +4 ClickHouse +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 ClickHouse +6 ClickHouse +7 ClickHouse +8 ClickHouse +9 ClickHouse +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 ClickHouse +1 ClickHouse +2 ClickHouse +3 ClickHouse +4 ClickHouse +5 Hello +6 Hello +7 Hello +8 Hello +9 Hello +0 World +1 World +2 World +3 World +4 World +5 ClickHouse +6 ClickHouse +7 ClickHouse +8 ClickHouse +9 ClickHouse +0 ClickHouse +1 ClickHouse +2 ClickHouse +3 ClickHouse +4 ClickHouse +5 Hello 5 +6 Hello 6 +7 Hello 7 +8 Hello 8 +9 Hello 9 +0 World 0 +1 World 1 +2 World 2 +3 World 3 +4 World 4 +5 ClickHouse +6 ClickHouse +7 ClickHouse +8 ClickHouse +9 ClickHouse +0 0 +1 2 +2 4 +3 6 +4 8 +5 +6 +7 +8 +9 +0 0 +1 1 +2 3 +3 4 +4 6 +5 7 +6 9 +7 +8 +9 +0 Hello +1 Hello +2 Hello +3 Hello +4 Hello +5 +6 +7 +8 +9 +0 +1 +2 +3 Hello +4 Hello +5 Hello +6 Hello +7 Hello +8 Hello +9 Hello +0 World +1 World +2 World +3 Hello +4 Hello +5 Hello +6 Hello +7 Hello +8 Hello +9 Hello diff --git a/dbms/tests/queries/0_stateless/00996_neigbor.sql b/dbms/tests/queries/0_stateless/00996_neigbor.sql new file mode 100644 index 00000000000..25c20b1b896 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00996_neigbor.sql @@ -0,0 +1,42 @@ +SELECT number, neighbor(toString(number), 0) FROM numbers(10); + +SELECT number, neighbor(toString(number), 5) FROM numbers(10); +SELECT number, neighbor(toString(number), -5) FROM numbers(10); + +SELECT number, neighbor(toString(number), 10) FROM numbers(10); +SELECT number, neighbor(toString(number), -10) FROM numbers(10); + +SELECT number, neighbor(toString(number), 15) FROM numbers(10); +SELECT number, neighbor(toString(number), -15) FROM numbers(10); + +SELECT number, neighbor(toString(number), 5, 'Hello') FROM numbers(10); +SELECT number, neighbor(toString(number), -5, 'World') FROM numbers(10); + +SELECT number, neighbor(toString(number), 5, concat('Hello ', toString(number))) FROM numbers(10); +SELECT number, neighbor(toString(number), -5, concat('World ', toString(number))) FROM numbers(10); + + +SELECT number, neighbor('ClickHouse', 0) FROM numbers(10); + +SELECT number, neighbor('ClickHouse', 5) FROM numbers(10); +SELECT number, neighbor('ClickHouse', -5) FROM numbers(10); + +SELECT number, neighbor('ClickHouse', 10) FROM numbers(10); +SELECT number, neighbor('ClickHouse', -10) FROM numbers(10); + +SELECT number, neighbor('ClickHouse', 15) FROM numbers(10); +SELECT number, neighbor('ClickHouse', -15) FROM numbers(10); + +SELECT number, neighbor('ClickHouse', 5, 'Hello') FROM numbers(10); +SELECT number, neighbor('ClickHouse', -5, 'World') FROM numbers(10); + +SELECT number, neighbor('ClickHouse', 5, concat('Hello ', toString(number))) FROM numbers(10); +SELECT number, neighbor('ClickHouse', -5, concat('World ', toString(number))) FROM numbers(10); + + +SELECT number, neighbor(toString(number), number) FROM numbers(10); +SELECT number, neighbor(toString(number), intDiv(number, 2)) FROM numbers(10); + +SELECT number, neighbor('Hello', number) FROM numbers(10); +SELECT number, neighbor('Hello', -3) FROM numbers(10); +SELECT number, neighbor('Hello', -3, 'World') FROM numbers(10); From 3d8613f8dfe1750e7f7c5be0b71921db2b2cea2a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 07:36:01 +0300 Subject: [PATCH 377/509] More tests --- .../{00996_neigbor.reference => 00996_neighbor.reference} | 0 .../queries/0_stateless/{00996_neigbor.sql => 00996_neighbor.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{00996_neigbor.reference => 00996_neighbor.reference} (100%) rename dbms/tests/queries/0_stateless/{00996_neigbor.sql => 00996_neighbor.sql} (100%) diff --git a/dbms/tests/queries/0_stateless/00996_neigbor.reference b/dbms/tests/queries/0_stateless/00996_neighbor.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00996_neigbor.reference rename to dbms/tests/queries/0_stateless/00996_neighbor.reference diff --git a/dbms/tests/queries/0_stateless/00996_neigbor.sql b/dbms/tests/queries/0_stateless/00996_neighbor.sql similarity index 100% rename from dbms/tests/queries/0_stateless/00996_neigbor.sql rename to dbms/tests/queries/0_stateless/00996_neighbor.sql From 84b0f709aa0c1b26d15e4de0abea45c865fb1934 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 08:08:06 +0300 Subject: [PATCH 378/509] Removed useless code --- dbms/src/Core/Defines.h | 2 - dbms/src/Parsers/ASTAlterQuery.cpp | 43 ------------------ dbms/src/Parsers/ASTAlterQuery.h | 19 -------- dbms/src/Parsers/ASTCreateQuery.cpp | 3 -- dbms/src/Parsers/ASTCreateQuery.h | 1 - dbms/src/Parsers/ParserAlterQuery.cpp | 63 ++------------------------ dbms/src/Parsers/ParserAlterQuery.h | 13 +----- dbms/src/Parsers/ParserCreateQuery.cpp | 47 ++++++------------- dbms/src/Parsers/ParserCreateQuery.h | 2 +- 9 files changed, 20 insertions(+), 173 deletions(-) diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 0c72c926006..a172cf6e243 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -33,8 +33,6 @@ #define DEFAULT_MERGE_BLOCK_SIZE 8192 #define DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC 5 -#define DEFAULT_TEMPORARY_LIVE_CHANNEL_TIMEOUT_SEC 15 -#define DEFAULT_ALTER_LIVE_CHANNEL_WAIT_MS 10000 #define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160) #define DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC 15 #define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024 diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index 54ba460d75a..6cfba1d6a79 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -45,11 +45,6 @@ ASTPtr ASTAlterCommand::clone() const res->ttl = ttl->clone(); res->children.push_back(res->ttl); } - if (values) - { - res->values = values->clone(); - res->children.push_back(res->values); - } return res; } @@ -226,42 +221,6 @@ void ASTAlterCommand::formatImpl( { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : ""); } - else if (type == ASTAlterCommand::LIVE_CHANNEL_ADD) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD " << (settings.hilite ? hilite_none : ""); - - values->formatImpl(settings, state, frame); - } - else if (type == ASTAlterCommand::LIVE_CHANNEL_DROP) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DROP " << (settings.hilite ? hilite_none : ""); - - values->formatImpl(settings, state, frame); - } - else if (type == ASTAlterCommand::LIVE_CHANNEL_MODIFY) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY " << (settings.hilite ? hilite_none : ""); - - values->formatImpl(settings, state, frame); - } - else if (type == ASTAlterCommand::LIVE_CHANNEL_SUSPEND) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "SUSPEND " << (settings.hilite ? hilite_none : ""); - - values->formatImpl(settings, state, frame); - } - else if (type == ASTAlterCommand::LIVE_CHANNEL_RESUME) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "RESUME " << (settings.hilite ? hilite_none : ""); - - values->formatImpl(settings, state, frame); - } - else if (type == ASTAlterCommand::LIVE_CHANNEL_REFRESH) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : ""); - - values->formatImpl(settings, state, frame); - } else throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE); } @@ -316,8 +275,6 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState if (is_live_view) settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER LIVE VIEW " << (settings.hilite ? hilite_none : ""); - else if (is_live_channel) - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER LIVE CHANNEL " << (settings.hilite ? hilite_none : ""); else settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (settings.hilite ? hilite_none : ""); diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index 5b04fcdffb9..2563abfac6e 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -17,13 +17,6 @@ namespace DB * COMMENT_COLUMN col_name 'comment', * ALTER LIVE VIEW [db.]name_type * REFRESH - * ALTER CHANNEL [db.]name_type - * ADD live_view,... - * DROP live_view,... - * SUSPEND live_view,... - * RESUME live_view,... - * REFRESH live_view,... - * MODIFY live_view,... */ class ASTAlterCommand : public IAST @@ -59,13 +52,6 @@ public: NO_TYPE, LIVE_VIEW_REFRESH, - - LIVE_CHANNEL_ADD, - LIVE_CHANNEL_DROP, - LIVE_CHANNEL_SUSPEND, - LIVE_CHANNEL_RESUME, - LIVE_CHANNEL_REFRESH, - LIVE_CHANNEL_MODIFY }; Type type = NO_TYPE; @@ -121,10 +107,6 @@ public: /// For MODIFY TTL query ASTPtr ttl; - /** In ALTER CHANNEL, ADD, DROP, SUSPEND, RESUME, REFRESH, MODIFY queries, the list of live views is stored here - */ - ASTPtr values; - bool detach = false; /// true for DETACH PARTITION bool part = false; /// true for ATTACH PART and DROP DETACHED PART @@ -182,7 +164,6 @@ class ASTAlterQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCl { public: bool is_live_view{false}; /// true for ALTER LIVE VIEW - bool is_live_channel{false}; /// true for ALTER LIVE CHANNEL ASTAlterCommandList * command_list = nullptr; diff --git a/dbms/src/Parsers/ASTCreateQuery.cpp b/dbms/src/Parsers/ASTCreateQuery.cpp index 87e9301329d..f22460d8716 100644 --- a/dbms/src/Parsers/ASTCreateQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuery.cpp @@ -220,9 +220,6 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat what = "MATERIALIZED VIEW"; if (is_live_view) what = "LIVE VIEW"; - if (is_live_channel) - what = "LIVE CHANNEL"; - settings.ostr << (settings.hilite ? hilite_keyword : "") diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 3893fa2c82a..5ca0c067a3c 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -57,7 +57,6 @@ public: bool is_view{false}; bool is_materialized_view{false}; bool is_live_view{false}; - bool is_live_channel{false}; bool is_populate{false}; bool replace_view{false}; /// CREATE OR REPLACE VIEW ASTColumns * columns_list = nullptr; diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 41b046e2877..21a4fd586c6 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -89,53 +89,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected else return false; } - else if (is_live_channel) - { - if (s_add.ignore(pos, expected)) - { - if (!values_p.parse(pos, command->values, expected)) - return false; - - command->type = ASTAlterCommand::LIVE_CHANNEL_ADD; - } - else if (s_drop.ignore(pos, expected)) - { - if (!values_p.parse(pos, command->values, expected)) - return false; - - command->type = ASTAlterCommand::LIVE_CHANNEL_DROP; - } - else if (s_suspend.ignore(pos, expected)) - { - if (!values_p.parse(pos, command->values, expected)) - return false; - - command->type = ASTAlterCommand::LIVE_CHANNEL_SUSPEND; - } - else if (s_resume.ignore(pos, expected)) - { - if (!values_p.parse(pos, command->values, expected)) - return false; - - command->type = ASTAlterCommand::LIVE_CHANNEL_RESUME; - } - else if (s_refresh.ignore(pos, expected)) - { - if (!values_p.parse(pos, command->values, expected)) - return false; - - command->type = ASTAlterCommand::LIVE_CHANNEL_REFRESH; - } - else if (s_modify.ignore(pos, expected)) - { - if (!values_p.parse(pos, command->values, expected)) - return false; - - command->type = ASTAlterCommand::LIVE_CHANNEL_MODIFY; - } - else - return false; - } else { if (s_add_column.ignore(pos, expected)) @@ -466,7 +419,7 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe node = command_list; ParserToken s_comma(TokenType::Comma); - ParserAlterCommand p_command(is_live_view, is_live_channel); + ParserAlterCommand p_command(is_live_view); do { @@ -516,20 +469,13 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_alter_table("ALTER TABLE"); ParserKeyword s_alter_live_view("ALTER LIVE VIEW"); - ParserKeyword s_alter_live_channel("ALTER LIVE CHANNEL"); bool is_live_view = false; - bool is_live_channel = false; if (!s_alter_table.ignore(pos, expected)) { if (!s_alter_live_view.ignore(pos, expected)) - { - if (!s_alter_live_channel.ignore(pos, expected)) - return false; - else - is_live_channel = true; - } + return false; else is_live_view = true; } @@ -537,9 +483,6 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (is_live_view) query->is_live_view = true; - if (is_live_channel) - query->is_live_channel = true; - if (!parseDatabaseAndTableName(pos, expected, query->database, query->table)) return false; @@ -551,7 +494,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } query->cluster = cluster_str; - ParserAlterCommandList p_command_list(is_live_view, is_live_channel); + ParserAlterCommandList p_command_list(is_live_view); ASTPtr command_list; if (!p_command_list.parse(pos, command_list, expected)) return false; diff --git a/dbms/src/Parsers/ParserAlterQuery.h b/dbms/src/Parsers/ParserAlterQuery.h index 13f5681a9da..450e64ee9f4 100644 --- a/dbms/src/Parsers/ParserAlterQuery.h +++ b/dbms/src/Parsers/ParserAlterQuery.h @@ -21,13 +21,6 @@ namespace DB * [UPDATE col_name = expr, ... WHERE ...] * ALTER LIVE VIEW [db.name] * [REFRESH] - * ALTER LIVE CHANNEL [db.name] [ON CLUSTER cluster] - * [ADD live_view, ...] - * [DROP live_view, ...] - * [SUSPEND live_view, ...] - * [RESUME live_view, ...] - * [REFRESH live_view, ...] - * [MODIFY live_view, ...] */ class ParserAlterQuery : public IParserBase @@ -46,9 +39,8 @@ protected: public: bool is_live_view; - bool is_live_channel; - ParserAlterCommandList(bool is_live_view_ = false, bool is_live_channel_ = false) : is_live_view(is_live_view_), is_live_channel(is_live_channel_) {} + ParserAlterCommandList(bool is_live_view_ = false) : is_live_view(is_live_view_) {} }; @@ -60,9 +52,8 @@ protected: public: bool is_live_view; - bool is_live_channel; - ParserAlterCommand(bool is_live_view_ = false, bool is_live_channel_ = false) : is_live_view(is_live_view_), is_live_channel(is_live_channel_) {} + ParserAlterCommand(bool is_live_view_ = false) : is_live_view(is_live_view_) {} }; diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index d2cbd920d0f..acbf2fb89ba 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -365,7 +365,6 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_with("WITH"); ParserKeyword s_materialized("MATERIALIZED"); ParserKeyword s_live("LIVE"); - ParserKeyword s_channel("CHANNEL"); ParserKeyword s_populate("POPULATE"); ParserKeyword s_or_replace("OR REPLACE"); ParserToken s_dot(TokenType::Dot); @@ -396,7 +395,6 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool is_view = false; bool is_materialized_view = false; bool is_live_view = false; - bool is_live_channel = false; bool is_populate = false; bool is_temporary = false; bool replace_view = false; @@ -494,9 +492,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else if (s_live.ignore(pos, expected)) { - if (s_channel.ignore(pos, expected)) - is_live_channel = true; - else if (s_view.ignore(pos, expected)) + if (s_view.ignore(pos, expected)) is_live_view = true; else return false; @@ -520,50 +516,36 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - if (!is_live_channel) + // TO [db.]table + if (ParserKeyword{"TO"}.ignore(pos, expected)) { - // TO [db.]table - if (ParserKeyword{"TO"}.ignore(pos, expected)) + if (!name_p.parse(pos, to_table, expected)) + return false; + + if (s_dot.ignore(pos, expected)) { + to_database = to_table; if (!name_p.parse(pos, to_table, expected)) return false; - - if (s_dot.ignore(pos, expected)) - { - to_database = to_table; - if (!name_p.parse(pos, to_table, expected)) - return false; - } } } /// Optional - a list of columns can be specified. It must fully comply with SELECT. if (s_lparen.ignore(pos, expected)) { - if (!columns_or_indices_p.parse(pos, columns_list, expected)) + if (!table_properties_p.parse(pos, columns_list, expected)) return false; if (!s_rparen.ignore(pos, expected)) return false; } - if (is_live_channel) - { - if (s_with.ignore(pos, expected)) - { - if (!names_p.parse(pos, tables, expected)) - return false; - } - } - else - { - /// AS SELECT ... - if (!s_as.ignore(pos, expected)) - return false; + /// AS SELECT ... + if (!s_as.ignore(pos, expected)) + return false; - if (!select_p.parse(pos, select, expected)) - return false; - } + if (!select_p.parse(pos, select, expected)) + return false; } else if (is_temporary) return false; @@ -673,7 +655,6 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->is_view = is_view; query->is_materialized_view = is_materialized_view; query->is_live_view = is_live_view; - query->is_live_channel = is_live_channel; query->is_populate = is_populate; query->temporary = is_temporary; query->replace_view = replace_view; diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 1199c712585..a4f4da8907e 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -321,7 +321,7 @@ protected: * CREATE|ATTACH DATABASE db [ENGINE = engine] * * Or: - * CREATE[OR REPLACE]|ATTACH [[MATERIALIZED] VIEW] | [[TEMPORARY] LIVE [CHANNEL] | [VIEW]] [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... + * CREATE[OR REPLACE]|ATTACH [[MATERIALIZED] VIEW] | [VIEW]] [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... */ class ParserCreateQuery : public IParserBase { From dae2aa61387a619cf5db94d8a046d5c3353fde37 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 08:11:11 +0300 Subject: [PATCH 379/509] Removed useless code --- dbms/src/Parsers/ASTAlterQuery.cpp | 5 +++++ dbms/src/Parsers/ASTAlterQuery.h | 4 ++++ 2 files changed, 9 insertions(+) diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index 6cfba1d6a79..e8fa3630442 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -45,6 +45,11 @@ ASTPtr ASTAlterCommand::clone() const res->ttl = ttl->clone(); res->children.push_back(res->ttl); } + if (values) + { + res->values = values->clone(); + res->children.push_back(res->values); + } return res; } diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index 2563abfac6e..a4962087a9b 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -107,6 +107,10 @@ public: /// For MODIFY TTL query ASTPtr ttl; + /** In ALTER CHANNEL, ADD, DROP, SUSPEND, RESUME, REFRESH, MODIFY queries, the list of live views is stored here + */ + ASTPtr values; + bool detach = false; /// true for DETACH PARTITION bool part = false; /// true for ATTACH PART and DROP DETACHED PART From 1cc8bf7fe040afe05978bec43fab525bd85e68d7 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Fri, 23 Aug 2019 12:11:43 +0300 Subject: [PATCH 380/509] DOCAPI-7783: RU Translation (#6623) --- docs/ru/query_language/misc.md | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/docs/ru/query_language/misc.md b/docs/ru/query_language/misc.md index 93f548bf73c..e366abf2f3d 100644 --- a/docs/ru/query_language/misc.md +++ b/docs/ru/query_language/misc.md @@ -193,18 +193,21 @@ RENAME TABLE [db11.]name11 TO [db12.]name12, [db21.]name21 TO [db22.]name22, ... Все таблицы переименовываются под глобальной блокировкой. Переименовывание таблицы является лёгкой операцией. Если вы указали после TO другую базу данных, то таблица будет перенесена в эту базу данных. При этом, директории с базами данных должны быть расположены в одной файловой системе (иначе возвращается ошибка). -## SET +## SET {#query-set} ```sql SET param = value ``` -Позволяет установить настройку `param` в значение `value`. Также можно одним запросом установить все настройки из заданного профиля настроек. Для этого укажите 'profile' в качестве имени настройки. Подробнее смотрите в разделе "Настройки". -Настройка устанавливается на сессию, или на сервер (глобально), если указано `GLOBAL`. -При установке глобальных настроек, эти настройки не применяются к уже запущенной сессии, включая текущую сессию. Она будет использована только для новых сессий. +Устанавливает значение `value` для настройки `param` в текущей сессии. [Конфигурационные параметры сервера](../operations/server_settings/index.md) нельзя изменить подобным образом. -При перезапуске сервера теряются настройки, установленные с помощью `SET`. -Установить настройки, которые переживут перезапуск сервера, можно только с помощью конфигурационного файла сервера. +Можно одним запросом установить все настройки из заданного профиля настроек. + +```sql +SET profile = 'profile-name-from-the-settings-file' +``` + +Подробности смотрите в разделе [Настройки](../operations/settings/settings.md). ## SHOW CREATE TABLE From baf121c864550cdcafdcd70b9b2ab36afc7e9c53 Mon Sep 17 00:00:00 2001 From: Winter Zhang Date: Fri, 23 Aug 2019 18:14:19 +0800 Subject: [PATCH 381/509] Translate database engine documentation(zh) (#6625) --- docs/zh/database_engines/index.md | 12 +- docs/zh/database_engines/mysql.md | 125 +++++++++++++++++- docs/zh/operations/table_engines/mergetree.md | 2 +- 3 files changed, 136 insertions(+), 3 deletions(-) diff --git a/docs/zh/database_engines/index.md b/docs/zh/database_engines/index.md index bbdb762a4ad..f8ae05e2520 120000 --- a/docs/zh/database_engines/index.md +++ b/docs/zh/database_engines/index.md @@ -1 +1,11 @@ -../../en/database_engines/index.md \ No newline at end of file +# 数据库引擎 + +您使用的所有表都是由数据库引擎所提供的 + +默认情况下,ClickHouse使用自己的数据库引擎,该引擎提供可配置的[表引擎](../operations/table_engines/index.md)和[所有支持的SQL语法](../query_language/syntax.md). + +除此之外,您还可以选择使用以下的数据库引擎: + +- [MySQL](mysql.md) + +[来源文章](https://clickhouse.yandex/docs/en/database_engines/) diff --git a/docs/zh/database_engines/mysql.md b/docs/zh/database_engines/mysql.md index 51ac4126e2d..38dfcb5ef64 120000 --- a/docs/zh/database_engines/mysql.md +++ b/docs/zh/database_engines/mysql.md @@ -1 +1,124 @@ -../../en/database_engines/mysql.md \ No newline at end of file +# MySQL + +MySQL引擎用于将远程的MySQL服务器中的表映射到ClickHouse中,并允许您对表进行`INSERT`和`SELECT`查询,以方便您在ClickHouse与MySQL之间进行数据交换。 + +`MySQL`数据库引擎会将对其的查询转换为MySQL语法并发送到MySQL服务器中,因此您可以执行诸如`SHOW TABLES`或`SHOW CREATE TABLE`之类的操作。 + +但您无法对其执行以下操作: + +- `ATTACH`/`DETACH` +- `DROP` +- `RENAME` +- `CREATE TABLE` +- `ALTER` + + +## CREATE DATABASE + +``` sql +CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] +ENGINE = MySQL('host:port', 'database', 'user', 'password') +``` + +**MySQL数据库引擎参数** + +- `host:port` — 链接的MySQL地址。 +- `database` — 链接的MySQL数据库。 +- `user` — 链接的MySQL用户。 +- `password` — 链接的MySQL用户密码。 + + +## 支持的类型对应 + +MySQL | ClickHouse +------|------------ +UNSIGNED TINYINT | [UInt8](../data_types/int_uint.md) +TINYINT | [Int8](../data_types/int_uint.md) +UNSIGNED SMALLINT | [UInt16](../data_types/int_uint.md) +SMALLINT | [Int16](../data_types/int_uint.md) +UNSIGNED INT, UNSIGNED MEDIUMINT | [UInt32](../data_types/int_uint.md) +INT, MEDIUMINT | [Int32](../data_types/int_uint.md) +UNSIGNED BIGINT | [UInt64](../data_types/int_uint.md) +BIGINT | [Int64](../data_types/int_uint.md) +FLOAT | [Float32](../data_types/float.md) +DOUBLE | [Float64](../data_types/float.md) +DATE | [Date](../data_types/date.md) +DATETIME, TIMESTAMP | [DateTime](../data_types/datetime.md) +BINARY | [FixedString](../data_types/fixedstring.md) + +其他的MySQL数据类型将全部都转换为[String](../data_types/string.md)。 + +同时以上的所有类型都支持[Nullable](../data_types/nullable.md)。 + + +## 使用示例 + +在MySQL中创建表: + +``` +mysql> USE test; +Database changed + +mysql> CREATE TABLE `mysql_table` ( + -> `int_id` INT NOT NULL AUTO_INCREMENT, + -> `float` FLOAT NOT NULL, + -> PRIMARY KEY (`int_id`)); +Query OK, 0 rows affected (0,09 sec) + +mysql> insert into mysql_table (`int_id`, `float`) VALUES (1,2); +Query OK, 1 row affected (0,00 sec) + +mysql> select * from mysql_table; ++--------+-------+ +| int_id | value | ++--------+-------+ +| 1 | 2 | ++--------+-------+ +1 row in set (0,00 sec) +``` + +在ClickHouse中创建MySQL类型的数据库,同时与MySQL服务器交换数据: + +```sql +CREATE DATABASE mysql_db ENGINE = MySQL('localhost:3306', 'test', 'my_user', 'user_password') +``` +```sql +SHOW DATABASES +``` +```text +┌─name─────┐ +│ default │ +│ mysql_db │ +│ system │ +└──────────┘ +``` +```sql +SHOW TABLES FROM mysql_db +``` +```text +┌─name─────────┐ +│ mysql_table │ +└──────────────┘ +``` +```sql +SELECT * FROM mysql_db.mysql_table +``` +```text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +└────────┴───────┘ +``` +```sql +INSERT INTO mysql_db.mysql_table VALUES (3,4) +``` +```sql +SELECT * FROM mysql_db.mysql_table +``` +```text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +│ 3 │ 4 │ +└────────┴───────┘ +``` + +[来源文章](https://clickhouse.yandex/docs/en/database_engines/mysql/) diff --git a/docs/zh/operations/table_engines/mergetree.md b/docs/zh/operations/table_engines/mergetree.md index 5ddf837708a..5e330164c5a 100644 --- a/docs/zh/operations/table_engines/mergetree.md +++ b/docs/zh/operations/table_engines/mergetree.md @@ -48,7 +48,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] **子句** -- `ENGINE` - 引擎名和参数。 `ENGINE = MergeTree()`. `MergeTree` 引擎没有参数。 +- `ENGINE` - 引擎名和参数。 `ENGINE = MergeTree()`。 `MergeTree` 引擎不需要其他参数。 - `PARTITION BY` — [分区键](custom_partitioning_key.md) 。 From a367f94923c13445d5ed46d4bfe58d7b94950b69 Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Fri, 23 Aug 2019 16:19:12 +0300 Subject: [PATCH 382/509] Fix after review --- dbms/programs/benchmark/Benchmark.cpp | 3 ++- dbms/programs/client/Client.cpp | 16 +++++------ dbms/programs/compressor/Compressor.cpp | 5 ++-- dbms/programs/format/Format.cpp | 4 +-- dbms/programs/local/LocalServer.cpp | 13 ++------- dbms/programs/obfuscator/Obfuscator.cpp | 3 ++- .../performance-test/PerformanceTestSuite.cpp | 4 +-- ...nsDescription.h => TerminalDisplaying.cpp} | 27 ++++++++++--------- dbms/src/Common/TerminalDisplaying.h | 17 ++++++++++++ 9 files changed, 52 insertions(+), 40 deletions(-) rename dbms/src/Common/{SetOptionsDescription.h => TerminalDisplaying.cpp} (55%) create mode 100644 dbms/src/Common/TerminalDisplaying.h diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index c69e9a54feb..7fc33559b54 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -32,6 +32,7 @@ #include #include #include +#include /** A tool for evaluating ClickHouse performance. @@ -439,7 +440,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) { using boost::program_options::value; - boost::program_options::options_description desc("Allowed options"); + boost::program_options::options_description desc = setOptionsDescription("Allowed options", getTerminalWidth()); desc.add_options() ("help", "produce help message") ("concurrency,c", value()->default_value(1), "number of parallel queries") diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 726a607812c..f77ddbea827 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -67,7 +67,7 @@ #include #include #include -#include +#include #if USE_READLINE #include "Suggest.h" @@ -131,7 +131,7 @@ private: bool print_time_to_stderr = false; /// Output execution time to stderr in batch mode. bool stdin_is_not_tty = false; /// stdin is not a terminal. - winsize terminal_size {}; /// Terminal size is needed to render progress bar. + unsigned short int terminal_width; /// Terminal width is needed to render progress bar. std::unique_ptr connection; /// Connection to DB. String query_id; /// Current query_id. @@ -1466,7 +1466,7 @@ private: if (show_progress_bar) { - ssize_t width_of_progress_bar = static_cast(terminal_size.ws_col) - written_progress_chars - strlen(" 99%"); + ssize_t width_of_progress_bar = static_cast(terminal_width) - written_progress_chars - strlen(" 99%"); if (width_of_progress_bar > 0) { std::string bar = UnicodeBar::render(UnicodeBar::getWidth(progress.read_rows, 0, total_rows_corrected, width_of_progress_bar)); @@ -1642,16 +1642,14 @@ public: } stdin_is_not_tty = !isatty(STDIN_FILENO); + if (!stdin_is_not_tty) - { - if (ioctl(STDIN_FILENO, TIOCGWINSZ, &terminal_size)) - throwFromErrno("Cannot obtain terminal window size (ioctl TIOCGWINSZ)", ErrorCodes::SYSTEM_ERROR); - } + terminal_width = getTerminalWidth(); namespace po = boost::program_options; /// Main commandline options related to client functionality and all parameters from Settings. - po::options_description main_description = setOptionsDescription("Main options"); + po::options_description main_description = setOptionsDescription("Main options", terminal_width); main_description.add_options() ("help", "produce help message") ("config-file,C", po::value(), "config-file path") @@ -1697,7 +1695,7 @@ public: context.getSettingsRef().addProgramOptions(main_description); /// Commandline options related to external tables. - po::options_description external_description("External tables options"); + po::options_description external_description = setOptionsDescription("External tables options", terminal_width); external_description.add_options() ("file", po::value(), "data file or - for stdin") ("name", po::value()->default_value("_data"), "name of the table") diff --git a/dbms/programs/compressor/Compressor.cpp b/dbms/programs/compressor/Compressor.cpp index 427d58cbdc6..f990b3ac690 100644 --- a/dbms/programs/compressor/Compressor.cpp +++ b/dbms/programs/compressor/Compressor.cpp @@ -12,8 +12,9 @@ #include #include #include - #include +#include + namespace DB { @@ -59,7 +60,7 @@ void checkAndWriteHeader(DB::ReadBuffer & in, DB::WriteBuffer & out) int mainEntryClickHouseCompressor(int argc, char ** argv) { - boost::program_options::options_description desc("Allowed options"); + boost::program_options::options_description desc = setOptionsDescription("Allowed options", getTerminalWidth()); desc.add_options() ("help,h", "produce help message") ("decompress,d", "decompress") diff --git a/dbms/programs/format/Format.cpp b/dbms/programs/format/Format.cpp index b7e2629df16..fbc17e44e63 100644 --- a/dbms/programs/format/Format.cpp +++ b/dbms/programs/format/Format.cpp @@ -6,13 +6,13 @@ #include #include #include - +#include int mainEntryClickHouseFormat(int argc, char ** argv) { using namespace DB; - boost::program_options::options_description desc("Allowed options"); + boost::program_options::options_description desc = setOptionsDescription("Allowed options", getTerminalWidth()); desc.add_options() ("help,h", "produce help message") ("hilite", "add syntax highlight with ANSI terminal escape sequences") diff --git a/dbms/programs/local/LocalServer.cpp b/dbms/programs/local/LocalServer.cpp index bed55a0fc5f..c894e13ed47 100644 --- a/dbms/programs/local/LocalServer.cpp +++ b/dbms/programs/local/LocalServer.cpp @@ -35,6 +35,7 @@ #include #include #include +#include namespace DB @@ -409,17 +410,7 @@ void LocalServer::init(int argc, char ** argv) /// Don't parse options with Poco library, we prefer neat boost::program_options stopOptionsProcessing(); - unsigned line_length = po::options_description::m_default_line_length; - unsigned min_description_length = line_length / 2; - if (isatty(STDIN_FILENO)) - { - winsize terminal_size{}; - ioctl(0, TIOCGWINSZ, &terminal_size); - line_length = std::max(3U, static_cast(terminal_size.ws_col)); - min_description_length = std::min(min_description_length, line_length - 2); - } - - po::options_description description("Main options", line_length, min_description_length); + po::options_description description = setOptionsDescription("Main options", getTerminalWidth()); description.add_options() ("help", "produce help message") ("config-file,c", po::value(), "config-file path") diff --git a/dbms/programs/obfuscator/Obfuscator.cpp b/dbms/programs/obfuscator/Obfuscator.cpp index a96c10072dc..a2cae4ed1a5 100644 --- a/dbms/programs/obfuscator/Obfuscator.cpp +++ b/dbms/programs/obfuscator/Obfuscator.cpp @@ -36,6 +36,7 @@ #include #include #include +#include static const char * documantation = R"( @@ -948,7 +949,7 @@ try using namespace DB; namespace po = boost::program_options; - po::options_description description("Options"); + po::options_description description = setOptionsDescription("Options", getTerminalWidth()); description.add_options() ("help", "produce help message") ("structure,S", po::value(), "structure of the initial table (list of column and type names)") diff --git a/dbms/programs/performance-test/PerformanceTestSuite.cpp b/dbms/programs/performance-test/PerformanceTestSuite.cpp index 99408c72f37..37a9e61d169 100644 --- a/dbms/programs/performance-test/PerformanceTestSuite.cpp +++ b/dbms/programs/performance-test/PerformanceTestSuite.cpp @@ -28,7 +28,7 @@ #include #include #include -#include +#include #include "TestStopConditions.h" #include "TestStats.h" @@ -325,7 +325,7 @@ try using po::value; using Strings = DB::Strings; - po::options_description desc = setOptionsDescription("Allowed options"); + po::options_description desc = setOptionsDescription("Allowed options", getTerminalWidth()); desc.add_options() ("help", "produce help message") ("lite", "use lite version of output") diff --git a/dbms/src/Common/SetOptionsDescription.h b/dbms/src/Common/TerminalDisplaying.cpp similarity index 55% rename from dbms/src/Common/SetOptionsDescription.h rename to dbms/src/Common/TerminalDisplaying.cpp index a9c4031b064..d9a8f5ebfa9 100644 --- a/dbms/src/Common/SetOptionsDescription.h +++ b/dbms/src/Common/TerminalDisplaying.cpp @@ -1,8 +1,7 @@ #include #include #include -#include - +#include namespace po = boost::program_options; @@ -12,11 +11,8 @@ namespace DB::ErrorCodes extern const int SYSTEM_ERROR; } -static po::options_description setOptionsDescription(const std::string & caption) +unsigned short int getTerminalWidth() { - unsigned line_length = po::options_description::m_default_line_length; - unsigned min_description_length = line_length / 2; - if (isatty(STDIN_FILENO)) { winsize terminal_size {}; @@ -24,12 +20,19 @@ static po::options_description setOptionsDescription(const std::string & caption if (ioctl(STDIN_FILENO, TIOCGWINSZ, &terminal_size)) DB::throwFromErrno("Cannot obtain terminal window size (ioctl TIOCGWINSZ)", DB::ErrorCodes::SYSTEM_ERROR); - std::string longest_option_desc = "--http_native_compression_disable_checksumming_on_decompress"; - - line_length = std::max(static_cast(longest_option_desc.size()), terminal_size.ws_col); - - min_description_length = std::min(min_description_length, line_length - 2); + return terminal_size.ws_col; } + return 0; +} + +po::options_description setOptionsDescription(const std::string & caption, unsigned short terminal_width) +{ + unsigned line_length = po::options_description::m_default_line_length; + unsigned min_description_length = line_length / 2; + std::string longest_option_desc = "--http_native_compression_disable_checksumming_on_decompress"; + + line_length = std::max(static_cast(longest_option_desc.size()), terminal_width); + min_description_length = std::min(min_description_length, line_length - 2); return po::options_description(caption, line_length, min_description_length); -} +} \ No newline at end of file diff --git a/dbms/src/Common/TerminalDisplaying.h b/dbms/src/Common/TerminalDisplaying.h new file mode 100644 index 00000000000..598969b62c7 --- /dev/null +++ b/dbms/src/Common/TerminalDisplaying.h @@ -0,0 +1,17 @@ +#pragma once + +#include +#include +#include + + +namespace po = boost::program_options; + + +unsigned short int getTerminalWidth(); + +/** Sets a name and an appropriate size for option displaying + * when program is called with option --help + * */ +po::options_description setOptionsDescription(const std::string & caption, unsigned short terminal_width); + From 6e098eb953c29dfb51fd81b35f06a4ff625ef52b Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Fri, 23 Aug 2019 16:25:46 +0300 Subject: [PATCH 383/509] New line added :) --- dbms/src/Common/TerminalDisplaying.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/TerminalDisplaying.cpp b/dbms/src/Common/TerminalDisplaying.cpp index d9a8f5ebfa9..243ba7f8288 100644 --- a/dbms/src/Common/TerminalDisplaying.cpp +++ b/dbms/src/Common/TerminalDisplaying.cpp @@ -35,4 +35,4 @@ po::options_description setOptionsDescription(const std::string & caption, unsig min_description_length = std::min(min_description_length, line_length - 2); return po::options_description(caption, line_length, min_description_length); -} \ No newline at end of file +} From d001c7e554ccedfba05b271a91687e918ab4e6d3 Mon Sep 17 00:00:00 2001 From: Alex Ryndin Date: Fri, 23 Aug 2019 17:27:07 +0300 Subject: [PATCH 384/509] Fix typo (#6631) --- docs/en/operations/settings/query_complexity.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/query_complexity.md b/docs/en/operations/settings/query_complexity.md index 77699c868b4..c00f2132ebd 100644 --- a/docs/en/operations/settings/query_complexity.md +++ b/docs/en/operations/settings/query_complexity.md @@ -79,7 +79,7 @@ Enables or disables execution of `GROUP BY` clauses in external memory. See [GRO Possible values: -- Maximum volume or RAM (in bytes) that can be used by the single [GROUP BY](../../query_language/select.md#select-group-by-clause) operation. +- Maximum volume of RAM (in bytes) that can be used by the single [GROUP BY](../../query_language/select.md#select-group-by-clause) operation. - 0 — `GROUP BY` in external memory disabled. Default value: 0. From 9e6c629134b14952a7c3b143809ffdf75bee8b0b Mon Sep 17 00:00:00 2001 From: Dmitry Rubashkin Date: Fri, 23 Aug 2019 17:54:10 +0300 Subject: [PATCH 385/509] Style fixed --- dbms/src/Common/T_test.h | 206 ++++++++++++++++++++------------------- 1 file changed, 104 insertions(+), 102 deletions(-) diff --git a/dbms/src/Common/T_test.h b/dbms/src/Common/T_test.h index 456ed464e8f..1160fd0805e 100644 --- a/dbms/src/Common/T_test.h +++ b/dbms/src/Common/T_test.h @@ -50,110 +50,112 @@ struct T_test std::vector data; - const std::vector> students_table = + /// First row corresponds to infinity size of distributions case + const double students_table[101][6] = { - /* inf */ { 1.282, 1.645, 1.960, 2.326, 2.576, 3.090 }, - /* 1. */ { 3.078, 6.314, 12.706, 31.821, 63.657, 318.313}, - /* 2. */ { 1.886, 2.920, 4.303, 6.965, 9.925, 22.327 }, - /* 3. */ { 1.638, 2.353, 3.182, 4.541, 5.841, 10.215 }, - /* 4. */ { 1.533, 2.132, 2.776, 3.747, 4.604, 7.173 }, - /* 5. */ { 1.476, 2.015, 2.571, 3.365, 4.032, 5.893 }, - /* 6. */ { 1.440, 1.943, 2.447, 3.143, 3.707, 5.208 }, - /* 7. */ { 1.415, 1.895, 2.365, 2.998, 3.499, 4.782 }, - /* 8. */ { 1.397, 1.860, 2.306, 2.896, 3.355, 4.499 }, - /* 9. */ { 1.383, 1.833, 2.262, 2.821, 3.250, 4.296 }, - /* 10. */ { 1.372, 1.812, 2.228, 2.764, 3.169, 4.143 }, - /* 11. */ { 1.363, 1.796, 2.201, 2.718, 3.106, 4.024 }, - /* 12. */ { 1.356, 1.782, 2.179, 2.681, 3.055, 3.929 }, - /* 13. */ { 1.350, 1.771, 2.160, 2.650, 3.012, 3.852 }, - /* 14. */ { 1.345, 1.761, 2.145, 2.624, 2.977, 3.787 }, - /* 15. */ { 1.341, 1.753, 2.131, 2.602, 2.947, 3.733 }, - /* 16. */ { 1.337, 1.746, 2.120, 2.583, 2.921, 3.686 }, - /* 17. */ { 1.333, 1.740, 2.110, 2.567, 2.898, 3.646 }, - /* 18. */ { 1.330, 1.734, 2.101, 2.552, 2.878, 3.610 }, - /* 19. */ { 1.328, 1.729, 2.093, 2.539, 2.861, 3.579 }, - /* 20. */ { 1.325, 1.725, 2.086, 2.528, 2.845, 3.552 }, - /* 21. */ { 1.323, 1.721, 2.080, 2.518, 2.831, 3.527 }, - /* 22. */ { 1.321, 1.717, 2.074, 2.508, 2.819, 3.505 }, - /* 23. */ { 1.319, 1.714, 2.069, 2.500, 2.807, 3.485 }, - /* 24. */ { 1.318, 1.711, 2.064, 2.492, 2.797, 3.467 }, - /* 25. */ { 1.316, 1.708, 2.060, 2.485, 2.787, 3.450 }, - /* 26. */ { 1.315, 1.706, 2.056, 2.479, 2.779, 3.435 }, - /* 27. */ { 1.314, 1.703, 2.052, 2.473, 2.771, 3.421 }, - /* 28. */ { 1.313, 1.701, 2.048, 2.467, 2.763, 3.408 }, - /* 29. */ { 1.311, 1.699, 2.045, 2.462, 2.756, 3.396 }, - /* 30. */ { 1.310, 1.697, 2.042, 2.457, 2.750, 3.385 }, - /* 31. */ { 1.309, 1.696, 2.040, 2.453, 2.744, 3.375 }, - /* 32. */ { 1.309, 1.694, 2.037, 2.449, 2.738, 3.365 }, - /* 33. */ { 1.308, 1.692, 2.035, 2.445, 2.733, 3.356 }, - /* 34. */ { 1.307, 1.691, 2.032, 2.441, 2.728, 3.348 }, - /* 35. */ { 1.306, 1.690, 2.030, 2.438, 2.724, 3.340 }, - /* 36. */ { 1.306, 1.688, 2.028, 2.434, 2.719, 3.333 }, - /* 37. */ { 1.305, 1.687, 2.026, 2.431, 2.715, 3.326 }, - /* 38. */ { 1.304, 1.686, 2.024, 2.429, 2.712, 3.319 }, - /* 39. */ { 1.304, 1.685, 2.023, 2.426, 2.708, 3.313 }, - /* 40. */ { 1.303, 1.684, 2.021, 2.423, 2.704, 3.307 }, - /* 41. */ { 1.303, 1.683, 2.020, 2.421, 2.701, 3.301 }, - /* 42. */ { 1.302, 1.682, 2.018, 2.418, 2.698, 3.296 }, - /* 43. */ { 1.302, 1.681, 2.017, 2.416, 2.695, 3.291 }, - /* 44. */ { 1.301, 1.680, 2.015, 2.414, 2.692, 3.286 }, - /* 45. */ { 1.301, 1.679, 2.014, 2.412, 2.690, 3.281 }, - /* 46. */ { 1.300, 1.679, 2.013, 2.410, 2.687, 3.277 }, - /* 47. */ { 1.300, 1.678, 2.012, 2.408, 2.685, 3.273 }, - /* 48. */ { 1.299, 1.677, 2.011, 2.407, 2.682, 3.269 }, - /* 49. */ { 1.299, 1.677, 2.010, 2.405, 2.680, 3.265 }, - /* 50. */ { 1.299, 1.676, 2.009, 2.403, 2.678, 3.261 }, - /* 51. */ { 1.298, 1.675, 2.008, 2.402, 2.676, 3.258 }, - /* 52. */ { 1.298, 1.675, 2.007, 2.400, 2.674, 3.255 }, - /* 53. */ { 1.298, 1.674, 2.006, 2.399, 2.672, 3.251 }, - /* 54. */ { 1.297, 1.674, 2.005, 2.397, 2.670, 3.248 }, - /* 55. */ { 1.297, 1.673, 2.004, 2.396, 2.668, 3.245 }, - /* 56. */ { 1.297, 1.673, 2.003, 2.395, 2.667, 3.242 }, - /* 57. */ { 1.297, 1.672, 2.002, 2.394, 2.665, 3.239 }, - /* 58. */ { 1.296, 1.672, 2.002, 2.392, 2.663, 3.237 }, - /* 59. */ { 1.296, 1.671, 2.001, 2.391, 2.662, 3.234 }, - /* 60. */ { 1.296, 1.671, 2.000, 2.390, 2.660, 3.232 }, - /* 61. */ { 1.296, 1.670, 2.000, 2.389, 2.659, 3.229 }, - /* 62. */ { 1.295, 1.670, 1.999, 2.388, 2.657, 3.227 }, - /* 63. */ { 1.295, 1.669, 1.998, 2.387, 2.656, 3.225 }, - /* 64. */ { 1.295, 1.669, 1.998, 2.386, 2.655, 3.223 }, - /* 65. */ { 1.295, 1.669, 1.997, 2.385, 2.654, 3.220 }, - /* 66. */ { 1.295, 1.668, 1.997, 2.384, 2.652, 3.218 }, - /* 67. */ { 1.294, 1.668, 1.996, 2.383, 2.651, 3.216 }, - /* 68. */ { 1.294, 1.668, 1.995, 2.382, 2.650, 3.214 }, - /* 69. */ { 1.294, 1.667, 1.995, 2.382, 2.649, 3.213 }, - /* 70. */ { 1.294, 1.667, 1.994, 2.381, 2.648, 3.211 }, - /* 71. */ { 1.294, 1.667, 1.994, 2.380, 2.647, 3.209 }, - /* 72. */ { 1.293, 1.666, 1.993, 2.379, 2.646, 3.207 }, - /* 73. */ { 1.293, 1.666, 1.993, 2.379, 2.645, 3.206 }, - /* 74. */ { 1.293, 1.666, 1.993, 2.378, 2.644, 3.204 }, - /* 75. */ { 1.293, 1.665, 1.992, 2.377, 2.643, 3.202 }, - /* 76. */ { 1.293, 1.665, 1.992, 2.376, 2.642, 3.201 }, - /* 77. */ { 1.293, 1.665, 1.991, 2.376, 2.641, 3.199 }, - /* 78. */ { 1.292, 1.665, 1.991, 2.375, 2.640, 3.198 }, - /* 79. */ { 1.292, 1.664, 1.990, 2.374, 2.640, 3.197 }, - /* 80. */ { 1.292, 1.664, 1.990, 2.374, 2.639, 3.195 }, - /* 81. */ { 1.292, 1.664, 1.990, 2.373, 2.638, 3.194 }, - /* 82. */ { 1.292, 1.664, 1.989, 2.373, 2.637, 3.193 }, - /* 83. */ { 1.292, 1.663, 1.989, 2.372, 2.636, 3.191 }, - /* 84. */ { 1.292, 1.663, 1.989, 2.372, 2.636, 3.190 }, - /* 85. */ { 1.292, 1.663, 1.988, 2.371, 2.635, 3.189 }, - /* 86. */ { 1.291, 1.663, 1.988, 2.370, 2.634, 3.188 }, - /* 87. */ { 1.291, 1.663, 1.988, 2.370, 2.634, 3.187 }, - /* 88. */ { 1.291, 1.662, 1.987, 2.369, 2.633, 3.185 }, - /* 89. */ { 1.291, 1.662, 1.987, 2.369, 2.632, 3.184 }, - /* 90. */ { 1.291, 1.662, 1.987, 2.368, 2.632, 3.183 }, - /* 91. */ { 1.291, 1.662, 1.986, 2.368, 2.631, 3.182 }, - /* 92. */ { 1.291, 1.662, 1.986, 2.368, 2.630, 3.181 }, - /* 93. */ { 1.291, 1.661, 1.986, 2.367, 2.630, 3.180 }, - /* 94. */ { 1.291, 1.661, 1.986, 2.367, 2.629, 3.179 }, - /* 95. */ { 1.291, 1.661, 1.985, 2.366, 2.629, 3.178 }, - /* 96. */ { 1.290, 1.661, 1.985, 2.366, 2.628, 3.177 }, - /* 97. */ { 1.290, 1.661, 1.985, 2.365, 2.627, 3.176 }, - /* 98. */ { 1.290, 1.661, 1.984, 2.365, 2.627, 3.175 }, - /* 99. */ { 1.290, 1.660, 1.984, 2.365, 2.626, 3.175 }, - /* 100. */ { 1.290, 1.660, 1.984, 2.364, 2.626, 3.174 } + { 1.282, 1.645, 1.960, 2.326, 2.576, 3.090 }, + { 3.078, 6.314, 12.706, 31.821, 63.657, 318.313 }, + { 1.886, 2.920, 4.303, 6.965, 9.925, 22.327 }, + { 1.638, 2.353, 3.182, 4.541, 5.841, 10.215 }, + { 1.533, 2.132, 2.776, 3.747, 4.604, 7.173 }, + { 1.476, 2.015, 2.571, 3.365, 4.032, 5.893 }, + { 1.440, 1.943, 2.447, 3.143, 3.707, 5.208 }, + { 1.415, 1.895, 2.365, 2.998, 3.499, 4.782 }, + { 1.397, 1.860, 2.306, 2.896, 3.355, 4.499 }, + { 1.383, 1.833, 2.262, 2.821, 3.250, 4.296 }, + { 1.372, 1.812, 2.228, 2.764, 3.169, 4.143 }, + { 1.363, 1.796, 2.201, 2.718, 3.106, 4.024 }, + { 1.356, 1.782, 2.179, 2.681, 3.055, 3.929 }, + { 1.350, 1.771, 2.160, 2.650, 3.012, 3.852 }, + { 1.345, 1.761, 2.145, 2.624, 2.977, 3.787 }, + { 1.341, 1.753, 2.131, 2.602, 2.947, 3.733 }, + { 1.337, 1.746, 2.120, 2.583, 2.921, 3.686 }, + { 1.333, 1.740, 2.110, 2.567, 2.898, 3.646 }, + { 1.330, 1.734, 2.101, 2.552, 2.878, 3.610 }, + { 1.328, 1.729, 2.093, 2.539, 2.861, 3.579 }, + { 1.325, 1.725, 2.086, 2.528, 2.845, 3.552 }, + { 1.323, 1.721, 2.080, 2.518, 2.831, 3.527 }, + { 1.321, 1.717, 2.074, 2.508, 2.819, 3.505 }, + { 1.319, 1.714, 2.069, 2.500, 2.807, 3.485 }, + { 1.318, 1.711, 2.064, 2.492, 2.797, 3.467 }, + { 1.316, 1.708, 2.060, 2.485, 2.787, 3.450 }, + { 1.315, 1.706, 2.056, 2.479, 2.779, 3.435 }, + { 1.314, 1.703, 2.052, 2.473, 2.771, 3.421 }, + { 1.313, 1.701, 2.048, 2.467, 2.763, 3.408 }, + { 1.311, 1.699, 2.045, 2.462, 2.756, 3.396 }, + { 1.310, 1.697, 2.042, 2.457, 2.750, 3.385 }, + { 1.309, 1.696, 2.040, 2.453, 2.744, 3.375 }, + { 1.309, 1.694, 2.037, 2.449, 2.738, 3.365 }, + { 1.308, 1.692, 2.035, 2.445, 2.733, 3.356 }, + { 1.307, 1.691, 2.032, 2.441, 2.728, 3.348 }, + { 1.306, 1.690, 2.030, 2.438, 2.724, 3.340 }, + { 1.306, 1.688, 2.028, 2.434, 2.719, 3.333 }, + { 1.305, 1.687, 2.026, 2.431, 2.715, 3.326 }, + { 1.304, 1.686, 2.024, 2.429, 2.712, 3.319 }, + { 1.304, 1.685, 2.023, 2.426, 2.708, 3.313 }, + { 1.303, 1.684, 2.021, 2.423, 2.704, 3.307 }, + { 1.303, 1.683, 2.020, 2.421, 2.701, 3.301 }, + { 1.302, 1.682, 2.018, 2.418, 2.698, 3.296 }, + { 1.302, 1.681, 2.017, 2.416, 2.695, 3.291 }, + { 1.301, 1.680, 2.015, 2.414, 2.692, 3.286 }, + { 1.301, 1.679, 2.014, 2.412, 2.690, 3.281 }, + { 1.300, 1.679, 2.013, 2.410, 2.687, 3.277 }, + { 1.300, 1.678, 2.012, 2.408, 2.685, 3.273 }, + { 1.299, 1.677, 2.011, 2.407, 2.682, 3.269 }, + { 1.299, 1.677, 2.010, 2.405, 2.680, 3.265 }, + { 1.299, 1.676, 2.009, 2.403, 2.678, 3.261 }, + { 1.298, 1.675, 2.008, 2.402, 2.676, 3.258 }, + { 1.298, 1.675, 2.007, 2.400, 2.674, 3.255 }, + { 1.298, 1.674, 2.006, 2.399, 2.672, 3.251 }, + { 1.297, 1.674, 2.005, 2.397, 2.670, 3.248 }, + { 1.297, 1.673, 2.004, 2.396, 2.668, 3.245 }, + { 1.297, 1.673, 2.003, 2.395, 2.667, 3.242 }, + { 1.297, 1.672, 2.002, 2.394, 2.665, 3.239 }, + { 1.296, 1.672, 2.002, 2.392, 2.663, 3.237 }, + { 1.296, 1.671, 2.001, 2.391, 2.662, 3.234 }, + { 1.296, 1.671, 2.000, 2.390, 2.660, 3.232 }, + { 1.296, 1.670, 2.000, 2.389, 2.659, 3.229 }, + { 1.295, 1.670, 1.999, 2.388, 2.657, 3.227 }, + { 1.295, 1.669, 1.998, 2.387, 2.656, 3.225 }, + { 1.295, 1.669, 1.998, 2.386, 2.655, 3.223 }, + { 1.295, 1.669, 1.997, 2.385, 2.654, 3.220 }, + { 1.295, 1.668, 1.997, 2.384, 2.652, 3.218 }, + { 1.294, 1.668, 1.996, 2.383, 2.651, 3.216 }, + { 1.294, 1.668, 1.995, 2.382, 2.650, 3.214 }, + { 1.294, 1.667, 1.995, 2.382, 2.649, 3.213 }, + { 1.294, 1.667, 1.994, 2.381, 2.648, 3.211 }, + { 1.294, 1.667, 1.994, 2.380, 2.647, 3.209 }, + { 1.293, 1.666, 1.993, 2.379, 2.646, 3.207 }, + { 1.293, 1.666, 1.993, 2.379, 2.645, 3.206 }, + { 1.293, 1.666, 1.993, 2.378, 2.644, 3.204 }, + { 1.293, 1.665, 1.992, 2.377, 2.643, 3.202 }, + { 1.293, 1.665, 1.992, 2.376, 2.642, 3.201 }, + { 1.293, 1.665, 1.991, 2.376, 2.641, 3.199 }, + { 1.292, 1.665, 1.991, 2.375, 2.640, 3.198 }, + { 1.292, 1.664, 1.990, 2.374, 2.640, 3.197 }, + { 1.292, 1.664, 1.990, 2.374, 2.639, 3.195 }, + { 1.292, 1.664, 1.990, 2.373, 2.638, 3.194 }, + { 1.292, 1.664, 1.989, 2.373, 2.637, 3.193 }, + { 1.292, 1.663, 1.989, 2.372, 2.636, 3.191 }, + { 1.292, 1.663, 1.989, 2.372, 2.636, 3.190 }, + { 1.292, 1.663, 1.988, 2.371, 2.635, 3.189 }, + { 1.291, 1.663, 1.988, 2.370, 2.634, 3.188 }, + { 1.291, 1.663, 1.988, 2.370, 2.634, 3.187 }, + { 1.291, 1.662, 1.987, 2.369, 2.633, 3.185 }, + { 1.291, 1.662, 1.987, 2.369, 2.632, 3.184 }, + { 1.291, 1.662, 1.987, 2.368, 2.632, 3.183 }, + { 1.291, 1.662, 1.986, 2.368, 2.631, 3.182 }, + { 1.291, 1.662, 1.986, 2.368, 2.630, 3.181 }, + { 1.291, 1.661, 1.986, 2.367, 2.630, 3.180 }, + { 1.291, 1.661, 1.986, 2.367, 2.629, 3.179 }, + { 1.291, 1.661, 1.985, 2.366, 2.629, 3.178 }, + { 1.290, 1.661, 1.985, 2.366, 2.628, 3.177 }, + { 1.290, 1.661, 1.985, 2.365, 2.627, 3.176 }, + { 1.290, 1.661, 1.984, 2.365, 2.627, 3.175 }, + { 1.290, 1.660, 1.984, 2.365, 2.626, 3.175 }, + { 1.290, 1.660, 1.984, 2.364, 2.626, 3.174 }, }; + const std::vector confidence_level = { 80, 90, 95, 98, 99, 99.5 }; T_test() From f757de462584ec8cad5d6cba4c73f3fe2b480324 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Aug 2019 17:56:38 +0300 Subject: [PATCH 386/509] Revert "Translate database engine documentation(zh) (#6625)" This reverts commit baf121c864550cdcafdcd70b9b2ab36afc7e9c53. --- docs/zh/database_engines/index.md | 12 +- docs/zh/database_engines/mysql.md | 125 +----------------- docs/zh/operations/table_engines/mergetree.md | 2 +- 3 files changed, 3 insertions(+), 136 deletions(-) diff --git a/docs/zh/database_engines/index.md b/docs/zh/database_engines/index.md index f8ae05e2520..bbdb762a4ad 120000 --- a/docs/zh/database_engines/index.md +++ b/docs/zh/database_engines/index.md @@ -1,11 +1 @@ -# 数据库引擎 - -您使用的所有表都是由数据库引擎所提供的 - -默认情况下,ClickHouse使用自己的数据库引擎,该引擎提供可配置的[表引擎](../operations/table_engines/index.md)和[所有支持的SQL语法](../query_language/syntax.md). - -除此之外,您还可以选择使用以下的数据库引擎: - -- [MySQL](mysql.md) - -[来源文章](https://clickhouse.yandex/docs/en/database_engines/) +../../en/database_engines/index.md \ No newline at end of file diff --git a/docs/zh/database_engines/mysql.md b/docs/zh/database_engines/mysql.md index 38dfcb5ef64..51ac4126e2d 120000 --- a/docs/zh/database_engines/mysql.md +++ b/docs/zh/database_engines/mysql.md @@ -1,124 +1 @@ -# MySQL - -MySQL引擎用于将远程的MySQL服务器中的表映射到ClickHouse中,并允许您对表进行`INSERT`和`SELECT`查询,以方便您在ClickHouse与MySQL之间进行数据交换。 - -`MySQL`数据库引擎会将对其的查询转换为MySQL语法并发送到MySQL服务器中,因此您可以执行诸如`SHOW TABLES`或`SHOW CREATE TABLE`之类的操作。 - -但您无法对其执行以下操作: - -- `ATTACH`/`DETACH` -- `DROP` -- `RENAME` -- `CREATE TABLE` -- `ALTER` - - -## CREATE DATABASE - -``` sql -CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] -ENGINE = MySQL('host:port', 'database', 'user', 'password') -``` - -**MySQL数据库引擎参数** - -- `host:port` — 链接的MySQL地址。 -- `database` — 链接的MySQL数据库。 -- `user` — 链接的MySQL用户。 -- `password` — 链接的MySQL用户密码。 - - -## 支持的类型对应 - -MySQL | ClickHouse -------|------------ -UNSIGNED TINYINT | [UInt8](../data_types/int_uint.md) -TINYINT | [Int8](../data_types/int_uint.md) -UNSIGNED SMALLINT | [UInt16](../data_types/int_uint.md) -SMALLINT | [Int16](../data_types/int_uint.md) -UNSIGNED INT, UNSIGNED MEDIUMINT | [UInt32](../data_types/int_uint.md) -INT, MEDIUMINT | [Int32](../data_types/int_uint.md) -UNSIGNED BIGINT | [UInt64](../data_types/int_uint.md) -BIGINT | [Int64](../data_types/int_uint.md) -FLOAT | [Float32](../data_types/float.md) -DOUBLE | [Float64](../data_types/float.md) -DATE | [Date](../data_types/date.md) -DATETIME, TIMESTAMP | [DateTime](../data_types/datetime.md) -BINARY | [FixedString](../data_types/fixedstring.md) - -其他的MySQL数据类型将全部都转换为[String](../data_types/string.md)。 - -同时以上的所有类型都支持[Nullable](../data_types/nullable.md)。 - - -## 使用示例 - -在MySQL中创建表: - -``` -mysql> USE test; -Database changed - -mysql> CREATE TABLE `mysql_table` ( - -> `int_id` INT NOT NULL AUTO_INCREMENT, - -> `float` FLOAT NOT NULL, - -> PRIMARY KEY (`int_id`)); -Query OK, 0 rows affected (0,09 sec) - -mysql> insert into mysql_table (`int_id`, `float`) VALUES (1,2); -Query OK, 1 row affected (0,00 sec) - -mysql> select * from mysql_table; -+--------+-------+ -| int_id | value | -+--------+-------+ -| 1 | 2 | -+--------+-------+ -1 row in set (0,00 sec) -``` - -在ClickHouse中创建MySQL类型的数据库,同时与MySQL服务器交换数据: - -```sql -CREATE DATABASE mysql_db ENGINE = MySQL('localhost:3306', 'test', 'my_user', 'user_password') -``` -```sql -SHOW DATABASES -``` -```text -┌─name─────┐ -│ default │ -│ mysql_db │ -│ system │ -└──────────┘ -``` -```sql -SHOW TABLES FROM mysql_db -``` -```text -┌─name─────────┐ -│ mysql_table │ -└──────────────┘ -``` -```sql -SELECT * FROM mysql_db.mysql_table -``` -```text -┌─int_id─┬─value─┐ -│ 1 │ 2 │ -└────────┴───────┘ -``` -```sql -INSERT INTO mysql_db.mysql_table VALUES (3,4) -``` -```sql -SELECT * FROM mysql_db.mysql_table -``` -```text -┌─int_id─┬─value─┐ -│ 1 │ 2 │ -│ 3 │ 4 │ -└────────┴───────┘ -``` - -[来源文章](https://clickhouse.yandex/docs/en/database_engines/mysql/) +../../en/database_engines/mysql.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/mergetree.md b/docs/zh/operations/table_engines/mergetree.md index 5e330164c5a..5ddf837708a 100644 --- a/docs/zh/operations/table_engines/mergetree.md +++ b/docs/zh/operations/table_engines/mergetree.md @@ -48,7 +48,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] **子句** -- `ENGINE` - 引擎名和参数。 `ENGINE = MergeTree()`。 `MergeTree` 引擎不需要其他参数。 +- `ENGINE` - 引擎名和参数。 `ENGINE = MergeTree()`. `MergeTree` 引擎没有参数。 - `PARTITION BY` — [分区键](custom_partitioning_key.md) 。 From abdd70fcc4b02297618c8ed29751e6bbff917fa1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 18:01:36 +0300 Subject: [PATCH 387/509] Fixed "splitted" build --- contrib/arrow-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index bc229deeced..843ff9cd8af 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -219,7 +219,7 @@ endif() add_library(${ARROW_LIBRARY} ${ARROW_SRCS}) add_dependencies(${ARROW_LIBRARY} protoc) target_include_directories(${ARROW_LIBRARY} SYSTEM PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}/cpp/src ${Boost_INCLUDE_DIRS}) -target_link_libraries(${ARROW_LIBRARY} PRIVATE ${DOUBLE_CONVERSION_LIBRARIES} ${PROTOBUF_LIBRARIES} Threads::Threads) +target_link_libraries(${ARROW_LIBRARY} PRIVATE ${DOUBLE_CONVERSION_LIBRARIES} ${Protobuf_LIBRARY} Threads::Threads) if (ARROW_WITH_LZ4) target_link_libraries(${ARROW_LIBRARY} PRIVATE ${LZ4_LIBRARY}) endif() From 75e124f3909d1a6820938986f9d7f82b05acb309 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 18:05:27 +0300 Subject: [PATCH 388/509] Removed misleading flag from CMake --- CMakeLists.txt | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 6ac4d67f6ae..f84a181a39c 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -264,11 +264,10 @@ if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE) set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}") endif () -# TODO: only make this extra-checks in CI builds, since a lot of contrib libs won't link - -# CI works around this problem by explicitly adding GLIBC_COMPATIBILITY flag. -if (NOT SANITIZE AND YANDEX_OFFICIAL_BUILD) - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-undefined") - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined") +# Make this extra-checks for correct library dependencies. +if (NOT SANITIZE) + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-undefined") + set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined") endif () include (cmake/find_unwind.cmake) From 5fd649e663e126813b074f2c819baca8a6eedc7c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 18:10:33 +0300 Subject: [PATCH 389/509] Check for broken symlinks #6625 --- utils/check-style/check-style | 3 +++ 1 file changed, 3 insertions(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index fed4b6b8670..ef3bf6cfad4 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -25,3 +25,6 @@ find $ROOT_PATH/dbms -name '*.h' -or -name '*.cpp' | find $ROOT_PATH/dbms -name '*.h' -or -name '*.cpp' | grep -vP 'Compiler|build' | xargs grep $@ -P '}\s*//+\s*namespace\s*' + +# Broken symlinks +find -L $ROOT_PATH -type l | grep -v contrib && echo "^ Broken symlinks found" From 016b1ee2f77646806d95aa28b562ab7c8e0fd41e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 18:42:45 +0300 Subject: [PATCH 390/509] Increase timeout for "stack overflow" test because it may take a long time in debug build --- dbms/tests/queries/0_stateless/00984_parser_stack_overflow.sh | 2 ++ dbms/tests/queries/shell_config.sh | 3 ++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00984_parser_stack_overflow.sh b/dbms/tests/queries/0_stateless/00984_parser_stack_overflow.sh index 64fae3fb0f9..14f2a8e31fb 100755 --- a/dbms/tests/queries/0_stateless/00984_parser_stack_overflow.sh +++ b/dbms/tests/queries/0_stateless/00984_parser_stack_overflow.sh @@ -1,5 +1,7 @@ #!/usr/bin/env bash +CLICKHOUSE_CURL_TIMEOUT=30 + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh diff --git a/dbms/tests/queries/shell_config.sh b/dbms/tests/queries/shell_config.sh index d4ab11be927..b3058a6cdbe 100644 --- a/dbms/tests/queries/shell_config.sh +++ b/dbms/tests/queries/shell_config.sh @@ -46,7 +46,8 @@ export CLICKHOUSE_PORT_INTERSERVER=${CLICKHOUSE_PORT_INTERSERVER:="9009"} export CLICKHOUSE_URL_INTERSERVER=${CLICKHOUSE_URL_INTERSERVER:="${CLICKHOUSE_PORT_HTTP_PROTO}://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_INTERSERVER}/"} export CLICKHOUSE_CURL_COMMAND=${CLICKHOUSE_CURL_COMMAND:="curl"} -export CLICKHOUSE_CURL=${CLICKHOUSE_CURL:="${CLICKHOUSE_CURL_COMMAND} --max-time 10"} +export CLICKHOUSE_CURL_TIMEOUT=${CLICKHOUSE_CURL_TIMEOUT:="10"} +export CLICKHOUSE_CURL=${CLICKHOUSE_CURL:="${CLICKHOUSE_CURL_COMMAND} --max-time ${CLICKHOUSE_CURL_TIMEOUT}"} export CLICKHOUSE_TMP=${CLICKHOUSE_TMP:="."} mkdir -p ${CLICKHOUSE_TMP} From 859d97e8fdf4ef3d51d95de4047ee94621f19e0f Mon Sep 17 00:00:00 2001 From: dimarub2000 Date: Fri, 23 Aug 2019 18:47:27 +0300 Subject: [PATCH 391/509] Renaming --- dbms/programs/benchmark/Benchmark.cpp | 4 ++-- dbms/programs/client/Client.cpp | 6 +++--- dbms/programs/compressor/Compressor.cpp | 4 ++-- dbms/programs/format/Format.cpp | 4 ++-- dbms/programs/local/LocalServer.cpp | 4 ++-- dbms/programs/obfuscator/Obfuscator.cpp | 4 ++-- dbms/programs/performance-test/PerformanceTestSuite.cpp | 4 ++-- .../src/Common/{TerminalDisplaying.cpp => TerminalSize.cpp} | 4 ++-- dbms/src/Common/{TerminalDisplaying.h => TerminalSize.h} | 4 ++-- 9 files changed, 19 insertions(+), 19 deletions(-) rename dbms/src/Common/{TerminalDisplaying.cpp => TerminalSize.cpp} (87%) rename dbms/src/Common/{TerminalDisplaying.h => TerminalSize.h} (52%) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 7fc33559b54..f56e3e6da2c 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -32,7 +32,7 @@ #include #include #include -#include +#include /** A tool for evaluating ClickHouse performance. @@ -440,7 +440,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) { using boost::program_options::value; - boost::program_options::options_description desc = setOptionsDescription("Allowed options", getTerminalWidth()); + boost::program_options::options_description desc = createOptionsDescription("Allowed options", getTerminalWidth()); desc.add_options() ("help", "produce help message") ("concurrency,c", value()->default_value(1), "number of parallel queries") diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index f77ddbea827..3032c9a5a57 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -67,7 +67,7 @@ #include #include #include -#include +#include #if USE_READLINE #include "Suggest.h" @@ -1649,7 +1649,7 @@ public: namespace po = boost::program_options; /// Main commandline options related to client functionality and all parameters from Settings. - po::options_description main_description = setOptionsDescription("Main options", terminal_width); + po::options_description main_description = createOptionsDescription("Main options", terminal_width); main_description.add_options() ("help", "produce help message") ("config-file,C", po::value(), "config-file path") @@ -1695,7 +1695,7 @@ public: context.getSettingsRef().addProgramOptions(main_description); /// Commandline options related to external tables. - po::options_description external_description = setOptionsDescription("External tables options", terminal_width); + po::options_description external_description = createOptionsDescription("External tables options", terminal_width); external_description.add_options() ("file", po::value(), "data file or - for stdin") ("name", po::value()->default_value("_data"), "name of the table") diff --git a/dbms/programs/compressor/Compressor.cpp b/dbms/programs/compressor/Compressor.cpp index f990b3ac690..a073a79b416 100644 --- a/dbms/programs/compressor/Compressor.cpp +++ b/dbms/programs/compressor/Compressor.cpp @@ -13,7 +13,7 @@ #include #include #include -#include +#include namespace DB @@ -60,7 +60,7 @@ void checkAndWriteHeader(DB::ReadBuffer & in, DB::WriteBuffer & out) int mainEntryClickHouseCompressor(int argc, char ** argv) { - boost::program_options::options_description desc = setOptionsDescription("Allowed options", getTerminalWidth()); + boost::program_options::options_description desc = createOptionsDescription("Allowed options", getTerminalWidth()); desc.add_options() ("help,h", "produce help message") ("decompress,d", "decompress") diff --git a/dbms/programs/format/Format.cpp b/dbms/programs/format/Format.cpp index fbc17e44e63..ff415d88e1b 100644 --- a/dbms/programs/format/Format.cpp +++ b/dbms/programs/format/Format.cpp @@ -6,13 +6,13 @@ #include #include #include -#include +#include int mainEntryClickHouseFormat(int argc, char ** argv) { using namespace DB; - boost::program_options::options_description desc = setOptionsDescription("Allowed options", getTerminalWidth()); + boost::program_options::options_description desc = createOptionsDescription("Allowed options", getTerminalWidth()); desc.add_options() ("help,h", "produce help message") ("hilite", "add syntax highlight with ANSI terminal escape sequences") diff --git a/dbms/programs/local/LocalServer.cpp b/dbms/programs/local/LocalServer.cpp index c894e13ed47..54383050b6c 100644 --- a/dbms/programs/local/LocalServer.cpp +++ b/dbms/programs/local/LocalServer.cpp @@ -35,7 +35,7 @@ #include #include #include -#include +#include namespace DB @@ -410,7 +410,7 @@ void LocalServer::init(int argc, char ** argv) /// Don't parse options with Poco library, we prefer neat boost::program_options stopOptionsProcessing(); - po::options_description description = setOptionsDescription("Main options", getTerminalWidth()); + po::options_description description = createOptionsDescription("Main options", getTerminalWidth()); description.add_options() ("help", "produce help message") ("config-file,c", po::value(), "config-file path") diff --git a/dbms/programs/obfuscator/Obfuscator.cpp b/dbms/programs/obfuscator/Obfuscator.cpp index a2cae4ed1a5..0fb6b495982 100644 --- a/dbms/programs/obfuscator/Obfuscator.cpp +++ b/dbms/programs/obfuscator/Obfuscator.cpp @@ -36,7 +36,7 @@ #include #include #include -#include +#include static const char * documantation = R"( @@ -949,7 +949,7 @@ try using namespace DB; namespace po = boost::program_options; - po::options_description description = setOptionsDescription("Options", getTerminalWidth()); + po::options_description description = createOptionsDescription("Options", getTerminalWidth()); description.add_options() ("help", "produce help message") ("structure,S", po::value(), "structure of the initial table (list of column and type names)") diff --git a/dbms/programs/performance-test/PerformanceTestSuite.cpp b/dbms/programs/performance-test/PerformanceTestSuite.cpp index 37a9e61d169..eaa4e24cde9 100644 --- a/dbms/programs/performance-test/PerformanceTestSuite.cpp +++ b/dbms/programs/performance-test/PerformanceTestSuite.cpp @@ -28,7 +28,7 @@ #include #include #include -#include +#include #include "TestStopConditions.h" #include "TestStats.h" @@ -325,7 +325,7 @@ try using po::value; using Strings = DB::Strings; - po::options_description desc = setOptionsDescription("Allowed options", getTerminalWidth()); + po::options_description desc = createOptionsDescription("Allowed options", getTerminalWidth()); desc.add_options() ("help", "produce help message") ("lite", "use lite version of output") diff --git a/dbms/src/Common/TerminalDisplaying.cpp b/dbms/src/Common/TerminalSize.cpp similarity index 87% rename from dbms/src/Common/TerminalDisplaying.cpp rename to dbms/src/Common/TerminalSize.cpp index 243ba7f8288..ffc301cf77c 100644 --- a/dbms/src/Common/TerminalDisplaying.cpp +++ b/dbms/src/Common/TerminalSize.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include namespace po = boost::program_options; @@ -25,7 +25,7 @@ unsigned short int getTerminalWidth() return 0; } -po::options_description setOptionsDescription(const std::string & caption, unsigned short terminal_width) +po::options_description createOptionsDescription(const std::string &caption, unsigned short terminal_width) { unsigned line_length = po::options_description::m_default_line_length; unsigned min_description_length = line_length / 2; diff --git a/dbms/src/Common/TerminalDisplaying.h b/dbms/src/Common/TerminalSize.h similarity index 52% rename from dbms/src/Common/TerminalDisplaying.h rename to dbms/src/Common/TerminalSize.h index 598969b62c7..4dd58b5c8ca 100644 --- a/dbms/src/Common/TerminalDisplaying.h +++ b/dbms/src/Common/TerminalSize.h @@ -10,8 +10,8 @@ namespace po = boost::program_options; unsigned short int getTerminalWidth(); -/** Sets a name and an appropriate size for option displaying +/** Creates po::options_description with name and an appropriate size for option displaying * when program is called with option --help * */ -po::options_description setOptionsDescription(const std::string & caption, unsigned short terminal_width); +po::options_description createOptionsDescription(const std::string &caption, unsigned short terminal_width); From 50b927a9d74ed6de672ec9cc58e230c545c6ce43 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Aug 2019 19:08:27 +0300 Subject: [PATCH 392/509] Update StringSearcher.h --- dbms/src/Common/StringSearcher.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/StringSearcher.h b/dbms/src/Common/StringSearcher.h index fecf1a7ca81..3cb6e56ab78 100644 --- a/dbms/src/Common/StringSearcher.h +++ b/dbms/src/Common/StringSearcher.h @@ -717,7 +717,7 @@ public: { if (std::any_of(reinterpret_cast(needle_), reinterpret_cast(needle_) + needle_size_, isTokenSeparator)) { - throw Exception{"needle must not contain whitespace characters", ErrorCodes::BAD_ARGUMENTS}; + throw Exception{"Needle must not contain whitespace or separator characters", ErrorCodes::BAD_ARGUMENTS}; } } From c781e1c6a7640ddf29bb0af52608fa97a1793736 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Aug 2019 19:09:24 +0300 Subject: [PATCH 393/509] Update StringSearcher.h --- dbms/src/Common/StringSearcher.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Common/StringSearcher.h b/dbms/src/Common/StringSearcher.h index 3cb6e56ab78..25287db11f5 100644 --- a/dbms/src/Common/StringSearcher.h +++ b/dbms/src/Common/StringSearcher.h @@ -160,7 +160,7 @@ public: #endif } - ALWAYS_INLINE bool compare(const UInt8 * /*haystack*/, const UInt8 * /*haystack_end*/, const UInt8 * pos) const + ALWAYS_INLINE bool compare(const UInt8 * /*haystack*/, const UInt8 * /*haystack_end*/, const UInt8 * pos) const { static const Poco::UTF8Encoding utf8; @@ -377,7 +377,7 @@ public: #endif } - ALWAYS_INLINE bool compare(const UInt8 * /*haystack*/, const UInt8 * /*haystack_end*/, const UInt8 * pos) const + ALWAYS_INLINE bool compare(const UInt8 * /*haystack*/, const UInt8 * /*haystack_end*/, const UInt8 * pos) const { #ifdef __SSE4_1__ if (pageSafe(pos)) @@ -570,7 +570,7 @@ public: #endif } - ALWAYS_INLINE bool compare(const UInt8 * /*haystack*/, const UInt8 * /*haystack_end*/, const UInt8 * pos) const + ALWAYS_INLINE bool compare(const UInt8 * /*haystack*/, const UInt8 * /*haystack_end*/, const UInt8 * pos) const { #ifdef __SSE4_1__ if (pageSafe(pos)) @@ -722,7 +722,7 @@ public: } - ALWAYS_INLINE bool compare(const UInt8 * haystack, const UInt8 * haystack_end, const UInt8 * pos) const + ALWAYS_INLINE bool compare(const UInt8 * haystack, const UInt8 * haystack_end, const UInt8 * pos) const { // use searcher only if pos is in the beginning of token and pos + searcher.needle_size is end of token. if (isToken(haystack, haystack_end, pos)) From b42f85e16bd683b1f03fe4a0833370c742e1445b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 21:30:04 +0300 Subject: [PATCH 394/509] Added a check for double whitespaces --- dbms/src/AggregateFunctions/QuantileExact.h | 2 +- dbms/src/DataTypes/DataTypeEnum.cpp | 2 +- dbms/src/Functions/GeoUtils.cpp | 2 +- .../Formats/Impl/ProtobufRowInputFormat.cpp | 2 +- .../MergeTree/MergeTreeRangeReader.cpp | 2 +- .../Storages/MergeTree/MergeTreeRangeReader.h | 2 +- .../Storages/System/StorageSystemNumbers.cpp | 2 +- utils/check-style/check-style | 3 ++ utils/check-style/double-whitespaces.pl | 33 +++++++++++++++++++ 9 files changed, 43 insertions(+), 7 deletions(-) create mode 100755 utils/check-style/double-whitespaces.pl diff --git a/dbms/src/AggregateFunctions/QuantileExact.h b/dbms/src/AggregateFunctions/QuantileExact.h index 5a1343b1399..4a2aa574ae9 100644 --- a/dbms/src/AggregateFunctions/QuantileExact.h +++ b/dbms/src/AggregateFunctions/QuantileExact.h @@ -176,7 +176,7 @@ struct QuantileExactExclusive : public QuantileExact } }; -/// QuantileExactInclusive is equivalent to Excel PERCENTILE and PERCENTILE.INC, R-7, SciPy-(1,1) +/// QuantileExactInclusive is equivalent to Excel PERCENTILE and PERCENTILE.INC, R-7, SciPy-(1,1) template struct QuantileExactInclusive : public QuantileExact { diff --git a/dbms/src/DataTypes/DataTypeEnum.cpp b/dbms/src/DataTypes/DataTypeEnum.cpp index cffc29feaf8..add7052195a 100644 --- a/dbms/src/DataTypes/DataTypeEnum.cpp +++ b/dbms/src/DataTypes/DataTypeEnum.cpp @@ -234,7 +234,7 @@ void DataTypeEnum::deserializeBinaryBulk( } template -void DataTypeEnum::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const +void DataTypeEnum::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const { if (value_index) return; diff --git a/dbms/src/Functions/GeoUtils.cpp b/dbms/src/Functions/GeoUtils.cpp index 5134343dae0..847d934c6b4 100644 --- a/dbms/src/Functions/GeoUtils.cpp +++ b/dbms/src/Functions/GeoUtils.cpp @@ -18,7 +18,7 @@ const UInt8 geohash_base32_decode_lookup_table[256] = { 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, - 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 10, 11, 12, 13, 14, 15, 16, 0xFF, 17, 18, 0xFF, 19, 20, 0xFF, diff --git a/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index 25fecc5c642..1cd9d329c9d 100644 --- a/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -12,7 +12,7 @@ namespace DB ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSchemaInfo & info_) : IRowInputFormat(header_, in_, params_) - , data_types(header_.getDataTypes()) + , data_types(header_.getDataTypes()) , reader(in, ProtobufSchemas::instance().getMessageTypeForFormatSchema(info_), header_.getNames()) { } diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 2aae847217e..932721eb028 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -108,7 +108,7 @@ size_t MergeTreeRangeReader::DelayedStream::finalize(Block & block) MergeTreeRangeReader::Stream::Stream( - size_t from_mark, size_t to_mark, MergeTreeReader * merge_tree_reader_) + size_t from_mark, size_t to_mark, MergeTreeReader * merge_tree_reader_) : current_mark(from_mark), offset_after_current_mark(0) , last_mark(to_mark) , merge_tree_reader(merge_tree_reader_) diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 9552373901c..0eae69ee17e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -41,7 +41,7 @@ public: { public: DelayedStream() = default; - DelayedStream(size_t from_mark, MergeTreeReader * merge_tree_reader); + DelayedStream(size_t from_mark, MergeTreeReader * merge_tree_reader); /// Read @num_rows rows from @from_mark starting from @offset row /// Returns the number of rows added to block. diff --git a/dbms/src/Storages/System/StorageSystemNumbers.cpp b/dbms/src/Storages/System/StorageSystemNumbers.cpp index 2afe2a7c018..2f155e22a11 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.cpp +++ b/dbms/src/Storages/System/StorageSystemNumbers.cpp @@ -146,7 +146,7 @@ BlockInputStreams StorageSystemNumbers::read( res[i] = std::make_shared(max_block_size, offset + i * max_block_size, num_streams * max_block_size); if (limit) /// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly. - res[i] = std::make_shared(res[i], *limit * (i + 1) / num_streams - *limit * i / num_streams, 0, false, true); + res[i] = std::make_shared(res[i], *limit * (i + 1) / num_streams - *limit * i / num_streams, 0, false, true); } return res; diff --git a/utils/check-style/check-style b/utils/check-style/check-style index ef3bf6cfad4..deed481f043 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -28,3 +28,6 @@ find $ROOT_PATH/dbms -name '*.h' -or -name '*.cpp' | # Broken symlinks find -L $ROOT_PATH -type l | grep -v contrib && echo "^ Broken symlinks found" + +# Double whitespaces +find $ROOT_PATH/dbms -name '*.h' -or -name '*.cpp' | while read i; do $ROOT_PATH/utils/check-style/double-whitespaces.pl < $i || echo -e "^ File $i contains double whitespaces\n"; done diff --git a/utils/check-style/double-whitespaces.pl b/utils/check-style/double-whitespaces.pl new file mode 100755 index 00000000000..47b03cb74ab --- /dev/null +++ b/utils/check-style/double-whitespaces.pl @@ -0,0 +1,33 @@ +#!/usr/bin/perl + +use strict; + +# Find double whitespace such as "a, b, c" that looks very ugly and annoying. +# But skip double whitespaces if they are used as an alignment - by comparing to surrounding lines. + +my @array; + +while (<>) +{ + push @array, $_; +} + +my $ret = 0; + +for (my $i = 1; $i < $#array; ++$i) +{ + if ($array[$i] =~ ',( {2,3})[^ /]') + { + # https://stackoverflow.com/questions/87380/how-can-i-find-the-location-of-a-regex-match-in-perl + + if ((substr($array[$i - 1], $+[1] - 1, 2) !~ /^[ -][^ ]$/) # whitespaces are not part of alignment + && (substr($array[$i + 1], $+[1] - 1, 2) !~ /^[ -][^ ]$/) + && $array[$i] !~ /(-?\d+\w*,\s+){3,}/) # this is not a number table like { 10, -1, 2 } + { + print(($i + 1) . ":" . $array[$i]); + $ret = 1; + } + } +} + +exit $ret; From a21b43913fd97d0a97232764ad2f338da93a9561 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 23 Aug 2019 21:40:42 +0300 Subject: [PATCH 395/509] fix crash in OptimizedRegularExpression --- dbms/src/Common/OptimizedRegularExpression.cpp | 5 +++-- .../0_stateless/00997_extract_all_crash_6627.reference | 1 + .../queries/0_stateless/00997_extract_all_crash_6627.sql | 1 + 3 files changed, 5 insertions(+), 2 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.reference create mode 100644 dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.sql diff --git a/dbms/src/Common/OptimizedRegularExpression.cpp b/dbms/src/Common/OptimizedRegularExpression.cpp index c87d87fc2df..3a224709447 100644 --- a/dbms/src/Common/OptimizedRegularExpression.cpp +++ b/dbms/src/Common/OptimizedRegularExpression.cpp @@ -1,4 +1,5 @@ #include +#include #include #define MIN_LENGTH_FOR_STRSTR 3 @@ -413,9 +414,9 @@ unsigned OptimizedRegularExpressionImpl::match(const char * subject return 0; } - StringPieceType pieces[MAX_SUBPATTERNS]; + DB::PODArrayWithStackMemory pieces(limit); - if (!re2->Match(StringPieceType(subject, subject_size), 0, subject_size, RegexType::UNANCHORED, pieces, limit)) + if (!re2->Match(StringPieceType(subject, subject_size), 0, subject_size, RegexType::UNANCHORED, pieces.data(), pieces.size())) return 0; else { diff --git a/dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.reference b/dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.reference new file mode 100644 index 00000000000..acb53e80e6d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.reference @@ -0,0 +1 @@ +['9'] diff --git a/dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.sql b/dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.sql new file mode 100644 index 00000000000..06de4ec8afb --- /dev/null +++ b/dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.sql @@ -0,0 +1 @@ +SELECT extractAll('Mozilla/5.0 (Windows NT 10.0; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/75.0.3770.143 YaBrowser/19.7.2.455 Yowser/2.5 Safari/537.36', '[Y][a-zA-Z]{8}/[1-9]([1-9]+)?(((.?)([0-9]+)?){0,4})?'); From 7dffa0fe9fbca08d6f25bc83b53fdee1ffeadbf5 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Fri, 23 Aug 2019 22:19:36 +0300 Subject: [PATCH 396/509] added wait for mutation to indices tests --- dbms/tests/queries/0_stateless/00942_mutate_index.sh | 4 ++-- dbms/tests/queries/0_stateless/00943_materialize_index.sh | 6 ++++++ .../queries/0_stateless/00944_clear_index_in_partition.sh | 3 ++- .../00975_indices_mutation_replicated_zookeeper.sh | 7 +++++-- 4 files changed, 15 insertions(+), 5 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00942_mutate_index.sh b/dbms/tests/queries/0_stateless/00942_mutate_index.sh index c6dd1dfb836..467eb9ab671 100755 --- a/dbms/tests/queries/0_stateless/00942_mutate_index.sh +++ b/dbms/tests/queries/0_stateless/00942_mutate_index.sh @@ -2,6 +2,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh +. $CURDIR/mergetree_mutations.lib $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" @@ -35,8 +36,7 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 1;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 5;" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx UPDATE i64 = 5 WHERE i64 = 1;" - -sleep 0.1 +wait_for_mutation "minmax_idx" "mutation_2.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 1;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 5;" diff --git a/dbms/tests/queries/0_stateless/00943_materialize_index.sh b/dbms/tests/queries/0_stateless/00943_materialize_index.sh index f51b66993aa..ab2fd1e5355 100755 --- a/dbms/tests/queries/0_stateless/00943_materialize_index.sh +++ b/dbms/tests/queries/0_stateless/00943_materialize_index.sh @@ -2,6 +2,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh +. $CURDIR/mergetree_mutations.lib $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" @@ -38,22 +39,27 @@ SET allow_experimental_data_skipping_indices=1; ALTER TABLE test.minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1;" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" +wait_for_mutation "minmax_idx" "mutation_1.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 2;" +wait_for_mutation "minmax_idx" "mutation_2.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 1;" +wait_for_mutation "minmax_idx" "mutation_3.txt" "test" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 2;" +wait_for_mutation "minmax_idx" "mutation_4.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx;" +wait_for_mutation "minmax_idx" "mutation_5.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" diff --git a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh index 9047bbb3a72..7d68bac8c83 100755 --- a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh +++ b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh @@ -2,6 +2,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh +. $CURDIR/mergetree_mutations.lib $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" @@ -42,7 +43,7 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" -sleep 0.5 +wait_for_mutation "minmax_idx" "mutation_1.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" diff --git a/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh index 613226a3fb7..1bcb4f17edd 100755 --- a/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh @@ -2,6 +2,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh +. $CURDIR/mergetree_mutations.lib $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.indices_mutaions1;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.indices_mutaions2;" @@ -47,13 +48,15 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;" -sleep 1 +wait_for_mutation "indices_mutaions1" "mutation_1.txt" "test" +wait_for_mutation "indices_mutaions2" "mutation_1.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 MATERIALIZE INDEX idx IN PARTITION 1;" -sleep 1 +wait_for_mutation "indices_mutaions1" "mutation_2.txt" "test" +wait_for_mutation "indices_mutaions2" "mutation_2.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" From ba2d17c12a68ede2bb47e34fdfbdd6ea0ec33a8e Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Fri, 23 Aug 2019 22:36:17 +0300 Subject: [PATCH 397/509] fix --- .../0_stateless/00943_materialize_index.sh | 17 ++++++++--------- .../00944_clear_index_in_partition.sh | 2 +- ...975_indices_mutation_replicated_zookeeper.sh | 3 +-- 3 files changed, 10 insertions(+), 12 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00943_materialize_index.sh b/dbms/tests/queries/0_stateless/00943_materialize_index.sh index ab2fd1e5355..feab59b368e 100755 --- a/dbms/tests/queries/0_stateless/00943_materialize_index.sh +++ b/dbms/tests/queries/0_stateless/00943_materialize_index.sh @@ -39,27 +39,26 @@ SET allow_experimental_data_skipping_indices=1; ALTER TABLE test.minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1;" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" -wait_for_mutation "minmax_idx" "mutation_1.txt" "test" - -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" - -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 2;" wait_for_mutation "minmax_idx" "mutation_2.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 1;" +$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 2;" wait_for_mutation "minmax_idx" "mutation_3.txt" "test" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 1;" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 2;" -wait_for_mutation "minmax_idx" "mutation_4.txt" "test" +sleep 0.5 $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx;" -wait_for_mutation "minmax_idx" "mutation_5.txt" "test" +wait_for_mutation "minmax_idx" "mutation_4.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" diff --git a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh index 7d68bac8c83..5a7bdd8e3ae 100755 --- a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh +++ b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh @@ -43,7 +43,7 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" -wait_for_mutation "minmax_idx" "mutation_1.txt" "test" +wait_for_mutation "minmax_idx" "mutation_2.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" diff --git a/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh index 1bcb4f17edd..5e6159475f8 100755 --- a/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh @@ -48,8 +48,7 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;" -wait_for_mutation "indices_mutaions1" "mutation_1.txt" "test" -wait_for_mutation "indices_mutaions2" "mutation_1.txt" "test" +sleep 0.5 $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" From 13f59d3f6c470b41ada777a88625130cbf9fa910 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Aug 2019 23:13:08 +0300 Subject: [PATCH 398/509] Update Client.cpp --- dbms/programs/client/Client.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 3032c9a5a57..76f67bd6de3 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -131,7 +131,7 @@ private: bool print_time_to_stderr = false; /// Output execution time to stderr in batch mode. bool stdin_is_not_tty = false; /// stdin is not a terminal. - unsigned short int terminal_width; /// Terminal width is needed to render progress bar. + uint16_t terminal_width{}; /// Terminal width is needed to render progress bar. std::unique_ptr connection; /// Connection to DB. String query_id; /// Current query_id. From d38e9ee229cede07348b2531c3949a9f11b9abe3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Aug 2019 23:32:31 +0300 Subject: [PATCH 399/509] Fixed "trim" functions (in progress) --- dbms/src/Functions/trim.cpp | 64 ++---- libs/libcommon/include/common/find_symbols.h | 198 +++++++++++++------ 2 files changed, 157 insertions(+), 105 deletions(-) diff --git a/dbms/src/Functions/trim.cpp b/dbms/src/Functions/trim.cpp index f2e2543cc90..81916604d63 100644 --- a/dbms/src/Functions/trim.cpp +++ b/dbms/src/Functions/trim.cpp @@ -1,10 +1,8 @@ #include #include #include +#include -#ifdef __SSE4_2__ -#include -#endif namespace DB { @@ -60,7 +58,7 @@ public: execute(reinterpret_cast(&data[prev_offset]), offsets[i] - prev_offset - 1, start, length); res_data.resize(res_data.size() + length + 1); - memcpy(&res_data[res_offset], start, length); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], start, length); res_offset += length + 1; res_data[res_offset - 1] = '\0'; @@ -77,59 +75,27 @@ public: private: static void execute(const UInt8 * data, size_t size, const UInt8 *& res_data, size_t & res_size) { - size_t chars_to_trim_left = 0; - size_t chars_to_trim_right = 0; - char whitespace = ' '; -#ifdef __SSE4_2__ - const auto bytes_sse = sizeof(__m128i); - const auto size_sse = size - (size % bytes_sse); - const auto whitespace_mask = _mm_set1_epi8(whitespace); - constexpr auto base_sse_mode = _SIDD_UBYTE_OPS | _SIDD_CMP_EQUAL_EACH | _SIDD_NEGATIVE_POLARITY; - auto mask = bytes_sse; -#endif + const char * char_data = reinterpret_cast(data); + const char * char_end = char_data + size; if constexpr (mode::trim_left) { -#ifdef __SSE4_2__ - /// skip whitespace from left in blocks of up to 16 characters - - /// Avoid gcc bug: _mm_cmpistri: error: the third argument must be an 8-bit immediate - enum { left_sse_mode = base_sse_mode | _SIDD_LEAST_SIGNIFICANT }; - while (mask == bytes_sse && chars_to_trim_left < size_sse) - { - const auto chars = _mm_loadu_si128(reinterpret_cast(data + chars_to_trim_left)); - mask = _mm_cmpistri(whitespace_mask, chars, left_sse_mode); - chars_to_trim_left += mask; - } -#endif - /// skip remaining whitespace from left, character by character - while (chars_to_trim_left < size && data[chars_to_trim_left] == whitespace) - ++chars_to_trim_left; + const char * found = find_first_not_symbols<' '>(char_data, char_end); + size_t num_chars = found - char_data; + char_data += num_chars; } - if constexpr (mode::trim_right) + if constexpr (mode::trim_left) { - const auto trim_right_size = size - chars_to_trim_left; -#ifdef __SSE4_2__ - /// try to skip whitespace from right in blocks of up to 16 characters - - /// Avoid gcc bug: _mm_cmpistri: error: the third argument must be an 8-bit immediate - enum { right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT }; - const auto trim_right_size_sse = trim_right_size - (trim_right_size % bytes_sse); - while (mask == bytes_sse && chars_to_trim_right < trim_right_size_sse) - { - const auto chars = _mm_loadu_si128(reinterpret_cast(data + size - chars_to_trim_right - bytes_sse)); - mask = _mm_cmpistri(whitespace_mask, chars, right_sse_mode); - chars_to_trim_right += mask; - } -#endif - /// skip remaining whitespace from right, character by character - while (chars_to_trim_right < trim_right_size && data[size - chars_to_trim_right - 1] == whitespace) - ++chars_to_trim_right; + const char * found = find_last_not_symbols_or_null<' '>(char_data, char_end); + if (found) + char_end = found + 1; + else + char_end = char_data; } - res_data = data + chars_to_trim_left; - res_size = size - chars_to_trim_left - chars_to_trim_right; + res_data = reinterpret_cast(char_data); + res_size = char_end - char_data; } }; diff --git a/libs/libcommon/include/common/find_symbols.h b/libs/libcommon/include/common/find_symbols.h index 68b49397683..920a7df04c5 100644 --- a/libs/libcommon/include/common/find_symbols.h +++ b/libs/libcommon/include/common/find_symbols.h @@ -65,115 +65,153 @@ inline __m128i mm_is_in(__m128i bytes) } #endif - -template -inline const char * find_first_symbols_sse2(const char * begin, const char * end) +template +bool maybe_negate(bool x) { + if constexpr (positive) + return x; + else + return !x; +} + +template +uint16_t maybe_negate(uint16_t x) +{ + if constexpr (positive) + return x; + else + return ~x; +} + +enum class ReturnMode +{ + End, + Nullptr, +}; + + +template +inline const char * find_first_symbols_sse2(const char * const begin, const char * const end) +{ + const char * pos = begin; + #if defined(__SSE2__) - for (; begin + 15 < end; begin += 16) + for (; pos + 15 < end; pos += 16) { - __m128i bytes = _mm_loadu_si128(reinterpret_cast(begin)); + __m128i bytes = _mm_loadu_si128(reinterpret_cast(pos)); __m128i eq = mm_is_in(bytes); - uint16_t bit_mask = _mm_movemask_epi8(eq); + uint16_t bit_mask = maybe_negate(uint16_t(_mm_movemask_epi8(eq))); if (bit_mask) - return begin + __builtin_ctz(bit_mask); + return pos + __builtin_ctz(bit_mask); } #endif - for (; begin < end; ++begin) - if (is_in(*begin)) - return begin; - return end; + for (; pos < end; ++pos) + if (maybe_negate(is_in(*pos))) + return pos; + + return return_mode == ReturnMode::End ? end : nullptr; } -template -inline const char * find_last_symbols_or_null_sse2(const char * begin, const char * end) +template +inline const char * find_last_symbols_sse2(const char * const begin, const char * const end) { + const char * pos = end; + #if defined(__SSE2__) - for (; end - 16 >= begin; end -= 16) /// Assuming the pointer cannot overflow. Assuming we can compare these pointers. + for (; pos - 16 >= begin; pos -= 16) /// Assuming the pointer cannot overflow. Assuming we can compare these pointers. { - __m128i bytes = _mm_loadu_si128(reinterpret_cast(end - 16)); + __m128i bytes = _mm_loadu_si128(reinterpret_cast(pos - 16)); __m128i eq = mm_is_in(bytes); - uint16_t bit_mask = _mm_movemask_epi8(eq); + uint16_t bit_mask = maybe_negate(uint16_t(_mm_movemask_epi8(eq))); if (bit_mask) - return end - 1 - (__builtin_clz(bit_mask) - 16); /// because __builtin_clz works with mask as uint32. + return pos - 1 - (__builtin_clz(bit_mask) - 16); /// because __builtin_clz works with mask as uint32. } #endif - --end; - for (; end >= begin; --end) - if (is_in(*end)) - return end; + --pos; + for (; pos >= begin; --pos) + if (maybe_negate(is_in(*pos))) + return pos; - return nullptr; + return return_mode == ReturnMode::End ? end : nullptr; } -template -inline const char * find_first_symbols_sse42_impl(const char * begin, const char * end) +inline const char * find_first_symbols_sse42_impl(const char * const begin, const char * const end) { + const char * pos = begin; + #if defined(__SSE4_2__) #define MODE (_SIDD_UBYTE_OPS | _SIDD_CMP_EQUAL_ANY | _SIDD_LEAST_SIGNIFICANT) __m128i set = _mm_setr_epi8(c01, c02, c03, c04, c05, c06, c07, c08, c09, c10, c11, c12, c13, c14, c15, c16); - for (; begin + 15 < end; begin += 16) + for (; pos + 15 < end; pos += 16) { - __m128i bytes = _mm_loadu_si128(reinterpret_cast(begin)); + __m128i bytes = _mm_loadu_si128(reinterpret_cast(pos)); - if (_mm_cmpestrc(set, num_chars, bytes, 16, MODE)) - return begin + _mm_cmpestri(set, num_chars, bytes, 16, MODE); + if constexpr (positive) + { + if (_mm_cmpestrc(set, num_chars, bytes, 16, MODE)) + return pos + _mm_cmpestri(set, num_chars, bytes, 16, MODE); + } + else + { + if (_mm_cmpestrc(set, num_chars, bytes, 16, MODE | _SIDD_NEGATIVE_POLARITY)) + return pos + _mm_cmpestri(set, num_chars, bytes, 16, MODE | _SIDD_NEGATIVE_POLARITY); + } } #undef MODE #endif - for (; begin < end; ++begin) - if ( (num_chars >= 1 && *begin == c01) - || (num_chars >= 2 && *begin == c02) - || (num_chars >= 3 && *begin == c03) - || (num_chars >= 4 && *begin == c04) - || (num_chars >= 5 && *begin == c05) - || (num_chars >= 6 && *begin == c06) - || (num_chars >= 7 && *begin == c07) - || (num_chars >= 8 && *begin == c08) - || (num_chars >= 9 && *begin == c09) - || (num_chars >= 10 && *begin == c10) - || (num_chars >= 11 && *begin == c11) - || (num_chars >= 12 && *begin == c12) - || (num_chars >= 13 && *begin == c13) - || (num_chars >= 14 && *begin == c14) - || (num_chars >= 15 && *begin == c15) - || (num_chars >= 16 && *begin == c16)) - return begin; - return end; + for (; pos < end; ++pos) + if ( (num_chars >= 1 && maybe_negate(*pos == c01)) + || (num_chars >= 2 && maybe_negate(*pos == c02)) + || (num_chars >= 3 && maybe_negate(*pos == c03)) + || (num_chars >= 4 && maybe_negate(*pos == c04)) + || (num_chars >= 5 && maybe_negate(*pos == c05)) + || (num_chars >= 6 && maybe_negate(*pos == c06)) + || (num_chars >= 7 && maybe_negate(*pos == c07)) + || (num_chars >= 8 && maybe_negate(*pos == c08)) + || (num_chars >= 9 && maybe_negate(*pos == c09)) + || (num_chars >= 10 && maybe_negate(*pos == c10)) + || (num_chars >= 11 && maybe_negate(*pos == c11)) + || (num_chars >= 12 && maybe_negate(*pos == c12)) + || (num_chars >= 13 && maybe_negate(*pos == c13)) + || (num_chars >= 15 && maybe_negate(*pos == c15)) + || (num_chars >= 16 && maybe_negate(*pos == c16))) + return pos; + return return_mode == ReturnMode::End ? end : nullptr; } -template +template inline const char * find_first_symbols_sse42(const char * begin, const char * end) { - return find_first_symbols_sse42_impl(begin, end); + return find_first_symbols_sse42_impl(begin, end); } /// NOTE No SSE 4.2 implementation for find_last_symbols_or_null. Not worth to do. -template +template inline const char * find_first_symbols_dispatch(const char * begin, const char * end) { #if defined(__SSE4_2__) if (sizeof...(symbols) >= 5) - return find_first_symbols_sse42(begin, end); + return find_first_symbols_sse42(begin, end); else #endif - return find_first_symbols_sse2(begin, end); + return find_first_symbols_sse2(begin, end); } } @@ -182,7 +220,7 @@ inline const char * find_first_symbols_dispatch(const char * begin, const char * template inline const char * find_first_symbols(const char * begin, const char * end) { - return detail::find_first_symbols_dispatch(begin, end); + return detail::find_first_symbols_dispatch(begin, end); } /// Returning non const result for non const arguments. @@ -190,18 +228,66 @@ inline const char * find_first_symbols(const char * begin, const char * end) template inline char * find_first_symbols(char * begin, char * end) { - return const_cast(detail::find_first_symbols_dispatch(begin, end)); + return const_cast(detail::find_first_symbols_dispatch(begin, end)); +} + +template +inline const char * find_first_not_symbols(const char * begin, const char * end) +{ + return detail::find_first_symbols_dispatch(begin, end); +} + +template +inline char * find_first_not_symbols(char * begin, char * end) +{ + return const_cast(detail::find_first_symbols_dispatch(begin, end)); +} + +template +inline const char * find_first_symbols_or_null(const char * begin, const char * end) +{ + return detail::find_first_symbols_dispatch(begin, end); +} + +template +inline char * find_first_symbols_or_null(char * begin, char * end) +{ + return const_cast(detail::find_first_symbols_dispatch(begin, end)); +} + +template +inline const char * find_first_not_symbols_or_null(const char * begin, const char * end) +{ + return detail::find_first_symbols_dispatch(begin, end); +} + +template +inline char * find_first_not_symbols_or_null(char * begin, char * end) +{ + return const_cast(detail::find_first_symbols_dispatch(begin, end)); } template inline const char * find_last_symbols_or_null(const char * begin, const char * end) { - return detail::find_last_symbols_or_null_sse2(begin, end); + return detail::find_last_symbols_sse2(begin, end); } template inline char * find_last_symbols_or_null(char * begin, char * end) { - return const_cast(detail::find_last_symbols_or_null_sse2(begin, end)); + return const_cast(detail::find_last_symbols_sse2(begin, end)); +} + +template +inline const char * find_last_not_symbols_or_null(const char * begin, const char * end) +{ + return detail::find_last_symbols_sse2(begin, end); +} + +template +inline char * find_last_not_symbols_or_null(char * begin, char * end) +{ + return const_cast(detail::find_last_symbols_sse2(begin, end)); } From 44d3e1e837fd5e804c2d0452e8a8f1b4af6f93f8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Aug 2019 23:35:36 +0300 Subject: [PATCH 400/509] Update Client.cpp --- dbms/programs/client/Client.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 76f67bd6de3..e1f1bd92840 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -1664,7 +1664,7 @@ public: * the "\n" is used to distinguish this case because there is hardly a chance an user would use "\n" * as the password. */ - ("password", po::value()->implicit_value("\n","\\n"), "password") + ("password", po::value()->implicit_value("\n", ""), "password") ("ask-password", "ask-password") ("query_id", po::value(), "query_id") ("query,q", po::value(), "query") From cdd6dca51771520b70df52550b3ae427b4fc82f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Aug 2019 00:10:26 +0300 Subject: [PATCH 401/509] Remove Compiler --- dbms/programs/CMakeLists.txt | 28 +- dbms/programs/clang/CMakeLists.txt | 38 -- .../clang/Compiler-5.0.0/CMakeLists.txt | 53 -- .../programs/clang/Compiler-5.0.0/LICENSE.TXT | 63 -- .../clang/Compiler-5.0.0/cc1_main.cpp | 242 -------- .../clang/Compiler-5.0.0/cc1as_main.cpp | 540 ----------------- dbms/programs/clang/Compiler-5.0.0/driver.cpp | 519 ---------------- dbms/programs/clang/Compiler-5.0.0/lld.cpp | 23 - dbms/programs/clang/Compiler-5.0.1 | 1 - dbms/programs/clang/Compiler-5.0.2 | 1 - .../clang/Compiler-6.0.0/CMakeLists.txt | 54 -- .../programs/clang/Compiler-6.0.0/LICENSE.TXT | 63 -- .../clang/Compiler-6.0.0/cc1_main.cpp | 242 -------- .../clang/Compiler-6.0.0/cc1as_main.cpp | 540 ----------------- dbms/programs/clang/Compiler-6.0.0/driver.cpp | 520 ---------------- dbms/programs/clang/Compiler-6.0.0/lld.cpp | 23 - dbms/programs/clang/Compiler-6.0.0svn | 1 - dbms/programs/clang/Compiler-6.0.1 | 1 - .../clang/Compiler-7.0.0/CMakeLists.txt | 49 -- .../clang/Compiler-7.0.0/cc1_main.cpp | 239 -------- .../clang/Compiler-7.0.0/cc1as_main.cpp | 572 ------------------ .../Compiler-7.0.0/cc1gen_reproducer_main.cpp | 196 ------ dbms/programs/clang/Compiler-7.0.0/driver.cpp | 514 ---------------- dbms/programs/clang/Compiler-7.0.0/lld.cpp | 150 ----- .../Compiler-7.0.0bundled/CMakeLists.txt | 49 -- .../clang/Compiler-7.0.0bundled/cc1_main.cpp | 243 -------- .../Compiler-7.0.0bundled/cc1as_main.cpp | 555 ----------------- .../clang/Compiler-7.0.0bundled/driver.cpp | 512 ---------------- .../clang/Compiler-7.0.0bundled/lld.cpp | 10 - dbms/programs/clang/Compiler-7.0.0svn | 1 - dbms/programs/clang/Compiler-7.0.1 | 1 - dbms/programs/clang/clickhouse-clang.cpp | 2 - dbms/programs/clang/clickhouse-lld.cpp | 2 - dbms/programs/clang/copy_headers.sh | 100 --- dbms/programs/main.cpp | 16 - dbms/src/Core/Settings.h | 2 +- dbms/src/Interpreters/Aggregator.cpp | 254 +------- dbms/src/Interpreters/Aggregator.h | 58 +- dbms/src/Interpreters/CMakeLists.txt | 67 -- dbms/src/Interpreters/Compiler.cpp | 326 ---------- dbms/src/Interpreters/Compiler.h | 88 --- dbms/src/Interpreters/Context.cpp | 13 - .../Interpreters/InterpreterSelectQuery.cpp | 4 - dbms/src/Interpreters/SpecializedAggregator.h | 215 ------- dbms/src/Interpreters/config_compile.h.in | 26 - dbms/src/Interpreters/tests/CMakeLists.txt | 3 - dbms/src/Interpreters/tests/compiler_test.cpp | 57 -- .../00281_compile_sizeof_packed.reference | 2 - .../00281_compile_sizeof_packed.sql | 2 - .../00568_compile_catch_throw.reference | 2 - .../0_stateless/00568_compile_catch_throw.sh | 14 - 51 files changed, 8 insertions(+), 7288 deletions(-) delete mode 100644 dbms/programs/clang/CMakeLists.txt delete mode 100644 dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt delete mode 100644 dbms/programs/clang/Compiler-5.0.0/LICENSE.TXT delete mode 100644 dbms/programs/clang/Compiler-5.0.0/cc1_main.cpp delete mode 100644 dbms/programs/clang/Compiler-5.0.0/cc1as_main.cpp delete mode 100644 dbms/programs/clang/Compiler-5.0.0/driver.cpp delete mode 100644 dbms/programs/clang/Compiler-5.0.0/lld.cpp delete mode 120000 dbms/programs/clang/Compiler-5.0.1 delete mode 120000 dbms/programs/clang/Compiler-5.0.2 delete mode 100644 dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt delete mode 100644 dbms/programs/clang/Compiler-6.0.0/LICENSE.TXT delete mode 100644 dbms/programs/clang/Compiler-6.0.0/cc1_main.cpp delete mode 100644 dbms/programs/clang/Compiler-6.0.0/cc1as_main.cpp delete mode 100644 dbms/programs/clang/Compiler-6.0.0/driver.cpp delete mode 100644 dbms/programs/clang/Compiler-6.0.0/lld.cpp delete mode 120000 dbms/programs/clang/Compiler-6.0.0svn delete mode 120000 dbms/programs/clang/Compiler-6.0.1 delete mode 100644 dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt delete mode 100644 dbms/programs/clang/Compiler-7.0.0/cc1_main.cpp delete mode 100644 dbms/programs/clang/Compiler-7.0.0/cc1as_main.cpp delete mode 100644 dbms/programs/clang/Compiler-7.0.0/cc1gen_reproducer_main.cpp delete mode 100644 dbms/programs/clang/Compiler-7.0.0/driver.cpp delete mode 100644 dbms/programs/clang/Compiler-7.0.0/lld.cpp delete mode 100644 dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt delete mode 100644 dbms/programs/clang/Compiler-7.0.0bundled/cc1_main.cpp delete mode 100644 dbms/programs/clang/Compiler-7.0.0bundled/cc1as_main.cpp delete mode 100644 dbms/programs/clang/Compiler-7.0.0bundled/driver.cpp delete mode 100644 dbms/programs/clang/Compiler-7.0.0bundled/lld.cpp delete mode 120000 dbms/programs/clang/Compiler-7.0.0svn delete mode 120000 dbms/programs/clang/Compiler-7.0.1 delete mode 100644 dbms/programs/clang/clickhouse-clang.cpp delete mode 100644 dbms/programs/clang/clickhouse-lld.cpp delete mode 100755 dbms/programs/clang/copy_headers.sh delete mode 100644 dbms/src/Interpreters/Compiler.cpp delete mode 100644 dbms/src/Interpreters/Compiler.h delete mode 100644 dbms/src/Interpreters/SpecializedAggregator.h delete mode 100644 dbms/src/Interpreters/config_compile.h.in delete mode 100644 dbms/src/Interpreters/tests/compiler_test.cpp delete mode 100644 dbms/tests/queries/0_stateless/00281_compile_sizeof_packed.reference delete mode 100644 dbms/tests/queries/0_stateless/00281_compile_sizeof_packed.sql delete mode 100644 dbms/tests/queries/0_stateless/00568_compile_catch_throw.reference delete mode 100755 dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh diff --git a/dbms/programs/CMakeLists.txt b/dbms/programs/CMakeLists.txt index 03eba470949..0dcd4d7ab91 100644 --- a/dbms/programs/CMakeLists.txt +++ b/dbms/programs/CMakeLists.txt @@ -81,7 +81,6 @@ add_subdirectory (extract-from-config) add_subdirectory (compressor) add_subdirectory (copier) add_subdirectory (format) -add_subdirectory (clang) add_subdirectory (obfuscator) if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) @@ -89,9 +88,9 @@ if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) endif () if (CLICKHOUSE_ONE_SHARED) - add_library(clickhouse-lib SHARED ${CLICKHOUSE_SERVER_SOURCES} ${CLICKHOUSE_CLIENT_SOURCES} ${CLICKHOUSE_LOCAL_SOURCES} ${CLICKHOUSE_BENCHMARK_SOURCES} ${CLICKHOUSE_PERFORMANCE_TEST_SOURCES} ${CLICKHOUSE_COPIER_SOURCES} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_SOURCES} ${CLICKHOUSE_COMPRESSOR_SOURCES} ${CLICKHOUSE_FORMAT_SOURCES} ${CLICKHOUSE_OBFUSCATOR_SOURCES} ${CLICKHOUSE_COMPILER_SOURCES} ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) - target_link_libraries(clickhouse-lib ${CLICKHOUSE_SERVER_LINK} ${CLICKHOUSE_CLIENT_LINK} ${CLICKHOUSE_LOCAL_LINK} ${CLICKHOUSE_BENCHMARK_LINK} ${CLICKHOUSE_PERFORMANCE_TEST_LINK} ${CLICKHOUSE_COPIER_LINK} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_LINK} ${CLICKHOUSE_COMPRESSOR_LINK} ${CLICKHOUSE_FORMAT_LINK} ${CLICKHOUSE_OBFUSCATOR_LINK} ${CLICKHOUSE_COMPILER_LINK} ${CLICKHOUSE_ODBC_BRIDGE_LINK}) - target_include_directories(clickhouse-lib ${CLICKHOUSE_SERVER_INCLUDE} ${CLICKHOUSE_CLIENT_INCLUDE} ${CLICKHOUSE_LOCAL_INCLUDE} ${CLICKHOUSE_BENCHMARK_INCLUDE} ${CLICKHOUSE_PERFORMANCE_TEST_INCLUDE} ${CLICKHOUSE_COPIER_INCLUDE} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_INCLUDE} ${CLICKHOUSE_COMPRESSOR_INCLUDE} ${CLICKHOUSE_FORMAT_INCLUDE} ${CLICKHOUSE_OBFUSCATOR_INCLUDE} ${CLICKHOUSE_COMPILER_INCLUDE} ${CLICKHOUSE_ODBC_BRIDGE_INCLUDE}) + add_library(clickhouse-lib SHARED ${CLICKHOUSE_SERVER_SOURCES} ${CLICKHOUSE_CLIENT_SOURCES} ${CLICKHOUSE_LOCAL_SOURCES} ${CLICKHOUSE_BENCHMARK_SOURCES} ${CLICKHOUSE_PERFORMANCE_TEST_SOURCES} ${CLICKHOUSE_COPIER_SOURCES} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_SOURCES} ${CLICKHOUSE_COMPRESSOR_SOURCES} ${CLICKHOUSE_FORMAT_SOURCES} ${CLICKHOUSE_OBFUSCATOR_SOURCES} ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) + target_link_libraries(clickhouse-lib ${CLICKHOUSE_SERVER_LINK} ${CLICKHOUSE_CLIENT_LINK} ${CLICKHOUSE_LOCAL_LINK} ${CLICKHOUSE_BENCHMARK_LINK} ${CLICKHOUSE_PERFORMANCE_TEST_LINK} ${CLICKHOUSE_COPIER_LINK} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_LINK} ${CLICKHOUSE_COMPRESSOR_LINK} ${CLICKHOUSE_FORMAT_LINK} ${CLICKHOUSE_OBFUSCATOR_LINK} ${CLICKHOUSE_ODBC_BRIDGE_LINK}) + target_include_directories(clickhouse-lib ${CLICKHOUSE_SERVER_INCLUDE} ${CLICKHOUSE_CLIENT_INCLUDE} ${CLICKHOUSE_LOCAL_INCLUDE} ${CLICKHOUSE_BENCHMARK_INCLUDE} ${CLICKHOUSE_PERFORMANCE_TEST_INCLUDE} ${CLICKHOUSE_COPIER_INCLUDE} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_INCLUDE} ${CLICKHOUSE_COMPRESSOR_INCLUDE} ${CLICKHOUSE_FORMAT_INCLUDE} ${CLICKHOUSE_OBFUSCATOR_INCLUDE} ${CLICKHOUSE_ODBC_BRIDGE_INCLUDE}) set_target_properties(clickhouse-lib PROPERTIES SOVERSION ${VERSION_MAJOR}.${VERSION_MINOR} VERSION ${VERSION_SO} OUTPUT_NAME clickhouse DEBUG_POSTFIX "") install (TARGETS clickhouse-lib LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} COMPONENT clickhouse) endif() @@ -104,10 +103,6 @@ if (CLICKHOUSE_SPLIT_BINARY) list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-odbc-bridge) endif () - if (USE_EMBEDDED_COMPILER) - list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-clang clickhouse-lld) - endif () - set_target_properties(${CLICKHOUSE_ALL_TARGETS} PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..) add_custom_target (clickhouse-bundle ALL DEPENDS ${CLICKHOUSE_ALL_TARGETS}) @@ -115,10 +110,6 @@ if (CLICKHOUSE_SPLIT_BINARY) install(PROGRAMS clickhouse-split-helper DESTINATION ${CMAKE_INSTALL_BINDIR} RENAME clickhouse COMPONENT clickhouse) else () - if (USE_EMBEDDED_COMPILER) - # before add_executable ! - link_directories (${LLVM_LIBRARY_DIRS}) - endif () add_executable (clickhouse main.cpp) target_link_libraries (clickhouse PRIVATE clickhouse_common_io string_utils) target_include_directories (clickhouse BEFORE PRIVATE ${COMMON_INCLUDE_DIR}) @@ -154,9 +145,6 @@ else () if (ENABLE_CLICKHOUSE_OBFUSCATOR) clickhouse_target_link_split_lib(clickhouse obfuscator) endif () - if (USE_EMBEDDED_COMPILER) - target_link_libraries(clickhouse PRIVATE clickhouse-compiler-lib) - endif () set (CLICKHOUSE_BUNDLE) if (ENABLE_CLICKHOUSE_SERVER) @@ -213,18 +201,8 @@ else () list(APPEND CLICKHOUSE_BUNDLE clickhouse-odbc-bridge) endif() - # install always because depian package want this files: - add_custom_target (clickhouse-clang ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-clang DEPENDS clickhouse) - add_custom_target (clickhouse-lld ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-lld DEPENDS clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-clang clickhouse-lld) - install (TARGETS clickhouse RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - install (FILES - ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-clang - ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-lld - DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - add_custom_target (clickhouse-bundle ALL DEPENDS ${CLICKHOUSE_BUNDLE}) endif () diff --git a/dbms/programs/clang/CMakeLists.txt b/dbms/programs/clang/CMakeLists.txt deleted file mode 100644 index 82f520614f4..00000000000 --- a/dbms/programs/clang/CMakeLists.txt +++ /dev/null @@ -1,38 +0,0 @@ -if (USE_EMBEDDED_COMPILER) - add_subdirectory ("Compiler-${LLVM_VERSION}") -endif () - -if (CLICKHOUSE_SPLIT_BINARY) - if (USE_EMBEDDED_COMPILER) - link_directories (${LLVM_LIBRARY_DIRS}) - add_executable (clickhouse-clang clickhouse-clang.cpp) - target_link_libraries (clickhouse-clang PRIVATE clickhouse-compiler-lib) - add_executable (clickhouse-lld clickhouse-lld.cpp) - target_link_libraries (clickhouse-lld PRIVATE clickhouse-compiler-lib) - install (TARGETS clickhouse-clang clickhouse-lld RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - endif () -endif () - -set (TMP_HEADERS_DIR "${CMAKE_CURRENT_BINARY_DIR}/${INTERNAL_COMPILER_HEADERS_RELATIVE}") -# Make and install empty dir for debian package if compiler disabled -add_custom_target (make-headers-directory ALL COMMAND ${CMAKE_COMMAND} -E make_directory ${TMP_HEADERS_DIR}) -install (DIRECTORY ${TMP_HEADERS_DIR} DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse/${INTERNAL_COMPILER_HEADERS_DIR} COMPONENT clickhouse) -# TODO: fix on macos copy_headers.sh: sed --posix - -if (USE_EMBEDDED_COMPILER) - set (COPY_HEADERS_COMPILER "${CMAKE_CURRENT_BINARY_DIR}/../${INTERNAL_COMPILER_EXECUTABLE}") - set (COPY_HEADERS_DEPENDS clickhouse-clang) -elseif (EXISTS ${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE}) - set (COPY_HEADERS_COMPILER "${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE}") -endif () - -if (COPY_HEADERS_COMPILER) - add_custom_target (copy-headers [ -f ${TMP_HEADERS_DIR}/dbms/src/Interpreters/SpecializedAggregator.h ] || env CLANG=${COPY_HEADERS_COMPILER} BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} CMAKE_CXX_COMPILER_VERSION=${CMAKE_CXX_COMPILER_VERSION} ${CMAKE_CURRENT_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS ${COPY_HEADERS_DEPENDS} WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES copy_headers.sh) - - if (USE_INTERNAL_LLVM_LIBRARY) - set (CLANG_HEADERS_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm/clang/lib/Headers") - set (CLANG_HEADERS_DEST "${TMP_HEADERS_DIR}/usr/local/lib/clang/${LLVM_VERSION}/include") # original: ${LLVM_LIBRARY_OUTPUT_INTDIR}/clang/${CLANG_VERSION}/include - add_custom_target (copy-headers-clang ${CMAKE_COMMAND} -E make_directory ${CLANG_HEADERS_DEST} && ${CMAKE_COMMAND} -E copy_if_different ${CLANG_HEADERS_DIR}/* ${CLANG_HEADERS_DEST} ) - add_dependencies (copy-headers copy-headers-clang) - endif () -endif () diff --git a/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt deleted file mode 100644 index 83e38cea257..00000000000 --- a/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt +++ /dev/null @@ -1,53 +0,0 @@ -add_definitions(-Wno-error -Wno-unused-parameter -Wno-non-virtual-dtor -U_LIBCPP_DEBUG) - -link_directories(${LLVM_LIBRARY_DIRS}) - -add_library(clickhouse-compiler-lib - driver.cpp - cc1_main.cpp - cc1as_main.cpp - lld.cpp) - -target_compile_options(clickhouse-compiler-lib PRIVATE -fno-rtti -fno-exceptions -g0) - -string(REPLACE "${INCLUDE_DEBUG_HELPERS}" "" CMAKE_CXX_FLAGS ${CMAKE_CXX_FLAGS}) # cant compile with -fno-rtti - -llvm_libs_all(REQUIRED_LLVM_LIBRARIES) - -message(STATUS "Using LLVM ${LLVM_VERSION}: ${LLVM_INCLUDE_DIRS} : ${REQUIRED_LLVM_LIBRARIES}") - -target_include_directories(clickhouse-compiler-lib SYSTEM PRIVATE ${LLVM_INCLUDE_DIRS}) - -# This is extracted almost directly from CMakeFiles/.../link.txt in LLVM build directory. - -target_link_libraries(clickhouse-compiler-lib PRIVATE - -clangBasic clangCodeGen clangDriver clangFrontend clangFrontendTool -clangRewriteFrontend clangARCMigrate clangStaticAnalyzerFrontend -clangParse clangSerialization clangSema clangEdit clangStaticAnalyzerCheckers -clangASTMatchers clangStaticAnalyzerCore clangAnalysis clangAST clangRewrite clangLex clangBasic - -lldCOFF -lldDriver -lldELF -#lldMinGW -lldMachO -lldReaderWriter -lldYAML -#lldCommon -lldCore -lldConfig - -${REQUIRED_LLVM_LIBRARIES} - -LLVMSupport - -#Polly -#PollyISL -#PollyPPCG - -PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARIES} Threads::Threads -${MALLOC_LIBRARIES} -${GLIBC_COMPATIBILITY_LIBRARIES} -${MEMCPY_LIBRARIES} -) diff --git a/dbms/programs/clang/Compiler-5.0.0/LICENSE.TXT b/dbms/programs/clang/Compiler-5.0.0/LICENSE.TXT deleted file mode 100644 index b452ca2efd8..00000000000 --- a/dbms/programs/clang/Compiler-5.0.0/LICENSE.TXT +++ /dev/null @@ -1,63 +0,0 @@ -============================================================================== -LLVM Release License -============================================================================== -University of Illinois/NCSA -Open Source License - -Copyright (c) 2007-2016 University of Illinois at Urbana-Champaign. -All rights reserved. - -Developed by: - - LLVM Team - - University of Illinois at Urbana-Champaign - - http://llvm.org - -Permission is hereby granted, free of charge, to any person obtaining a copy of -this software and associated documentation files (the "Software"), to deal with -the Software without restriction, including without limitation the rights to -use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies -of the Software, and to permit persons to whom the Software is furnished to do -so, subject to the following conditions: - - * Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimers. - - * Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimers in the - documentation and/or other materials provided with the distribution. - - * Neither the names of the LLVM Team, University of Illinois at - Urbana-Champaign, nor the names of its contributors may be used to - endorse or promote products derived from this Software without specific - prior written permission. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS -FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -CONTRIBUTORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS WITH THE -SOFTWARE. - -============================================================================== -The LLVM software contains code written by third parties. Such software will -have its own individual LICENSE.TXT file in the directory in which it appears. -This file will describe the copyrights, license, and restrictions which apply -to that code. - -The disclaimer of warranty in the University of Illinois Open Source License -applies to all code in the LLVM Distribution, and nothing in any of the -other licenses gives permission to use the names of the LLVM Team or the -University of Illinois to endorse or promote products derived from this -Software. - -The following pieces of software have additional or alternate copyrights, -licenses, and/or restrictions: - -Program Directory -------- --------- - - diff --git a/dbms/programs/clang/Compiler-5.0.0/cc1_main.cpp b/dbms/programs/clang/Compiler-5.0.0/cc1_main.cpp deleted file mode 100644 index f6eabaf3387..00000000000 --- a/dbms/programs/clang/Compiler-5.0.0/cc1_main.cpp +++ /dev/null @@ -1,242 +0,0 @@ -//===-- cc1_main.cpp - Clang CC1 Compiler Frontend ------------------------===// -// -// The LLVM Compiler Infrastructure -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This is the entry point to the clang -cc1 functionality, which implements the -// core compiler functionality along with a number of additional tools for -// demonstration and testing purposes. -// -//===----------------------------------------------------------------------===// - -#include "llvm/Option/Arg.h" -#include "clang/CodeGen/ObjectFilePCHContainerOperations.h" -#include "clang/Config/config.h" -#include "clang/Driver/DriverDiagnostic.h" -#include "clang/Driver/Options.h" -#include "clang/Frontend/CompilerInstance.h" -#include "clang/Frontend/CompilerInvocation.h" -#include "clang/Frontend/FrontendDiagnostic.h" -#include "clang/Frontend/TextDiagnosticBuffer.h" -#include "clang/Frontend/TextDiagnosticPrinter.h" -#include "clang/Frontend/Utils.h" -#include "clang/FrontendTool/Utils.h" -#include "llvm/ADT/Statistic.h" -#include "llvm/LinkAllPasses.h" -#include "llvm/Option/ArgList.h" -#include "llvm/Option/OptTable.h" -#include "llvm/Support/Compiler.h" -#include "llvm/Support/ErrorHandling.h" -#include "llvm/Support/ManagedStatic.h" -#include "llvm/Support/Signals.h" -#include "llvm/Support/TargetSelect.h" -#include "llvm/Support/Timer.h" -#include "llvm/Support/raw_ostream.h" -#include - -#ifdef CLANG_HAVE_RLIMITS -#include -#endif - -// have no .a version in packages -#undef LINK_POLLY_INTO_TOOLS - -using namespace clang; -using namespace llvm::opt; - -//===----------------------------------------------------------------------===// -// Main driver -//===----------------------------------------------------------------------===// - -static void LLVMErrorHandler(void *UserData, const std::string &Message, - bool GenCrashDiag) { - DiagnosticsEngine &Diags = *static_cast(UserData); - - Diags.Report(diag::err_fe_error_backend) << Message; - - // Run the interrupt handlers to make sure any special cleanups get done, in - // particular that we remove files registered with RemoveFileOnSignal. - llvm::sys::RunInterruptHandlers(); - - // We cannot recover from llvm errors. When reporting a fatal error, exit - // with status 70 to generate crash diagnostics. For BSD systems this is - // defined as an internal software error. Otherwise, exit with status 1. - exit(GenCrashDiag ? 70 : 1); -} - -#ifdef LINK_POLLY_INTO_TOOLS -namespace polly { -void initializePollyPasses(llvm::PassRegistry &Registry); -} -#endif - -#ifdef CLANG_HAVE_RLIMITS -// The amount of stack we think is "sufficient". If less than this much is -// available, we may be unable to reach our template instantiation depth -// limit and other similar limits. -// FIXME: Unify this with the stack we request when spawning a thread to build -// a module. -static const int kSufficientStack = 8 << 20; - -#if defined(__linux__) && defined(__PIE__) -static size_t getCurrentStackAllocation() { - // If we can't compute the current stack usage, allow for 512K of command - // line arguments and environment. - size_t Usage = 512 * 1024; - if (FILE *StatFile = fopen("/proc/self/stat", "r")) { - // We assume that the stack extends from its current address to the end of - // the environment space. In reality, there is another string literal (the - // program name) after the environment, but this is close enough (we only - // need to be within 100K or so). - unsigned long StackPtr, EnvEnd; - // Disable silly GCC -Wformat warning that complains about length - // modifiers on ignored format specifiers. We want to retain these - // for documentation purposes even though they have no effect. -#if defined(__GNUC__) && !defined(__clang__) -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wformat" -#endif - if (fscanf(StatFile, - "%*d %*s %*c %*d %*d %*d %*d %*d %*u %*lu %*lu %*lu %*lu %*lu " - "%*lu %*ld %*ld %*ld %*ld %*ld %*ld %*llu %*lu %*ld %*lu %*lu " - "%*lu %*lu %lu %*lu %*lu %*lu %*lu %*lu %*llu %*lu %*lu %*d %*d " - "%*u %*u %*llu %*lu %*ld %*lu %*lu %*lu %*lu %*lu %*lu %lu %*d", - &StackPtr, &EnvEnd) == 2) { -#if defined(__GNUC__) && !defined(__clang__) -#pragma GCC diagnostic pop -#endif - Usage = StackPtr < EnvEnd ? EnvEnd - StackPtr : StackPtr - EnvEnd; - } - fclose(StatFile); - } - return Usage; -} - -#include - -LLVM_ATTRIBUTE_NOINLINE -static void ensureStackAddressSpace(int ExtraChunks = 0) { - // Linux kernels prior to 4.1 will sometimes locate the heap of a PIE binary - // relatively close to the stack (they are only guaranteed to be 128MiB - // apart). This results in crashes if we happen to heap-allocate more than - // 128MiB before we reach our stack high-water mark. - // - // To avoid these crashes, ensure that we have sufficient virtual memory - // pages allocated before we start running. - size_t Curr = getCurrentStackAllocation(); - const int kTargetStack = kSufficientStack - 256 * 1024; - if (Curr < kTargetStack) { - volatile char *volatile Alloc = - static_cast(alloca(kTargetStack - Curr)); - Alloc[0] = 0; - Alloc[kTargetStack - Curr - 1] = 0; - } -} -#else -static void ensureStackAddressSpace() {} -#endif - -/// Attempt to ensure that we have at least 8MiB of usable stack space. -static void ensureSufficientStack() { - struct rlimit rlim; - if (getrlimit(RLIMIT_STACK, &rlim) != 0) - return; - - // Increase the soft stack limit to our desired level, if necessary and - // possible. - if (rlim.rlim_cur != RLIM_INFINITY && rlim.rlim_cur < kSufficientStack) { - // Try to allocate sufficient stack. - if (rlim.rlim_max == RLIM_INFINITY || rlim.rlim_max >= kSufficientStack) - rlim.rlim_cur = kSufficientStack; - else if (rlim.rlim_cur == rlim.rlim_max) - return; - else - rlim.rlim_cur = rlim.rlim_max; - - if (setrlimit(RLIMIT_STACK, &rlim) != 0 || - rlim.rlim_cur != kSufficientStack) - return; - } - - // We should now have a stack of size at least kSufficientStack. Ensure - // that we can actually use that much, if necessary. - ensureStackAddressSpace(); -} -#else -static void ensureSufficientStack() {} -#endif - -int cc1_main(ArrayRef Argv, const char *Argv0, void *MainAddr) { - ensureSufficientStack(); - - std::unique_ptr Clang(new CompilerInstance()); - IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); - - // Register the support for object-file-wrapped Clang modules. - auto PCHOps = Clang->getPCHContainerOperations(); - PCHOps->registerWriter(llvm::make_unique()); - PCHOps->registerReader(llvm::make_unique()); - - // Initialize targets first, so that --version shows registered targets. - llvm::InitializeAllTargets(); - llvm::InitializeAllTargetMCs(); - llvm::InitializeAllAsmPrinters(); - llvm::InitializeAllAsmParsers(); - -#ifdef LINK_POLLY_INTO_TOOLS - llvm::PassRegistry &Registry = *llvm::PassRegistry::getPassRegistry(); - polly::initializePollyPasses(Registry); -#endif - - // Buffer diagnostics from argument parsing so that we can output them using a - // well formed diagnostic object. - IntrusiveRefCntPtr DiagOpts = new DiagnosticOptions(); - TextDiagnosticBuffer *DiagsBuffer = new TextDiagnosticBuffer; - DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagsBuffer); - bool Success = CompilerInvocation::CreateFromArgs( - Clang->getInvocation(), Argv.begin(), Argv.end(), Diags); - - // Infer the builtin include path if unspecified. - if (Clang->getHeaderSearchOpts().UseBuiltinIncludes && - Clang->getHeaderSearchOpts().ResourceDir.empty()) - Clang->getHeaderSearchOpts().ResourceDir = - CompilerInvocation::GetResourcesPath(Argv0, MainAddr); - - // Create the actual diagnostics engine. - Clang->createDiagnostics(); - if (!Clang->hasDiagnostics()) - return 1; - - // Set an error handler, so that any LLVM backend diagnostics go through our - // error handler. - llvm::install_fatal_error_handler(LLVMErrorHandler, - static_cast(&Clang->getDiagnostics())); - - DiagsBuffer->FlushDiagnostics(Clang->getDiagnostics()); - if (!Success) - return 1; - - // Execute the frontend actions. - Success = ExecuteCompilerInvocation(Clang.get()); - - // If any timers were active but haven't been destroyed yet, print their - // results now. This happens in -disable-free mode. - llvm::TimerGroup::printAll(llvm::errs()); - - // Our error handler depends on the Diagnostics object, which we're - // potentially about to delete. Uninstall the handler now so that any - // later errors use the default handling behavior instead. - llvm::remove_fatal_error_handler(); - - // When running with -disable-free, don't do any destruction or shutdown. - if (Clang->getFrontendOpts().DisableFree) { - BuryPointer(std::move(Clang)); - return !Success; - } - - return !Success; -} diff --git a/dbms/programs/clang/Compiler-5.0.0/cc1as_main.cpp b/dbms/programs/clang/Compiler-5.0.0/cc1as_main.cpp deleted file mode 100644 index 2fc2b508ef2..00000000000 --- a/dbms/programs/clang/Compiler-5.0.0/cc1as_main.cpp +++ /dev/null @@ -1,540 +0,0 @@ -//===-- cc1as_main.cpp - Clang Assembler ---------------------------------===// -// -// The LLVM Compiler Infrastructure -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This is the entry point to the clang -cc1as functionality, which implements -// the direct interface to the LLVM MC based assembler. -// -//===----------------------------------------------------------------------===// - -#include "clang/Basic/Diagnostic.h" -#include "clang/Basic/DiagnosticOptions.h" -#include "clang/Driver/DriverDiagnostic.h" -#include "clang/Driver/Options.h" -#include "clang/Frontend/FrontendDiagnostic.h" -#include "clang/Frontend/TextDiagnosticPrinter.h" -#include "clang/Frontend/Utils.h" -#include "llvm/ADT/STLExtras.h" -#include "llvm/ADT/StringSwitch.h" -#include "llvm/ADT/Triple.h" -#include "llvm/IR/DataLayout.h" -#include "llvm/MC/MCAsmBackend.h" -#include "llvm/MC/MCAsmInfo.h" -#include "llvm/MC/MCCodeEmitter.h" -#include "llvm/MC/MCContext.h" -#include "llvm/MC/MCInstrInfo.h" -#include "llvm/MC/MCObjectFileInfo.h" -#include "llvm/MC/MCParser/MCAsmParser.h" -#include "llvm/MC/MCParser/MCTargetAsmParser.h" -#include "llvm/MC/MCRegisterInfo.h" -#include "llvm/MC/MCStreamer.h" -#include "llvm/MC/MCSubtargetInfo.h" -#include "llvm/MC/MCTargetOptions.h" -#include "llvm/Option/Arg.h" -#include "llvm/Option/ArgList.h" -#include "llvm/Option/OptTable.h" -#include "llvm/Support/CommandLine.h" -#include "llvm/Support/ErrorHandling.h" -#include "llvm/Support/FileSystem.h" -#include "llvm/Support/FormattedStream.h" -#include "llvm/Support/Host.h" -#include "llvm/Support/MemoryBuffer.h" -#include "llvm/Support/Path.h" -#include "llvm/Support/Signals.h" -#include "llvm/Support/SourceMgr.h" -#include "llvm/Support/TargetRegistry.h" -#include "llvm/Support/TargetSelect.h" -#include "llvm/Support/Timer.h" -#include "llvm/Support/raw_ostream.h" -#include -#include -using namespace clang; -using namespace clang::driver; -using namespace clang::driver::options; -using namespace llvm; -using namespace llvm::opt; - -namespace { - -/// \brief Helper class for representing a single invocation of the assembler. -struct AssemblerInvocation { - /// @name Target Options - /// @{ - - /// The name of the target triple to assemble for. - std::string Triple; - - /// If given, the name of the target CPU to determine which instructions - /// are legal. - std::string CPU; - - /// The list of target specific features to enable or disable -- this should - /// be a list of strings starting with '+' or '-'. - std::vector Features; - - /// The list of symbol definitions. - std::vector SymbolDefs; - - /// @} - /// @name Language Options - /// @{ - - std::vector IncludePaths; - unsigned NoInitialTextSection : 1; - unsigned SaveTemporaryLabels : 1; - unsigned GenDwarfForAssembly : 1; - unsigned RelaxELFRelocations : 1; - unsigned DwarfVersion; - std::string DwarfDebugFlags; - std::string DwarfDebugProducer; - std::string DebugCompilationDir; - llvm::DebugCompressionType CompressDebugSections = - llvm::DebugCompressionType::None; - std::string MainFileName; - - /// @} - /// @name Frontend Options - /// @{ - - std::string InputFile; - std::vector LLVMArgs; - std::string OutputPath; - enum FileType { - FT_Asm, ///< Assembly (.s) output, transliterate mode. - FT_Null, ///< No output, for timing purposes. - FT_Obj ///< Object file output. - }; - FileType OutputType; - unsigned ShowHelp : 1; - unsigned ShowVersion : 1; - - /// @} - /// @name Transliterate Options - /// @{ - - unsigned OutputAsmVariant; - unsigned ShowEncoding : 1; - unsigned ShowInst : 1; - - /// @} - /// @name Assembler Options - /// @{ - - unsigned RelaxAll : 1; - unsigned NoExecStack : 1; - unsigned FatalWarnings : 1; - unsigned IncrementalLinkerCompatible : 1; - - /// The name of the relocation model to use. - std::string RelocationModel; - - /// @} - -public: - AssemblerInvocation() { - Triple = ""; - NoInitialTextSection = 0; - InputFile = "-"; - OutputPath = "-"; - OutputType = FT_Asm; - OutputAsmVariant = 0; - ShowInst = 0; - ShowEncoding = 0; - RelaxAll = 0; - NoExecStack = 0; - FatalWarnings = 0; - IncrementalLinkerCompatible = 0; - DwarfVersion = 0; - } - - static bool CreateFromArgs(AssemblerInvocation &Res, - ArrayRef Argv, - DiagnosticsEngine &Diags); -}; - -} - -bool AssemblerInvocation::CreateFromArgs(AssemblerInvocation &Opts, - ArrayRef Argv, - DiagnosticsEngine &Diags) { - bool Success = true; - - // Parse the arguments. - std::unique_ptr OptTbl(createDriverOptTable()); - - const unsigned IncludedFlagsBitmask = options::CC1AsOption; - unsigned MissingArgIndex, MissingArgCount; - InputArgList Args = OptTbl->ParseArgs(Argv, MissingArgIndex, MissingArgCount, - IncludedFlagsBitmask); - - // Check for missing argument error. - if (MissingArgCount) { - Diags.Report(diag::err_drv_missing_argument) - << Args.getArgString(MissingArgIndex) << MissingArgCount; - Success = false; - } - - // Issue errors on unknown arguments. - for (const Arg *A : Args.filtered(OPT_UNKNOWN)) { - Diags.Report(diag::err_drv_unknown_argument) << A->getAsString(Args); - Success = false; - } - - // Construct the invocation. - - // Target Options - Opts.Triple = llvm::Triple::normalize(Args.getLastArgValue(OPT_triple)); - Opts.CPU = Args.getLastArgValue(OPT_target_cpu); - Opts.Features = Args.getAllArgValues(OPT_target_feature); - - // Use the default target triple if unspecified. - if (Opts.Triple.empty()) - Opts.Triple = llvm::sys::getDefaultTargetTriple(); - - // Language Options - Opts.IncludePaths = Args.getAllArgValues(OPT_I); - Opts.NoInitialTextSection = Args.hasArg(OPT_n); - Opts.SaveTemporaryLabels = Args.hasArg(OPT_msave_temp_labels); - // Any DebugInfoKind implies GenDwarfForAssembly. - Opts.GenDwarfForAssembly = Args.hasArg(OPT_debug_info_kind_EQ); - - if (const Arg *A = Args.getLastArg(OPT_compress_debug_sections, - OPT_compress_debug_sections_EQ)) { - if (A->getOption().getID() == OPT_compress_debug_sections) { - // TODO: be more clever about the compression type auto-detection - Opts.CompressDebugSections = llvm::DebugCompressionType::GNU; - } else { - Opts.CompressDebugSections = - llvm::StringSwitch(A->getValue()) - .Case("none", llvm::DebugCompressionType::None) - .Case("zlib", llvm::DebugCompressionType::Z) - .Case("zlib-gnu", llvm::DebugCompressionType::GNU) - .Default(llvm::DebugCompressionType::None); - } - } - - Opts.RelaxELFRelocations = Args.hasArg(OPT_mrelax_relocations); - Opts.DwarfVersion = getLastArgIntValue(Args, OPT_dwarf_version_EQ, 2, Diags); - Opts.DwarfDebugFlags = Args.getLastArgValue(OPT_dwarf_debug_flags); - Opts.DwarfDebugProducer = Args.getLastArgValue(OPT_dwarf_debug_producer); - Opts.DebugCompilationDir = Args.getLastArgValue(OPT_fdebug_compilation_dir); - Opts.MainFileName = Args.getLastArgValue(OPT_main_file_name); - - // Frontend Options - if (Args.hasArg(OPT_INPUT)) { - bool First = true; - for (const Arg *A : Args.filtered(OPT_INPUT)) { - if (First) { - Opts.InputFile = A->getValue(); - First = false; - } else { - Diags.Report(diag::err_drv_unknown_argument) << A->getAsString(Args); - Success = false; - } - } - } - Opts.LLVMArgs = Args.getAllArgValues(OPT_mllvm); - Opts.OutputPath = Args.getLastArgValue(OPT_o); - if (Arg *A = Args.getLastArg(OPT_filetype)) { - StringRef Name = A->getValue(); - unsigned OutputType = StringSwitch(Name) - .Case("asm", FT_Asm) - .Case("null", FT_Null) - .Case("obj", FT_Obj) - .Default(~0U); - if (OutputType == ~0U) { - Diags.Report(diag::err_drv_invalid_value) << A->getAsString(Args) << Name; - Success = false; - } else - Opts.OutputType = FileType(OutputType); - } - Opts.ShowHelp = Args.hasArg(OPT_help); - Opts.ShowVersion = Args.hasArg(OPT_version); - - // Transliterate Options - Opts.OutputAsmVariant = - getLastArgIntValue(Args, OPT_output_asm_variant, 0, Diags); - Opts.ShowEncoding = Args.hasArg(OPT_show_encoding); - Opts.ShowInst = Args.hasArg(OPT_show_inst); - - // Assemble Options - Opts.RelaxAll = Args.hasArg(OPT_mrelax_all); - Opts.NoExecStack = Args.hasArg(OPT_mno_exec_stack); - Opts.FatalWarnings = Args.hasArg(OPT_massembler_fatal_warnings); - Opts.RelocationModel = Args.getLastArgValue(OPT_mrelocation_model, "pic"); - Opts.IncrementalLinkerCompatible = - Args.hasArg(OPT_mincremental_linker_compatible); - Opts.SymbolDefs = Args.getAllArgValues(OPT_defsym); - - return Success; -} - -static std::unique_ptr -getOutputStream(AssemblerInvocation &Opts, DiagnosticsEngine &Diags, - bool Binary) { - if (Opts.OutputPath.empty()) - Opts.OutputPath = "-"; - - // Make sure that the Out file gets unlinked from the disk if we get a - // SIGINT. - if (Opts.OutputPath != "-") - sys::RemoveFileOnSignal(Opts.OutputPath); - - std::error_code EC; - auto Out = llvm::make_unique( - Opts.OutputPath, EC, (Binary ? sys::fs::F_None : sys::fs::F_Text)); - if (EC) { - Diags.Report(diag::err_fe_unable_to_open_output) << Opts.OutputPath - << EC.message(); - return nullptr; - } - - return Out; -} - -static bool ExecuteAssembler(AssemblerInvocation &Opts, - DiagnosticsEngine &Diags) { - // Get the target specific parser. - std::string Error; - const Target *TheTarget = TargetRegistry::lookupTarget(Opts.Triple, Error); - if (!TheTarget) - return Diags.Report(diag::err_target_unknown_triple) << Opts.Triple; - - ErrorOr> Buffer = - MemoryBuffer::getFileOrSTDIN(Opts.InputFile); - - if (std::error_code EC = Buffer.getError()) { - Error = EC.message(); - return Diags.Report(diag::err_fe_error_reading) << Opts.InputFile; - } - - SourceMgr SrcMgr; - - // Tell SrcMgr about this buffer, which is what the parser will pick up. - SrcMgr.AddNewSourceBuffer(std::move(*Buffer), SMLoc()); - - // Record the location of the include directories so that the lexer can find - // it later. - SrcMgr.setIncludeDirs(Opts.IncludePaths); - - std::unique_ptr MRI(TheTarget->createMCRegInfo(Opts.Triple)); - assert(MRI && "Unable to create target register info!"); - - std::unique_ptr MAI(TheTarget->createMCAsmInfo(*MRI, Opts.Triple)); - assert(MAI && "Unable to create target asm info!"); - - // Ensure MCAsmInfo initialization occurs before any use, otherwise sections - // may be created with a combination of default and explicit settings. - MAI->setCompressDebugSections(Opts.CompressDebugSections); - - MAI->setRelaxELFRelocations(Opts.RelaxELFRelocations); - - bool IsBinary = Opts.OutputType == AssemblerInvocation::FT_Obj; - std::unique_ptr FDOS = getOutputStream(Opts, Diags, IsBinary); - if (!FDOS) - return true; - - // FIXME: This is not pretty. MCContext has a ptr to MCObjectFileInfo and - // MCObjectFileInfo needs a MCContext reference in order to initialize itself. - std::unique_ptr MOFI(new MCObjectFileInfo()); - - MCContext Ctx(MAI.get(), MRI.get(), MOFI.get(), &SrcMgr); - - bool PIC = false; - if (Opts.RelocationModel == "static") { - PIC = false; - } else if (Opts.RelocationModel == "pic") { - PIC = true; - } else { - assert(Opts.RelocationModel == "dynamic-no-pic" && - "Invalid PIC model!"); - PIC = false; - } - - MOFI->InitMCObjectFileInfo(Triple(Opts.Triple), PIC, CodeModel::Default, Ctx); - if (Opts.SaveTemporaryLabels) - Ctx.setAllowTemporaryLabels(false); - if (Opts.GenDwarfForAssembly) - Ctx.setGenDwarfForAssembly(true); - if (!Opts.DwarfDebugFlags.empty()) - Ctx.setDwarfDebugFlags(StringRef(Opts.DwarfDebugFlags)); - if (!Opts.DwarfDebugProducer.empty()) - Ctx.setDwarfDebugProducer(StringRef(Opts.DwarfDebugProducer)); - if (!Opts.DebugCompilationDir.empty()) - Ctx.setCompilationDir(Opts.DebugCompilationDir); - if (!Opts.MainFileName.empty()) - Ctx.setMainFileName(StringRef(Opts.MainFileName)); - Ctx.setDwarfVersion(Opts.DwarfVersion); - - // Build up the feature string from the target feature list. - std::string FS; - if (!Opts.Features.empty()) { - FS = Opts.Features[0]; - for (unsigned i = 1, e = Opts.Features.size(); i != e; ++i) - FS += "," + Opts.Features[i]; - } - - std::unique_ptr Str; - - std::unique_ptr MCII(TheTarget->createMCInstrInfo()); - std::unique_ptr STI( - TheTarget->createMCSubtargetInfo(Opts.Triple, Opts.CPU, FS)); - - raw_pwrite_stream *Out = FDOS.get(); - std::unique_ptr BOS; - - // FIXME: There is a bit of code duplication with addPassesToEmitFile. - if (Opts.OutputType == AssemblerInvocation::FT_Asm) { - MCInstPrinter *IP = TheTarget->createMCInstPrinter( - llvm::Triple(Opts.Triple), Opts.OutputAsmVariant, *MAI, *MCII, *MRI); - MCCodeEmitter *CE = nullptr; - MCAsmBackend *MAB = nullptr; - if (Opts.ShowEncoding) { - CE = TheTarget->createMCCodeEmitter(*MCII, *MRI, Ctx); - MCTargetOptions Options; - MAB = TheTarget->createMCAsmBackend(*MRI, Opts.Triple, Opts.CPU, Options); - } - auto FOut = llvm::make_unique(*Out); - Str.reset(TheTarget->createAsmStreamer( - Ctx, std::move(FOut), /*asmverbose*/ true, - /*useDwarfDirectory*/ true, IP, CE, MAB, Opts.ShowInst)); - } else if (Opts.OutputType == AssemblerInvocation::FT_Null) { - Str.reset(createNullStreamer(Ctx)); - } else { - assert(Opts.OutputType == AssemblerInvocation::FT_Obj && - "Invalid file type!"); - if (!FDOS->supportsSeeking()) { - BOS = make_unique(*FDOS); - Out = BOS.get(); - } - - MCCodeEmitter *CE = TheTarget->createMCCodeEmitter(*MCII, *MRI, Ctx); - MCTargetOptions Options; - MCAsmBackend *MAB = TheTarget->createMCAsmBackend(*MRI, Opts.Triple, - Opts.CPU, Options); - Triple T(Opts.Triple); - Str.reset(TheTarget->createMCObjectStreamer( - T, Ctx, *MAB, *Out, CE, *STI, Opts.RelaxAll, - Opts.IncrementalLinkerCompatible, - /*DWARFMustBeAtTheEnd*/ true)); - Str.get()->InitSections(Opts.NoExecStack); - } - - bool Failed = false; - - std::unique_ptr Parser( - createMCAsmParser(SrcMgr, Ctx, *Str.get(), *MAI)); - - // FIXME: init MCTargetOptions from sanitizer flags here. - MCTargetOptions Options; - std::unique_ptr TAP( - TheTarget->createMCAsmParser(*STI, *Parser, *MCII, Options)); - if (!TAP) - Failed = Diags.Report(diag::err_target_unknown_triple) << Opts.Triple; - - // Set values for symbols, if any. - for (auto &S : Opts.SymbolDefs) { - auto Pair = StringRef(S).split('='); - auto Sym = Pair.first; - auto Val = Pair.second; - int64_t Value; - // We have already error checked this in the driver. - Val.getAsInteger(0, Value); - Ctx.setSymbolValue(Parser->getStreamer(), Sym, Value); - } - - if (!Failed) { - Parser->setTargetParser(*TAP.get()); - Failed = Parser->Run(Opts.NoInitialTextSection); - } - - // Close Streamer first. - // It might have a reference to the output stream. - Str.reset(); - // Close the output stream early. - BOS.reset(); - FDOS.reset(); - - // Delete output file if there were errors. - if (Failed && Opts.OutputPath != "-") - sys::fs::remove(Opts.OutputPath); - - return Failed; -} - -static void LLVMErrorHandler(void *UserData, const std::string &Message, - bool GenCrashDiag) { - DiagnosticsEngine &Diags = *static_cast(UserData); - - Diags.Report(diag::err_fe_error_backend) << Message; - - // We cannot recover from llvm errors. - exit(1); -} - -int cc1as_main(ArrayRef Argv, const char *Argv0, void *MainAddr) { - // Initialize targets and assembly printers/parsers. - InitializeAllTargetInfos(); - InitializeAllTargetMCs(); - InitializeAllAsmParsers(); - - // Construct our diagnostic client. - IntrusiveRefCntPtr DiagOpts = new DiagnosticOptions(); - TextDiagnosticPrinter *DiagClient - = new TextDiagnosticPrinter(errs(), &*DiagOpts); - DiagClient->setPrefix("clang -cc1as"); - IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); - DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagClient); - - // Set an error handler, so that any LLVM backend diagnostics go through our - // error handler. - ScopedFatalErrorHandler FatalErrorHandler - (LLVMErrorHandler, static_cast(&Diags)); - - // Parse the arguments. - AssemblerInvocation Asm; - if (!AssemblerInvocation::CreateFromArgs(Asm, Argv, Diags)) - return 1; - - if (Asm.ShowHelp) { - std::unique_ptr Opts(driver::createDriverOptTable()); - Opts->PrintHelp(llvm::outs(), "clang -cc1as", "Clang Integrated Assembler", - /*Include=*/driver::options::CC1AsOption, /*Exclude=*/0); - return 0; - } - - // Honor -version. - // - // FIXME: Use a better -version message? - if (Asm.ShowVersion) { - llvm::cl::PrintVersionMessage(); - return 0; - } - - // Honor -mllvm. - // - // FIXME: Remove this, one day. - if (!Asm.LLVMArgs.empty()) { - unsigned NumArgs = Asm.LLVMArgs.size(); - auto Args = llvm::make_unique(NumArgs + 2); - Args[0] = "clang (LLVM option parsing)"; - for (unsigned i = 0; i != NumArgs; ++i) - Args[i + 1] = Asm.LLVMArgs[i].c_str(); - Args[NumArgs + 1] = nullptr; - llvm::cl::ParseCommandLineOptions(NumArgs + 1, Args.get()); - } - - // Execute the invocation, unless there were parsing errors. - bool Failed = Diags.hasErrorOccurred() || ExecuteAssembler(Asm, Diags); - - // If any timers were active but haven't been destroyed yet, print their - // results now. - TimerGroup::printAll(errs()); - - return !!Failed; -} diff --git a/dbms/programs/clang/Compiler-5.0.0/driver.cpp b/dbms/programs/clang/Compiler-5.0.0/driver.cpp deleted file mode 100644 index 5aec2759f9e..00000000000 --- a/dbms/programs/clang/Compiler-5.0.0/driver.cpp +++ /dev/null @@ -1,519 +0,0 @@ -//===-- driver.cpp - Clang GCC-Compatible Driver --------------------------===// -// -// The LLVM Compiler Infrastructure -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This is the entry point to the clang driver; it is a thin wrapper -// for functionality in the Driver clang library. -// -//===----------------------------------------------------------------------===// - -#include "clang/Basic/DiagnosticOptions.h" -#include "clang/Driver/Compilation.h" -#include "clang/Driver/Driver.h" -#include "clang/Driver/DriverDiagnostic.h" -#include "clang/Driver/Options.h" -#include "clang/Driver/ToolChain.h" -#include "clang/Frontend/ChainedDiagnosticConsumer.h" -#include "clang/Frontend/CompilerInvocation.h" -#include "clang/Frontend/SerializedDiagnosticPrinter.h" -#include "clang/Frontend/TextDiagnosticPrinter.h" -#include "clang/Frontend/Utils.h" -#include "llvm/ADT/ArrayRef.h" -#include "llvm/ADT/SmallString.h" -#include "llvm/ADT/SmallVector.h" -#include "llvm/Config/llvm-config.h" -#include "llvm/Option/ArgList.h" -#include "llvm/Option/OptTable.h" -#include "llvm/Option/Option.h" -#include "llvm/Support/CommandLine.h" -#include "llvm/Support/ErrorHandling.h" -#include "llvm/Support/FileSystem.h" -#include "llvm/Support/Host.h" -#include "llvm/Support/ManagedStatic.h" -#include "llvm/Support/Path.h" -#include "llvm/Support/PrettyStackTrace.h" -#include "llvm/Support/Process.h" -#include "llvm/Support/Program.h" -#include "llvm/Support/Regex.h" -#include "llvm/Support/Signals.h" -#include "llvm/Support/StringSaver.h" -#include "llvm/Support/TargetSelect.h" -#include "llvm/Support/Timer.h" -#include "llvm/Support/raw_ostream.h" -#include -#include -#include -using namespace clang; -using namespace clang::driver; -using namespace llvm::opt; - -std::string GetExecutablePath(const char *Argv0, bool CanonicalPrefixes) { - if (!CanonicalPrefixes) { - SmallString<128> ExecutablePath(Argv0); - // Do a PATH lookup if Argv0 isn't a valid path. - if (!llvm::sys::fs::exists(ExecutablePath)) - if (llvm::ErrorOr P = - llvm::sys::findProgramByName(ExecutablePath)) - ExecutablePath = *P; - return ExecutablePath.str(); - } - - // This just needs to be some symbol in the binary; C++ doesn't - // allow taking the address of ::main however. - void *P = (void*) (intptr_t) GetExecutablePath; - return llvm::sys::fs::getMainExecutable(Argv0, P); -} - -static const char *GetStableCStr(std::set &SavedStrings, - StringRef S) { - return SavedStrings.insert(S).first->c_str(); -} - -/// ApplyQAOverride - Apply a list of edits to the input argument lists. -/// -/// The input string is a space separate list of edits to perform, -/// they are applied in order to the input argument lists. Edits -/// should be one of the following forms: -/// -/// '#': Silence information about the changes to the command line arguments. -/// -/// '^': Add FOO as a new argument at the beginning of the command line. -/// -/// '+': Add FOO as a new argument at the end of the command line. -/// -/// 's/XXX/YYY/': Substitute the regular expression XXX with YYY in the command -/// line. -/// -/// 'xOPTION': Removes all instances of the literal argument OPTION. -/// -/// 'XOPTION': Removes all instances of the literal argument OPTION, -/// and the following argument. -/// -/// 'Ox': Removes all flags matching 'O' or 'O[sz0-9]' and adds 'Ox' -/// at the end of the command line. -/// -/// \param OS - The stream to write edit information to. -/// \param Args - The vector of command line arguments. -/// \param Edit - The override command to perform. -/// \param SavedStrings - Set to use for storing string representations. -static void ApplyOneQAOverride(raw_ostream &OS, - SmallVectorImpl &Args, - StringRef Edit, - std::set &SavedStrings) { - // This does not need to be efficient. - - if (Edit[0] == '^') { - const char *Str = - GetStableCStr(SavedStrings, Edit.substr(1)); - OS << "### Adding argument " << Str << " at beginning\n"; - Args.insert(Args.begin() + 1, Str); - } else if (Edit[0] == '+') { - const char *Str = - GetStableCStr(SavedStrings, Edit.substr(1)); - OS << "### Adding argument " << Str << " at end\n"; - Args.push_back(Str); - } else if (Edit[0] == 's' && Edit[1] == '/' && Edit.endswith("/") && - Edit.slice(2, Edit.size()-1).find('/') != StringRef::npos) { - StringRef MatchPattern = Edit.substr(2).split('/').first; - StringRef ReplPattern = Edit.substr(2).split('/').second; - ReplPattern = ReplPattern.slice(0, ReplPattern.size()-1); - - for (unsigned i = 1, e = Args.size(); i != e; ++i) { - // Ignore end-of-line response file markers - if (Args[i] == nullptr) - continue; - std::string Repl = llvm::Regex(MatchPattern).sub(ReplPattern, Args[i]); - - if (Repl != Args[i]) { - OS << "### Replacing '" << Args[i] << "' with '" << Repl << "'\n"; - Args[i] = GetStableCStr(SavedStrings, Repl); - } - } - } else if (Edit[0] == 'x' || Edit[0] == 'X') { - auto Option = Edit.substr(1); - for (unsigned i = 1; i < Args.size();) { - if (Option == Args[i]) { - OS << "### Deleting argument " << Args[i] << '\n'; - Args.erase(Args.begin() + i); - if (Edit[0] == 'X') { - if (i < Args.size()) { - OS << "### Deleting argument " << Args[i] << '\n'; - Args.erase(Args.begin() + i); - } else - OS << "### Invalid X edit, end of command line!\n"; - } - } else - ++i; - } - } else if (Edit[0] == 'O') { - for (unsigned i = 1; i < Args.size();) { - const char *A = Args[i]; - // Ignore end-of-line response file markers - if (A == nullptr) - continue; - if (A[0] == '-' && A[1] == 'O' && - (A[2] == '\0' || - (A[3] == '\0' && (A[2] == 's' || A[2] == 'z' || - ('0' <= A[2] && A[2] <= '9'))))) { - OS << "### Deleting argument " << Args[i] << '\n'; - Args.erase(Args.begin() + i); - } else - ++i; - } - OS << "### Adding argument " << Edit << " at end\n"; - Args.push_back(GetStableCStr(SavedStrings, '-' + Edit.str())); - } else { - OS << "### Unrecognized edit: " << Edit << "\n"; - } -} - -/// ApplyQAOverride - Apply a comma separate list of edits to the -/// input argument lists. See ApplyOneQAOverride. -static void ApplyQAOverride(SmallVectorImpl &Args, - const char *OverrideStr, - std::set &SavedStrings) { - raw_ostream *OS = &llvm::errs(); - - if (OverrideStr[0] == '#') { - ++OverrideStr; - OS = &llvm::nulls(); - } - - *OS << "### CCC_OVERRIDE_OPTIONS: " << OverrideStr << "\n"; - - // This does not need to be efficient. - - const char *S = OverrideStr; - while (*S) { - const char *End = ::strchr(S, ' '); - if (!End) - End = S + strlen(S); - if (End != S) - ApplyOneQAOverride(*OS, Args, std::string(S, End), SavedStrings); - S = End; - if (*S != '\0') - ++S; - } -} - -extern int cc1_main(ArrayRef Argv, const char *Argv0, - void *MainAddr); -extern int cc1as_main(ArrayRef Argv, const char *Argv0, - void *MainAddr); - -static void insertTargetAndModeArgs(StringRef Target, StringRef Mode, - SmallVectorImpl &ArgVector, - std::set &SavedStrings) { - if (!Mode.empty()) { - // Add the mode flag to the arguments. - auto it = ArgVector.begin(); - if (it != ArgVector.end()) - ++it; - ArgVector.insert(it, GetStableCStr(SavedStrings, Mode)); - } - - if (!Target.empty()) { - auto it = ArgVector.begin(); - if (it != ArgVector.end()) - ++it; - const char *arr[] = {"-target", GetStableCStr(SavedStrings, Target)}; - ArgVector.insert(it, std::begin(arr), std::end(arr)); - } -} - -static void getCLEnvVarOptions(std::string &EnvValue, llvm::StringSaver &Saver, - SmallVectorImpl &Opts) { - llvm::cl::TokenizeWindowsCommandLine(EnvValue, Saver, Opts); - // The first instance of '#' should be replaced with '=' in each option. - for (const char *Opt : Opts) - if (char *NumberSignPtr = const_cast(::strchr(Opt, '#'))) - *NumberSignPtr = '='; -} - -static void SetBackdoorDriverOutputsFromEnvVars(Driver &TheDriver) { - // Handle CC_PRINT_OPTIONS and CC_PRINT_OPTIONS_FILE. - TheDriver.CCPrintOptions = !!::getenv("CC_PRINT_OPTIONS"); - if (TheDriver.CCPrintOptions) - TheDriver.CCPrintOptionsFilename = ::getenv("CC_PRINT_OPTIONS_FILE"); - - // Handle CC_PRINT_HEADERS and CC_PRINT_HEADERS_FILE. - TheDriver.CCPrintHeaders = !!::getenv("CC_PRINT_HEADERS"); - if (TheDriver.CCPrintHeaders) - TheDriver.CCPrintHeadersFilename = ::getenv("CC_PRINT_HEADERS_FILE"); - - // Handle CC_LOG_DIAGNOSTICS and CC_LOG_DIAGNOSTICS_FILE. - TheDriver.CCLogDiagnostics = !!::getenv("CC_LOG_DIAGNOSTICS"); - if (TheDriver.CCLogDiagnostics) - TheDriver.CCLogDiagnosticsFilename = ::getenv("CC_LOG_DIAGNOSTICS_FILE"); -} - -static void FixupDiagPrefixExeName(TextDiagnosticPrinter *DiagClient, - const std::string &Path) { - // If the clang binary happens to be named cl.exe for compatibility reasons, - // use clang-cl.exe as the prefix to avoid confusion between clang and MSVC. - StringRef ExeBasename(llvm::sys::path::filename(Path)); - if (ExeBasename.equals_lower("cl.exe")) - ExeBasename = "clang-cl.exe"; - DiagClient->setPrefix(ExeBasename); -} - -// This lets us create the DiagnosticsEngine with a properly-filled-out -// DiagnosticOptions instance. -static DiagnosticOptions * -CreateAndPopulateDiagOpts(ArrayRef argv) { - auto *DiagOpts = new DiagnosticOptions; - std::unique_ptr Opts(createDriverOptTable()); - unsigned MissingArgIndex, MissingArgCount; - InputArgList Args = - Opts->ParseArgs(argv.slice(1), MissingArgIndex, MissingArgCount); - // We ignore MissingArgCount and the return value of ParseDiagnosticArgs. - // Any errors that would be diagnosed here will also be diagnosed later, - // when the DiagnosticsEngine actually exists. - (void)ParseDiagnosticArgs(*DiagOpts, Args); - return DiagOpts; -} - -static void SetInstallDir(SmallVectorImpl &argv, - Driver &TheDriver, bool CanonicalPrefixes) { - // Attempt to find the original path used to invoke the driver, to determine - // the installed path. We do this manually, because we want to support that - // path being a symlink. - SmallString<128> InstalledPath(argv[0]); - - // Do a PATH lookup, if there are no directory components. - if (llvm::sys::path::filename(InstalledPath) == InstalledPath) - if (llvm::ErrorOr Tmp = llvm::sys::findProgramByName( - llvm::sys::path::filename(InstalledPath.str()))) - InstalledPath = *Tmp; - - // FIXME: We don't actually canonicalize this, we just make it absolute. - if (CanonicalPrefixes) - llvm::sys::fs::make_absolute(InstalledPath); - - StringRef InstalledPathParent(llvm::sys::path::parent_path(InstalledPath)); - if (llvm::sys::fs::exists(InstalledPathParent)) - TheDriver.setInstalledDir(InstalledPathParent); -} - -static int ExecuteCC1Tool(ArrayRef argv, StringRef Tool) { - void *GetExecutablePathVP = (void *)(intptr_t) GetExecutablePath; - if (Tool == "") - return cc1_main(argv.slice(2), argv[0], GetExecutablePathVP); - if (Tool == "as") - return cc1as_main(argv.slice(2), argv[0], GetExecutablePathVP); - - // Reject unknown tools. - llvm::errs() << "error: unknown integrated tool '" << Tool << "'\n"; - return 1; -} - -int mainEntryClickHouseClang(int argc_, char **argv_) { - llvm::sys::PrintStackTraceOnErrorSignal(argv_[0]); - llvm::PrettyStackTraceProgram X(argc_, argv_); - llvm::llvm_shutdown_obj Y; // Call llvm_shutdown() on exit. - - if (llvm::sys::Process::FixupStandardFileDescriptors()) - return 1; - - SmallVector argv; - llvm::SpecificBumpPtrAllocator ArgAllocator; - std::error_code EC = llvm::sys::Process::GetArgumentVector( - argv, llvm::makeArrayRef(argv_, argc_), ArgAllocator); - if (EC) { - llvm::errs() << "error: couldn't get arguments: " << EC.message() << '\n'; - return 1; - } - - llvm::InitializeAllTargets(); - std::string ProgName = argv[0]; - std::pair TargetAndMode = - ToolChain::getTargetAndModeFromProgramName(ProgName); - - llvm::BumpPtrAllocator A; - llvm::StringSaver Saver(A); - - // Parse response files using the GNU syntax, unless we're in CL mode. There - // are two ways to put clang in CL compatibility mode: argv[0] is either - // clang-cl or cl, or --driver-mode=cl is on the command line. The normal - // command line parsing can't happen until after response file parsing, so we - // have to manually search for a --driver-mode=cl argument the hard way. - // Finally, our -cc1 tools don't care which tokenization mode we use because - // response files written by clang will tokenize the same way in either mode. - bool ClangCLMode = false; - if (TargetAndMode.second == "--driver-mode=cl" || - std::find_if(argv.begin(), argv.end(), [](const char *F) { - return F && strcmp(F, "--driver-mode=cl") == 0; - }) != argv.end()) { - ClangCLMode = true; - } - enum { Default, POSIX, Windows } RSPQuoting = Default; - for (const char *F : argv) { - if (strcmp(F, "--rsp-quoting=posix") == 0) - RSPQuoting = POSIX; - else if (strcmp(F, "--rsp-quoting=windows") == 0) - RSPQuoting = Windows; - } - - // Determines whether we want nullptr markers in argv to indicate response - // files end-of-lines. We only use this for the /LINK driver argument with - // clang-cl.exe on Windows. - bool MarkEOLs = ClangCLMode; - - llvm::cl::TokenizerCallback Tokenizer; - if (RSPQuoting == Windows || (RSPQuoting == Default && ClangCLMode)) - Tokenizer = &llvm::cl::TokenizeWindowsCommandLine; - else - Tokenizer = &llvm::cl::TokenizeGNUCommandLine; - - if (MarkEOLs && argv.size() > 1 && StringRef(argv[1]).startswith("-cc1")) - MarkEOLs = false; - llvm::cl::ExpandResponseFiles(Saver, Tokenizer, argv, MarkEOLs); - - // Handle -cc1 integrated tools, even if -cc1 was expanded from a response - // file. - auto FirstArg = std::find_if(argv.begin() + 1, argv.end(), - [](const char *A) { return A != nullptr; }); - if (FirstArg != argv.end() && StringRef(*FirstArg).startswith("-cc1")) { - // If -cc1 came from a response file, remove the EOL sentinels. - if (MarkEOLs) { - auto newEnd = std::remove(argv.begin(), argv.end(), nullptr); - argv.resize(newEnd - argv.begin()); - } - return ExecuteCC1Tool(argv, argv[1] + 4); - } - - bool CanonicalPrefixes = true; - for (int i = 1, size = argv.size(); i < size; ++i) { - // Skip end-of-line response file markers - if (argv[i] == nullptr) - continue; - if (StringRef(argv[i]) == "-no-canonical-prefixes") { - CanonicalPrefixes = false; - break; - } - } - - // Handle CL and _CL_ which permits additional command line options to be - // prepended or appended. - if (ClangCLMode) { - // Arguments in "CL" are prepended. - llvm::Optional OptCL = llvm::sys::Process::GetEnv("CL"); - if (OptCL.hasValue()) { - SmallVector PrependedOpts; - getCLEnvVarOptions(OptCL.getValue(), Saver, PrependedOpts); - - // Insert right after the program name to prepend to the argument list. - argv.insert(argv.begin() + 1, PrependedOpts.begin(), PrependedOpts.end()); - } - // Arguments in "_CL_" are appended. - llvm::Optional Opt_CL_ = llvm::sys::Process::GetEnv("_CL_"); - if (Opt_CL_.hasValue()) { - SmallVector AppendedOpts; - getCLEnvVarOptions(Opt_CL_.getValue(), Saver, AppendedOpts); - - // Insert at the end of the argument list to append. - argv.append(AppendedOpts.begin(), AppendedOpts.end()); - } - } - - std::set SavedStrings; - // Handle CCC_OVERRIDE_OPTIONS, used for editing a command line behind the - // scenes. - if (const char *OverrideStr = ::getenv("CCC_OVERRIDE_OPTIONS")) { - // FIXME: Driver shouldn't take extra initial argument. - ApplyQAOverride(argv, OverrideStr, SavedStrings); - } - - std::string Path = GetExecutablePath(argv[0], CanonicalPrefixes); - - IntrusiveRefCntPtr DiagOpts = - CreateAndPopulateDiagOpts(argv); - - TextDiagnosticPrinter *DiagClient - = new TextDiagnosticPrinter(llvm::errs(), &*DiagOpts); - FixupDiagPrefixExeName(DiagClient, Path); - - IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); - - DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagClient); - - if (!DiagOpts->DiagnosticSerializationFile.empty()) { - auto SerializedConsumer = - clang::serialized_diags::create(DiagOpts->DiagnosticSerializationFile, - &*DiagOpts, /*MergeChildRecords=*/true); - Diags.setClient(new ChainedDiagnosticConsumer( - Diags.takeClient(), std::move(SerializedConsumer))); - } - - ProcessWarningOptions(Diags, *DiagOpts, /*ReportDiags=*/false); - - Driver TheDriver(Path, llvm::sys::getDefaultTargetTriple(), Diags); - SetInstallDir(argv, TheDriver, CanonicalPrefixes); - - insertTargetAndModeArgs(TargetAndMode.first, TargetAndMode.second, argv, - SavedStrings); - - SetBackdoorDriverOutputsFromEnvVars(TheDriver); - - std::unique_ptr C(TheDriver.BuildCompilation(argv)); - int Res = 1; - if (C && !C->containsError()) { - SmallVector, 4> FailingCommands; - Res = TheDriver.ExecuteCompilation(*C, FailingCommands); - - // Force a crash to test the diagnostics. - if (TheDriver.GenReproducer) { - Diags.Report(diag::err_drv_force_crash) - << !::getenv("FORCE_CLANG_DIAGNOSTICS_CRASH"); - - // Pretend that every command failed. - FailingCommands.clear(); - for (const auto &J : C->getJobs()) - if (const Command *C = dyn_cast(&J)) - FailingCommands.push_back(std::make_pair(-1, C)); - } - - for (const auto &P : FailingCommands) { - int CommandRes = P.first; - const Command *FailingCommand = P.second; - if (!Res) - Res = CommandRes; - - // If result status is < 0, then the driver command signalled an error. - // If result status is 70, then the driver command reported a fatal error. - // On Windows, abort will return an exit code of 3. In these cases, - // generate additional diagnostic information if possible. - bool DiagnoseCrash = CommandRes < 0 || CommandRes == 70; -#ifdef LLVM_ON_WIN32 - DiagnoseCrash |= CommandRes == 3; -#endif - if (DiagnoseCrash) { - TheDriver.generateCompilationDiagnostics(*C, *FailingCommand); - break; - } - } - } - - Diags.getClient()->finish(); - - // If any timers were active but haven't been destroyed yet, print their - // results now. This happens in -disable-free mode. - llvm::TimerGroup::printAll(llvm::errs()); - -#ifdef LLVM_ON_WIN32 - // Exit status should not be negative on Win32, unless abnormal termination. - // Once abnormal termiation was caught, negative status should not be - // propagated. - if (Res < 0) - Res = 1; -#endif - - // If we have multiple failing commands, we return the result of the first - // failing command. - return Res; -} diff --git a/dbms/programs/clang/Compiler-5.0.0/lld.cpp b/dbms/programs/clang/Compiler-5.0.0/lld.cpp deleted file mode 100644 index 5af29868864..00000000000 --- a/dbms/programs/clang/Compiler-5.0.0/lld.cpp +++ /dev/null @@ -1,23 +0,0 @@ -#include "lld/Driver/Driver.h" -#include "llvm/ADT/STLExtras.h" -#include "llvm/ADT/StringSwitch.h" -#include "llvm/ADT/Twine.h" -#include "llvm/Support/ManagedStatic.h" -#include "llvm/Support/Path.h" -#include "llvm/Support/PrettyStackTrace.h" -#include "llvm/Support/Signals.h" - -using namespace lld; -using namespace llvm; -using namespace llvm::sys; - -int mainEntryClickHouseLLD(int Argc, char **Argv) -{ - // Standard set up, so program fails gracefully. - sys::PrintStackTraceOnErrorSignal(Argv[0]); - PrettyStackTraceProgram StackPrinter(Argc, Argv); - llvm_shutdown_obj Shutdown; - - std::vector Args(Argv, Argv + Argc); - return !elf::link(Args, true); -} diff --git a/dbms/programs/clang/Compiler-5.0.1 b/dbms/programs/clang/Compiler-5.0.1 deleted file mode 120000 index 7c8af57399f..00000000000 --- a/dbms/programs/clang/Compiler-5.0.1 +++ /dev/null @@ -1 +0,0 @@ -Compiler-5.0.0 \ No newline at end of file diff --git a/dbms/programs/clang/Compiler-5.0.2 b/dbms/programs/clang/Compiler-5.0.2 deleted file mode 120000 index 7c8af57399f..00000000000 --- a/dbms/programs/clang/Compiler-5.0.2 +++ /dev/null @@ -1 +0,0 @@ -Compiler-5.0.0 \ No newline at end of file diff --git a/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt deleted file mode 100644 index 4a046674afc..00000000000 --- a/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt +++ /dev/null @@ -1,54 +0,0 @@ - -add_definitions(-Wno-error -Wno-unused-parameter -Wno-non-virtual-dtor -U_LIBCPP_DEBUG) - -link_directories(${LLVM_LIBRARY_DIRS}) - -add_library(clickhouse-compiler-lib - driver.cpp - cc1_main.cpp - cc1as_main.cpp - lld.cpp) - -target_compile_options(clickhouse-compiler-lib PRIVATE -fno-rtti -fno-exceptions -g0) - -string(REPLACE "${INCLUDE_DEBUG_HELPERS}" "" CMAKE_CXX_FLAGS ${CMAKE_CXX_FLAGS}) # cant compile with -fno-rtti - -llvm_libs_all(REQUIRED_LLVM_LIBRARIES) - -message(STATUS "Using LLVM ${LLVM_VERSION}: ${LLVM_INCLUDE_DIRS} : ${REQUIRED_LLVM_LIBRARIES}") - -target_include_directories(clickhouse-compiler-lib SYSTEM PRIVATE ${LLVM_INCLUDE_DIRS}) - -# This is extracted almost directly from CMakeFiles/.../link.txt in LLVM build directory. - -target_link_libraries(clickhouse-compiler-lib PRIVATE - -clangBasic clangCodeGen clangDriver -clangFrontend -clangFrontendTool -clangRewriteFrontend clangARCMigrate clangStaticAnalyzerFrontend -clangParse clangSerialization clangSema clangEdit clangStaticAnalyzerCheckers -clangASTMatchers clangStaticAnalyzerCore clangAnalysis clangAST clangRewrite clangLex clangBasic - -lldCOFF -lldDriver -lldELF -lldMinGW -lldMachO -lldReaderWriter -lldYAML -lldCommon -lldCore -#lldWasm - -${REQUIRED_LLVM_LIBRARIES} - -#Polly -#PollyISL -#PollyPPCG - -PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARIES} Threads::Threads -${MALLOC_LIBRARIES} -${GLIBC_COMPATIBILITY_LIBRARIES} -${MEMCPY_LIBRARIES} -) diff --git a/dbms/programs/clang/Compiler-6.0.0/LICENSE.TXT b/dbms/programs/clang/Compiler-6.0.0/LICENSE.TXT deleted file mode 100644 index b452ca2efd8..00000000000 --- a/dbms/programs/clang/Compiler-6.0.0/LICENSE.TXT +++ /dev/null @@ -1,63 +0,0 @@ -============================================================================== -LLVM Release License -============================================================================== -University of Illinois/NCSA -Open Source License - -Copyright (c) 2007-2016 University of Illinois at Urbana-Champaign. -All rights reserved. - -Developed by: - - LLVM Team - - University of Illinois at Urbana-Champaign - - http://llvm.org - -Permission is hereby granted, free of charge, to any person obtaining a copy of -this software and associated documentation files (the "Software"), to deal with -the Software without restriction, including without limitation the rights to -use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies -of the Software, and to permit persons to whom the Software is furnished to do -so, subject to the following conditions: - - * Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimers. - - * Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimers in the - documentation and/or other materials provided with the distribution. - - * Neither the names of the LLVM Team, University of Illinois at - Urbana-Champaign, nor the names of its contributors may be used to - endorse or promote products derived from this Software without specific - prior written permission. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS -FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -CONTRIBUTORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS WITH THE -SOFTWARE. - -============================================================================== -The LLVM software contains code written by third parties. Such software will -have its own individual LICENSE.TXT file in the directory in which it appears. -This file will describe the copyrights, license, and restrictions which apply -to that code. - -The disclaimer of warranty in the University of Illinois Open Source License -applies to all code in the LLVM Distribution, and nothing in any of the -other licenses gives permission to use the names of the LLVM Team or the -University of Illinois to endorse or promote products derived from this -Software. - -The following pieces of software have additional or alternate copyrights, -licenses, and/or restrictions: - -Program Directory -------- --------- - - diff --git a/dbms/programs/clang/Compiler-6.0.0/cc1_main.cpp b/dbms/programs/clang/Compiler-6.0.0/cc1_main.cpp deleted file mode 100644 index f6eabaf3387..00000000000 --- a/dbms/programs/clang/Compiler-6.0.0/cc1_main.cpp +++ /dev/null @@ -1,242 +0,0 @@ -//===-- cc1_main.cpp - Clang CC1 Compiler Frontend ------------------------===// -// -// The LLVM Compiler Infrastructure -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This is the entry point to the clang -cc1 functionality, which implements the -// core compiler functionality along with a number of additional tools for -// demonstration and testing purposes. -// -//===----------------------------------------------------------------------===// - -#include "llvm/Option/Arg.h" -#include "clang/CodeGen/ObjectFilePCHContainerOperations.h" -#include "clang/Config/config.h" -#include "clang/Driver/DriverDiagnostic.h" -#include "clang/Driver/Options.h" -#include "clang/Frontend/CompilerInstance.h" -#include "clang/Frontend/CompilerInvocation.h" -#include "clang/Frontend/FrontendDiagnostic.h" -#include "clang/Frontend/TextDiagnosticBuffer.h" -#include "clang/Frontend/TextDiagnosticPrinter.h" -#include "clang/Frontend/Utils.h" -#include "clang/FrontendTool/Utils.h" -#include "llvm/ADT/Statistic.h" -#include "llvm/LinkAllPasses.h" -#include "llvm/Option/ArgList.h" -#include "llvm/Option/OptTable.h" -#include "llvm/Support/Compiler.h" -#include "llvm/Support/ErrorHandling.h" -#include "llvm/Support/ManagedStatic.h" -#include "llvm/Support/Signals.h" -#include "llvm/Support/TargetSelect.h" -#include "llvm/Support/Timer.h" -#include "llvm/Support/raw_ostream.h" -#include - -#ifdef CLANG_HAVE_RLIMITS -#include -#endif - -// have no .a version in packages -#undef LINK_POLLY_INTO_TOOLS - -using namespace clang; -using namespace llvm::opt; - -//===----------------------------------------------------------------------===// -// Main driver -//===----------------------------------------------------------------------===// - -static void LLVMErrorHandler(void *UserData, const std::string &Message, - bool GenCrashDiag) { - DiagnosticsEngine &Diags = *static_cast(UserData); - - Diags.Report(diag::err_fe_error_backend) << Message; - - // Run the interrupt handlers to make sure any special cleanups get done, in - // particular that we remove files registered with RemoveFileOnSignal. - llvm::sys::RunInterruptHandlers(); - - // We cannot recover from llvm errors. When reporting a fatal error, exit - // with status 70 to generate crash diagnostics. For BSD systems this is - // defined as an internal software error. Otherwise, exit with status 1. - exit(GenCrashDiag ? 70 : 1); -} - -#ifdef LINK_POLLY_INTO_TOOLS -namespace polly { -void initializePollyPasses(llvm::PassRegistry &Registry); -} -#endif - -#ifdef CLANG_HAVE_RLIMITS -// The amount of stack we think is "sufficient". If less than this much is -// available, we may be unable to reach our template instantiation depth -// limit and other similar limits. -// FIXME: Unify this with the stack we request when spawning a thread to build -// a module. -static const int kSufficientStack = 8 << 20; - -#if defined(__linux__) && defined(__PIE__) -static size_t getCurrentStackAllocation() { - // If we can't compute the current stack usage, allow for 512K of command - // line arguments and environment. - size_t Usage = 512 * 1024; - if (FILE *StatFile = fopen("/proc/self/stat", "r")) { - // We assume that the stack extends from its current address to the end of - // the environment space. In reality, there is another string literal (the - // program name) after the environment, but this is close enough (we only - // need to be within 100K or so). - unsigned long StackPtr, EnvEnd; - // Disable silly GCC -Wformat warning that complains about length - // modifiers on ignored format specifiers. We want to retain these - // for documentation purposes even though they have no effect. -#if defined(__GNUC__) && !defined(__clang__) -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wformat" -#endif - if (fscanf(StatFile, - "%*d %*s %*c %*d %*d %*d %*d %*d %*u %*lu %*lu %*lu %*lu %*lu " - "%*lu %*ld %*ld %*ld %*ld %*ld %*ld %*llu %*lu %*ld %*lu %*lu " - "%*lu %*lu %lu %*lu %*lu %*lu %*lu %*lu %*llu %*lu %*lu %*d %*d " - "%*u %*u %*llu %*lu %*ld %*lu %*lu %*lu %*lu %*lu %*lu %lu %*d", - &StackPtr, &EnvEnd) == 2) { -#if defined(__GNUC__) && !defined(__clang__) -#pragma GCC diagnostic pop -#endif - Usage = StackPtr < EnvEnd ? EnvEnd - StackPtr : StackPtr - EnvEnd; - } - fclose(StatFile); - } - return Usage; -} - -#include - -LLVM_ATTRIBUTE_NOINLINE -static void ensureStackAddressSpace(int ExtraChunks = 0) { - // Linux kernels prior to 4.1 will sometimes locate the heap of a PIE binary - // relatively close to the stack (they are only guaranteed to be 128MiB - // apart). This results in crashes if we happen to heap-allocate more than - // 128MiB before we reach our stack high-water mark. - // - // To avoid these crashes, ensure that we have sufficient virtual memory - // pages allocated before we start running. - size_t Curr = getCurrentStackAllocation(); - const int kTargetStack = kSufficientStack - 256 * 1024; - if (Curr < kTargetStack) { - volatile char *volatile Alloc = - static_cast(alloca(kTargetStack - Curr)); - Alloc[0] = 0; - Alloc[kTargetStack - Curr - 1] = 0; - } -} -#else -static void ensureStackAddressSpace() {} -#endif - -/// Attempt to ensure that we have at least 8MiB of usable stack space. -static void ensureSufficientStack() { - struct rlimit rlim; - if (getrlimit(RLIMIT_STACK, &rlim) != 0) - return; - - // Increase the soft stack limit to our desired level, if necessary and - // possible. - if (rlim.rlim_cur != RLIM_INFINITY && rlim.rlim_cur < kSufficientStack) { - // Try to allocate sufficient stack. - if (rlim.rlim_max == RLIM_INFINITY || rlim.rlim_max >= kSufficientStack) - rlim.rlim_cur = kSufficientStack; - else if (rlim.rlim_cur == rlim.rlim_max) - return; - else - rlim.rlim_cur = rlim.rlim_max; - - if (setrlimit(RLIMIT_STACK, &rlim) != 0 || - rlim.rlim_cur != kSufficientStack) - return; - } - - // We should now have a stack of size at least kSufficientStack. Ensure - // that we can actually use that much, if necessary. - ensureStackAddressSpace(); -} -#else -static void ensureSufficientStack() {} -#endif - -int cc1_main(ArrayRef Argv, const char *Argv0, void *MainAddr) { - ensureSufficientStack(); - - std::unique_ptr Clang(new CompilerInstance()); - IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); - - // Register the support for object-file-wrapped Clang modules. - auto PCHOps = Clang->getPCHContainerOperations(); - PCHOps->registerWriter(llvm::make_unique()); - PCHOps->registerReader(llvm::make_unique()); - - // Initialize targets first, so that --version shows registered targets. - llvm::InitializeAllTargets(); - llvm::InitializeAllTargetMCs(); - llvm::InitializeAllAsmPrinters(); - llvm::InitializeAllAsmParsers(); - -#ifdef LINK_POLLY_INTO_TOOLS - llvm::PassRegistry &Registry = *llvm::PassRegistry::getPassRegistry(); - polly::initializePollyPasses(Registry); -#endif - - // Buffer diagnostics from argument parsing so that we can output them using a - // well formed diagnostic object. - IntrusiveRefCntPtr DiagOpts = new DiagnosticOptions(); - TextDiagnosticBuffer *DiagsBuffer = new TextDiagnosticBuffer; - DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagsBuffer); - bool Success = CompilerInvocation::CreateFromArgs( - Clang->getInvocation(), Argv.begin(), Argv.end(), Diags); - - // Infer the builtin include path if unspecified. - if (Clang->getHeaderSearchOpts().UseBuiltinIncludes && - Clang->getHeaderSearchOpts().ResourceDir.empty()) - Clang->getHeaderSearchOpts().ResourceDir = - CompilerInvocation::GetResourcesPath(Argv0, MainAddr); - - // Create the actual diagnostics engine. - Clang->createDiagnostics(); - if (!Clang->hasDiagnostics()) - return 1; - - // Set an error handler, so that any LLVM backend diagnostics go through our - // error handler. - llvm::install_fatal_error_handler(LLVMErrorHandler, - static_cast(&Clang->getDiagnostics())); - - DiagsBuffer->FlushDiagnostics(Clang->getDiagnostics()); - if (!Success) - return 1; - - // Execute the frontend actions. - Success = ExecuteCompilerInvocation(Clang.get()); - - // If any timers were active but haven't been destroyed yet, print their - // results now. This happens in -disable-free mode. - llvm::TimerGroup::printAll(llvm::errs()); - - // Our error handler depends on the Diagnostics object, which we're - // potentially about to delete. Uninstall the handler now so that any - // later errors use the default handling behavior instead. - llvm::remove_fatal_error_handler(); - - // When running with -disable-free, don't do any destruction or shutdown. - if (Clang->getFrontendOpts().DisableFree) { - BuryPointer(std::move(Clang)); - return !Success; - } - - return !Success; -} diff --git a/dbms/programs/clang/Compiler-6.0.0/cc1as_main.cpp b/dbms/programs/clang/Compiler-6.0.0/cc1as_main.cpp deleted file mode 100644 index caf8409054a..00000000000 --- a/dbms/programs/clang/Compiler-6.0.0/cc1as_main.cpp +++ /dev/null @@ -1,540 +0,0 @@ -//===-- cc1as_main.cpp - Clang Assembler ---------------------------------===// -// -// The LLVM Compiler Infrastructure -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This is the entry point to the clang -cc1as functionality, which implements -// the direct interface to the LLVM MC based assembler. -// -//===----------------------------------------------------------------------===// - -#include "clang/Basic/Diagnostic.h" -#include "clang/Basic/DiagnosticOptions.h" -#include "clang/Driver/DriverDiagnostic.h" -#include "clang/Driver/Options.h" -#include "clang/Frontend/FrontendDiagnostic.h" -#include "clang/Frontend/TextDiagnosticPrinter.h" -#include "clang/Frontend/Utils.h" -#include "llvm/ADT/STLExtras.h" -#include "llvm/ADT/StringSwitch.h" -#include "llvm/ADT/Triple.h" -#include "llvm/IR/DataLayout.h" -#include "llvm/MC/MCAsmBackend.h" -#include "llvm/MC/MCAsmInfo.h" -#include "llvm/MC/MCCodeEmitter.h" -#include "llvm/MC/MCContext.h" -#include "llvm/MC/MCInstrInfo.h" -#include "llvm/MC/MCObjectFileInfo.h" -#include "llvm/MC/MCParser/MCAsmParser.h" -#include "llvm/MC/MCParser/MCTargetAsmParser.h" -#include "llvm/MC/MCRegisterInfo.h" -#include "llvm/MC/MCStreamer.h" -#include "llvm/MC/MCSubtargetInfo.h" -#include "llvm/MC/MCTargetOptions.h" -#include "llvm/Option/Arg.h" -#include "llvm/Option/ArgList.h" -#include "llvm/Option/OptTable.h" -#include "llvm/Support/CommandLine.h" -#include "llvm/Support/ErrorHandling.h" -#include "llvm/Support/FileSystem.h" -#include "llvm/Support/FormattedStream.h" -#include "llvm/Support/Host.h" -#include "llvm/Support/MemoryBuffer.h" -#include "llvm/Support/Path.h" -#include "llvm/Support/Signals.h" -#include "llvm/Support/SourceMgr.h" -#include "llvm/Support/TargetRegistry.h" -#include "llvm/Support/TargetSelect.h" -#include "llvm/Support/Timer.h" -#include "llvm/Support/raw_ostream.h" -#include -#include -using namespace clang; -using namespace clang::driver; -using namespace clang::driver::options; -using namespace llvm; -using namespace llvm::opt; - -namespace { - -/// \brief Helper class for representing a single invocation of the assembler. -struct AssemblerInvocation { - /// @name Target Options - /// @{ - - /// The name of the target triple to assemble for. - std::string Triple; - - /// If given, the name of the target CPU to determine which instructions - /// are legal. - std::string CPU; - - /// The list of target specific features to enable or disable -- this should - /// be a list of strings starting with '+' or '-'. - std::vector Features; - - /// The list of symbol definitions. - std::vector SymbolDefs; - - /// @} - /// @name Language Options - /// @{ - - std::vector IncludePaths; - unsigned NoInitialTextSection : 1; - unsigned SaveTemporaryLabels : 1; - unsigned GenDwarfForAssembly : 1; - unsigned RelaxELFRelocations : 1; - unsigned DwarfVersion; - std::string DwarfDebugFlags; - std::string DwarfDebugProducer; - std::string DebugCompilationDir; - llvm::DebugCompressionType CompressDebugSections = - llvm::DebugCompressionType::None; - std::string MainFileName; - - /// @} - /// @name Frontend Options - /// @{ - - std::string InputFile; - std::vector LLVMArgs; - std::string OutputPath; - enum FileType { - FT_Asm, ///< Assembly (.s) output, transliterate mode. - FT_Null, ///< No output, for timing purposes. - FT_Obj ///< Object file output. - }; - FileType OutputType; - unsigned ShowHelp : 1; - unsigned ShowVersion : 1; - - /// @} - /// @name Transliterate Options - /// @{ - - unsigned OutputAsmVariant; - unsigned ShowEncoding : 1; - unsigned ShowInst : 1; - - /// @} - /// @name Assembler Options - /// @{ - - unsigned RelaxAll : 1; - unsigned NoExecStack : 1; - unsigned FatalWarnings : 1; - unsigned IncrementalLinkerCompatible : 1; - - /// The name of the relocation model to use. - std::string RelocationModel; - - /// @} - -public: - AssemblerInvocation() { - Triple = ""; - NoInitialTextSection = 0; - InputFile = "-"; - OutputPath = "-"; - OutputType = FT_Asm; - OutputAsmVariant = 0; - ShowInst = 0; - ShowEncoding = 0; - RelaxAll = 0; - NoExecStack = 0; - FatalWarnings = 0; - IncrementalLinkerCompatible = 0; - DwarfVersion = 0; - } - - static bool CreateFromArgs(AssemblerInvocation &Res, - ArrayRef Argv, - DiagnosticsEngine &Diags); -}; - -} - -bool AssemblerInvocation::CreateFromArgs(AssemblerInvocation &Opts, - ArrayRef Argv, - DiagnosticsEngine &Diags) { - bool Success = true; - - // Parse the arguments. - std::unique_ptr OptTbl(createDriverOptTable()); - - const unsigned IncludedFlagsBitmask = options::CC1AsOption; - unsigned MissingArgIndex, MissingArgCount; - InputArgList Args = OptTbl->ParseArgs(Argv, MissingArgIndex, MissingArgCount, - IncludedFlagsBitmask); - - // Check for missing argument error. - if (MissingArgCount) { - Diags.Report(diag::err_drv_missing_argument) - << Args.getArgString(MissingArgIndex) << MissingArgCount; - Success = false; - } - - // Issue errors on unknown arguments. - for (const Arg *A : Args.filtered(OPT_UNKNOWN)) { - Diags.Report(diag::err_drv_unknown_argument) << A->getAsString(Args); - Success = false; - } - - // Construct the invocation. - - // Target Options - Opts.Triple = llvm::Triple::normalize(Args.getLastArgValue(OPT_triple)); - Opts.CPU = Args.getLastArgValue(OPT_target_cpu); - Opts.Features = Args.getAllArgValues(OPT_target_feature); - - // Use the default target triple if unspecified. - if (Opts.Triple.empty()) - Opts.Triple = llvm::sys::getDefaultTargetTriple(); - - // Language Options - Opts.IncludePaths = Args.getAllArgValues(OPT_I); - Opts.NoInitialTextSection = Args.hasArg(OPT_n); - Opts.SaveTemporaryLabels = Args.hasArg(OPT_msave_temp_labels); - // Any DebugInfoKind implies GenDwarfForAssembly. - Opts.GenDwarfForAssembly = Args.hasArg(OPT_debug_info_kind_EQ); - - if (const Arg *A = Args.getLastArg(OPT_compress_debug_sections, - OPT_compress_debug_sections_EQ)) { - if (A->getOption().getID() == OPT_compress_debug_sections) { - // TODO: be more clever about the compression type auto-detection - Opts.CompressDebugSections = llvm::DebugCompressionType::GNU; - } else { - Opts.CompressDebugSections = - llvm::StringSwitch(A->getValue()) - .Case("none", llvm::DebugCompressionType::None) - .Case("zlib", llvm::DebugCompressionType::Z) - .Case("zlib-gnu", llvm::DebugCompressionType::GNU) - .Default(llvm::DebugCompressionType::None); - } - } - - Opts.RelaxELFRelocations = Args.hasArg(OPT_mrelax_relocations); - Opts.DwarfVersion = getLastArgIntValue(Args, OPT_dwarf_version_EQ, 2, Diags); - Opts.DwarfDebugFlags = Args.getLastArgValue(OPT_dwarf_debug_flags); - Opts.DwarfDebugProducer = Args.getLastArgValue(OPT_dwarf_debug_producer); - Opts.DebugCompilationDir = Args.getLastArgValue(OPT_fdebug_compilation_dir); - Opts.MainFileName = Args.getLastArgValue(OPT_main_file_name); - - // Frontend Options - if (Args.hasArg(OPT_INPUT)) { - bool First = true; - for (const Arg *A : Args.filtered(OPT_INPUT)) { - if (First) { - Opts.InputFile = A->getValue(); - First = false; - } else { - Diags.Report(diag::err_drv_unknown_argument) << A->getAsString(Args); - Success = false; - } - } - } - Opts.LLVMArgs = Args.getAllArgValues(OPT_mllvm); - Opts.OutputPath = Args.getLastArgValue(OPT_o); - if (Arg *A = Args.getLastArg(OPT_filetype)) { - StringRef Name = A->getValue(); - unsigned OutputType = StringSwitch(Name) - .Case("asm", FT_Asm) - .Case("null", FT_Null) - .Case("obj", FT_Obj) - .Default(~0U); - if (OutputType == ~0U) { - Diags.Report(diag::err_drv_invalid_value) << A->getAsString(Args) << Name; - Success = false; - } else - Opts.OutputType = FileType(OutputType); - } - Opts.ShowHelp = Args.hasArg(OPT_help); - Opts.ShowVersion = Args.hasArg(OPT_version); - - // Transliterate Options - Opts.OutputAsmVariant = - getLastArgIntValue(Args, OPT_output_asm_variant, 0, Diags); - Opts.ShowEncoding = Args.hasArg(OPT_show_encoding); - Opts.ShowInst = Args.hasArg(OPT_show_inst); - - // Assemble Options - Opts.RelaxAll = Args.hasArg(OPT_mrelax_all); - Opts.NoExecStack = Args.hasArg(OPT_mno_exec_stack); - Opts.FatalWarnings = Args.hasArg(OPT_massembler_fatal_warnings); - Opts.RelocationModel = Args.getLastArgValue(OPT_mrelocation_model, "pic"); - Opts.IncrementalLinkerCompatible = - Args.hasArg(OPT_mincremental_linker_compatible); - Opts.SymbolDefs = Args.getAllArgValues(OPT_defsym); - - return Success; -} - -static std::unique_ptr -getOutputStream(AssemblerInvocation &Opts, DiagnosticsEngine &Diags, - bool Binary) { - if (Opts.OutputPath.empty()) - Opts.OutputPath = "-"; - - // Make sure that the Out file gets unlinked from the disk if we get a - // SIGINT. - if (Opts.OutputPath != "-") - sys::RemoveFileOnSignal(Opts.OutputPath); - - std::error_code EC; - auto Out = llvm::make_unique( - Opts.OutputPath, EC, (Binary ? sys::fs::F_None : sys::fs::F_Text)); - if (EC) { - Diags.Report(diag::err_fe_unable_to_open_output) << Opts.OutputPath - << EC.message(); - return nullptr; - } - - return Out; -} - -static bool ExecuteAssembler(AssemblerInvocation &Opts, - DiagnosticsEngine &Diags) { - // Get the target specific parser. - std::string Error; - const Target *TheTarget = TargetRegistry::lookupTarget(Opts.Triple, Error); - if (!TheTarget) - return Diags.Report(diag::err_target_unknown_triple) << Opts.Triple; - - ErrorOr> Buffer = - MemoryBuffer::getFileOrSTDIN(Opts.InputFile); - - if (std::error_code EC = Buffer.getError()) { - Error = EC.message(); - return Diags.Report(diag::err_fe_error_reading) << Opts.InputFile; - } - - SourceMgr SrcMgr; - - // Tell SrcMgr about this buffer, which is what the parser will pick up. - SrcMgr.AddNewSourceBuffer(std::move(*Buffer), SMLoc()); - - // Record the location of the include directories so that the lexer can find - // it later. - SrcMgr.setIncludeDirs(Opts.IncludePaths); - - std::unique_ptr MRI(TheTarget->createMCRegInfo(Opts.Triple)); - assert(MRI && "Unable to create target register info!"); - - std::unique_ptr MAI(TheTarget->createMCAsmInfo(*MRI, Opts.Triple)); - assert(MAI && "Unable to create target asm info!"); - - // Ensure MCAsmInfo initialization occurs before any use, otherwise sections - // may be created with a combination of default and explicit settings. - MAI->setCompressDebugSections(Opts.CompressDebugSections); - - MAI->setRelaxELFRelocations(Opts.RelaxELFRelocations); - - bool IsBinary = Opts.OutputType == AssemblerInvocation::FT_Obj; - std::unique_ptr FDOS = getOutputStream(Opts, Diags, IsBinary); - if (!FDOS) - return true; - - // FIXME: This is not pretty. MCContext has a ptr to MCObjectFileInfo and - // MCObjectFileInfo needs a MCContext reference in order to initialize itself. - std::unique_ptr MOFI(new MCObjectFileInfo()); - - MCContext Ctx(MAI.get(), MRI.get(), MOFI.get(), &SrcMgr); - - bool PIC = false; - if (Opts.RelocationModel == "static") { - PIC = false; - } else if (Opts.RelocationModel == "pic") { - PIC = true; - } else { - assert(Opts.RelocationModel == "dynamic-no-pic" && - "Invalid PIC model!"); - PIC = false; - } - - MOFI->InitMCObjectFileInfo(Triple(Opts.Triple), PIC, Ctx); - if (Opts.SaveTemporaryLabels) - Ctx.setAllowTemporaryLabels(false); - if (Opts.GenDwarfForAssembly) - Ctx.setGenDwarfForAssembly(true); - if (!Opts.DwarfDebugFlags.empty()) - Ctx.setDwarfDebugFlags(StringRef(Opts.DwarfDebugFlags)); - if (!Opts.DwarfDebugProducer.empty()) - Ctx.setDwarfDebugProducer(StringRef(Opts.DwarfDebugProducer)); - if (!Opts.DebugCompilationDir.empty()) - Ctx.setCompilationDir(Opts.DebugCompilationDir); - if (!Opts.MainFileName.empty()) - Ctx.setMainFileName(StringRef(Opts.MainFileName)); - Ctx.setDwarfVersion(Opts.DwarfVersion); - - // Build up the feature string from the target feature list. - std::string FS; - if (!Opts.Features.empty()) { - FS = Opts.Features[0]; - for (unsigned i = 1, e = Opts.Features.size(); i != e; ++i) - FS += "," + Opts.Features[i]; - } - - std::unique_ptr Str; - - std::unique_ptr MCII(TheTarget->createMCInstrInfo()); - std::unique_ptr STI( - TheTarget->createMCSubtargetInfo(Opts.Triple, Opts.CPU, FS)); - - raw_pwrite_stream *Out = FDOS.get(); - std::unique_ptr BOS; - - // FIXME: There is a bit of code duplication with addPassesToEmitFile. - if (Opts.OutputType == AssemblerInvocation::FT_Asm) { - MCInstPrinter *IP = TheTarget->createMCInstPrinter( - llvm::Triple(Opts.Triple), Opts.OutputAsmVariant, *MAI, *MCII, *MRI); - MCCodeEmitter *CE = nullptr; - MCAsmBackend *MAB = nullptr; - if (Opts.ShowEncoding) { - CE = TheTarget->createMCCodeEmitter(*MCII, *MRI, Ctx); - MCTargetOptions Options; - MAB = TheTarget->createMCAsmBackend(*STI, *MRI, Options); - } - auto FOut = llvm::make_unique(*Out); - Str.reset(TheTarget->createAsmStreamer( - Ctx, std::move(FOut), /*asmverbose*/ true, - /*useDwarfDirectory*/ true, IP, CE, MAB, Opts.ShowInst)); - } else if (Opts.OutputType == AssemblerInvocation::FT_Null) { - Str.reset(createNullStreamer(Ctx)); - } else { - assert(Opts.OutputType == AssemblerInvocation::FT_Obj && - "Invalid file type!"); - if (!FDOS->supportsSeeking()) { - BOS = make_unique(*FDOS); - Out = BOS.get(); - } - - MCCodeEmitter *CE = TheTarget->createMCCodeEmitter(*MCII, *MRI, Ctx); - MCTargetOptions Options; - MCAsmBackend *MAB = TheTarget->createMCAsmBackend(*STI, *MRI, Options); - Triple T(Opts.Triple); - Str.reset(TheTarget->createMCObjectStreamer( - T, Ctx, std::unique_ptr(MAB), *Out, std::unique_ptr(CE), *STI, - Opts.RelaxAll, Opts.IncrementalLinkerCompatible, - /*DWARFMustBeAtTheEnd*/ true)); - Str.get()->InitSections(Opts.NoExecStack); - } - - bool Failed = false; - - std::unique_ptr Parser( - createMCAsmParser(SrcMgr, Ctx, *Str.get(), *MAI)); - - // FIXME: init MCTargetOptions from sanitizer flags here. - MCTargetOptions Options; - std::unique_ptr TAP( - TheTarget->createMCAsmParser(*STI, *Parser, *MCII, Options)); - if (!TAP) - Failed = Diags.Report(diag::err_target_unknown_triple) << Opts.Triple; - - // Set values for symbols, if any. - for (auto &S : Opts.SymbolDefs) { - auto Pair = StringRef(S).split('='); - auto Sym = Pair.first; - auto Val = Pair.second; - int64_t Value = 0; - // We have already error checked this in the driver. - Val.getAsInteger(0, Value); - Ctx.setSymbolValue(Parser->getStreamer(), Sym, Value); - } - - if (!Failed) { - Parser->setTargetParser(*TAP.get()); - Failed = Parser->Run(Opts.NoInitialTextSection); - } - - // Close Streamer first. - // It might have a reference to the output stream. - Str.reset(); - // Close the output stream early. - BOS.reset(); - FDOS.reset(); - - // Delete output file if there were errors. - if (Failed && Opts.OutputPath != "-") - sys::fs::remove(Opts.OutputPath); - - return Failed; -} - -static void LLVMErrorHandler(void *UserData, const std::string &Message, - bool GenCrashDiag) { - DiagnosticsEngine &Diags = *static_cast(UserData); - - Diags.Report(diag::err_fe_error_backend) << Message; - - // We cannot recover from llvm errors. - exit(1); -} - -int cc1as_main(ArrayRef Argv, const char *Argv0, void *MainAddr) { - // Initialize targets and assembly printers/parsers. - InitializeAllTargetInfos(); - InitializeAllTargetMCs(); - InitializeAllAsmParsers(); - - // Construct our diagnostic client. - IntrusiveRefCntPtr DiagOpts = new DiagnosticOptions(); - TextDiagnosticPrinter *DiagClient - = new TextDiagnosticPrinter(errs(), &*DiagOpts); - DiagClient->setPrefix("clang -cc1as"); - IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); - DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagClient); - - // Set an error handler, so that any LLVM backend diagnostics go through our - // error handler. - ScopedFatalErrorHandler FatalErrorHandler - (LLVMErrorHandler, static_cast(&Diags)); - - // Parse the arguments. - AssemblerInvocation Asm; - if (!AssemblerInvocation::CreateFromArgs(Asm, Argv, Diags)) - return 1; - - if (Asm.ShowHelp) { - std::unique_ptr Opts(driver::createDriverOptTable()); - Opts->PrintHelp(llvm::outs(), "clang -cc1as", "Clang Integrated Assembler", - /*Include=*/driver::options::CC1AsOption, /*Exclude=*/0, - /*ShowAllAliases=*/false); - return 0; - } - - // Honor -version. - // - // FIXME: Use a better -version message? - if (Asm.ShowVersion) { - llvm::cl::PrintVersionMessage(); - return 0; - } - - // Honor -mllvm. - // - // FIXME: Remove this, one day. - if (!Asm.LLVMArgs.empty()) { - unsigned NumArgs = Asm.LLVMArgs.size(); - auto Args = llvm::make_unique(NumArgs + 2); - Args[0] = "clang (LLVM option parsing)"; - for (unsigned i = 0; i != NumArgs; ++i) - Args[i + 1] = Asm.LLVMArgs[i].c_str(); - Args[NumArgs + 1] = nullptr; - llvm::cl::ParseCommandLineOptions(NumArgs + 1, Args.get()); - } - - // Execute the invocation, unless there were parsing errors. - bool Failed = Diags.hasErrorOccurred() || ExecuteAssembler(Asm, Diags); - - // If any timers were active but haven't been destroyed yet, print their - // results now. - TimerGroup::printAll(errs()); - - return !!Failed; -} diff --git a/dbms/programs/clang/Compiler-6.0.0/driver.cpp b/dbms/programs/clang/Compiler-6.0.0/driver.cpp deleted file mode 100644 index 30511b8253a..00000000000 --- a/dbms/programs/clang/Compiler-6.0.0/driver.cpp +++ /dev/null @@ -1,520 +0,0 @@ -//===-- driver.cpp - Clang GCC-Compatible Driver --------------------------===// -// -// The LLVM Compiler Infrastructure -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This is the entry point to the clang driver; it is a thin wrapper -// for functionality in the Driver clang library. -// -//===----------------------------------------------------------------------===// - -#include "clang/Basic/DiagnosticOptions.h" -#include "clang/Driver/Compilation.h" -#include "clang/Driver/Driver.h" -#include "clang/Driver/DriverDiagnostic.h" -#include "clang/Driver/Options.h" -#include "clang/Driver/ToolChain.h" -#include "clang/Frontend/ChainedDiagnosticConsumer.h" -#include "clang/Frontend/CompilerInvocation.h" -#include "clang/Frontend/SerializedDiagnosticPrinter.h" -#include "clang/Frontend/TextDiagnosticPrinter.h" -#include "clang/Frontend/Utils.h" -#include "llvm/ADT/ArrayRef.h" -#include "llvm/ADT/SmallString.h" -#include "llvm/ADT/SmallVector.h" -#include "llvm/Config/llvm-config.h" -#include "llvm/Option/ArgList.h" -#include "llvm/Option/OptTable.h" -#include "llvm/Option/Option.h" -#include "llvm/Support/CommandLine.h" -#include "llvm/Support/ErrorHandling.h" -#include "llvm/Support/FileSystem.h" -#include "llvm/Support/Host.h" -#include "llvm/Support/ManagedStatic.h" -#include "llvm/Support/Path.h" -#include "llvm/Support/PrettyStackTrace.h" -#include "llvm/Support/Process.h" -#include "llvm/Support/Program.h" -#include "llvm/Support/Regex.h" -#include "llvm/Support/Signals.h" -#include "llvm/Support/StringSaver.h" -#include "llvm/Support/TargetSelect.h" -#include "llvm/Support/Timer.h" -#include "llvm/Support/raw_ostream.h" -#include -#include -#include -using namespace clang; -using namespace clang::driver; -using namespace llvm::opt; - -std::string GetExecutablePath(const char *Argv0, bool CanonicalPrefixes) { - if (!CanonicalPrefixes) { - SmallString<128> ExecutablePath(Argv0); - // Do a PATH lookup if Argv0 isn't a valid path. - if (!llvm::sys::fs::exists(ExecutablePath)) - if (llvm::ErrorOr P = - llvm::sys::findProgramByName(ExecutablePath)) - ExecutablePath = *P; - return ExecutablePath.str(); - } - - // This just needs to be some symbol in the binary; C++ doesn't - // allow taking the address of ::main however. - void *P = (void*) (intptr_t) GetExecutablePath; - return llvm::sys::fs::getMainExecutable(Argv0, P); -} - -static const char *GetStableCStr(std::set &SavedStrings, - StringRef S) { - return SavedStrings.insert(S).first->c_str(); -} - -/// ApplyQAOverride - Apply a list of edits to the input argument lists. -/// -/// The input string is a space separate list of edits to perform, -/// they are applied in order to the input argument lists. Edits -/// should be one of the following forms: -/// -/// '#': Silence information about the changes to the command line arguments. -/// -/// '^': Add FOO as a new argument at the beginning of the command line. -/// -/// '+': Add FOO as a new argument at the end of the command line. -/// -/// 's/XXX/YYY/': Substitute the regular expression XXX with YYY in the command -/// line. -/// -/// 'xOPTION': Removes all instances of the literal argument OPTION. -/// -/// 'XOPTION': Removes all instances of the literal argument OPTION, -/// and the following argument. -/// -/// 'Ox': Removes all flags matching 'O' or 'O[sz0-9]' and adds 'Ox' -/// at the end of the command line. -/// -/// \param OS - The stream to write edit information to. -/// \param Args - The vector of command line arguments. -/// \param Edit - The override command to perform. -/// \param SavedStrings - Set to use for storing string representations. -static void ApplyOneQAOverride(raw_ostream &OS, - SmallVectorImpl &Args, - StringRef Edit, - std::set &SavedStrings) { - // This does not need to be efficient. - - if (Edit[0] == '^') { - const char *Str = - GetStableCStr(SavedStrings, Edit.substr(1)); - OS << "### Adding argument " << Str << " at beginning\n"; - Args.insert(Args.begin() + 1, Str); - } else if (Edit[0] == '+') { - const char *Str = - GetStableCStr(SavedStrings, Edit.substr(1)); - OS << "### Adding argument " << Str << " at end\n"; - Args.push_back(Str); - } else if (Edit[0] == 's' && Edit[1] == '/' && Edit.endswith("/") && - Edit.slice(2, Edit.size()-1).find('/') != StringRef::npos) { - StringRef MatchPattern = Edit.substr(2).split('/').first; - StringRef ReplPattern = Edit.substr(2).split('/').second; - ReplPattern = ReplPattern.slice(0, ReplPattern.size()-1); - - for (unsigned i = 1, e = Args.size(); i != e; ++i) { - // Ignore end-of-line response file markers - if (Args[i] == nullptr) - continue; - std::string Repl = llvm::Regex(MatchPattern).sub(ReplPattern, Args[i]); - - if (Repl != Args[i]) { - OS << "### Replacing '" << Args[i] << "' with '" << Repl << "'\n"; - Args[i] = GetStableCStr(SavedStrings, Repl); - } - } - } else if (Edit[0] == 'x' || Edit[0] == 'X') { - auto Option = Edit.substr(1); - for (unsigned i = 1; i < Args.size();) { - if (Option == Args[i]) { - OS << "### Deleting argument " << Args[i] << '\n'; - Args.erase(Args.begin() + i); - if (Edit[0] == 'X') { - if (i < Args.size()) { - OS << "### Deleting argument " << Args[i] << '\n'; - Args.erase(Args.begin() + i); - } else - OS << "### Invalid X edit, end of command line!\n"; - } - } else - ++i; - } - } else if (Edit[0] == 'O') { - for (unsigned i = 1; i < Args.size();) { - const char *A = Args[i]; - // Ignore end-of-line response file markers - if (A == nullptr) - continue; - if (A[0] == '-' && A[1] == 'O' && - (A[2] == '\0' || - (A[3] == '\0' && (A[2] == 's' || A[2] == 'z' || - ('0' <= A[2] && A[2] <= '9'))))) { - OS << "### Deleting argument " << Args[i] << '\n'; - Args.erase(Args.begin() + i); - } else - ++i; - } - OS << "### Adding argument " << Edit << " at end\n"; - Args.push_back(GetStableCStr(SavedStrings, '-' + Edit.str())); - } else { - OS << "### Unrecognized edit: " << Edit << "\n"; - } -} - -/// ApplyQAOverride - Apply a comma separate list of edits to the -/// input argument lists. See ApplyOneQAOverride. -static void ApplyQAOverride(SmallVectorImpl &Args, - const char *OverrideStr, - std::set &SavedStrings) { - raw_ostream *OS = &llvm::errs(); - - if (OverrideStr[0] == '#') { - ++OverrideStr; - OS = &llvm::nulls(); - } - - *OS << "### CCC_OVERRIDE_OPTIONS: " << OverrideStr << "\n"; - - // This does not need to be efficient. - - const char *S = OverrideStr; - while (*S) { - const char *End = ::strchr(S, ' '); - if (!End) - End = S + strlen(S); - if (End != S) - ApplyOneQAOverride(*OS, Args, std::string(S, End), SavedStrings); - S = End; - if (*S != '\0') - ++S; - } -} - -extern int cc1_main(ArrayRef Argv, const char *Argv0, - void *MainAddr); -extern int cc1as_main(ArrayRef Argv, const char *Argv0, - void *MainAddr); - -static void insertTargetAndModeArgs(const ParsedClangName &NameParts, - SmallVectorImpl &ArgVector, - std::set &SavedStrings) { - // Put target and mode arguments at the start of argument list so that - // arguments specified in command line could override them. Avoid putting - // them at index 0, as an option like '-cc1' must remain the first. - auto InsertionPoint = ArgVector.begin(); - if (InsertionPoint != ArgVector.end()) - ++InsertionPoint; - - if (NameParts.DriverMode) { - // Add the mode flag to the arguments. - ArgVector.insert(InsertionPoint, - GetStableCStr(SavedStrings, NameParts.DriverMode)); - } - - if (NameParts.TargetIsValid) { - const char *arr[] = {"-target", GetStableCStr(SavedStrings, - NameParts.TargetPrefix)}; - ArgVector.insert(InsertionPoint, std::begin(arr), std::end(arr)); - } -} - -static void getCLEnvVarOptions(std::string &EnvValue, llvm::StringSaver &Saver, - SmallVectorImpl &Opts) { - llvm::cl::TokenizeWindowsCommandLine(EnvValue, Saver, Opts); - // The first instance of '#' should be replaced with '=' in each option. - for (const char *Opt : Opts) - if (char *NumberSignPtr = const_cast(::strchr(Opt, '#'))) - *NumberSignPtr = '='; -} - -static void SetBackdoorDriverOutputsFromEnvVars(Driver &TheDriver) { - // Handle CC_PRINT_OPTIONS and CC_PRINT_OPTIONS_FILE. - TheDriver.CCPrintOptions = !!::getenv("CC_PRINT_OPTIONS"); - if (TheDriver.CCPrintOptions) - TheDriver.CCPrintOptionsFilename = ::getenv("CC_PRINT_OPTIONS_FILE"); - - // Handle CC_PRINT_HEADERS and CC_PRINT_HEADERS_FILE. - TheDriver.CCPrintHeaders = !!::getenv("CC_PRINT_HEADERS"); - if (TheDriver.CCPrintHeaders) - TheDriver.CCPrintHeadersFilename = ::getenv("CC_PRINT_HEADERS_FILE"); - - // Handle CC_LOG_DIAGNOSTICS and CC_LOG_DIAGNOSTICS_FILE. - TheDriver.CCLogDiagnostics = !!::getenv("CC_LOG_DIAGNOSTICS"); - if (TheDriver.CCLogDiagnostics) - TheDriver.CCLogDiagnosticsFilename = ::getenv("CC_LOG_DIAGNOSTICS_FILE"); -} - -static void FixupDiagPrefixExeName(TextDiagnosticPrinter *DiagClient, - const std::string &Path) { - // If the clang binary happens to be named cl.exe for compatibility reasons, - // use clang-cl.exe as the prefix to avoid confusion between clang and MSVC. - StringRef ExeBasename(llvm::sys::path::filename(Path)); - if (ExeBasename.equals_lower("cl.exe")) - ExeBasename = "clang-cl.exe"; - DiagClient->setPrefix(ExeBasename); -} - -// This lets us create the DiagnosticsEngine with a properly-filled-out -// DiagnosticOptions instance. -static DiagnosticOptions * -CreateAndPopulateDiagOpts(ArrayRef argv) { - auto *DiagOpts = new DiagnosticOptions; - std::unique_ptr Opts(createDriverOptTable()); - unsigned MissingArgIndex, MissingArgCount; - InputArgList Args = - Opts->ParseArgs(argv.slice(1), MissingArgIndex, MissingArgCount); - // We ignore MissingArgCount and the return value of ParseDiagnosticArgs. - // Any errors that would be diagnosed here will also be diagnosed later, - // when the DiagnosticsEngine actually exists. - (void)ParseDiagnosticArgs(*DiagOpts, Args); - return DiagOpts; -} - -static void SetInstallDir(SmallVectorImpl &argv, - Driver &TheDriver, bool CanonicalPrefixes) { - // Attempt to find the original path used to invoke the driver, to determine - // the installed path. We do this manually, because we want to support that - // path being a symlink. - SmallString<128> InstalledPath(argv[0]); - - // Do a PATH lookup, if there are no directory components. - if (llvm::sys::path::filename(InstalledPath) == InstalledPath) - if (llvm::ErrorOr Tmp = llvm::sys::findProgramByName( - llvm::sys::path::filename(InstalledPath.str()))) - InstalledPath = *Tmp; - - // FIXME: We don't actually canonicalize this, we just make it absolute. - if (CanonicalPrefixes) - llvm::sys::fs::make_absolute(InstalledPath); - - StringRef InstalledPathParent(llvm::sys::path::parent_path(InstalledPath)); - if (llvm::sys::fs::exists(InstalledPathParent)) - TheDriver.setInstalledDir(InstalledPathParent); -} - -static int ExecuteCC1Tool(ArrayRef argv, StringRef Tool) { - void *GetExecutablePathVP = (void *)(intptr_t) GetExecutablePath; - if (Tool == "") - return cc1_main(argv.slice(2), argv[0], GetExecutablePathVP); - if (Tool == "as") - return cc1as_main(argv.slice(2), argv[0], GetExecutablePathVP); - - // Reject unknown tools. - llvm::errs() << "error: unknown integrated tool '" << Tool << "'\n"; - return 1; -} - -int mainEntryClickHouseClang(int argc_, char **argv_) { - llvm::sys::PrintStackTraceOnErrorSignal(argv_[0]); - llvm::PrettyStackTraceProgram X(argc_, argv_); - llvm::llvm_shutdown_obj Y; // Call llvm_shutdown() on exit. - - if (llvm::sys::Process::FixupStandardFileDescriptors()) - return 1; - - SmallVector argv; - llvm::SpecificBumpPtrAllocator ArgAllocator; - std::error_code EC = llvm::sys::Process::GetArgumentVector( - argv, llvm::makeArrayRef(argv_, argc_), ArgAllocator); - if (EC) { - llvm::errs() << "error: couldn't get arguments: " << EC.message() << '\n'; - return 1; - } - - llvm::InitializeAllTargets(); - auto TargetAndMode = ToolChain::getTargetAndModeFromProgramName(argv[0]); - - llvm::BumpPtrAllocator A; - llvm::StringSaver Saver(A); - - // Parse response files using the GNU syntax, unless we're in CL mode. There - // are two ways to put clang in CL compatibility mode: argv[0] is either - // clang-cl or cl, or --driver-mode=cl is on the command line. The normal - // command line parsing can't happen until after response file parsing, so we - // have to manually search for a --driver-mode=cl argument the hard way. - // Finally, our -cc1 tools don't care which tokenization mode we use because - // response files written by clang will tokenize the same way in either mode. - bool ClangCLMode = false; - if (StringRef(TargetAndMode.DriverMode).equals("--driver-mode=cl") || - std::find_if(argv.begin(), argv.end(), [](const char *F) { - return F && strcmp(F, "--driver-mode=cl") == 0; - }) != argv.end()) { - ClangCLMode = true; - } - enum { Default, POSIX, Windows } RSPQuoting = Default; - for (const char *F : argv) { - if (strcmp(F, "--rsp-quoting=posix") == 0) - RSPQuoting = POSIX; - else if (strcmp(F, "--rsp-quoting=windows") == 0) - RSPQuoting = Windows; - } - - // Determines whether we want nullptr markers in argv to indicate response - // files end-of-lines. We only use this for the /LINK driver argument with - // clang-cl.exe on Windows. - bool MarkEOLs = ClangCLMode; - - llvm::cl::TokenizerCallback Tokenizer; - if (RSPQuoting == Windows || (RSPQuoting == Default && ClangCLMode)) - Tokenizer = &llvm::cl::TokenizeWindowsCommandLine; - else - Tokenizer = &llvm::cl::TokenizeGNUCommandLine; - - if (MarkEOLs && argv.size() > 1 && StringRef(argv[1]).startswith("-cc1")) - MarkEOLs = false; - llvm::cl::ExpandResponseFiles(Saver, Tokenizer, argv, MarkEOLs); - - // Handle -cc1 integrated tools, even if -cc1 was expanded from a response - // file. - auto FirstArg = std::find_if(argv.begin() + 1, argv.end(), - [](const char *A) { return A != nullptr; }); - if (FirstArg != argv.end() && StringRef(*FirstArg).startswith("-cc1")) { - // If -cc1 came from a response file, remove the EOL sentinels. - if (MarkEOLs) { - auto newEnd = std::remove(argv.begin(), argv.end(), nullptr); - argv.resize(newEnd - argv.begin()); - } - return ExecuteCC1Tool(argv, argv[1] + 4); - } - - bool CanonicalPrefixes = true; - for (int i = 1, size = argv.size(); i < size; ++i) { - // Skip end-of-line response file markers - if (argv[i] == nullptr) - continue; - if (StringRef(argv[i]) == "-no-canonical-prefixes") { - CanonicalPrefixes = false; - break; - } - } - - // Handle CL and _CL_ which permits additional command line options to be - // prepended or appended. - if (ClangCLMode) { - // Arguments in "CL" are prepended. - llvm::Optional OptCL = llvm::sys::Process::GetEnv("CL"); - if (OptCL.hasValue()) { - SmallVector PrependedOpts; - getCLEnvVarOptions(OptCL.getValue(), Saver, PrependedOpts); - - // Insert right after the program name to prepend to the argument list. - argv.insert(argv.begin() + 1, PrependedOpts.begin(), PrependedOpts.end()); - } - // Arguments in "_CL_" are appended. - llvm::Optional Opt_CL_ = llvm::sys::Process::GetEnv("_CL_"); - if (Opt_CL_.hasValue()) { - SmallVector AppendedOpts; - getCLEnvVarOptions(Opt_CL_.getValue(), Saver, AppendedOpts); - - // Insert at the end of the argument list to append. - argv.append(AppendedOpts.begin(), AppendedOpts.end()); - } - } - - std::set SavedStrings; - // Handle CCC_OVERRIDE_OPTIONS, used for editing a command line behind the - // scenes. - if (const char *OverrideStr = ::getenv("CCC_OVERRIDE_OPTIONS")) { - // FIXME: Driver shouldn't take extra initial argument. - ApplyQAOverride(argv, OverrideStr, SavedStrings); - } - - std::string Path = GetExecutablePath(argv[0], CanonicalPrefixes); - - IntrusiveRefCntPtr DiagOpts = - CreateAndPopulateDiagOpts(argv); - - TextDiagnosticPrinter *DiagClient - = new TextDiagnosticPrinter(llvm::errs(), &*DiagOpts); - FixupDiagPrefixExeName(DiagClient, Path); - - IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); - - DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagClient); - - if (!DiagOpts->DiagnosticSerializationFile.empty()) { - auto SerializedConsumer = - clang::serialized_diags::create(DiagOpts->DiagnosticSerializationFile, - &*DiagOpts, /*MergeChildRecords=*/true); - Diags.setClient(new ChainedDiagnosticConsumer( - Diags.takeClient(), std::move(SerializedConsumer))); - } - - ProcessWarningOptions(Diags, *DiagOpts, /*ReportDiags=*/false); - - Driver TheDriver(Path, llvm::sys::getDefaultTargetTriple(), Diags); - SetInstallDir(argv, TheDriver, CanonicalPrefixes); - TheDriver.setTargetAndMode(TargetAndMode); - - insertTargetAndModeArgs(TargetAndMode, argv, SavedStrings); - - SetBackdoorDriverOutputsFromEnvVars(TheDriver); - - std::unique_ptr C(TheDriver.BuildCompilation(argv)); - int Res = 1; - if (C && !C->containsError()) { - SmallVector, 4> FailingCommands; - Res = TheDriver.ExecuteCompilation(*C, FailingCommands); - - // Force a crash to test the diagnostics. - if (TheDriver.GenReproducer) { - Diags.Report(diag::err_drv_force_crash) - << !::getenv("FORCE_CLANG_DIAGNOSTICS_CRASH"); - - // Pretend that every command failed. - FailingCommands.clear(); - for (const auto &J : C->getJobs()) - if (const Command *C = dyn_cast(&J)) - FailingCommands.push_back(std::make_pair(-1, C)); - } - - for (const auto &P : FailingCommands) { - int CommandRes = P.first; - const Command *FailingCommand = P.second; - if (!Res) - Res = CommandRes; - - // If result status is < 0, then the driver command signalled an error. - // If result status is 70, then the driver command reported a fatal error. - // On Windows, abort will return an exit code of 3. In these cases, - // generate additional diagnostic information if possible. - bool DiagnoseCrash = CommandRes < 0 || CommandRes == 70; -#ifdef LLVM_ON_WIN32 - DiagnoseCrash |= CommandRes == 3; -#endif - if (DiagnoseCrash) { - TheDriver.generateCompilationDiagnostics(*C, *FailingCommand); - break; - } - } - } - - Diags.getClient()->finish(); - - // If any timers were active but haven't been destroyed yet, print their - // results now. This happens in -disable-free mode. - llvm::TimerGroup::printAll(llvm::errs()); - -#ifdef LLVM_ON_WIN32 - // Exit status should not be negative on Win32, unless abnormal termination. - // Once abnormal termiation was caught, negative status should not be - // propagated. - if (Res < 0) - Res = 1; -#endif - - // If we have multiple failing commands, we return the result of the first - // failing command. - return Res; -} diff --git a/dbms/programs/clang/Compiler-6.0.0/lld.cpp b/dbms/programs/clang/Compiler-6.0.0/lld.cpp deleted file mode 100644 index 696ff84dfe6..00000000000 --- a/dbms/programs/clang/Compiler-6.0.0/lld.cpp +++ /dev/null @@ -1,23 +0,0 @@ -#include "lld/Common/Driver.h" -#include "llvm/ADT/STLExtras.h" -#include "llvm/ADT/StringSwitch.h" -#include "llvm/ADT/Twine.h" -#include "llvm/Support/ManagedStatic.h" -#include "llvm/Support/Path.h" -#include "llvm/Support/PrettyStackTrace.h" -#include "llvm/Support/Signals.h" - -using namespace lld; -using namespace llvm; -using namespace llvm::sys; - -int mainEntryClickHouseLLD(int Argc, char **Argv) -{ - // Standard set up, so program fails gracefully. - sys::PrintStackTraceOnErrorSignal(Argv[0]); - PrettyStackTraceProgram StackPrinter(Argc, Argv); - llvm_shutdown_obj Shutdown; - - std::vector Args(Argv, Argv + Argc); - return !elf::link(Args, true); -} diff --git a/dbms/programs/clang/Compiler-6.0.0svn b/dbms/programs/clang/Compiler-6.0.0svn deleted file mode 120000 index 7eba9cc37d0..00000000000 --- a/dbms/programs/clang/Compiler-6.0.0svn +++ /dev/null @@ -1 +0,0 @@ -Compiler-6.0.0 \ No newline at end of file diff --git a/dbms/programs/clang/Compiler-6.0.1 b/dbms/programs/clang/Compiler-6.0.1 deleted file mode 120000 index 7eba9cc37d0..00000000000 --- a/dbms/programs/clang/Compiler-6.0.1 +++ /dev/null @@ -1 +0,0 @@ -Compiler-6.0.0 \ No newline at end of file diff --git a/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt deleted file mode 100644 index a042c821ec4..00000000000 --- a/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt +++ /dev/null @@ -1,49 +0,0 @@ -add_definitions(-Wno-error -Wno-unused-parameter -Wno-non-virtual-dtor -U_LIBCPP_DEBUG) - -link_directories(${LLVM_LIBRARY_DIRS}) - -add_library(clickhouse-compiler-lib - driver.cpp - cc1_main.cpp - cc1gen_reproducer_main.cpp - cc1as_main.cpp - lld.cpp) - -target_compile_options(clickhouse-compiler-lib PRIVATE -fno-rtti -fno-exceptions -g0) - -string(REPLACE "${INCLUDE_DEBUG_HELPERS}" "" CMAKE_CXX_FLAGS ${CMAKE_CXX_FLAGS}) # cant compile with -fno-rtti - -llvm_libs_all(REQUIRED_LLVM_LIBRARIES) - -message(STATUS "Using LLVM ${LLVM_VERSION}: ${LLVM_INCLUDE_DIRS} : ${REQUIRED_LLVM_LIBRARIES}") - -target_include_directories(clickhouse-compiler-lib SYSTEM PRIVATE ${LLVM_INCLUDE_DIRS}) - -# This is extracted almost directly from CMakeFiles/.../link.txt in LLVM build directory. - -target_link_libraries(clickhouse-compiler-lib PRIVATE -clangBasic clangCodeGen clangDriver -clangFrontend -clangFrontendTool -clangRewriteFrontend clangARCMigrate clangStaticAnalyzerFrontend -clangParse clangSerialization clangSema clangEdit clangStaticAnalyzerCheckers -clangASTMatchers clangStaticAnalyzerCore clangAnalysis clangAST clangRewrite clangLex clangBasic -clangCrossTU clangIndex - -lldCOFF -lldDriver -lldELF -lldMinGW -lldMachO -lldReaderWriter -lldYAML -lldCommon -lldCore - -${REQUIRED_LLVM_LIBRARIES} - -PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARIES} Threads::Threads -${MALLOC_LIBRARIES} -${GLIBC_COMPATIBILITY_LIBRARIES} -${MEMCPY_LIBRARIES} -) diff --git a/dbms/programs/clang/Compiler-7.0.0/cc1_main.cpp b/dbms/programs/clang/Compiler-7.0.0/cc1_main.cpp deleted file mode 100644 index 214bfa72476..00000000000 --- a/dbms/programs/clang/Compiler-7.0.0/cc1_main.cpp +++ /dev/null @@ -1,239 +0,0 @@ -//===-- cc1_main.cpp - Clang CC1 Compiler Frontend ------------------------===// -// -// The LLVM Compiler Infrastructure -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This is the entry point to the clang -cc1 functionality, which implements the -// core compiler functionality along with a number of additional tools for -// demonstration and testing purposes. -// -//===----------------------------------------------------------------------===// - -#include "llvm/Option/Arg.h" -#include "clang/CodeGen/ObjectFilePCHContainerOperations.h" -#include "clang/Config/config.h" -#include "clang/Basic/Stack.h" -#include "clang/Driver/DriverDiagnostic.h" -#include "clang/Driver/Options.h" -#include "clang/Frontend/CompilerInstance.h" -#include "clang/Frontend/CompilerInvocation.h" -#include "clang/Frontend/FrontendDiagnostic.h" -#include "clang/Frontend/TextDiagnosticBuffer.h" -#include "clang/Frontend/TextDiagnosticPrinter.h" -#include "clang/Frontend/Utils.h" -#include "clang/FrontendTool/Utils.h" -#include "llvm/ADT/Statistic.h" -#include "llvm/Config/llvm-config.h" -#include "llvm/LinkAllPasses.h" -#include "llvm/Option/ArgList.h" -#include "llvm/Option/OptTable.h" -#include "llvm/Support/Compiler.h" -#include "llvm/Support/ErrorHandling.h" -#include "llvm/Support/ManagedStatic.h" -#include "llvm/Support/Signals.h" -#include "llvm/Support/TargetSelect.h" -#include "llvm/Support/Timer.h" -#include "llvm/Support/raw_ostream.h" -#include - -#ifdef CLANG_HAVE_RLIMITS -#include -#endif - -// have no .a version in packages -#undef LINK_POLLY_INTO_TOOLS - -using namespace clang; -using namespace llvm::opt; - -//===----------------------------------------------------------------------===// -// Main driver -//===----------------------------------------------------------------------===// - -static void LLVMErrorHandler(void *UserData, const std::string &Message, - bool GenCrashDiag) { - DiagnosticsEngine &Diags = *static_cast(UserData); - - Diags.Report(diag::err_fe_error_backend) << Message; - - // Run the interrupt handlers to make sure any special cleanups get done, in - // particular that we remove files registered with RemoveFileOnSignal. - llvm::sys::RunInterruptHandlers(); - - // We cannot recover from llvm errors. When reporting a fatal error, exit - // with status 70 to generate crash diagnostics. For BSD systems this is - // defined as an internal software error. Otherwise, exit with status 1. - exit(GenCrashDiag ? 70 : 1); -} - -#ifdef LINK_POLLY_INTO_TOOLS -namespace polly { -void initializePollyPasses(llvm::PassRegistry &Registry); -} -#endif - -#ifdef CLANG_HAVE_RLIMITS -#if defined(__linux__) && defined(__PIE__) -static size_t getCurrentStackAllocation() { - // If we can't compute the current stack usage, allow for 512K of command - // line arguments and environment. - size_t Usage = 512 * 1024; - if (FILE *StatFile = fopen("/proc/self/stat", "r")) { - // We assume that the stack extends from its current address to the end of - // the environment space. In reality, there is another string literal (the - // program name) after the environment, but this is close enough (we only - // need to be within 100K or so). - unsigned long StackPtr, EnvEnd; - // Disable silly GCC -Wformat warning that complains about length - // modifiers on ignored format specifiers. We want to retain these - // for documentation purposes even though they have no effect. -#if defined(__GNUC__) && !defined(__clang__) -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wformat" -#endif - if (fscanf(StatFile, - "%*d %*s %*c %*d %*d %*d %*d %*d %*u %*lu %*lu %*lu %*lu %*lu " - "%*lu %*ld %*ld %*ld %*ld %*ld %*ld %*llu %*lu %*ld %*lu %*lu " - "%*lu %*lu %lu %*lu %*lu %*lu %*lu %*lu %*llu %*lu %*lu %*d %*d " - "%*u %*u %*llu %*lu %*ld %*lu %*lu %*lu %*lu %*lu %*lu %lu %*d", - &StackPtr, &EnvEnd) == 2) { -#if defined(__GNUC__) && !defined(__clang__) -#pragma GCC diagnostic pop -#endif - Usage = StackPtr < EnvEnd ? EnvEnd - StackPtr : StackPtr - EnvEnd; - } - fclose(StatFile); - } - return Usage; -} - -#include - -LLVM_ATTRIBUTE_NOINLINE -static void ensureStackAddressSpace() { - // Linux kernels prior to 4.1 will sometimes locate the heap of a PIE binary - // relatively close to the stack (they are only guaranteed to be 128MiB - // apart). This results in crashes if we happen to heap-allocate more than - // 128MiB before we reach our stack high-water mark. - // - // To avoid these crashes, ensure that we have sufficient virtual memory - // pages allocated before we start running. - size_t Curr = getCurrentStackAllocation(); - const int kTargetStack = DesiredStackSize - 256 * 1024; - if (Curr < kTargetStack) { - volatile char *volatile Alloc = - static_cast(alloca(kTargetStack - Curr)); - Alloc[0] = 0; - Alloc[kTargetStack - Curr - 1] = 0; - } -} -#else -static void ensureStackAddressSpace() {} -#endif - -/// Attempt to ensure that we have at least 8MiB of usable stack space. -static void ensureSufficientStack() { - struct rlimit rlim; - if (getrlimit(RLIMIT_STACK, &rlim) != 0) - return; - - // Increase the soft stack limit to our desired level, if necessary and - // possible. - if (rlim.rlim_cur != RLIM_INFINITY && - rlim.rlim_cur < rlim_t(DesiredStackSize)) { - // Try to allocate sufficient stack. - if (rlim.rlim_max == RLIM_INFINITY || - rlim.rlim_max >= rlim_t(DesiredStackSize)) - rlim.rlim_cur = DesiredStackSize; - else if (rlim.rlim_cur == rlim.rlim_max) - return; - else - rlim.rlim_cur = rlim.rlim_max; - - if (setrlimit(RLIMIT_STACK, &rlim) != 0 || - rlim.rlim_cur != DesiredStackSize) - return; - } - - // We should now have a stack of size at least DesiredStackSize. Ensure - // that we can actually use that much, if necessary. - ensureStackAddressSpace(); -} -#else -static void ensureSufficientStack() {} -#endif - -int cc1_main(ArrayRef Argv, const char *Argv0, void *MainAddr) { - ensureSufficientStack(); - - std::unique_ptr Clang(new CompilerInstance()); - IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); - - // Register the support for object-file-wrapped Clang modules. - auto PCHOps = Clang->getPCHContainerOperations(); - PCHOps->registerWriter(llvm::make_unique()); - PCHOps->registerReader(llvm::make_unique()); - - // Initialize targets first, so that --version shows registered targets. - llvm::InitializeAllTargets(); - llvm::InitializeAllTargetMCs(); - llvm::InitializeAllAsmPrinters(); - llvm::InitializeAllAsmParsers(); - -#ifdef LINK_POLLY_INTO_TOOLS - llvm::PassRegistry &Registry = *llvm::PassRegistry::getPassRegistry(); - polly::initializePollyPasses(Registry); -#endif - - // Buffer diagnostics from argument parsing so that we can output them using a - // well formed diagnostic object. - IntrusiveRefCntPtr DiagOpts = new DiagnosticOptions(); - TextDiagnosticBuffer *DiagsBuffer = new TextDiagnosticBuffer; - DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagsBuffer); - bool Success = CompilerInvocation::CreateFromArgs( - Clang->getInvocation(), Argv.begin(), Argv.end(), Diags); - - // Infer the builtin include path if unspecified. - if (Clang->getHeaderSearchOpts().UseBuiltinIncludes && - Clang->getHeaderSearchOpts().ResourceDir.empty()) - Clang->getHeaderSearchOpts().ResourceDir = - CompilerInvocation::GetResourcesPath(Argv0, MainAddr); - - // Create the actual diagnostics engine. - Clang->createDiagnostics(); - if (!Clang->hasDiagnostics()) - return 1; - - // Set an error handler, so that any LLVM backend diagnostics go through our - // error handler. - llvm::install_fatal_error_handler(LLVMErrorHandler, - static_cast(&Clang->getDiagnostics())); - - DiagsBuffer->FlushDiagnostics(Clang->getDiagnostics()); - if (!Success) - return 1; - - // Execute the frontend actions. - Success = ExecuteCompilerInvocation(Clang.get()); - - // If any timers were active but haven't been destroyed yet, print their - // results now. This happens in -disable-free mode. - llvm::TimerGroup::printAll(llvm::errs()); - - // Our error handler depends on the Diagnostics object, which we're - // potentially about to delete. Uninstall the handler now so that any - // later errors use the default handling behavior instead. - llvm::remove_fatal_error_handler(); - - // When running with -disable-free, don't do any destruction or shutdown. - if (Clang->getFrontendOpts().DisableFree) { - BuryPointer(std::move(Clang)); - return !Success; - } - - return !Success; -} diff --git a/dbms/programs/clang/Compiler-7.0.0/cc1as_main.cpp b/dbms/programs/clang/Compiler-7.0.0/cc1as_main.cpp deleted file mode 100644 index d93b1f5cb1d..00000000000 --- a/dbms/programs/clang/Compiler-7.0.0/cc1as_main.cpp +++ /dev/null @@ -1,572 +0,0 @@ -//===-- cc1as_main.cpp - Clang Assembler ---------------------------------===// -// -// The LLVM Compiler Infrastructure -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This is the entry point to the clang -cc1as functionality, which implements -// the direct interface to the LLVM MC based assembler. -// -//===----------------------------------------------------------------------===// - -#include "clang/Basic/Diagnostic.h" -#include "clang/Basic/DiagnosticOptions.h" -#include "clang/Driver/DriverDiagnostic.h" -#include "clang/Driver/Options.h" -#include "clang/Frontend/FrontendDiagnostic.h" -#include "clang/Frontend/TextDiagnosticPrinter.h" -#include "clang/Frontend/Utils.h" -#include "llvm/ADT/STLExtras.h" -#include "llvm/ADT/StringSwitch.h" -#include "llvm/ADT/Triple.h" -#include "llvm/IR/DataLayout.h" -#include "llvm/MC/MCAsmBackend.h" -#include "llvm/MC/MCAsmInfo.h" -#include "llvm/MC/MCCodeEmitter.h" -#include "llvm/MC/MCContext.h" -#include "llvm/MC/MCInstrInfo.h" -#include "llvm/MC/MCObjectFileInfo.h" -#include "llvm/MC/MCObjectWriter.h" -#include "llvm/MC/MCParser/MCAsmParser.h" -#include "llvm/MC/MCParser/MCTargetAsmParser.h" -#include "llvm/MC/MCRegisterInfo.h" -#include "llvm/MC/MCStreamer.h" -#include "llvm/MC/MCSubtargetInfo.h" -#include "llvm/MC/MCTargetOptions.h" -#include "llvm/Option/Arg.h" -#include "llvm/Option/ArgList.h" -#include "llvm/Option/OptTable.h" -#include "llvm/Support/CommandLine.h" -#include "llvm/Support/ErrorHandling.h" -#include "llvm/Support/FileSystem.h" -#include "llvm/Support/FormattedStream.h" -#include "llvm/Support/Host.h" -#include "llvm/Support/MemoryBuffer.h" -#include "llvm/Support/Path.h" -#include "llvm/Support/Signals.h" -#include "llvm/Support/SourceMgr.h" -#include "llvm/Support/TargetRegistry.h" -#include "llvm/Support/TargetSelect.h" -#include "llvm/Support/Timer.h" -#include "llvm/Support/raw_ostream.h" -#include -#include -using namespace clang; -using namespace clang::driver; -using namespace clang::driver::options; -using namespace llvm; -using namespace llvm::opt; - -namespace { - -/// Helper class for representing a single invocation of the assembler. -struct AssemblerInvocation { - /// @name Target Options - /// @{ - - /// The name of the target triple to assemble for. - std::string Triple; - - /// If given, the name of the target CPU to determine which instructions - /// are legal. - std::string CPU; - - /// The list of target specific features to enable or disable -- this should - /// be a list of strings starting with '+' or '-'. - std::vector Features; - - /// The list of symbol definitions. - std::vector SymbolDefs; - - /// @} - /// @name Language Options - /// @{ - - std::vector IncludePaths; - unsigned NoInitialTextSection : 1; - unsigned SaveTemporaryLabels : 1; - unsigned GenDwarfForAssembly : 1; - unsigned RelaxELFRelocations : 1; - unsigned DwarfVersion; - std::string DwarfDebugFlags; - std::string DwarfDebugProducer; - std::string DebugCompilationDir; - std::map DebugPrefixMap; - llvm::DebugCompressionType CompressDebugSections = - llvm::DebugCompressionType::None; - std::string MainFileName; - std::string SplitDwarfFile; - - /// @} - /// @name Frontend Options - /// @{ - - std::string InputFile; - std::vector LLVMArgs; - std::string OutputPath; - enum FileType { - FT_Asm, ///< Assembly (.s) output, transliterate mode. - FT_Null, ///< No output, for timing purposes. - FT_Obj ///< Object file output. - }; - FileType OutputType; - unsigned ShowHelp : 1; - unsigned ShowVersion : 1; - - /// @} - /// @name Transliterate Options - /// @{ - - unsigned OutputAsmVariant; - unsigned ShowEncoding : 1; - unsigned ShowInst : 1; - - /// @} - /// @name Assembler Options - /// @{ - - unsigned RelaxAll : 1; - unsigned NoExecStack : 1; - unsigned FatalWarnings : 1; - unsigned IncrementalLinkerCompatible : 1; - - /// The name of the relocation model to use. - std::string RelocationModel; - - /// @} - -public: - AssemblerInvocation() { - Triple = ""; - NoInitialTextSection = 0; - InputFile = "-"; - OutputPath = "-"; - OutputType = FT_Asm; - OutputAsmVariant = 0; - ShowInst = 0; - ShowEncoding = 0; - RelaxAll = 0; - NoExecStack = 0; - FatalWarnings = 0; - IncrementalLinkerCompatible = 0; - DwarfVersion = 0; - } - - static bool CreateFromArgs(AssemblerInvocation &Res, - ArrayRef Argv, - DiagnosticsEngine &Diags); -}; - -} - -bool AssemblerInvocation::CreateFromArgs(AssemblerInvocation &Opts, - ArrayRef Argv, - DiagnosticsEngine &Diags) { - bool Success = true; - - // Parse the arguments. - std::unique_ptr OptTbl(createDriverOptTable()); - - const unsigned IncludedFlagsBitmask = options::CC1AsOption; - unsigned MissingArgIndex, MissingArgCount; - InputArgList Args = OptTbl->ParseArgs(Argv, MissingArgIndex, MissingArgCount, - IncludedFlagsBitmask); - - // Check for missing argument error. - if (MissingArgCount) { - Diags.Report(diag::err_drv_missing_argument) - << Args.getArgString(MissingArgIndex) << MissingArgCount; - Success = false; - } - - // Issue errors on unknown arguments. - for (const Arg *A : Args.filtered(OPT_UNKNOWN)) { - auto ArgString = A->getAsString(Args); - std::string Nearest; - if (OptTbl->findNearest(ArgString, Nearest, IncludedFlagsBitmask) > 1) - Diags.Report(diag::err_drv_unknown_argument) << ArgString; - else - Diags.Report(diag::err_drv_unknown_argument_with_suggestion) - << ArgString << Nearest; - Success = false; - } - - // Construct the invocation. - - // Target Options - Opts.Triple = llvm::Triple::normalize(Args.getLastArgValue(OPT_triple)); - Opts.CPU = Args.getLastArgValue(OPT_target_cpu); - Opts.Features = Args.getAllArgValues(OPT_target_feature); - - // Use the default target triple if unspecified. - if (Opts.Triple.empty()) - Opts.Triple = llvm::sys::getDefaultTargetTriple(); - - // Language Options - Opts.IncludePaths = Args.getAllArgValues(OPT_I); - Opts.NoInitialTextSection = Args.hasArg(OPT_n); - Opts.SaveTemporaryLabels = Args.hasArg(OPT_msave_temp_labels); - // Any DebugInfoKind implies GenDwarfForAssembly. - Opts.GenDwarfForAssembly = Args.hasArg(OPT_debug_info_kind_EQ); - - if (const Arg *A = Args.getLastArg(OPT_compress_debug_sections, - OPT_compress_debug_sections_EQ)) { - if (A->getOption().getID() == OPT_compress_debug_sections) { - // TODO: be more clever about the compression type auto-detection - Opts.CompressDebugSections = llvm::DebugCompressionType::GNU; - } else { - Opts.CompressDebugSections = - llvm::StringSwitch(A->getValue()) - .Case("none", llvm::DebugCompressionType::None) - .Case("zlib", llvm::DebugCompressionType::Z) - .Case("zlib-gnu", llvm::DebugCompressionType::GNU) - .Default(llvm::DebugCompressionType::None); - } - } - - Opts.RelaxELFRelocations = Args.hasArg(OPT_mrelax_relocations); - Opts.DwarfVersion = getLastArgIntValue(Args, OPT_dwarf_version_EQ, 2, Diags); - Opts.DwarfDebugFlags = Args.getLastArgValue(OPT_dwarf_debug_flags); - Opts.DwarfDebugProducer = Args.getLastArgValue(OPT_dwarf_debug_producer); - Opts.DebugCompilationDir = Args.getLastArgValue(OPT_fdebug_compilation_dir); - Opts.MainFileName = Args.getLastArgValue(OPT_main_file_name); - - for (const auto &Arg : Args.getAllArgValues(OPT_fdebug_prefix_map_EQ)) - Opts.DebugPrefixMap.insert(StringRef(Arg).split('=')); - - // Frontend Options - if (Args.hasArg(OPT_INPUT)) { - bool First = true; - for (const Arg *A : Args.filtered(OPT_INPUT)) { - if (First) { - Opts.InputFile = A->getValue(); - First = false; - } else { - Diags.Report(diag::err_drv_unknown_argument) << A->getAsString(Args); - Success = false; - } - } - } - Opts.LLVMArgs = Args.getAllArgValues(OPT_mllvm); - Opts.OutputPath = Args.getLastArgValue(OPT_o); - Opts.SplitDwarfFile = Args.getLastArgValue(OPT_split_dwarf_file); - if (Arg *A = Args.getLastArg(OPT_filetype)) { - StringRef Name = A->getValue(); - unsigned OutputType = StringSwitch(Name) - .Case("asm", FT_Asm) - .Case("null", FT_Null) - .Case("obj", FT_Obj) - .Default(~0U); - if (OutputType == ~0U) { - Diags.Report(diag::err_drv_invalid_value) << A->getAsString(Args) << Name; - Success = false; - } else - Opts.OutputType = FileType(OutputType); - } - Opts.ShowHelp = Args.hasArg(OPT_help); - Opts.ShowVersion = Args.hasArg(OPT_version); - - // Transliterate Options - Opts.OutputAsmVariant = - getLastArgIntValue(Args, OPT_output_asm_variant, 0, Diags); - Opts.ShowEncoding = Args.hasArg(OPT_show_encoding); - Opts.ShowInst = Args.hasArg(OPT_show_inst); - - // Assemble Options - Opts.RelaxAll = Args.hasArg(OPT_mrelax_all); - Opts.NoExecStack = Args.hasArg(OPT_mno_exec_stack); - Opts.FatalWarnings = Args.hasArg(OPT_massembler_fatal_warnings); - Opts.RelocationModel = Args.getLastArgValue(OPT_mrelocation_model, "pic"); - Opts.IncrementalLinkerCompatible = - Args.hasArg(OPT_mincremental_linker_compatible); - Opts.SymbolDefs = Args.getAllArgValues(OPT_defsym); - - return Success; -} - -static std::unique_ptr -getOutputStream(StringRef Path, DiagnosticsEngine &Diags, bool Binary) { - // Make sure that the Out file gets unlinked from the disk if we get a - // SIGINT. - if (Path != "-") - sys::RemoveFileOnSignal(Path); - - std::error_code EC; - auto Out = llvm::make_unique( - Path, EC, (Binary ? sys::fs::F_None : sys::fs::F_Text)); - if (EC) { - Diags.Report(diag::err_fe_unable_to_open_output) << Path << EC.message(); - return nullptr; - } - - return Out; -} - -static bool ExecuteAssembler(AssemblerInvocation &Opts, - DiagnosticsEngine &Diags) { - // Get the target specific parser. - std::string Error; - const Target *TheTarget = TargetRegistry::lookupTarget(Opts.Triple, Error); - if (!TheTarget) - return Diags.Report(diag::err_target_unknown_triple) << Opts.Triple; - - ErrorOr> Buffer = - MemoryBuffer::getFileOrSTDIN(Opts.InputFile); - - if (std::error_code EC = Buffer.getError()) { - Error = EC.message(); - return Diags.Report(diag::err_fe_error_reading) << Opts.InputFile; - } - - SourceMgr SrcMgr; - - // Tell SrcMgr about this buffer, which is what the parser will pick up. - SrcMgr.AddNewSourceBuffer(std::move(*Buffer), SMLoc()); - - // Record the location of the include directories so that the lexer can find - // it later. - SrcMgr.setIncludeDirs(Opts.IncludePaths); - - std::unique_ptr MRI(TheTarget->createMCRegInfo(Opts.Triple)); - assert(MRI && "Unable to create target register info!"); - - std::unique_ptr MAI(TheTarget->createMCAsmInfo(*MRI, Opts.Triple)); - assert(MAI && "Unable to create target asm info!"); - - // Ensure MCAsmInfo initialization occurs before any use, otherwise sections - // may be created with a combination of default and explicit settings. - MAI->setCompressDebugSections(Opts.CompressDebugSections); - - MAI->setRelaxELFRelocations(Opts.RelaxELFRelocations); - - bool IsBinary = Opts.OutputType == AssemblerInvocation::FT_Obj; - if (Opts.OutputPath.empty()) - Opts.OutputPath = "-"; - std::unique_ptr FDOS = - getOutputStream(Opts.OutputPath, Diags, IsBinary); - if (!FDOS) - return true; - std::unique_ptr DwoOS; - if (!Opts.SplitDwarfFile.empty()) - DwoOS = getOutputStream(Opts.SplitDwarfFile, Diags, IsBinary); - - // FIXME: This is not pretty. MCContext has a ptr to MCObjectFileInfo and - // MCObjectFileInfo needs a MCContext reference in order to initialize itself. - std::unique_ptr MOFI(new MCObjectFileInfo()); - - MCContext Ctx(MAI.get(), MRI.get(), MOFI.get(), &SrcMgr); - - bool PIC = false; - if (Opts.RelocationModel == "static") { - PIC = false; - } else if (Opts.RelocationModel == "pic") { - PIC = true; - } else { - assert(Opts.RelocationModel == "dynamic-no-pic" && - "Invalid PIC model!"); - PIC = false; - } - - MOFI->InitMCObjectFileInfo(Triple(Opts.Triple), PIC, Ctx); - if (Opts.SaveTemporaryLabels) - Ctx.setAllowTemporaryLabels(false); - if (Opts.GenDwarfForAssembly) - Ctx.setGenDwarfForAssembly(true); - if (!Opts.DwarfDebugFlags.empty()) - Ctx.setDwarfDebugFlags(StringRef(Opts.DwarfDebugFlags)); - if (!Opts.DwarfDebugProducer.empty()) - Ctx.setDwarfDebugProducer(StringRef(Opts.DwarfDebugProducer)); - if (!Opts.DebugCompilationDir.empty()) - Ctx.setCompilationDir(Opts.DebugCompilationDir); - if (!Opts.DebugPrefixMap.empty()) - for (const auto &KV : Opts.DebugPrefixMap) - Ctx.addDebugPrefixMapEntry(KV.first, KV.second); - if (!Opts.MainFileName.empty()) - Ctx.setMainFileName(StringRef(Opts.MainFileName)); - Ctx.setDwarfVersion(Opts.DwarfVersion); - - // Build up the feature string from the target feature list. - std::string FS; - if (!Opts.Features.empty()) { - FS = Opts.Features[0]; - for (unsigned i = 1, e = Opts.Features.size(); i != e; ++i) - FS += "," + Opts.Features[i]; - } - - std::unique_ptr Str; - - std::unique_ptr MCII(TheTarget->createMCInstrInfo()); - std::unique_ptr STI( - TheTarget->createMCSubtargetInfo(Opts.Triple, Opts.CPU, FS)); - - raw_pwrite_stream *Out = FDOS.get(); - std::unique_ptr BOS; - - // FIXME: There is a bit of code duplication with addPassesToEmitFile. - if (Opts.OutputType == AssemblerInvocation::FT_Asm) { - MCInstPrinter *IP = TheTarget->createMCInstPrinter( - llvm::Triple(Opts.Triple), Opts.OutputAsmVariant, *MAI, *MCII, *MRI); - - std::unique_ptr CE; - if (Opts.ShowEncoding) - CE.reset(TheTarget->createMCCodeEmitter(*MCII, *MRI, Ctx)); - MCTargetOptions MCOptions; - std::unique_ptr MAB( - TheTarget->createMCAsmBackend(*STI, *MRI, MCOptions)); - - auto FOut = llvm::make_unique(*Out); - Str.reset(TheTarget->createAsmStreamer( - Ctx, std::move(FOut), /*asmverbose*/ true, - /*useDwarfDirectory*/ true, IP, std::move(CE), std::move(MAB), - Opts.ShowInst)); - } else if (Opts.OutputType == AssemblerInvocation::FT_Null) { - Str.reset(createNullStreamer(Ctx)); - } else { - assert(Opts.OutputType == AssemblerInvocation::FT_Obj && - "Invalid file type!"); - if (!FDOS->supportsSeeking()) { - BOS = make_unique(*FDOS); - Out = BOS.get(); - } - - std::unique_ptr CE( - TheTarget->createMCCodeEmitter(*MCII, *MRI, Ctx)); - MCTargetOptions MCOptions; - std::unique_ptr MAB( - TheTarget->createMCAsmBackend(*STI, *MRI, MCOptions)); - std::unique_ptr OW = - DwoOS ? MAB->createDwoObjectWriter(*Out, *DwoOS) - : MAB->createObjectWriter(*Out); - - Triple T(Opts.Triple); - Str.reset(TheTarget->createMCObjectStreamer( - T, Ctx, std::move(MAB), std::move(OW), std::move(CE), *STI, - Opts.RelaxAll, Opts.IncrementalLinkerCompatible, - /*DWARFMustBeAtTheEnd*/ true)); - Str.get()->InitSections(Opts.NoExecStack); - } - - // Assembly to object compilation should leverage assembly info. - Str->setUseAssemblerInfoForParsing(true); - - bool Failed = false; - - std::unique_ptr Parser( - createMCAsmParser(SrcMgr, Ctx, *Str.get(), *MAI)); - - // FIXME: init MCTargetOptions from sanitizer flags here. - MCTargetOptions Options; - std::unique_ptr TAP( - TheTarget->createMCAsmParser(*STI, *Parser, *MCII, Options)); - if (!TAP) - Failed = Diags.Report(diag::err_target_unknown_triple) << Opts.Triple; - - // Set values for symbols, if any. - for (auto &S : Opts.SymbolDefs) { - auto Pair = StringRef(S).split('='); - auto Sym = Pair.first; - auto Val = Pair.second; - int64_t Value = 1; - // We have already error checked this in the driver. - Val.getAsInteger(0, Value); - Ctx.setSymbolValue(Parser->getStreamer(), Sym, Value); - } - - if (!Failed) { - Parser->setTargetParser(*TAP.get()); - Failed = Parser->Run(Opts.NoInitialTextSection); - } - - // Close Streamer first. - // It might have a reference to the output stream. - Str.reset(); - // Close the output stream early. - BOS.reset(); - FDOS.reset(); - - // Delete output file if there were errors. - if (Failed) { - if (Opts.OutputPath != "-") - sys::fs::remove(Opts.OutputPath); - if (!Opts.SplitDwarfFile.empty() && Opts.SplitDwarfFile != "-") - sys::fs::remove(Opts.SplitDwarfFile); - } - - return Failed; -} - -static void LLVMErrorHandler(void *UserData, const std::string &Message, - bool GenCrashDiag) { - DiagnosticsEngine &Diags = *static_cast(UserData); - - Diags.Report(diag::err_fe_error_backend) << Message; - - // We cannot recover from llvm errors. - exit(1); -} - -int cc1as_main(ArrayRef Argv, const char *Argv0, void *MainAddr) { - // Initialize targets and assembly printers/parsers. - InitializeAllTargetInfos(); - InitializeAllTargetMCs(); - InitializeAllAsmParsers(); - - // Construct our diagnostic client. - IntrusiveRefCntPtr DiagOpts = new DiagnosticOptions(); - TextDiagnosticPrinter *DiagClient - = new TextDiagnosticPrinter(errs(), &*DiagOpts); - DiagClient->setPrefix("clang -cc1as"); - IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); - DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagClient); - - // Set an error handler, so that any LLVM backend diagnostics go through our - // error handler. - ScopedFatalErrorHandler FatalErrorHandler - (LLVMErrorHandler, static_cast(&Diags)); - - // Parse the arguments. - AssemblerInvocation Asm; - if (!AssemblerInvocation::CreateFromArgs(Asm, Argv, Diags)) - return 1; - - if (Asm.ShowHelp) { - std::unique_ptr Opts(driver::createDriverOptTable()); - Opts->PrintHelp(llvm::outs(), "clang -cc1as", "Clang Integrated Assembler", - /*Include=*/driver::options::CC1AsOption, /*Exclude=*/0, - /*ShowAllAliases=*/false); - return 0; - } - - // Honor -version. - // - // FIXME: Use a better -version message? - if (Asm.ShowVersion) { - llvm::cl::PrintVersionMessage(); - return 0; - } - - // Honor -mllvm. - // - // FIXME: Remove this, one day. - if (!Asm.LLVMArgs.empty()) { - unsigned NumArgs = Asm.LLVMArgs.size(); - auto Args = llvm::make_unique(NumArgs + 2); - Args[0] = "clang (LLVM option parsing)"; - for (unsigned i = 0; i != NumArgs; ++i) - Args[i + 1] = Asm.LLVMArgs[i].c_str(); - Args[NumArgs + 1] = nullptr; - llvm::cl::ParseCommandLineOptions(NumArgs + 1, Args.get()); - } - - // Execute the invocation, unless there were parsing errors. - bool Failed = Diags.hasErrorOccurred() || ExecuteAssembler(Asm, Diags); - - // If any timers were active but haven't been destroyed yet, print their - // results now. - TimerGroup::printAll(errs()); - - return !!Failed; -} diff --git a/dbms/programs/clang/Compiler-7.0.0/cc1gen_reproducer_main.cpp b/dbms/programs/clang/Compiler-7.0.0/cc1gen_reproducer_main.cpp deleted file mode 100644 index a4c034d8d35..00000000000 --- a/dbms/programs/clang/Compiler-7.0.0/cc1gen_reproducer_main.cpp +++ /dev/null @@ -1,196 +0,0 @@ -//===-- cc1gen_reproducer_main.cpp - Clang reproducer generator ----------===// -// -// The LLVM Compiler Infrastructure -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This is the entry point to the clang -cc1gen-reproducer functionality, which -// generates reproducers for invocations for clang-based tools. -// -//===----------------------------------------------------------------------===// - -#include "clang/Basic/Diagnostic.h" -#include "clang/Basic/LLVM.h" -#include "clang/Basic/VirtualFileSystem.h" -#include "clang/Driver/Compilation.h" -#include "clang/Driver/Driver.h" -#include "llvm/ADT/ArrayRef.h" -#include "llvm/ADT/STLExtras.h" -#include "llvm/Support/FileSystem.h" -#include "llvm/Support/TargetSelect.h" -#include "llvm/Support/YAMLTraits.h" -#include "llvm/Support/raw_ostream.h" - -using namespace clang; - -namespace { - -struct UnsavedFileHash { - std::string Name; - std::string MD5; -}; - -struct ClangInvocationInfo { - std::string Toolchain; - std::string LibclangOperation; - std::string LibclangOptions; - std::vector Arguments; - std::vector InvocationArguments; - std::vector UnsavedFileHashes; - bool Dump = false; -}; - -} // end anonymous namespace - -LLVM_YAML_IS_SEQUENCE_VECTOR(UnsavedFileHash) - -namespace llvm { -namespace yaml { - -template <> struct MappingTraits { - static void mapping(IO &IO, UnsavedFileHash &Info) { - IO.mapRequired("name", Info.Name); - IO.mapRequired("md5", Info.MD5); - } -}; - -template <> struct MappingTraits { - static void mapping(IO &IO, ClangInvocationInfo &Info) { - IO.mapRequired("toolchain", Info.Toolchain); - IO.mapOptional("libclang.operation", Info.LibclangOperation); - IO.mapOptional("libclang.opts", Info.LibclangOptions); - IO.mapRequired("args", Info.Arguments); - IO.mapOptional("invocation-args", Info.InvocationArguments); - IO.mapOptional("unsaved_file_hashes", Info.UnsavedFileHashes); - } -}; - -} // end namespace yaml -} // end namespace llvm - -static std::string generateReproducerMetaInfo(const ClangInvocationInfo &Info) { - std::string Result; - llvm::raw_string_ostream OS(Result); - OS << '{'; - bool NeedComma = false; - auto EmitKey = [&](StringRef Key) { - if (NeedComma) - OS << ", "; - NeedComma = true; - OS << '"' << Key << "\": "; - }; - auto EmitStringKey = [&](StringRef Key, StringRef Value) { - if (Value.empty()) - return; - EmitKey(Key); - OS << '"' << Value << '"'; - }; - EmitStringKey("libclang.operation", Info.LibclangOperation); - EmitStringKey("libclang.opts", Info.LibclangOptions); - if (!Info.InvocationArguments.empty()) { - EmitKey("invocation-args"); - OS << '['; - for (const auto &Arg : llvm::enumerate(Info.InvocationArguments)) { - if (Arg.index()) - OS << ','; - OS << '"' << Arg.value() << '"'; - } - OS << ']'; - } - OS << '}'; - // FIXME: Compare unsaved file hashes and report mismatch in the reproducer. - if (Info.Dump) - llvm::outs() << "REPRODUCER METAINFO: " << OS.str() << "\n"; - return std::move(OS.str()); -} - -/// Generates a reproducer for a set of arguments from a specific invocation. -static llvm::Optional -generateReproducerForInvocationArguments(ArrayRef Argv, - const ClangInvocationInfo &Info) { - using namespace driver; - auto TargetAndMode = ToolChain::getTargetAndModeFromProgramName(Argv[0]); - - IntrusiveRefCntPtr DiagOpts = new DiagnosticOptions; - - IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); - DiagnosticsEngine Diags(DiagID, &*DiagOpts, new IgnoringDiagConsumer()); - ProcessWarningOptions(Diags, *DiagOpts, /*ReportDiags=*/false); - Driver TheDriver(Argv[0], llvm::sys::getDefaultTargetTriple(), Diags); - TheDriver.setTargetAndMode(TargetAndMode); - - std::unique_ptr C(TheDriver.BuildCompilation(Argv)); - if (C && !C->containsError()) { - for (const auto &J : C->getJobs()) { - if (const Command *Cmd = dyn_cast(&J)) { - Driver::CompilationDiagnosticReport Report; - TheDriver.generateCompilationDiagnostics( - *C, *Cmd, generateReproducerMetaInfo(Info), &Report); - return Report; - } - } - } - - return None; -} - -std::string GetExecutablePath(const char *Argv0, bool CanonicalPrefixes); - -static void printReproducerInformation( - llvm::raw_ostream &OS, const ClangInvocationInfo &Info, - const driver::Driver::CompilationDiagnosticReport &Report) { - OS << "REPRODUCER:\n"; - OS << "{\n"; - OS << R"("files":[)"; - for (const auto &File : llvm::enumerate(Report.TemporaryFiles)) { - if (File.index()) - OS << ','; - OS << '"' << File.value() << '"'; - } - OS << "]\n}\n"; -} - -int cc1gen_reproducer_main(ArrayRef Argv, const char *Argv0, - void *MainAddr) { - if (Argv.size() < 1) { - llvm::errs() << "error: missing invocation file\n"; - return 1; - } - // Parse the invocation descriptor. - StringRef Input = Argv[0]; - llvm::ErrorOr> Buffer = - llvm::MemoryBuffer::getFile(Input); - if (!Buffer) { - llvm::errs() << "error: failed to read " << Input << ": " - << Buffer.getError().message() << "\n"; - return 1; - } - llvm::yaml::Input YAML(Buffer.get()->getBuffer()); - ClangInvocationInfo InvocationInfo; - YAML >> InvocationInfo; - if (Argv.size() > 1 && Argv[1] == StringRef("-v")) - InvocationInfo.Dump = true; - - // Create an invocation that will produce the reproducer. - std::vector DriverArgs; - for (const auto &Arg : InvocationInfo.Arguments) - DriverArgs.push_back(Arg.c_str()); - std::string Path = GetExecutablePath(Argv0, /*CanonicalPrefixes=*/true); - DriverArgs[0] = Path.c_str(); - llvm::Optional Report = - generateReproducerForInvocationArguments(DriverArgs, InvocationInfo); - - // Emit the information about the reproduce files to stdout. - int Result = 1; - if (Report) { - printReproducerInformation(llvm::outs(), InvocationInfo, *Report); - Result = 0; - } - - // Remove the input file. - llvm::sys::fs::remove(Input); - return Result; -} diff --git a/dbms/programs/clang/Compiler-7.0.0/driver.cpp b/dbms/programs/clang/Compiler-7.0.0/driver.cpp deleted file mode 100644 index 79d71b08ba7..00000000000 --- a/dbms/programs/clang/Compiler-7.0.0/driver.cpp +++ /dev/null @@ -1,514 +0,0 @@ -//===-- driver.cpp - Clang GCC-Compatible Driver --------------------------===// -// -// The LLVM Compiler Infrastructure -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This is the entry point to the clang driver; it is a thin wrapper -// for functionality in the Driver clang library. -// -//===----------------------------------------------------------------------===// - -#include "clang/Driver/Driver.h" -#include "clang/Basic/DiagnosticOptions.h" -#include "clang/Driver/Compilation.h" -#include "clang/Driver/DriverDiagnostic.h" -#include "clang/Driver/Options.h" -#include "clang/Driver/ToolChain.h" -#include "clang/Frontend/ChainedDiagnosticConsumer.h" -#include "clang/Frontend/CompilerInvocation.h" -#include "clang/Frontend/SerializedDiagnosticPrinter.h" -#include "clang/Frontend/TextDiagnosticPrinter.h" -#include "clang/Frontend/Utils.h" -#include "llvm/ADT/ArrayRef.h" -#include "llvm/ADT/SmallString.h" -#include "llvm/ADT/SmallVector.h" -#include "llvm/Option/ArgList.h" -#include "llvm/Option/OptTable.h" -#include "llvm/Option/Option.h" -#include "llvm/Support/CommandLine.h" -#include "llvm/Support/ErrorHandling.h" -#include "llvm/Support/FileSystem.h" -#include "llvm/Support/Host.h" -#include "llvm/Support/InitLLVM.h" -#include "llvm/Support/Path.h" -#include "llvm/Support/Process.h" -#include "llvm/Support/Program.h" -#include "llvm/Support/Regex.h" -#include "llvm/Support/Signals.h" -#include "llvm/Support/StringSaver.h" -#include "llvm/Support/TargetSelect.h" -#include "llvm/Support/Timer.h" -#include "llvm/Support/raw_ostream.h" -#include -#include -#include -using namespace clang; -using namespace clang::driver; -using namespace llvm::opt; - -std::string GetExecutablePath(const char *Argv0, bool CanonicalPrefixes) { - if (!CanonicalPrefixes) { - SmallString<128> ExecutablePath(Argv0); - // Do a PATH lookup if Argv0 isn't a valid path. - if (!llvm::sys::fs::exists(ExecutablePath)) - if (llvm::ErrorOr P = - llvm::sys::findProgramByName(ExecutablePath)) - ExecutablePath = *P; - return ExecutablePath.str(); - } - - // This just needs to be some symbol in the binary; C++ doesn't - // allow taking the address of ::main however. - void *P = (void*) (intptr_t) GetExecutablePath; - return llvm::sys::fs::getMainExecutable(Argv0, P); -} - -static const char *GetStableCStr(std::set &SavedStrings, - StringRef S) { - return SavedStrings.insert(S).first->c_str(); -} - -/// ApplyQAOverride - Apply a list of edits to the input argument lists. -/// -/// The input string is a space separate list of edits to perform, -/// they are applied in order to the input argument lists. Edits -/// should be one of the following forms: -/// -/// '#': Silence information about the changes to the command line arguments. -/// -/// '^': Add FOO as a new argument at the beginning of the command line. -/// -/// '+': Add FOO as a new argument at the end of the command line. -/// -/// 's/XXX/YYY/': Substitute the regular expression XXX with YYY in the command -/// line. -/// -/// 'xOPTION': Removes all instances of the literal argument OPTION. -/// -/// 'XOPTION': Removes all instances of the literal argument OPTION, -/// and the following argument. -/// -/// 'Ox': Removes all flags matching 'O' or 'O[sz0-9]' and adds 'Ox' -/// at the end of the command line. -/// -/// \param OS - The stream to write edit information to. -/// \param Args - The vector of command line arguments. -/// \param Edit - The override command to perform. -/// \param SavedStrings - Set to use for storing string representations. -static void ApplyOneQAOverride(raw_ostream &OS, - SmallVectorImpl &Args, - StringRef Edit, - std::set &SavedStrings) { - // This does not need to be efficient. - - if (Edit[0] == '^') { - const char *Str = - GetStableCStr(SavedStrings, Edit.substr(1)); - OS << "### Adding argument " << Str << " at beginning\n"; - Args.insert(Args.begin() + 1, Str); - } else if (Edit[0] == '+') { - const char *Str = - GetStableCStr(SavedStrings, Edit.substr(1)); - OS << "### Adding argument " << Str << " at end\n"; - Args.push_back(Str); - } else if (Edit[0] == 's' && Edit[1] == '/' && Edit.endswith("/") && - Edit.slice(2, Edit.size()-1).find('/') != StringRef::npos) { - StringRef MatchPattern = Edit.substr(2).split('/').first; - StringRef ReplPattern = Edit.substr(2).split('/').second; - ReplPattern = ReplPattern.slice(0, ReplPattern.size()-1); - - for (unsigned i = 1, e = Args.size(); i != e; ++i) { - // Ignore end-of-line response file markers - if (Args[i] == nullptr) - continue; - std::string Repl = llvm::Regex(MatchPattern).sub(ReplPattern, Args[i]); - - if (Repl != Args[i]) { - OS << "### Replacing '" << Args[i] << "' with '" << Repl << "'\n"; - Args[i] = GetStableCStr(SavedStrings, Repl); - } - } - } else if (Edit[0] == 'x' || Edit[0] == 'X') { - auto Option = Edit.substr(1); - for (unsigned i = 1; i < Args.size();) { - if (Option == Args[i]) { - OS << "### Deleting argument " << Args[i] << '\n'; - Args.erase(Args.begin() + i); - if (Edit[0] == 'X') { - if (i < Args.size()) { - OS << "### Deleting argument " << Args[i] << '\n'; - Args.erase(Args.begin() + i); - } else - OS << "### Invalid X edit, end of command line!\n"; - } - } else - ++i; - } - } else if (Edit[0] == 'O') { - for (unsigned i = 1; i < Args.size();) { - const char *A = Args[i]; - // Ignore end-of-line response file markers - if (A == nullptr) - continue; - if (A[0] == '-' && A[1] == 'O' && - (A[2] == '\0' || - (A[3] == '\0' && (A[2] == 's' || A[2] == 'z' || - ('0' <= A[2] && A[2] <= '9'))))) { - OS << "### Deleting argument " << Args[i] << '\n'; - Args.erase(Args.begin() + i); - } else - ++i; - } - OS << "### Adding argument " << Edit << " at end\n"; - Args.push_back(GetStableCStr(SavedStrings, '-' + Edit.str())); - } else { - OS << "### Unrecognized edit: " << Edit << "\n"; - } -} - -/// ApplyQAOverride - Apply a comma separate list of edits to the -/// input argument lists. See ApplyOneQAOverride. -static void ApplyQAOverride(SmallVectorImpl &Args, - const char *OverrideStr, - std::set &SavedStrings) { - raw_ostream *OS = &llvm::errs(); - - if (OverrideStr[0] == '#') { - ++OverrideStr; - OS = &llvm::nulls(); - } - - *OS << "### CCC_OVERRIDE_OPTIONS: " << OverrideStr << "\n"; - - // This does not need to be efficient. - - const char *S = OverrideStr; - while (*S) { - const char *End = ::strchr(S, ' '); - if (!End) - End = S + strlen(S); - if (End != S) - ApplyOneQAOverride(*OS, Args, std::string(S, End), SavedStrings); - S = End; - if (*S != '\0') - ++S; - } -} - -extern int cc1_main(ArrayRef Argv, const char *Argv0, - void *MainAddr); -extern int cc1as_main(ArrayRef Argv, const char *Argv0, - void *MainAddr); -extern int cc1gen_reproducer_main(ArrayRef Argv, - const char *Argv0, void *MainAddr); - -static void insertTargetAndModeArgs(const ParsedClangName &NameParts, - SmallVectorImpl &ArgVector, - std::set &SavedStrings) { - // Put target and mode arguments at the start of argument list so that - // arguments specified in command line could override them. Avoid putting - // them at index 0, as an option like '-cc1' must remain the first. - int InsertionPoint = 0; - if (ArgVector.size() > 0) - ++InsertionPoint; - - if (NameParts.DriverMode) { - // Add the mode flag to the arguments. - ArgVector.insert(ArgVector.begin() + InsertionPoint, - GetStableCStr(SavedStrings, NameParts.DriverMode)); - } - - if (NameParts.TargetIsValid) { - const char *arr[] = {"-target", GetStableCStr(SavedStrings, - NameParts.TargetPrefix)}; - ArgVector.insert(ArgVector.begin() + InsertionPoint, - std::begin(arr), std::end(arr)); - } -} - -static void getCLEnvVarOptions(std::string &EnvValue, llvm::StringSaver &Saver, - SmallVectorImpl &Opts) { - llvm::cl::TokenizeWindowsCommandLine(EnvValue, Saver, Opts); - // The first instance of '#' should be replaced with '=' in each option. - for (const char *Opt : Opts) - if (char *NumberSignPtr = const_cast(::strchr(Opt, '#'))) - *NumberSignPtr = '='; -} - -static void SetBackdoorDriverOutputsFromEnvVars(Driver &TheDriver) { - // Handle CC_PRINT_OPTIONS and CC_PRINT_OPTIONS_FILE. - TheDriver.CCPrintOptions = !!::getenv("CC_PRINT_OPTIONS"); - if (TheDriver.CCPrintOptions) - TheDriver.CCPrintOptionsFilename = ::getenv("CC_PRINT_OPTIONS_FILE"); - - // Handle CC_PRINT_HEADERS and CC_PRINT_HEADERS_FILE. - TheDriver.CCPrintHeaders = !!::getenv("CC_PRINT_HEADERS"); - if (TheDriver.CCPrintHeaders) - TheDriver.CCPrintHeadersFilename = ::getenv("CC_PRINT_HEADERS_FILE"); - - // Handle CC_LOG_DIAGNOSTICS and CC_LOG_DIAGNOSTICS_FILE. - TheDriver.CCLogDiagnostics = !!::getenv("CC_LOG_DIAGNOSTICS"); - if (TheDriver.CCLogDiagnostics) - TheDriver.CCLogDiagnosticsFilename = ::getenv("CC_LOG_DIAGNOSTICS_FILE"); -} - -static void FixupDiagPrefixExeName(TextDiagnosticPrinter *DiagClient, - const std::string &Path) { - // If the clang binary happens to be named cl.exe for compatibility reasons, - // use clang-cl.exe as the prefix to avoid confusion between clang and MSVC. - StringRef ExeBasename(llvm::sys::path::filename(Path)); - if (ExeBasename.equals_lower("cl.exe")) - ExeBasename = "clang-cl.exe"; - DiagClient->setPrefix(ExeBasename); -} - -// This lets us create the DiagnosticsEngine with a properly-filled-out -// DiagnosticOptions instance. -static DiagnosticOptions * -CreateAndPopulateDiagOpts(ArrayRef argv) { - auto *DiagOpts = new DiagnosticOptions; - std::unique_ptr Opts(createDriverOptTable()); - unsigned MissingArgIndex, MissingArgCount; - InputArgList Args = - Opts->ParseArgs(argv.slice(1), MissingArgIndex, MissingArgCount); - // We ignore MissingArgCount and the return value of ParseDiagnosticArgs. - // Any errors that would be diagnosed here will also be diagnosed later, - // when the DiagnosticsEngine actually exists. - (void)ParseDiagnosticArgs(*DiagOpts, Args); - return DiagOpts; -} - -static void SetInstallDir(SmallVectorImpl &argv, - Driver &TheDriver, bool CanonicalPrefixes) { - // Attempt to find the original path used to invoke the driver, to determine - // the installed path. We do this manually, because we want to support that - // path being a symlink. - SmallString<128> InstalledPath(argv[0]); - - // Do a PATH lookup, if there are no directory components. - if (llvm::sys::path::filename(InstalledPath) == InstalledPath) - if (llvm::ErrorOr Tmp = llvm::sys::findProgramByName( - llvm::sys::path::filename(InstalledPath.str()))) - InstalledPath = *Tmp; - - // FIXME: We don't actually canonicalize this, we just make it absolute. - if (CanonicalPrefixes) - llvm::sys::fs::make_absolute(InstalledPath); - - StringRef InstalledPathParent(llvm::sys::path::parent_path(InstalledPath)); - if (llvm::sys::fs::exists(InstalledPathParent)) - TheDriver.setInstalledDir(InstalledPathParent); -} - -static int ExecuteCC1Tool(ArrayRef argv, StringRef Tool) { - void *GetExecutablePathVP = (void *)(intptr_t) GetExecutablePath; - if (Tool == "") - return cc1_main(argv.slice(2), argv[0], GetExecutablePathVP); - if (Tool == "as") - return cc1as_main(argv.slice(2), argv[0], GetExecutablePathVP); - if (Tool == "gen-reproducer") - return cc1gen_reproducer_main(argv.slice(2), argv[0], GetExecutablePathVP); - - // Reject unknown tools. - llvm::errs() << "error: unknown integrated tool '" << Tool << "'. " - << "Valid tools include '-cc1' and '-cc1as'.\n"; - return 1; -} - -int mainEntryClickHouseClang(int argc_, /* const */ char **argv_) { - llvm::InitLLVM X(argc_, argv_); - SmallVector argv(argv_, argv_ + argc_); - - if (llvm::sys::Process::FixupStandardFileDescriptors()) - return 1; - - llvm::InitializeAllTargets(); - auto TargetAndMode = ToolChain::getTargetAndModeFromProgramName(argv[0]); - - llvm::BumpPtrAllocator A; - llvm::StringSaver Saver(A); - - // Parse response files using the GNU syntax, unless we're in CL mode. There - // are two ways to put clang in CL compatibility mode: argv[0] is either - // clang-cl or cl, or --driver-mode=cl is on the command line. The normal - // command line parsing can't happen until after response file parsing, so we - // have to manually search for a --driver-mode=cl argument the hard way. - // Finally, our -cc1 tools don't care which tokenization mode we use because - // response files written by clang will tokenize the same way in either mode. - bool ClangCLMode = false; - if (StringRef(TargetAndMode.DriverMode).equals("--driver-mode=cl") || - std::find_if(argv.begin(), argv.end(), [](const char *F) { - return F && strcmp(F, "--driver-mode=cl") == 0; - }) != argv.end()) { - ClangCLMode = true; - } - enum { Default, POSIX, Windows } RSPQuoting = Default; - for (const char *F : argv) { - if (strcmp(F, "--rsp-quoting=posix") == 0) - RSPQuoting = POSIX; - else if (strcmp(F, "--rsp-quoting=windows") == 0) - RSPQuoting = Windows; - } - - // Determines whether we want nullptr markers in argv to indicate response - // files end-of-lines. We only use this for the /LINK driver argument with - // clang-cl.exe on Windows. - bool MarkEOLs = ClangCLMode; - - llvm::cl::TokenizerCallback Tokenizer; - if (RSPQuoting == Windows || (RSPQuoting == Default && ClangCLMode)) - Tokenizer = &llvm::cl::TokenizeWindowsCommandLine; - else - Tokenizer = &llvm::cl::TokenizeGNUCommandLine; - - if (MarkEOLs && argv.size() > 1 && StringRef(argv[1]).startswith("-cc1")) - MarkEOLs = false; - llvm::cl::ExpandResponseFiles(Saver, Tokenizer, argv, MarkEOLs); - - // Handle -cc1 integrated tools, even if -cc1 was expanded from a response - // file. - auto FirstArg = std::find_if(argv.begin() + 1, argv.end(), - [](const char *A) { return A != nullptr; }); - if (FirstArg != argv.end() && StringRef(*FirstArg).startswith("-cc1")) { - // If -cc1 came from a response file, remove the EOL sentinels. - if (MarkEOLs) { - auto newEnd = std::remove(argv.begin(), argv.end(), nullptr); - argv.resize(newEnd - argv.begin()); - } - return ExecuteCC1Tool(argv, argv[1] + 4); - } - - bool CanonicalPrefixes = true; - for (int i = 1, size = argv.size(); i < size; ++i) { - // Skip end-of-line response file markers - if (argv[i] == nullptr) - continue; - if (StringRef(argv[i]) == "-no-canonical-prefixes") { - CanonicalPrefixes = false; - break; - } - } - - // Handle CL and _CL_ which permits additional command line options to be - // prepended or appended. - if (ClangCLMode) { - // Arguments in "CL" are prepended. - llvm::Optional OptCL = llvm::sys::Process::GetEnv("CL"); - if (OptCL.hasValue()) { - SmallVector PrependedOpts; - getCLEnvVarOptions(OptCL.getValue(), Saver, PrependedOpts); - - // Insert right after the program name to prepend to the argument list. - argv.insert(argv.begin() + 1, PrependedOpts.begin(), PrependedOpts.end()); - } - // Arguments in "_CL_" are appended. - llvm::Optional Opt_CL_ = llvm::sys::Process::GetEnv("_CL_"); - if (Opt_CL_.hasValue()) { - SmallVector AppendedOpts; - getCLEnvVarOptions(Opt_CL_.getValue(), Saver, AppendedOpts); - - // Insert at the end of the argument list to append. - argv.append(AppendedOpts.begin(), AppendedOpts.end()); - } - } - - std::set SavedStrings; - // Handle CCC_OVERRIDE_OPTIONS, used for editing a command line behind the - // scenes. - if (const char *OverrideStr = ::getenv("CCC_OVERRIDE_OPTIONS")) { - // FIXME: Driver shouldn't take extra initial argument. - ApplyQAOverride(argv, OverrideStr, SavedStrings); - } - - std::string Path = GetExecutablePath(argv[0], CanonicalPrefixes); - - IntrusiveRefCntPtr DiagOpts = - CreateAndPopulateDiagOpts(argv); - - TextDiagnosticPrinter *DiagClient - = new TextDiagnosticPrinter(llvm::errs(), &*DiagOpts); - FixupDiagPrefixExeName(DiagClient, Path); - - IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); - - DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagClient); - - if (!DiagOpts->DiagnosticSerializationFile.empty()) { - auto SerializedConsumer = - clang::serialized_diags::create(DiagOpts->DiagnosticSerializationFile, - &*DiagOpts, /*MergeChildRecords=*/true); - Diags.setClient(new ChainedDiagnosticConsumer( - Diags.takeClient(), std::move(SerializedConsumer))); - } - - ProcessWarningOptions(Diags, *DiagOpts, /*ReportDiags=*/false); - - Driver TheDriver(Path, llvm::sys::getDefaultTargetTriple(), Diags); - SetInstallDir(argv, TheDriver, CanonicalPrefixes); - TheDriver.setTargetAndMode(TargetAndMode); - - insertTargetAndModeArgs(TargetAndMode, argv, SavedStrings); - - SetBackdoorDriverOutputsFromEnvVars(TheDriver); - - std::unique_ptr C(TheDriver.BuildCompilation(argv)); - int Res = 1; - if (C && !C->containsError()) { - SmallVector, 4> FailingCommands; - Res = TheDriver.ExecuteCompilation(*C, FailingCommands); - - // Force a crash to test the diagnostics. - if (TheDriver.GenReproducer) { - Diags.Report(diag::err_drv_force_crash) - << !::getenv("FORCE_CLANG_DIAGNOSTICS_CRASH"); - - // Pretend that every command failed. - FailingCommands.clear(); - for (const auto &J : C->getJobs()) - if (const Command *C = dyn_cast(&J)) - FailingCommands.push_back(std::make_pair(-1, C)); - } - - for (const auto &P : FailingCommands) { - int CommandRes = P.first; - const Command *FailingCommand = P.second; - if (!Res) - Res = CommandRes; - - // If result status is < 0, then the driver command signalled an error. - // If result status is 70, then the driver command reported a fatal error. - // On Windows, abort will return an exit code of 3. In these cases, - // generate additional diagnostic information if possible. - bool DiagnoseCrash = CommandRes < 0 || CommandRes == 70; -#ifdef _WIN32 - DiagnoseCrash |= CommandRes == 3; -#endif - if (DiagnoseCrash) { - TheDriver.generateCompilationDiagnostics(*C, *FailingCommand); - break; - } - } - } - - Diags.getClient()->finish(); - - // If any timers were active but haven't been destroyed yet, print their - // results now. This happens in -disable-free mode. - llvm::TimerGroup::printAll(llvm::errs()); - -#ifdef _WIN32 - // Exit status should not be negative on Win32, unless abnormal termination. - // Once abnormal termiation was caught, negative status should not be - // propagated. - if (Res < 0) - Res = 1; -#endif - - // If we have multiple failing commands, we return the result of the first - // failing command. - return Res; -} diff --git a/dbms/programs/clang/Compiler-7.0.0/lld.cpp b/dbms/programs/clang/Compiler-7.0.0/lld.cpp deleted file mode 100644 index 8e118b6e24b..00000000000 --- a/dbms/programs/clang/Compiler-7.0.0/lld.cpp +++ /dev/null @@ -1,150 +0,0 @@ -//===- tools/lld/lld.cpp - Linker Driver Dispatcher -----------------------===// -// -// The LLVM Linker -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This file contains the main function of the lld executable. The main -// function is a thin wrapper which dispatches to the platform specific -// driver. -// -// lld is a single executable that contains four different linkers for ELF, -// COFF, WebAssembly and Mach-O. The main function dispatches according to -// argv[0] (i.e. command name). The most common name for each target is shown -// below: -// -// - ld.lld: ELF (Unix) -// - ld64: Mach-O (macOS) -// - lld-link: COFF (Windows) -// - ld-wasm: WebAssembly -// -// lld can be invoked as "lld" along with "-flavor" option. This is for -// backward compatibility and not recommended. -// -//===----------------------------------------------------------------------===// - -#include "lld/Common/Driver.h" -#include "llvm/ADT/STLExtras.h" -#include "llvm/ADT/StringSwitch.h" -#include "llvm/ADT/Twine.h" -#include "llvm/Support/InitLLVM.h" -#include "llvm/Support/Path.h" -#include -using namespace lld; -using namespace llvm; -using namespace llvm::sys; - -/* - -enum Flavor { - Invalid, - Gnu, // -flavor gnu - WinLink, // -flavor link - Darwin, // -flavor darwin - Wasm, // -flavor wasm -}; - -LLVM_ATTRIBUTE_NORETURN static void die(const Twine &S) { - errs() << S << "\n"; - exit(1); -} - -static Flavor getFlavor(StringRef S) { - return StringSwitch(S) - .CasesLower("ld", "ld.lld", "gnu", Gnu) - .CasesLower("wasm", "ld-wasm", Wasm) - .CaseLower("link", WinLink) - .CasesLower("ld64", "ld64.lld", "darwin", Darwin) - .Default(Invalid); -} - -static bool isPETarget(const std::vector &V) { - for (auto It = V.begin(); It + 1 != V.end(); ++It) { - if (StringRef(*It) != "-m") - continue; - StringRef S = *(It + 1); - return S == "i386pe" || S == "i386pep" || S == "thumb2pe" || S == "arm64pe"; - } - return false; -} - -static Flavor parseProgname(StringRef Progname) { -#if __APPLE__ - // Use Darwin driver for "ld" on Darwin. - if (Progname == "ld") - return Darwin; -#endif - -#if LLVM_ON_UNIX - // Use GNU driver for "ld" on other Unix-like system. - if (Progname == "ld") - return Gnu; -#endif - - // Progname may be something like "lld-gnu". Parse it. - SmallVector V; - Progname.split(V, "-"); - for (StringRef S : V) - if (Flavor F = getFlavor(S)) - return F; - return Invalid; -} - -static Flavor parseFlavor(std::vector &V) { - // Parse -flavor option. - if (V.size() > 1 && V[1] == StringRef("-flavor")) { - if (V.size() <= 2) - die("missing arg value for '-flavor'"); - Flavor F = getFlavor(V[2]); - if (F == Invalid) - die("Unknown flavor: " + StringRef(V[2])); - V.erase(V.begin() + 1, V.begin() + 3); - return F; - } - - // Deduct the flavor from argv[0]. - StringRef Arg0 = path::filename(V[0]); - if (Arg0.endswith_lower(".exe")) - Arg0 = Arg0.drop_back(4); - return parseProgname(Arg0); -} -*/ - -// If this function returns true, lld calls _exit() so that it quickly -// exits without invoking destructors of globally allocated objects. -// -// We don't want to do that if we are running tests though, because -// doing that breaks leak sanitizer. So, lit sets this environment variable, -// and we use it to detect whether we are running tests or not. -static bool canExitEarly() { return StringRef(getenv("LLD_IN_TEST")) != "1"; } - -/// Universal linker main(). This linker emulates the gnu, darwin, or -/// windows linker based on the argv[0] or -flavor option. -int mainEntryClickHouseLLD(int Argc, /* const */ char **Argv) { - InitLLVM X(Argc, Argv); - - std::vector Args(Argv, Argv + Argc); -/* - switch (parseFlavor(Args)) { - case Gnu: - if (isPETarget(Args)) - return !mingw::link(Args); -*/ - return !elf::link(Args, canExitEarly()); -/* - case WinLink: - return !coff::link(Args, canExitEarly()); - case Darwin: - return !mach_o::link(Args, canExitEarly()); - case Wasm: - return !wasm::link(Args, canExitEarly()); - default: - die("lld is a generic driver.\n" - "Invoke ld.lld (Unix), ld64.lld (macOS), lld-link (Windows), wasm-lld" - " (WebAssembly) instead"); - } -*/ -} diff --git a/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt b/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt deleted file mode 100644 index a5f8314b862..00000000000 --- a/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt +++ /dev/null @@ -1,49 +0,0 @@ -add_definitions(-Wno-error -Wno-unused-parameter -Wno-non-virtual-dtor -U_LIBCPP_DEBUG) - -link_directories(${LLVM_LIBRARY_DIRS}) - -add_library(clickhouse-compiler-lib - driver.cpp - cc1_main.cpp - cc1as_main.cpp - lld.cpp) - -target_compile_options(clickhouse-compiler-lib PRIVATE -fno-rtti -fno-exceptions -g0) - -string(REPLACE "${INCLUDE_DEBUG_HELPERS}" "" CMAKE_CXX_FLAGS ${CMAKE_CXX_FLAGS}) # cant compile with -fno-rtti - -llvm_libs_all(REQUIRED_LLVM_LIBRARIES) - -message(STATUS "Using LLVM ${LLVM_VERSION}: ${LLVM_INCLUDE_DIRS} : ${REQUIRED_LLVM_LIBRARIES}") - -target_include_directories(clickhouse-compiler-lib SYSTEM PRIVATE ${LLVM_INCLUDE_DIRS}) - -# This is extracted almost directly from CMakeFiles/.../link.txt in LLVM build directory. - -target_link_libraries(clickhouse-compiler-lib PRIVATE - -clangBasic clangCodeGen clangDriver -clangFrontend -clangFrontendTool -clangRewriteFrontend clangARCMigrate clangStaticAnalyzerFrontend -clangParse clangSerialization clangSema clangEdit clangStaticAnalyzerCheckers -clangASTMatchers clangStaticAnalyzerCore clangAnalysis clangAST clangRewrite clangLex clangBasic -clangCrossTU clangIndex - -lldCOFF -lldDriver -lldELF -lldMinGW -lldMachO -lldReaderWriter -lldYAML -lldCommon -lldCore - -${REQUIRED_LLVM_LIBRARIES} - -PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARIES} Threads::Threads -${MALLOC_LIBRARIES} -${GLIBC_COMPATIBILITY_LIBRARIES} -${MEMCPY_LIBRARIES} -) diff --git a/dbms/programs/clang/Compiler-7.0.0bundled/cc1_main.cpp b/dbms/programs/clang/Compiler-7.0.0bundled/cc1_main.cpp deleted file mode 100644 index 3686475dd42..00000000000 --- a/dbms/programs/clang/Compiler-7.0.0bundled/cc1_main.cpp +++ /dev/null @@ -1,243 +0,0 @@ -//===-- cc1_main.cpp - Clang CC1 Compiler Frontend ------------------------===// -// -// The LLVM Compiler Infrastructure -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This is the entry point to the clang -cc1 functionality, which implements the -// core compiler functionality along with a number of additional tools for -// demonstration and testing purposes. -// -//===----------------------------------------------------------------------===// - -#include "llvm/Option/Arg.h" -#include "clang/CodeGen/ObjectFilePCHContainerOperations.h" -#include "clang/Config/config.h" -#include "clang/Driver/DriverDiagnostic.h" -#include "clang/Driver/Options.h" -#include "clang/Frontend/CompilerInstance.h" -#include "clang/Frontend/CompilerInvocation.h" -#include "clang/Frontend/FrontendDiagnostic.h" -#include "clang/Frontend/TextDiagnosticBuffer.h" -#include "clang/Frontend/TextDiagnosticPrinter.h" -#include "clang/Frontend/Utils.h" -#include "clang/FrontendTool/Utils.h" -#include "llvm/ADT/Statistic.h" -#include "llvm/Config/llvm-config.h" -#include "llvm/LinkAllPasses.h" -#include "llvm/Option/ArgList.h" -#include "llvm/Option/OptTable.h" -#include "llvm/Support/Compiler.h" -#include "llvm/Support/ErrorHandling.h" -#include "llvm/Support/ManagedStatic.h" -#include "llvm/Support/Signals.h" -#include "llvm/Support/TargetSelect.h" -#include "llvm/Support/Timer.h" -#include "llvm/Support/raw_ostream.h" -#include - -#ifdef CLANG_HAVE_RLIMITS -#include -#endif - -// have no .a version in packages -#undef LINK_POLLY_INTO_TOOLS - -using namespace clang; -using namespace llvm::opt; - -//===----------------------------------------------------------------------===// -// Main driver -//===----------------------------------------------------------------------===// - -static void LLVMErrorHandler(void *UserData, const std::string &Message, - bool GenCrashDiag) { - DiagnosticsEngine &Diags = *static_cast(UserData); - - Diags.Report(diag::err_fe_error_backend) << Message; - - // Run the interrupt handlers to make sure any special cleanups get done, in - // particular that we remove files registered with RemoveFileOnSignal. - llvm::sys::RunInterruptHandlers(); - - // We cannot recover from llvm errors. When reporting a fatal error, exit - // with status 70 to generate crash diagnostics. For BSD systems this is - // defined as an internal software error. Otherwise, exit with status 1. - exit(GenCrashDiag ? 70 : 1); -} - -#ifdef LINK_POLLY_INTO_TOOLS -namespace polly { -void initializePollyPasses(llvm::PassRegistry &Registry); -} -#endif - -#ifdef CLANG_HAVE_RLIMITS -// The amount of stack we think is "sufficient". If less than this much is -// available, we may be unable to reach our template instantiation depth -// limit and other similar limits. -// FIXME: Unify this with the stack we request when spawning a thread to build -// a module. -static const int kSufficientStack = 8 << 20; - -#if defined(__linux__) && defined(__PIE__) -static size_t getCurrentStackAllocation() { - // If we can't compute the current stack usage, allow for 512K of command - // line arguments and environment. - size_t Usage = 512 * 1024; - if (FILE *StatFile = fopen("/proc/self/stat", "r")) { - // We assume that the stack extends from its current address to the end of - // the environment space. In reality, there is another string literal (the - // program name) after the environment, but this is close enough (we only - // need to be within 100K or so). - unsigned long StackPtr, EnvEnd; - // Disable silly GCC -Wformat warning that complains about length - // modifiers on ignored format specifiers. We want to retain these - // for documentation purposes even though they have no effect. -#if defined(__GNUC__) && !defined(__clang__) -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wformat" -#endif - if (fscanf(StatFile, - "%*d %*s %*c %*d %*d %*d %*d %*d %*u %*lu %*lu %*lu %*lu %*lu " - "%*lu %*ld %*ld %*ld %*ld %*ld %*ld %*llu %*lu %*ld %*lu %*lu " - "%*lu %*lu %lu %*lu %*lu %*lu %*lu %*lu %*llu %*lu %*lu %*d %*d " - "%*u %*u %*llu %*lu %*ld %*lu %*lu %*lu %*lu %*lu %*lu %lu %*d", - &StackPtr, &EnvEnd) == 2) { -#if defined(__GNUC__) && !defined(__clang__) -#pragma GCC diagnostic pop -#endif - Usage = StackPtr < EnvEnd ? EnvEnd - StackPtr : StackPtr - EnvEnd; - } - fclose(StatFile); - } - return Usage; -} - -#include - -LLVM_ATTRIBUTE_NOINLINE -static void ensureStackAddressSpace(int ExtraChunks = 0) { - // Linux kernels prior to 4.1 will sometimes locate the heap of a PIE binary - // relatively close to the stack (they are only guaranteed to be 128MiB - // apart). This results in crashes if we happen to heap-allocate more than - // 128MiB before we reach our stack high-water mark. - // - // To avoid these crashes, ensure that we have sufficient virtual memory - // pages allocated before we start running. - size_t Curr = getCurrentStackAllocation(); - const int kTargetStack = kSufficientStack - 256 * 1024; - if (Curr < kTargetStack) { - volatile char *volatile Alloc = - static_cast(alloca(kTargetStack - Curr)); - Alloc[0] = 0; - Alloc[kTargetStack - Curr - 1] = 0; - } -} -#else -static void ensureStackAddressSpace() {} -#endif - -/// Attempt to ensure that we have at least 8MiB of usable stack space. -static void ensureSufficientStack() { - struct rlimit rlim; - if (getrlimit(RLIMIT_STACK, &rlim) != 0) - return; - - // Increase the soft stack limit to our desired level, if necessary and - // possible. - if (rlim.rlim_cur != RLIM_INFINITY && rlim.rlim_cur < kSufficientStack) { - // Try to allocate sufficient stack. - if (rlim.rlim_max == RLIM_INFINITY || rlim.rlim_max >= kSufficientStack) - rlim.rlim_cur = kSufficientStack; - else if (rlim.rlim_cur == rlim.rlim_max) - return; - else - rlim.rlim_cur = rlim.rlim_max; - - if (setrlimit(RLIMIT_STACK, &rlim) != 0 || - rlim.rlim_cur != kSufficientStack) - return; - } - - // We should now have a stack of size at least kSufficientStack. Ensure - // that we can actually use that much, if necessary. - ensureStackAddressSpace(); -} -#else -static void ensureSufficientStack() {} -#endif - -int cc1_main(ArrayRef Argv, const char *Argv0, void *MainAddr) { - ensureSufficientStack(); - - std::unique_ptr Clang(new CompilerInstance()); - IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); - - // Register the support for object-file-wrapped Clang modules. - auto PCHOps = Clang->getPCHContainerOperations(); - PCHOps->registerWriter(llvm::make_unique()); - PCHOps->registerReader(llvm::make_unique()); - - // Initialize targets first, so that --version shows registered targets. - llvm::InitializeAllTargets(); - llvm::InitializeAllTargetMCs(); - llvm::InitializeAllAsmPrinters(); - llvm::InitializeAllAsmParsers(); - -#ifdef LINK_POLLY_INTO_TOOLS - llvm::PassRegistry &Registry = *llvm::PassRegistry::getPassRegistry(); - polly::initializePollyPasses(Registry); -#endif - - // Buffer diagnostics from argument parsing so that we can output them using a - // well formed diagnostic object. - IntrusiveRefCntPtr DiagOpts = new DiagnosticOptions(); - TextDiagnosticBuffer *DiagsBuffer = new TextDiagnosticBuffer; - DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagsBuffer); - bool Success = CompilerInvocation::CreateFromArgs( - Clang->getInvocation(), Argv.begin(), Argv.end(), Diags); - - // Infer the builtin include path if unspecified. - if (Clang->getHeaderSearchOpts().UseBuiltinIncludes && - Clang->getHeaderSearchOpts().ResourceDir.empty()) - Clang->getHeaderSearchOpts().ResourceDir = - CompilerInvocation::GetResourcesPath(Argv0, MainAddr); - - // Create the actual diagnostics engine. - Clang->createDiagnostics(); - if (!Clang->hasDiagnostics()) - return 1; - - // Set an error handler, so that any LLVM backend diagnostics go through our - // error handler. - llvm::install_fatal_error_handler(LLVMErrorHandler, - static_cast(&Clang->getDiagnostics())); - - DiagsBuffer->FlushDiagnostics(Clang->getDiagnostics()); - if (!Success) - return 1; - - // Execute the frontend actions. - Success = ExecuteCompilerInvocation(Clang.get()); - - // If any timers were active but haven't been destroyed yet, print their - // results now. This happens in -disable-free mode. - llvm::TimerGroup::printAll(llvm::errs()); - - // Our error handler depends on the Diagnostics object, which we're - // potentially about to delete. Uninstall the handler now so that any - // later errors use the default handling behavior instead. - llvm::remove_fatal_error_handler(); - - // When running with -disable-free, don't do any destruction or shutdown. - if (Clang->getFrontendOpts().DisableFree) { - BuryPointer(std::move(Clang)); - return !Success; - } - - return !Success; -} diff --git a/dbms/programs/clang/Compiler-7.0.0bundled/cc1as_main.cpp b/dbms/programs/clang/Compiler-7.0.0bundled/cc1as_main.cpp deleted file mode 100644 index ce23422077f..00000000000 --- a/dbms/programs/clang/Compiler-7.0.0bundled/cc1as_main.cpp +++ /dev/null @@ -1,555 +0,0 @@ -//===-- cc1as_main.cpp - Clang Assembler ---------------------------------===// -// -// The LLVM Compiler Infrastructure -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This is the entry point to the clang -cc1as functionality, which implements -// the direct interface to the LLVM MC based assembler. -// -//===----------------------------------------------------------------------===// - -#include "clang/Basic/Diagnostic.h" -#include "clang/Basic/DiagnosticOptions.h" -#include "clang/Driver/DriverDiagnostic.h" -#include "clang/Driver/Options.h" -#include "clang/Frontend/FrontendDiagnostic.h" -#include "clang/Frontend/TextDiagnosticPrinter.h" -#include "clang/Frontend/Utils.h" -#include "llvm/ADT/STLExtras.h" -#include "llvm/ADT/StringSwitch.h" -#include "llvm/ADT/Triple.h" -#include "llvm/IR/DataLayout.h" -#include "llvm/MC/MCAsmBackend.h" -#include "llvm/MC/MCAsmInfo.h" -#include "llvm/MC/MCCodeEmitter.h" -#include "llvm/MC/MCContext.h" -#include "llvm/MC/MCInstrInfo.h" -#include "llvm/MC/MCObjectFileInfo.h" -#include "llvm/MC/MCParser/MCAsmParser.h" -#include "llvm/MC/MCParser/MCTargetAsmParser.h" -#include "llvm/MC/MCRegisterInfo.h" -#include "llvm/MC/MCStreamer.h" -#include "llvm/MC/MCSubtargetInfo.h" -#include "llvm/MC/MCTargetOptions.h" -#include "llvm/Option/Arg.h" -#include "llvm/Option/ArgList.h" -#include "llvm/Option/OptTable.h" -#include "llvm/Support/CommandLine.h" -#include "llvm/Support/ErrorHandling.h" -#include "llvm/Support/FileSystem.h" -#include "llvm/Support/FormattedStream.h" -#include "llvm/Support/Host.h" -#include "llvm/Support/MemoryBuffer.h" -#include "llvm/Support/Path.h" -#include "llvm/Support/Signals.h" -#include "llvm/Support/SourceMgr.h" -#include "llvm/Support/TargetRegistry.h" -#include "llvm/Support/TargetSelect.h" -#include "llvm/Support/Timer.h" -#include "llvm/Support/raw_ostream.h" -#include -#include -using namespace clang; -using namespace clang::driver; -using namespace clang::driver::options; -using namespace llvm; -using namespace llvm::opt; - - -namespace { - -/// \brief Helper class for representing a single invocation of the assembler. -struct AssemblerInvocation { - /// @name Target Options - /// @{ - - /// The name of the target triple to assemble for. - std::string Triple; - - /// If given, the name of the target CPU to determine which instructions - /// are legal. - std::string CPU; - - /// The list of target specific features to enable or disable -- this should - /// be a list of strings starting with '+' or '-'. - std::vector Features; - - /// The list of symbol definitions. - std::vector SymbolDefs; - - /// @} - /// @name Language Options - /// @{ - - std::vector IncludePaths; - unsigned NoInitialTextSection : 1; - unsigned SaveTemporaryLabels : 1; - unsigned GenDwarfForAssembly : 1; - unsigned RelaxELFRelocations : 1; - unsigned DwarfVersion; - std::string DwarfDebugFlags; - std::string DwarfDebugProducer; - std::string DebugCompilationDir; - llvm::DebugCompressionType CompressDebugSections = - llvm::DebugCompressionType::None; - std::string MainFileName; - - /// @} - /// @name Frontend Options - /// @{ - - std::string InputFile; - std::vector LLVMArgs; - std::string OutputPath; - enum FileType { - FT_Asm, ///< Assembly (.s) output, transliterate mode. - FT_Null, ///< No output, for timing purposes. - FT_Obj ///< Object file output. - }; - FileType OutputType; - unsigned ShowHelp : 1; - unsigned ShowVersion : 1; - - /// @} - /// @name Transliterate Options - /// @{ - - unsigned OutputAsmVariant; - unsigned ShowEncoding : 1; - unsigned ShowInst : 1; - - /// @} - /// @name Assembler Options - /// @{ - - unsigned RelaxAll : 1; - unsigned NoExecStack : 1; - unsigned FatalWarnings : 1; - unsigned IncrementalLinkerCompatible : 1; - - /// The name of the relocation model to use. - std::string RelocationModel; - - /// @} - -public: - AssemblerInvocation() { - Triple = ""; - NoInitialTextSection = 0; - InputFile = "-"; - OutputPath = "-"; - OutputType = FT_Asm; - OutputAsmVariant = 0; - ShowInst = 0; - ShowEncoding = 0; - RelaxAll = 0; - NoExecStack = 0; - FatalWarnings = 0; - IncrementalLinkerCompatible = 0; - DwarfVersion = 0; - } - - static bool CreateFromArgs(AssemblerInvocation &Res, - ArrayRef Argv, - DiagnosticsEngine &Diags); -}; - -} - -bool AssemblerInvocation::CreateFromArgs(AssemblerInvocation &Opts, - ArrayRef Argv, - DiagnosticsEngine &Diags) { - bool Success = true; - - // Parse the arguments. - std::unique_ptr OptTbl(createDriverOptTable()); - - const unsigned IncludedFlagsBitmask = options::CC1AsOption; - unsigned MissingArgIndex, MissingArgCount; - InputArgList Args = OptTbl->ParseArgs(Argv, MissingArgIndex, MissingArgCount, - IncludedFlagsBitmask); - - // Check for missing argument error. - if (MissingArgCount) { - Diags.Report(diag::err_drv_missing_argument) - << Args.getArgString(MissingArgIndex) << MissingArgCount; - Success = false; - } - - // Issue errors on unknown arguments. - for (const Arg *A : Args.filtered(OPT_UNKNOWN)) { - auto ArgString = A->getAsString(Args); - std::string Nearest; - if (OptTbl->findNearest(ArgString, Nearest, IncludedFlagsBitmask) > 1) - Diags.Report(diag::err_drv_unknown_argument) << ArgString; - else - Diags.Report(diag::err_drv_unknown_argument_with_suggestion) - << ArgString << Nearest; - Success = false; - } - - // Construct the invocation. - - // Target Options - Opts.Triple = llvm::Triple::normalize(Args.getLastArgValue(OPT_triple)); - Opts.CPU = Args.getLastArgValue(OPT_target_cpu); - Opts.Features = Args.getAllArgValues(OPT_target_feature); - - // Use the default target triple if unspecified. - if (Opts.Triple.empty()) - Opts.Triple = llvm::sys::getDefaultTargetTriple(); - - // Language Options - Opts.IncludePaths = Args.getAllArgValues(OPT_I); - Opts.NoInitialTextSection = Args.hasArg(OPT_n); - Opts.SaveTemporaryLabels = Args.hasArg(OPT_msave_temp_labels); - // Any DebugInfoKind implies GenDwarfForAssembly. - Opts.GenDwarfForAssembly = Args.hasArg(OPT_debug_info_kind_EQ); - - if (const Arg *A = Args.getLastArg(OPT_compress_debug_sections, - OPT_compress_debug_sections_EQ)) { - if (A->getOption().getID() == OPT_compress_debug_sections) { - // TODO: be more clever about the compression type auto-detection - Opts.CompressDebugSections = llvm::DebugCompressionType::GNU; - } else { - Opts.CompressDebugSections = - llvm::StringSwitch(A->getValue()) - .Case("none", llvm::DebugCompressionType::None) - .Case("zlib", llvm::DebugCompressionType::Z) - .Case("zlib-gnu", llvm::DebugCompressionType::GNU) - .Default(llvm::DebugCompressionType::None); - } - } - - Opts.RelaxELFRelocations = Args.hasArg(OPT_mrelax_relocations); - Opts.DwarfVersion = getLastArgIntValue(Args, OPT_dwarf_version_EQ, 2, Diags); - Opts.DwarfDebugFlags = Args.getLastArgValue(OPT_dwarf_debug_flags); - Opts.DwarfDebugProducer = Args.getLastArgValue(OPT_dwarf_debug_producer); - Opts.DebugCompilationDir = Args.getLastArgValue(OPT_fdebug_compilation_dir); - Opts.MainFileName = Args.getLastArgValue(OPT_main_file_name); - - // Frontend Options - if (Args.hasArg(OPT_INPUT)) { - bool First = true; - for (const Arg *A : Args.filtered(OPT_INPUT)) { - if (First) { - Opts.InputFile = A->getValue(); - First = false; - } else { - Diags.Report(diag::err_drv_unknown_argument) << A->getAsString(Args); - Success = false; - } - } - } - Opts.LLVMArgs = Args.getAllArgValues(OPT_mllvm); - Opts.OutputPath = Args.getLastArgValue(OPT_o); - if (Arg *A = Args.getLastArg(OPT_filetype)) { - StringRef Name = A->getValue(); - unsigned OutputType = StringSwitch(Name) - .Case("asm", FT_Asm) - .Case("null", FT_Null) - .Case("obj", FT_Obj) - .Default(~0U); - if (OutputType == ~0U) { - Diags.Report(diag::err_drv_invalid_value) << A->getAsString(Args) << Name; - Success = false; - } else - Opts.OutputType = FileType(OutputType); - } - Opts.ShowHelp = Args.hasArg(OPT_help); - Opts.ShowVersion = Args.hasArg(OPT_version); - - // Transliterate Options - Opts.OutputAsmVariant = - getLastArgIntValue(Args, OPT_output_asm_variant, 0, Diags); - Opts.ShowEncoding = Args.hasArg(OPT_show_encoding); - Opts.ShowInst = Args.hasArg(OPT_show_inst); - - // Assemble Options - Opts.RelaxAll = Args.hasArg(OPT_mrelax_all); - Opts.NoExecStack = Args.hasArg(OPT_mno_exec_stack); - Opts.FatalWarnings = Args.hasArg(OPT_massembler_fatal_warnings); - Opts.RelocationModel = Args.getLastArgValue(OPT_mrelocation_model, "pic"); - Opts.IncrementalLinkerCompatible = - Args.hasArg(OPT_mincremental_linker_compatible); - Opts.SymbolDefs = Args.getAllArgValues(OPT_defsym); - - return Success; -} - -static std::unique_ptr -getOutputStream(AssemblerInvocation &Opts, DiagnosticsEngine &Diags, - bool Binary) { - if (Opts.OutputPath.empty()) - Opts.OutputPath = "-"; - - // Make sure that the Out file gets unlinked from the disk if we get a - // SIGINT. - if (Opts.OutputPath != "-") - sys::RemoveFileOnSignal(Opts.OutputPath); - - std::error_code EC; - auto Out = llvm::make_unique( - Opts.OutputPath, EC, (Binary ? sys::fs::F_None : sys::fs::F_Text)); - if (EC) { - Diags.Report(diag::err_fe_unable_to_open_output) << Opts.OutputPath - << EC.message(); - return nullptr; - } - - return Out; -} - -static bool ExecuteAssembler(AssemblerInvocation &Opts, - DiagnosticsEngine &Diags) { - // Get the target specific parser. - std::string Error; - const Target *TheTarget = TargetRegistry::lookupTarget(Opts.Triple, Error); - if (!TheTarget) - return Diags.Report(diag::err_target_unknown_triple) << Opts.Triple; - - ErrorOr> Buffer = - MemoryBuffer::getFileOrSTDIN(Opts.InputFile); - - if (std::error_code EC = Buffer.getError()) { - Error = EC.message(); - return Diags.Report(diag::err_fe_error_reading) << Opts.InputFile; - } - - SourceMgr SrcMgr; - - // Tell SrcMgr about this buffer, which is what the parser will pick up. - SrcMgr.AddNewSourceBuffer(std::move(*Buffer), SMLoc()); - - // Record the location of the include directories so that the lexer can find - // it later. - SrcMgr.setIncludeDirs(Opts.IncludePaths); - - std::unique_ptr MRI(TheTarget->createMCRegInfo(Opts.Triple)); - assert(MRI && "Unable to create target register info!"); - - std::unique_ptr MAI(TheTarget->createMCAsmInfo(*MRI, Opts.Triple)); - assert(MAI && "Unable to create target asm info!"); - - // Ensure MCAsmInfo initialization occurs before any use, otherwise sections - // may be created with a combination of default and explicit settings. - MAI->setCompressDebugSections(Opts.CompressDebugSections); - - MAI->setRelaxELFRelocations(Opts.RelaxELFRelocations); - - bool IsBinary = Opts.OutputType == AssemblerInvocation::FT_Obj; - std::unique_ptr FDOS = getOutputStream(Opts, Diags, IsBinary); - if (!FDOS) - return true; - - // FIXME: This is not pretty. MCContext has a ptr to MCObjectFileInfo and - // MCObjectFileInfo needs a MCContext reference in order to initialize itself. - std::unique_ptr MOFI(new MCObjectFileInfo()); - - MCContext Ctx(MAI.get(), MRI.get(), MOFI.get(), &SrcMgr); - - bool PIC = false; - if (Opts.RelocationModel == "static") { - PIC = false; - } else if (Opts.RelocationModel == "pic") { - PIC = true; - } else { - assert(Opts.RelocationModel == "dynamic-no-pic" && - "Invalid PIC model!"); - PIC = false; - } - - MOFI->InitMCObjectFileInfo(Triple(Opts.Triple), PIC, Ctx); - if (Opts.SaveTemporaryLabels) - Ctx.setAllowTemporaryLabels(false); - if (Opts.GenDwarfForAssembly) - Ctx.setGenDwarfForAssembly(true); - if (!Opts.DwarfDebugFlags.empty()) - Ctx.setDwarfDebugFlags(StringRef(Opts.DwarfDebugFlags)); - if (!Opts.DwarfDebugProducer.empty()) - Ctx.setDwarfDebugProducer(StringRef(Opts.DwarfDebugProducer)); - if (!Opts.DebugCompilationDir.empty()) - Ctx.setCompilationDir(Opts.DebugCompilationDir); - if (!Opts.MainFileName.empty()) - Ctx.setMainFileName(StringRef(Opts.MainFileName)); - Ctx.setDwarfVersion(Opts.DwarfVersion); - - // Build up the feature string from the target feature list. - std::string FS; - if (!Opts.Features.empty()) { - FS = Opts.Features[0]; - for (unsigned i = 1, e = Opts.Features.size(); i != e; ++i) - FS += "," + Opts.Features[i]; - } - - std::unique_ptr Str; - - std::unique_ptr MCII(TheTarget->createMCInstrInfo()); - std::unique_ptr STI( - TheTarget->createMCSubtargetInfo(Opts.Triple, Opts.CPU, FS)); - - raw_pwrite_stream *Out = FDOS.get(); - std::unique_ptr BOS; - - // FIXME: There is a bit of code duplication with addPassesToEmitFile. - if (Opts.OutputType == AssemblerInvocation::FT_Asm) { - MCInstPrinter *IP = TheTarget->createMCInstPrinter( - llvm::Triple(Opts.Triple), Opts.OutputAsmVariant, *MAI, *MCII, *MRI); - - std::unique_ptr CE; - if (Opts.ShowEncoding) - CE.reset(TheTarget->createMCCodeEmitter(*MCII, *MRI, Ctx)); - MCTargetOptions MCOptions; - std::unique_ptr MAB( - TheTarget->createMCAsmBackend(*STI, *MRI, MCOptions)); - - auto FOut = llvm::make_unique(*Out); - Str.reset(TheTarget->createAsmStreamer( - Ctx, std::move(FOut), /*asmverbose*/ true, - /*useDwarfDirectory*/ true, IP, std::move(CE), std::move(MAB), - Opts.ShowInst)); - } else if (Opts.OutputType == AssemblerInvocation::FT_Null) { - Str.reset(createNullStreamer(Ctx)); - } else { - assert(Opts.OutputType == AssemblerInvocation::FT_Obj && - "Invalid file type!"); - if (!FDOS->supportsSeeking()) { - BOS = make_unique(*FDOS); - Out = BOS.get(); - } - - std::unique_ptr CE( - TheTarget->createMCCodeEmitter(*MCII, *MRI, Ctx)); - MCTargetOptions MCOptions; - std::unique_ptr MAB( - TheTarget->createMCAsmBackend(*STI, *MRI, MCOptions)); - - Triple T(Opts.Triple); - Str.reset(TheTarget->createMCObjectStreamer( - T, Ctx, std::move(MAB), *Out, std::move(CE), *STI, Opts.RelaxAll, - Opts.IncrementalLinkerCompatible, - /*DWARFMustBeAtTheEnd*/ true)); - Str.get()->InitSections(Opts.NoExecStack); - } - - // Assembly to object compilation should leverage assembly info. - Str->setUseAssemblerInfoForParsing(true); - - bool Failed = false; - - std::unique_ptr Parser( - createMCAsmParser(SrcMgr, Ctx, *Str.get(), *MAI)); - - // FIXME: init MCTargetOptions from sanitizer flags here. - MCTargetOptions Options; - std::unique_ptr TAP( - TheTarget->createMCAsmParser(*STI, *Parser, *MCII, Options)); - if (!TAP) - Failed = Diags.Report(diag::err_target_unknown_triple) << Opts.Triple; - - // Set values for symbols, if any. - for (auto &S : Opts.SymbolDefs) { - auto Pair = StringRef(S).split('='); - auto Sym = Pair.first; - auto Val = Pair.second; - int64_t Value; - // We have already error checked this in the driver. - Val.getAsInteger(0, Value); - Ctx.setSymbolValue(Parser->getStreamer(), Sym, Value); - } - - if (!Failed) { - Parser->setTargetParser(*TAP.get()); - Failed = Parser->Run(Opts.NoInitialTextSection); - } - - // Close Streamer first. - // It might have a reference to the output stream. - Str.reset(); - // Close the output stream early. - BOS.reset(); - FDOS.reset(); - - // Delete output file if there were errors. - if (Failed && Opts.OutputPath != "-") - sys::fs::remove(Opts.OutputPath); - - return Failed; -} - -static void LLVMErrorHandler(void *UserData, const std::string &Message, - bool /*GenCrashDiag*/) { - DiagnosticsEngine &Diags = *static_cast(UserData); - - Diags.Report(diag::err_fe_error_backend) << Message; - - // We cannot recover from llvm errors. - exit(1); -} - -int cc1as_main(ArrayRef Argv, const char */*Argv0*/, void */*MainAddr*/) { - // Initialize targets and assembly printers/parsers. - InitializeAllTargetInfos(); - InitializeAllTargetMCs(); - InitializeAllAsmParsers(); - - // Construct our diagnostic client. - IntrusiveRefCntPtr DiagOpts = new DiagnosticOptions(); - TextDiagnosticPrinter *DiagClient - = new TextDiagnosticPrinter(errs(), &*DiagOpts); - DiagClient->setPrefix("clang -cc1as"); - IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); - DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagClient); - - // Set an error handler, so that any LLVM backend diagnostics go through our - // error handler. - ScopedFatalErrorHandler FatalErrorHandler - (LLVMErrorHandler, static_cast(&Diags)); - - // Parse the arguments. - AssemblerInvocation Asm; - if (!AssemblerInvocation::CreateFromArgs(Asm, Argv, Diags)) - return 1; - - if (Asm.ShowHelp) { - std::unique_ptr Opts(driver::createDriverOptTable()); - Opts->PrintHelp(llvm::outs(), "clang -cc1as", "Clang Integrated Assembler", - /*Include=*/driver::options::CC1AsOption, /*Exclude=*/0, - /*ShowAllAliases=*/false); - return 0; - } - - // Honor -version. - // - // FIXME: Use a better -version message? - if (Asm.ShowVersion) { - llvm::cl::PrintVersionMessage(); - return 0; - } - - // Honor -mllvm. - // - // FIXME: Remove this, one day. - if (!Asm.LLVMArgs.empty()) { - unsigned NumArgs = Asm.LLVMArgs.size(); - auto Args = llvm::make_unique(NumArgs + 2); - Args[0] = "clang (LLVM option parsing)"; - for (unsigned i = 0; i != NumArgs; ++i) - Args[i + 1] = Asm.LLVMArgs[i].c_str(); - Args[NumArgs + 1] = nullptr; - llvm::cl::ParseCommandLineOptions(NumArgs + 1, Args.get()); - } - - // Execute the invocation, unless there were parsing errors. - bool Failed = Diags.hasErrorOccurred() || ExecuteAssembler(Asm, Diags); - - // If any timers were active but haven't been destroyed yet, print their - // results now. - TimerGroup::printAll(errs()); - - return !!Failed; -} diff --git a/dbms/programs/clang/Compiler-7.0.0bundled/driver.cpp b/dbms/programs/clang/Compiler-7.0.0bundled/driver.cpp deleted file mode 100644 index 9a061b9d137..00000000000 --- a/dbms/programs/clang/Compiler-7.0.0bundled/driver.cpp +++ /dev/null @@ -1,512 +0,0 @@ -//===-- driver.cpp - Clang GCC-Compatible Driver --------------------------===// -// -// The LLVM Compiler Infrastructure -// -// This file is distributed under the University of Illinois Open Source -// License. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// -// -// This is the entry point to the clang driver; it is a thin wrapper -// for functionality in the Driver clang library. -// -//===----------------------------------------------------------------------===// - -#include "clang/Driver/Driver.h" -#include "clang/Basic/DiagnosticOptions.h" -#include "clang/Driver/Compilation.h" -#include "clang/Driver/DriverDiagnostic.h" -#include "clang/Driver/Options.h" -#include "clang/Driver/ToolChain.h" -#include "clang/Frontend/ChainedDiagnosticConsumer.h" -#include "clang/Frontend/CompilerInvocation.h" -#include "clang/Frontend/SerializedDiagnosticPrinter.h" -#include "clang/Frontend/TextDiagnosticPrinter.h" -#include "clang/Frontend/Utils.h" -#include "llvm/ADT/ArrayRef.h" -#include "llvm/ADT/SmallString.h" -#include "llvm/ADT/SmallVector.h" -#include "llvm/Option/ArgList.h" -#include "llvm/Option/OptTable.h" -#include "llvm/Option/Option.h" -#include "llvm/Support/CommandLine.h" -#include "llvm/Support/ErrorHandling.h" -#include "llvm/Support/FileSystem.h" -#include "llvm/Support/Host.h" -#include "llvm/Support/InitLLVM.h" -#include "llvm/Support/Path.h" -#include "llvm/Support/Process.h" -#include "llvm/Support/Program.h" -#include "llvm/Support/Regex.h" -#include "llvm/Support/Signals.h" -#include "llvm/Support/StringSaver.h" -#include "llvm/Support/TargetSelect.h" -#include "llvm/Support/Timer.h" -#include "llvm/Support/raw_ostream.h" -#include -#include -#include -using namespace clang; -using namespace clang::driver; -using namespace llvm::opt; - -std::string GetExecutablePath(const char *Argv0, bool CanonicalPrefixes) { - if (!CanonicalPrefixes) { - SmallString<128> ExecutablePath(Argv0); - // Do a PATH lookup if Argv0 isn't a valid path. - if (!llvm::sys::fs::exists(ExecutablePath)) - if (llvm::ErrorOr P = - llvm::sys::findProgramByName(ExecutablePath)) - ExecutablePath = *P; - return ExecutablePath.str(); - } - - // This just needs to be some symbol in the binary; C++ doesn't - // allow taking the address of ::main however. - void *P = (void*) (intptr_t) GetExecutablePath; - return llvm::sys::fs::getMainExecutable(Argv0, P); -} - -static const char *GetStableCStr(std::set &SavedStrings, - StringRef S) { - return SavedStrings.insert(S).first->c_str(); -} - -/// ApplyQAOverride - Apply a list of edits to the input argument lists. -/// -/// The input string is a space separate list of edits to perform, -/// they are applied in order to the input argument lists. Edits -/// should be one of the following forms: -/// -/// '#': Silence information about the changes to the command line arguments. -/// -/// '^': Add FOO as a new argument at the beginning of the command line. -/// -/// '+': Add FOO as a new argument at the end of the command line. -/// -/// 's/XXX/YYY/': Substitute the regular expression XXX with YYY in the command -/// line. -/// -/// 'xOPTION': Removes all instances of the literal argument OPTION. -/// -/// 'XOPTION': Removes all instances of the literal argument OPTION, -/// and the following argument. -/// -/// 'Ox': Removes all flags matching 'O' or 'O[sz0-9]' and adds 'Ox' -/// at the end of the command line. -/// -/// \param OS - The stream to write edit information to. -/// \param Args - The vector of command line arguments. -/// \param Edit - The override command to perform. -/// \param SavedStrings - Set to use for storing string representations. -static void ApplyOneQAOverride(raw_ostream &OS, - SmallVectorImpl &Args, - StringRef Edit, - std::set &SavedStrings) { - // This does not need to be efficient. - - if (Edit[0] == '^') { - const char *Str = - GetStableCStr(SavedStrings, Edit.substr(1)); - OS << "### Adding argument " << Str << " at beginning\n"; - Args.insert(Args.begin() + 1, Str); - } else if (Edit[0] == '+') { - const char *Str = - GetStableCStr(SavedStrings, Edit.substr(1)); - OS << "### Adding argument " << Str << " at end\n"; - Args.push_back(Str); - } else if (Edit[0] == 's' && Edit[1] == '/' && Edit.endswith("/") && - Edit.slice(2, Edit.size()-1).find('/') != StringRef::npos) { - StringRef MatchPattern = Edit.substr(2).split('/').first; - StringRef ReplPattern = Edit.substr(2).split('/').second; - ReplPattern = ReplPattern.slice(0, ReplPattern.size()-1); - - for (unsigned i = 1, e = Args.size(); i != e; ++i) { - // Ignore end-of-line response file markers - if (Args[i] == nullptr) - continue; - std::string Repl = llvm::Regex(MatchPattern).sub(ReplPattern, Args[i]); - - if (Repl != Args[i]) { - OS << "### Replacing '" << Args[i] << "' with '" << Repl << "'\n"; - Args[i] = GetStableCStr(SavedStrings, Repl); - } - } - } else if (Edit[0] == 'x' || Edit[0] == 'X') { - auto Option = Edit.substr(1); - for (unsigned i = 1; i < Args.size();) { - if (Option == Args[i]) { - OS << "### Deleting argument " << Args[i] << '\n'; - Args.erase(Args.begin() + i); - if (Edit[0] == 'X') { - if (i < Args.size()) { - OS << "### Deleting argument " << Args[i] << '\n'; - Args.erase(Args.begin() + i); - } else - OS << "### Invalid X edit, end of command line!\n"; - } - } else - ++i; - } - } else if (Edit[0] == 'O') { - for (unsigned i = 1; i < Args.size();) { - const char *A = Args[i]; - // Ignore end-of-line response file markers - if (A == nullptr) - continue; - if (A[0] == '-' && A[1] == 'O' && - (A[2] == '\0' || - (A[3] == '\0' && (A[2] == 's' || A[2] == 'z' || - ('0' <= A[2] && A[2] <= '9'))))) { - OS << "### Deleting argument " << Args[i] << '\n'; - Args.erase(Args.begin() + i); - } else - ++i; - } - OS << "### Adding argument " << Edit << " at end\n"; - Args.push_back(GetStableCStr(SavedStrings, '-' + Edit.str())); - } else { - OS << "### Unrecognized edit: " << Edit << "\n"; - } -} - -/// ApplyQAOverride - Apply a comma separate list of edits to the -/// input argument lists. See ApplyOneQAOverride. -static void ApplyQAOverride(SmallVectorImpl &Args, - const char *OverrideStr, - std::set &SavedStrings) { - raw_ostream *OS = &llvm::errs(); - - if (OverrideStr[0] == '#') { - ++OverrideStr; - OS = &llvm::nulls(); - } - - *OS << "### CCC_OVERRIDE_OPTIONS: " << OverrideStr << "\n"; - - // This does not need to be efficient. - - const char *S = OverrideStr; - while (*S) { - const char *End = ::strchr(S, ' '); - if (!End) - End = S + strlen(S); - if (End != S) - ApplyOneQAOverride(*OS, Args, std::string(S, End), SavedStrings); - S = End; - if (*S != '\0') - ++S; - } -} - -extern int cc1_main(ArrayRef Argv, const char *Argv0, - void *MainAddr); -extern int cc1as_main(ArrayRef Argv, const char *Argv0, - void *MainAddr); -extern int cc1gen_reproducer_main(ArrayRef Argv, - const char *Argv0, void *MainAddr); - -static void insertTargetAndModeArgs(const ParsedClangName &NameParts, - SmallVectorImpl &ArgVector, - std::set &SavedStrings) { - // Put target and mode arguments at the start of argument list so that - // arguments specified in command line could override them. Avoid putting - // them at index 0, as an option like '-cc1' must remain the first. - int InsertionPoint = 0; - if (ArgVector.size() > 0) - ++InsertionPoint; - - if (NameParts.DriverMode) { - // Add the mode flag to the arguments. - ArgVector.insert(ArgVector.begin() + InsertionPoint, - GetStableCStr(SavedStrings, NameParts.DriverMode)); - } - - if (NameParts.TargetIsValid) { - const char *arr[] = {"-target", GetStableCStr(SavedStrings, - NameParts.TargetPrefix)}; - ArgVector.insert(ArgVector.begin() + InsertionPoint, - std::begin(arr), std::end(arr)); - } -} - -static void getCLEnvVarOptions(std::string &EnvValue, llvm::StringSaver &Saver, - SmallVectorImpl &Opts) { - llvm::cl::TokenizeWindowsCommandLine(EnvValue, Saver, Opts); - // The first instance of '#' should be replaced with '=' in each option. - for (const char *Opt : Opts) - if (char *NumberSignPtr = const_cast(::strchr(Opt, '#'))) - *NumberSignPtr = '='; -} - -static void SetBackdoorDriverOutputsFromEnvVars(Driver &TheDriver) { - // Handle CC_PRINT_OPTIONS and CC_PRINT_OPTIONS_FILE. - TheDriver.CCPrintOptions = !!::getenv("CC_PRINT_OPTIONS"); - if (TheDriver.CCPrintOptions) - TheDriver.CCPrintOptionsFilename = ::getenv("CC_PRINT_OPTIONS_FILE"); - - // Handle CC_PRINT_HEADERS and CC_PRINT_HEADERS_FILE. - TheDriver.CCPrintHeaders = !!::getenv("CC_PRINT_HEADERS"); - if (TheDriver.CCPrintHeaders) - TheDriver.CCPrintHeadersFilename = ::getenv("CC_PRINT_HEADERS_FILE"); - - // Handle CC_LOG_DIAGNOSTICS and CC_LOG_DIAGNOSTICS_FILE. - TheDriver.CCLogDiagnostics = !!::getenv("CC_LOG_DIAGNOSTICS"); - if (TheDriver.CCLogDiagnostics) - TheDriver.CCLogDiagnosticsFilename = ::getenv("CC_LOG_DIAGNOSTICS_FILE"); -} - -static void FixupDiagPrefixExeName(TextDiagnosticPrinter *DiagClient, - const std::string &Path) { - // If the clang binary happens to be named cl.exe for compatibility reasons, - // use clang-cl.exe as the prefix to avoid confusion between clang and MSVC. - StringRef ExeBasename(llvm::sys::path::filename(Path)); - if (ExeBasename.equals_lower("cl.exe")) - ExeBasename = "clang-cl.exe"; - DiagClient->setPrefix(ExeBasename); -} - -// This lets us create the DiagnosticsEngine with a properly-filled-out -// DiagnosticOptions instance. -static DiagnosticOptions * -CreateAndPopulateDiagOpts(ArrayRef argv) { - auto *DiagOpts = new DiagnosticOptions; - std::unique_ptr Opts(createDriverOptTable()); - unsigned MissingArgIndex, MissingArgCount; - InputArgList Args = - Opts->ParseArgs(argv.slice(1), MissingArgIndex, MissingArgCount); - // We ignore MissingArgCount and the return value of ParseDiagnosticArgs. - // Any errors that would be diagnosed here will also be diagnosed later, - // when the DiagnosticsEngine actually exists. - (void)ParseDiagnosticArgs(*DiagOpts, Args); - return DiagOpts; -} - -static void SetInstallDir(SmallVectorImpl &argv, - Driver &TheDriver, bool CanonicalPrefixes) { - // Attempt to find the original path used to invoke the driver, to determine - // the installed path. We do this manually, because we want to support that - // path being a symlink. - SmallString<128> InstalledPath(argv[0]); - - // Do a PATH lookup, if there are no directory components. - if (llvm::sys::path::filename(InstalledPath) == InstalledPath) - if (llvm::ErrorOr Tmp = llvm::sys::findProgramByName( - llvm::sys::path::filename(InstalledPath.str()))) - InstalledPath = *Tmp; - - // FIXME: We don't actually canonicalize this, we just make it absolute. - if (CanonicalPrefixes) - llvm::sys::fs::make_absolute(InstalledPath); - - StringRef InstalledPathParent(llvm::sys::path::parent_path(InstalledPath)); - if (llvm::sys::fs::exists(InstalledPathParent)) - TheDriver.setInstalledDir(InstalledPathParent); -} - -static int ExecuteCC1Tool(ArrayRef argv, StringRef Tool) { - void *GetExecutablePathVP = (void *)(intptr_t) GetExecutablePath; - if (Tool == "") - return cc1_main(argv.slice(2), argv[0], GetExecutablePathVP); - if (Tool == "as") - return cc1as_main(argv.slice(2), argv[0], GetExecutablePathVP); - - // Reject unknown tools. - llvm::errs() << "error: unknown integrated tool '" << Tool << "'. " - << "Valid tools include '-cc1' and '-cc1as'.\n"; - return 1; -} - -int mainEntryClickHouseClang(int argc_, char **argv_) { - llvm::InitLLVM X(argc_, argv_); - SmallVector argv(argv_, argv_ + argc_); - - if (llvm::sys::Process::FixupStandardFileDescriptors()) - return 1; - - llvm::InitializeAllTargets(); - auto TargetAndMode = ToolChain::getTargetAndModeFromProgramName(argv[0]); - - llvm::BumpPtrAllocator A; - llvm::StringSaver Saver(A); - - // Parse response files using the GNU syntax, unless we're in CL mode. There - // are two ways to put clang in CL compatibility mode: argv[0] is either - // clang-cl or cl, or --driver-mode=cl is on the command line. The normal - // command line parsing can't happen until after response file parsing, so we - // have to manually search for a --driver-mode=cl argument the hard way. - // Finally, our -cc1 tools don't care which tokenization mode we use because - // response files written by clang will tokenize the same way in either mode. - bool ClangCLMode = false; - if (StringRef(TargetAndMode.DriverMode).equals("--driver-mode=cl") || - std::find_if(argv.begin(), argv.end(), [](const char *F) { - return F && strcmp(F, "--driver-mode=cl") == 0; - }) != argv.end()) { - ClangCLMode = true; - } - enum { Default, POSIX, Windows } RSPQuoting = Default; - for (const char *F : argv) { - if (strcmp(F, "--rsp-quoting=posix") == 0) - RSPQuoting = POSIX; - else if (strcmp(F, "--rsp-quoting=windows") == 0) - RSPQuoting = Windows; - } - - // Determines whether we want nullptr markers in argv to indicate response - // files end-of-lines. We only use this for the /LINK driver argument with - // clang-cl.exe on Windows. - bool MarkEOLs = ClangCLMode; - - llvm::cl::TokenizerCallback Tokenizer; - if (RSPQuoting == Windows || (RSPQuoting == Default && ClangCLMode)) - Tokenizer = &llvm::cl::TokenizeWindowsCommandLine; - else - Tokenizer = &llvm::cl::TokenizeGNUCommandLine; - - if (MarkEOLs && argv.size() > 1 && StringRef(argv[1]).startswith("-cc1")) - MarkEOLs = false; - llvm::cl::ExpandResponseFiles(Saver, Tokenizer, argv, MarkEOLs); - - // Handle -cc1 integrated tools, even if -cc1 was expanded from a response - // file. - auto FirstArg = std::find_if(argv.begin() + 1, argv.end(), - [](const char *A) { return A != nullptr; }); - if (FirstArg != argv.end() && StringRef(*FirstArg).startswith("-cc1")) { - // If -cc1 came from a response file, remove the EOL sentinels. - if (MarkEOLs) { - auto newEnd = std::remove(argv.begin(), argv.end(), nullptr); - argv.resize(newEnd - argv.begin()); - } - return ExecuteCC1Tool(argv, argv[1] + 4); - } - - bool CanonicalPrefixes = true; - for (int i = 1, size = argv.size(); i < size; ++i) { - // Skip end-of-line response file markers - if (argv[i] == nullptr) - continue; - if (StringRef(argv[i]) == "-no-canonical-prefixes") { - CanonicalPrefixes = false; - break; - } - } - - // Handle CL and _CL_ which permits additional command line options to be - // prepended or appended. - if (ClangCLMode) { - // Arguments in "CL" are prepended. - llvm::Optional OptCL = llvm::sys::Process::GetEnv("CL"); - if (OptCL.hasValue()) { - SmallVector PrependedOpts; - getCLEnvVarOptions(OptCL.getValue(), Saver, PrependedOpts); - - // Insert right after the program name to prepend to the argument list. - argv.insert(argv.begin() + 1, PrependedOpts.begin(), PrependedOpts.end()); - } - // Arguments in "_CL_" are appended. - llvm::Optional Opt_CL_ = llvm::sys::Process::GetEnv("_CL_"); - if (Opt_CL_.hasValue()) { - SmallVector AppendedOpts; - getCLEnvVarOptions(Opt_CL_.getValue(), Saver, AppendedOpts); - - // Insert at the end of the argument list to append. - argv.append(AppendedOpts.begin(), AppendedOpts.end()); - } - } - - std::set SavedStrings; - // Handle CCC_OVERRIDE_OPTIONS, used for editing a command line behind the - // scenes. - if (const char *OverrideStr = ::getenv("CCC_OVERRIDE_OPTIONS")) { - // FIXME: Driver shouldn't take extra initial argument. - ApplyQAOverride(argv, OverrideStr, SavedStrings); - } - - std::string Path = GetExecutablePath(argv[0], CanonicalPrefixes); - - IntrusiveRefCntPtr DiagOpts = - CreateAndPopulateDiagOpts(argv); - - TextDiagnosticPrinter *DiagClient - = new TextDiagnosticPrinter(llvm::errs(), &*DiagOpts); - FixupDiagPrefixExeName(DiagClient, Path); - - IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); - - DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagClient); - - if (!DiagOpts->DiagnosticSerializationFile.empty()) { - auto SerializedConsumer = - clang::serialized_diags::create(DiagOpts->DiagnosticSerializationFile, - &*DiagOpts, /*MergeChildRecords=*/true); - Diags.setClient(new ChainedDiagnosticConsumer( - Diags.takeClient(), std::move(SerializedConsumer))); - } - - ProcessWarningOptions(Diags, *DiagOpts, /*ReportDiags=*/false); - - Driver TheDriver(Path, llvm::sys::getDefaultTargetTriple(), Diags); - SetInstallDir(argv, TheDriver, CanonicalPrefixes); - TheDriver.setTargetAndMode(TargetAndMode); - - insertTargetAndModeArgs(TargetAndMode, argv, SavedStrings); - - SetBackdoorDriverOutputsFromEnvVars(TheDriver); - - std::unique_ptr C(TheDriver.BuildCompilation(argv)); - int Res = 1; - if (C && !C->containsError()) { - SmallVector, 4> FailingCommands; - Res = TheDriver.ExecuteCompilation(*C, FailingCommands); - - // Force a crash to test the diagnostics. - if (TheDriver.GenReproducer) { - Diags.Report(diag::err_drv_force_crash) - << !::getenv("FORCE_CLANG_DIAGNOSTICS_CRASH"); - - // Pretend that every command failed. - FailingCommands.clear(); - for (const auto &J : C->getJobs()) - if (const Command *C = dyn_cast(&J)) - FailingCommands.push_back(std::make_pair(-1, C)); - } - - for (const auto &P : FailingCommands) { - int CommandRes = P.first; - const Command *FailingCommand = P.second; - if (!Res) - Res = CommandRes; - - // If result status is < 0, then the driver command signalled an error. - // If result status is 70, then the driver command reported a fatal error. - // On Windows, abort will return an exit code of 3. In these cases, - // generate additional diagnostic information if possible. - bool DiagnoseCrash = CommandRes < 0 || CommandRes == 70; -#ifdef _WIN32 - DiagnoseCrash |= CommandRes == 3; -#endif - if (DiagnoseCrash) { - TheDriver.generateCompilationDiagnostics(*C, *FailingCommand); - break; - } - } - } - - Diags.getClient()->finish(); - - // If any timers were active but haven't been destroyed yet, print their - // results now. This happens in -disable-free mode. - llvm::TimerGroup::printAll(llvm::errs()); - -#ifdef _WIN32 - // Exit status should not be negative on Win32, unless abnormal termination. - // Once abnormal termiation was caught, negative status should not be - // propagated. - if (Res < 0) - Res = 1; -#endif - - // If we have multiple failing commands, we return the result of the first - // failing command. - return Res; -} diff --git a/dbms/programs/clang/Compiler-7.0.0bundled/lld.cpp b/dbms/programs/clang/Compiler-7.0.0bundled/lld.cpp deleted file mode 100644 index 203e50d42a9..00000000000 --- a/dbms/programs/clang/Compiler-7.0.0bundled/lld.cpp +++ /dev/null @@ -1,10 +0,0 @@ -#include "lld/Common/Driver.h" -#include "llvm/Support/InitLLVM.h" -#include - -int mainEntryClickHouseLLD(int argc, char ** argv) -{ - llvm::InitLLVM X(argc, argv); - std::vector args(argv, argv + argc); - return !lld::elf::link(args, false); -} diff --git a/dbms/programs/clang/Compiler-7.0.0svn b/dbms/programs/clang/Compiler-7.0.0svn deleted file mode 120000 index eeeb5bbc2c0..00000000000 --- a/dbms/programs/clang/Compiler-7.0.0svn +++ /dev/null @@ -1 +0,0 @@ -Compiler-7.0.0 \ No newline at end of file diff --git a/dbms/programs/clang/Compiler-7.0.1 b/dbms/programs/clang/Compiler-7.0.1 deleted file mode 120000 index eeeb5bbc2c0..00000000000 --- a/dbms/programs/clang/Compiler-7.0.1 +++ /dev/null @@ -1 +0,0 @@ -Compiler-7.0.0 \ No newline at end of file diff --git a/dbms/programs/clang/clickhouse-clang.cpp b/dbms/programs/clang/clickhouse-clang.cpp deleted file mode 100644 index 261ae18b6d3..00000000000 --- a/dbms/programs/clang/clickhouse-clang.cpp +++ /dev/null @@ -1,2 +0,0 @@ -int mainEntryClickHouseClang(int argc, char ** argv); -int main(int argc_, char ** argv_) { return mainEntryClickHouseClang(argc_, argv_); } diff --git a/dbms/programs/clang/clickhouse-lld.cpp b/dbms/programs/clang/clickhouse-lld.cpp deleted file mode 100644 index baa6182d66d..00000000000 --- a/dbms/programs/clang/clickhouse-lld.cpp +++ /dev/null @@ -1,2 +0,0 @@ -int mainEntryClickHouseLLD(int argc, char ** argv); -int main(int argc_, char ** argv_) { return mainEntryClickHouseLLD(argc_, argv_); } diff --git a/dbms/programs/clang/copy_headers.sh b/dbms/programs/clang/copy_headers.sh deleted file mode 100755 index 45a58855c91..00000000000 --- a/dbms/programs/clang/copy_headers.sh +++ /dev/null @@ -1,100 +0,0 @@ -#!/usr/bin/env bash - -set -e -#set -x -#echo "Args: $*"; env | sort - -# Этот скрипт собирает все заголовочные файлы, нужные для компиляции некоторого translation unit-а -# и копирует их с сохранением путей в директорию DST. -# Это затем может быть использовано, чтобы скомпилировать translation unit на другом сервере, -# используя ровно такой же набор заголовочных файлов. -# -# Требуется clang, желательно наиболее свежий (trunk). -# -# Используется при сборке пакетов. -# Заголовочные файлы записываются в пакет clickhouse-common, в директорию /usr/share/clickhouse/headers. -# -# Если вы хотите установить их самостоятельно, без сборки пакета, -# чтобы clickhouse-server видел их там, где ожидается, выполните: -# -# sudo ./copy_headers.sh . /usr/share/clickhouse/headers/ - -SOURCE_PATH=${1:-../../..} -DST=${2:-$SOURCE_PATH/../headers} -BUILD_PATH=${BUILD_PATH=${3:-$SOURCE_PATH/build}} - -PATH="/usr/local/bin:/usr/local/sbin:/usr/bin:$PATH" - -if [[ -z $CLANG ]]; then - CLANG="clang" -fi - -START_HEADERS=$(echo \ - $BUILD_PATH/dbms/src/Common/config_version.h \ - $SOURCE_PATH/dbms/src/Interpreters/SpecializedAggregator.h \ - $SOURCE_PATH/dbms/src/AggregateFunctions/AggregateFunction*.h) - -for header in $START_HEADERS; do - START_HEADERS_INCLUDE+="-include $header " -done - - -GCC_ROOT=`$CLANG -v 2>&1 | grep "Selected GCC installation"| sed -n -e 's/^.*: //p'` - -# TODO: Does not work on macos? -GCC_ROOT=${GCC_ROOT:=/usr/lib/clang/${CMAKE_CXX_COMPILER_VERSION}} - -# Опция -mcx16 для того, чтобы выбиралось больше заголовочных файлов (с запасом). -# The latter options are the same that are added while building packages. -for src_file in $(echo | $CLANG -M -xc++ -std=c++1z -Wall -Werror -msse2 -msse4 -mcx16 -mpopcnt -O3 -g -fPIC -fstack-protector -D_FORTIFY_SOURCE=2 \ - -I $GCC_ROOT/include \ - -I $GCC_ROOT/include-fixed \ - $(cat "$BUILD_PATH/include_directories.txt") \ - $START_HEADERS_INCLUDE \ - - | - tr -d '\\' | - sed -E -e 's/^-\.o://'); -do - dst_file=$src_file; - [ -n $BUILD_PATH ] && dst_file=$(echo $dst_file | sed -E -e "s!^$BUILD_PATH!!") - [ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed -E -e "s!^$DESTDIR!!") - dst_file=$(echo $dst_file | sed -E -e 's/build\///') # for simplicity reasons, will put generated headers near the rest. - mkdir -p "$DST/$(echo $dst_file | sed -E -e 's/\/[^/]*$/\//')"; - cp "$src_file" "$DST/$dst_file"; -done - - -# Копируем больше заголовочных файлов с интринсиками, так как на серверах, куда будут устанавливаться -# заголовочные файлы, будет использоваться опция -march=native. - -for src_file in $(ls -1 $($CLANG -v -xc++ - <<<'' 2>&1 | grep '^ /' | grep 'include' | grep -E '/lib/clang/|/include/clang/')/*.h | grep -vE 'arm|altivec|Intrin'); -do - dst_file=$src_file; - [ -n $BUILD_PATH ] && dst_file=$(echo $dst_file | sed -E -e "s!^$BUILD_PATH!!") - [ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed -E -e "s!^$DESTDIR!!") - mkdir -p "$DST/$(echo $dst_file | sed -E -e 's/\/[^/]*$/\//')"; - cp "$src_file" "$DST/$dst_file"; -done - -if [ -d "$SOURCE_PATH/contrib/boost/libs/smart_ptr/include/boost/smart_ptr/detail" ]; then - # Even more platform-specific headers - for src_file in $(ls -1 $SOURCE_PATH/contrib/boost/libs/smart_ptr/include/boost/smart_ptr/detail/*); - do - dst_file=$src_file; - [ -n $BUILD_PATH ] && dst_file=$(echo $dst_file | sed -E -e "s!^$BUILD_PATH!!") - [ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed -E -e "s!^$DESTDIR!!") - mkdir -p "$DST/$(echo $dst_file | sed -E -e 's/\/[^/]*$/\//')"; - cp "$src_file" "$DST/$dst_file"; - done -fi - -if [ -d "$SOURCE_PATH/contrib/boost/boost/smart_ptr/detail" ]; then - for src_file in $(ls -1 $SOURCE_PATH/contrib/boost/boost/smart_ptr/detail/*); - do - dst_file=$src_file; - [ -n $BUILD_PATH ] && dst_file=$(echo $dst_file | sed -E -e "s!^$BUILD_PATH!!") - [ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed -E -e "s!^$DESTDIR!!") - mkdir -p "$DST/$(echo $dst_file | sed -E -e 's/\/[^/]*$/\//')"; - cp "$src_file" "$DST/$dst_file"; - done -fi diff --git a/dbms/programs/main.cpp b/dbms/programs/main.cpp index 57821d854e9..3fbbcee0f15 100644 --- a/dbms/programs/main.cpp +++ b/dbms/programs/main.cpp @@ -56,11 +56,6 @@ int mainEntryClickHouseObfuscator(int argc, char ** argv); #endif -#if USE_EMBEDDED_COMPILER - int mainEntryClickHouseClang(int argc, char ** argv); - int mainEntryClickHouseLLD(int argc, char ** argv); -#endif - namespace { @@ -100,12 +95,6 @@ std::pair clickhouse_applications[] = #if ENABLE_CLICKHOUSE_OBFUSCATOR || !defined(ENABLE_CLICKHOUSE_OBFUSCATOR) {"obfuscator", mainEntryClickHouseObfuscator}, #endif - -#if USE_EMBEDDED_COMPILER - {"clang", mainEntryClickHouseClang}, - {"clang++", mainEntryClickHouseClang}, - {"lld", mainEntryClickHouseLLD}, -#endif }; @@ -152,11 +141,6 @@ int main(int argc_, char ** argv_) /// will work only after additional call of this function. updatePHDRCache(); -#if USE_EMBEDDED_COMPILER - if (argc_ >= 2 && 0 == strcmp(argv_[1], "-cc1")) - return mainEntryClickHouseClang(argc_, argv_); -#endif - #if USE_TCMALLOC /** Without this option, tcmalloc returns memory to OS too frequently for medium-sized memory allocations * (like IO buffers, column vectors, hash tables, etc.), diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index b7db44700bb..3adae6d9e93 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -85,7 +85,6 @@ struct Settings : public SettingsCollection M(SettingTotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE, "How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = ‘any’ are present.") \ M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.") \ \ - M(SettingBool, compile, false, "Whether query compilation is enabled.") \ M(SettingBool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.") \ M(SettingBool, compile_expressions, false, "Compile some scalar functions and operators to native code.") \ M(SettingUInt64, min_count_to_compile, 3, "The number of structurally identical queries before they are compiled.") \ @@ -351,6 +350,7 @@ struct Settings : public SettingsCollection /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") \ + M(SettingBool, compile, false, "Whether query compilation is enabled. Will be removed after 2020-03-13") \ DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 33fbb903497..373b47f7315 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -25,10 +25,6 @@ #include #include -#if __has_include() -#include -#endif - namespace ProfileEvents { @@ -47,7 +43,6 @@ namespace DB namespace ErrorCodes { - extern const int CANNOT_COMPILE_CODE; extern const int TOO_MANY_ROWS; extern const int EMPTY_DATA_PASSED; extern const int CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS; @@ -195,200 +190,6 @@ Aggregator::Aggregator(const Params & params_) } -void Aggregator::compileIfPossible(AggregatedDataVariants::Type type) -{ - std::lock_guard lock(mutex); - - if (compiled_if_possible) - return; - - compiled_if_possible = true; - -#if !defined(INTERNAL_COMPILER_HEADERS) - throw Exception("Cannot compile code: Compiler disabled", ErrorCodes::CANNOT_COMPILE_CODE); -#else - std::string method_typename_single_level; - std::string method_typename_two_level; - - if (false) {} -#define M(NAME) \ - else if (type == AggregatedDataVariants::Type::NAME) \ - { \ - method_typename_single_level = "decltype(AggregatedDataVariants::" #NAME ")::element_type"; \ - method_typename_two_level = "decltype(AggregatedDataVariants::" #NAME "_two_level)::element_type"; \ - } - - APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M) -#undef M - -#define M(NAME) \ - else if (type == AggregatedDataVariants::Type::NAME) \ - method_typename_single_level = "decltype(AggregatedDataVariants::" #NAME ")::element_type"; - - APPLY_FOR_VARIANTS_NOT_CONVERTIBLE_TO_TWO_LEVEL(M) -#undef M - else if (type == AggregatedDataVariants::Type::without_key) {} - else - throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); - - auto compiler_headers = Poco::Util::Application::instance().config().getString("compiler_headers", INTERNAL_COMPILER_HEADERS); - - /// List of types of aggregate functions. - std::stringstream aggregate_functions_typenames_str; - std::stringstream aggregate_functions_headers_args; - for (size_t i = 0; i < params.aggregates_size; ++i) - { - IAggregateFunction & func = *aggregate_functions[i]; - - int status = 0; - std::string type_name = demangle(typeid(func).name(), status); - - if (status) - throw Exception("Cannot compile code: cannot demangle name " + String(typeid(func).name()) - + ", status: " + toString(status), ErrorCodes::CANNOT_COMPILE_CODE); - - aggregate_functions_typenames_str << ((i != 0) ? ", " : "") << type_name; - - std::string header_path = func.getHeaderFilePath(); - auto pos = header_path.find("/AggregateFunctions/"); - - if (pos == std::string::npos) - throw Exception("Cannot compile code: unusual path of header file for aggregate function: " + header_path, - ErrorCodes::CANNOT_COMPILE_CODE); - - aggregate_functions_headers_args << "-include '" << compiler_headers << "/dbms/src"; - aggregate_functions_headers_args.write(&header_path[pos], header_path.size() - pos); - aggregate_functions_headers_args << "' "; - } - - aggregate_functions_headers_args << "-include '" << compiler_headers << "/dbms/src/Interpreters/SpecializedAggregator.h'"; - - std::string aggregate_functions_typenames = aggregate_functions_typenames_str.str(); - - std::stringstream key_str; - key_str << "Aggregate: "; - if (!method_typename_single_level.empty()) - key_str << method_typename_single_level + ", "; - key_str << aggregate_functions_typenames; - std::string key = key_str.str(); - - auto get_code = [method_typename_single_level, method_typename_two_level, aggregate_functions_typenames] - { - /// A short piece of code, which is an explicit instantiation of the template. - std::stringstream code; - code << /// No explicit inclusion of the header file. It is included using the -include compiler option. - "namespace DB\n" - "{\n" - "\n"; - - /// There can be up to two instantiations for the template - for normal and two_level options. - auto append_code_for_specialization = - [&code, &aggregate_functions_typenames] (const std::string & method_typename, const std::string & suffix) - { - code << - "template void Aggregator::executeSpecialized<\n" - " " << method_typename << ", TypeList<" << aggregate_functions_typenames << ">>(\n" - " " << method_typename << " &, Arena *, size_t, ColumnRawPtrs &,\n" - " AggregateColumns &, bool, AggregateDataPtr) const;\n" - "\n" - "static void wrapper" << suffix << "(\n" - " const Aggregator & aggregator,\n" - " " << method_typename << " & method,\n" - " Arena * arena,\n" - " size_t rows,\n" - " ColumnRawPtrs & key_columns,\n" - " Aggregator::AggregateColumns & aggregate_columns,\n" - " bool no_more_keys,\n" - " AggregateDataPtr overflow_row)\n" - "{\n" - " aggregator.executeSpecialized<\n" - " " << method_typename << ", TypeList<" << aggregate_functions_typenames << ">>(\n" - " method, arena, rows, key_columns, aggregate_columns, no_more_keys, overflow_row);\n" - "}\n" - "\n" - "void * getPtr" << suffix << "() __attribute__((__visibility__(\"default\")));\n" - "void * getPtr" << suffix << "()\n" /// Without this wrapper, it's not clear how to get the desired symbol from the compiled library. - "{\n" - " return reinterpret_cast(&wrapper" << suffix << ");\n" - "}\n"; - }; - - if (!method_typename_single_level.empty()) - append_code_for_specialization(method_typename_single_level, ""); - else - { - /// For `without_key` method. - code << - "template void Aggregator::executeSpecializedWithoutKey<\n" - " " << "TypeList<" << aggregate_functions_typenames << ">>(\n" - " AggregatedDataWithoutKey &, size_t, AggregateColumns &, Arena *) const;\n" - "\n" - "static void wrapper(\n" - " const Aggregator & aggregator,\n" - " AggregatedDataWithoutKey & method,\n" - " size_t rows,\n" - " Aggregator::AggregateColumns & aggregate_columns,\n" - " Arena * arena)\n" - "{\n" - " aggregator.executeSpecializedWithoutKey<\n" - " TypeList<" << aggregate_functions_typenames << ">>(\n" - " method, rows, aggregate_columns, arena);\n" - "}\n" - "\n" - "void * getPtr() __attribute__((__visibility__(\"default\")));\n" - "void * getPtr()\n" - "{\n" - " return reinterpret_cast(&wrapper);\n" - "}\n"; - } - - if (!method_typename_two_level.empty()) - append_code_for_specialization(method_typename_two_level, "TwoLevel"); - else - { - /// The stub. - code << - "void * getPtrTwoLevel() __attribute__((__visibility__(\"default\")));\n" - "void * getPtrTwoLevel()\n" - "{\n" - " return nullptr;\n" - "}\n"; - } - - code << - "}\n"; - - return code.str(); - }; - - auto compiled_data_owned_by_callback = compiled_data; - auto on_ready = [compiled_data_owned_by_callback] (SharedLibraryPtr & lib) - { - if (compiled_data_owned_by_callback.unique()) /// Aggregator is already destroyed. - return; - - compiled_data_owned_by_callback->compiled_aggregator = lib; - compiled_data_owned_by_callback->compiled_method_ptr = lib->get("_ZN2DB6getPtrEv")(); - compiled_data_owned_by_callback->compiled_two_level_method_ptr = lib->get("_ZN2DB14getPtrTwoLevelEv")(); - }; - - /** If the library has already been compiled, a non-zero SharedLibraryPtr is returned. - * If the library was not compiled, then the counter is incremented, and nullptr is returned. - * If the counter has reached the value min_count_to_compile, then the compilation starts asynchronously (in a separate thread) - * at the end of which `on_ready` callback is called. - */ - aggregate_functions_headers_args << " -Wno-unused-function"; - SharedLibraryPtr lib = params.compiler->getOrCount(key, params.min_count_to_compile, - aggregate_functions_headers_args.str(), - get_code, on_ready); - - /// If the result is already ready. - if (lib) - on_ready(lib); -#endif -} - - AggregatedDataVariants::Type Aggregator::chooseAggregationMethod() { /// If no keys. All aggregating to single row. @@ -720,9 +521,6 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re result.keys_size = params.keys_size; result.key_sizes = key_sizes; LOG_TRACE(log, "Aggregation method: " << result.getMethodName()); - - if (params.compiler) - compileIfPossible(result.type); } if (isCancelled()) @@ -794,67 +592,21 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re /// For the case when there are no keys (all aggregate into one row). if (result.type == AggregatedDataVariants::Type::without_key) { - /// If there is a dynamically compiled code. - if (compiled_data->compiled_method_ptr) - { - reinterpret_cast< - void (*)(const Aggregator &, AggregatedDataWithoutKey &, size_t, AggregateColumns &, Arena *)> - (compiled_data->compiled_method_ptr)(*this, result.without_key, rows, aggregate_columns, result.aggregates_pool); - } - else - executeWithoutKeyImpl(result.without_key, rows, aggregate_functions_instructions.data(), result.aggregates_pool); + executeWithoutKeyImpl(result.without_key, rows, aggregate_functions_instructions.data(), result.aggregates_pool); } else { /// This is where data is written that does not fit in `max_rows_to_group_by` with `group_by_overflow_mode = any`. AggregateDataPtr overflow_row_ptr = params.overflow_row ? result.without_key : nullptr; - bool is_two_level = result.isTwoLevel(); - - /// Compiled code, for the normal structure. - if (!is_two_level && compiled_data->compiled_method_ptr) - { - #define M(NAME, IS_TWO_LEVEL) \ - else if (result.type == AggregatedDataVariants::Type::NAME) \ - reinterpret_cast(compiled_data->compiled_method_ptr) \ - (*this, *result.NAME, result.aggregates_pool, rows, key_columns, aggregate_columns, \ - no_more_keys, overflow_row_ptr); - - if (false) {} - APPLY_FOR_AGGREGATED_VARIANTS(M) - #undef M - } - /// Compiled code, for a two-level structure. - else if (is_two_level && compiled_data->compiled_two_level_method_ptr) - { - #define M(NAME) \ - else if (result.type == AggregatedDataVariants::Type::NAME) \ - reinterpret_cast(compiled_data->compiled_two_level_method_ptr) \ - (*this, *result.NAME, result.aggregates_pool, rows, key_columns, aggregate_columns, \ - no_more_keys, overflow_row_ptr); - - if (false) {} - APPLY_FOR_VARIANTS_TWO_LEVEL(M) - #undef M - } - /// When there is no dynamically compiled code. - else - { #define M(NAME, IS_TWO_LEVEL) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ executeImpl(*result.NAME, result.aggregates_pool, rows, key_columns, aggregate_functions_instructions.data(), \ no_more_keys, overflow_row_ptr); - if (false) {} - APPLY_FOR_AGGREGATED_VARIANTS(M) + if (false) {} + APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M - } } size_t result_size = result.sizeWithoutOverflowRow(); diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 41fd957345e..b48663ff689 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -24,7 +24,6 @@ #include #include -#include #include #include @@ -778,10 +777,6 @@ public: const size_t max_rows_to_group_by; const OverflowMode group_by_overflow_mode; - /// For dynamic compilation. - Compiler * compiler; - const UInt32 min_count_to_compile; - /// Two-level aggregation settings (used for a large number of keys). /** With how many keys or the size of the aggregation state in bytes, * two-level aggregation begins to be used. Enough to reach of at least one of the thresholds. @@ -805,7 +800,6 @@ public: const Block & src_header_, const ColumnNumbers & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, - Compiler * compiler_, UInt32 min_count_to_compile_, size_t group_by_two_level_threshold_, size_t group_by_two_level_threshold_bytes_, size_t max_bytes_before_external_group_by_, bool empty_result_for_aggregation_by_empty_set_, @@ -813,7 +807,6 @@ public: : src_header(src_header_), keys(keys_), aggregates(aggregates_), keys_size(keys.size()), aggregates_size(aggregates.size()), overflow_row(overflow_row_), max_rows_to_group_by(max_rows_to_group_by_), group_by_overflow_mode(group_by_overflow_mode_), - compiler(compiler_), min_count_to_compile(min_count_to_compile_), group_by_two_level_threshold(group_by_two_level_threshold_), group_by_two_level_threshold_bytes(group_by_two_level_threshold_bytes_), max_bytes_before_external_group_by(max_bytes_before_external_group_by_), empty_result_for_aggregation_by_empty_set(empty_result_for_aggregation_by_empty_set_), @@ -824,7 +817,7 @@ public: /// Only parameters that matter during merge. Params(const Block & intermediate_header_, const ColumnNumbers & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_) - : Params(Block(), keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0, 0, 0, 0, false, "", max_threads_) + : Params(Block(), keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, 0, 0, 0, false, "", max_threads_) { intermediate_header = intermediate_header_; } @@ -956,26 +949,6 @@ protected: Logger * log = &Logger::get("Aggregator"); - /** Dynamically compiled library for aggregation, if any. - * The meaning of dynamic compilation is to specialize code - * for a specific list of aggregate functions. - * This allows you to expand the loop to create and update states of aggregate functions, - * and also use inline-code instead of virtual calls. - */ - struct CompiledData - { - SharedLibraryPtr compiled_aggregator; - - /// Obtained with dlsym. It is still necessary to make reinterpret_cast to the function pointer. - void * compiled_method_ptr = nullptr; - void * compiled_two_level_method_ptr = nullptr; - }; - /// shared_ptr - to pass into a callback, that can survive Aggregator. - std::shared_ptr compiled_data { new CompiledData }; - - bool compiled_if_possible = false; - void compileIfPossible(AggregatedDataVariants::Type type); - /// Returns true if you can abort the current task. CancellationHook isCancelled; @@ -1037,35 +1010,6 @@ protected: Method & method, IBlockOutputStream & out); -public: - /// Templates that are instantiated by dynamic code compilation - see SpecializedAggregator.h - - template - void executeSpecialized( - Method & method, - Arena * aggregates_pool, - size_t rows, - ColumnRawPtrs & key_columns, - AggregateColumns & aggregate_columns, - bool no_more_keys, - AggregateDataPtr overflow_row) const; - - template - void executeSpecializedCase( - Method & method, - typename Method::State & state, - Arena * aggregates_pool, - size_t rows, - AggregateColumns & aggregate_columns, - AggregateDataPtr overflow_row) const; - - template - void executeSpecializedWithoutKey( - AggregatedDataWithoutKey & res, - size_t rows, - AggregateColumns & aggregate_columns, - Arena * arena) const; - protected: /// Merge NULL key data from hash table `src` into `dst`. template diff --git a/dbms/src/Interpreters/CMakeLists.txt b/dbms/src/Interpreters/CMakeLists.txt index 75771a07027..65172356645 100644 --- a/dbms/src/Interpreters/CMakeLists.txt +++ b/dbms/src/Interpreters/CMakeLists.txt @@ -1,70 +1,3 @@ - -if (OS_FREEBSD) - set (PATH_SHARE "/usr/local/share" CACHE STRING "") -else () - set (PATH_SHARE "/usr/share" CACHE STRING "") -endif () - -set (INTERNAL_COMPILER_BIN_ROOT "${CMAKE_INSTALL_FULL_BINDIR}/" CACHE STRING "") -set (INTERNAL_COMPILER_EXECUTABLE "clickhouse-clang" CACHE STRING "") -set (INTERNAL_LINKER_EXECUTABLE "clickhouse-lld" CACHE STRING "") - -# Disabling leak reporting for these tools -if (SANITIZE STREQUAL "address") - # Note that this doesn't work for setuid and setcap binaries - set(INTERNAL_COMPILER_ENV "env ASAN_OPTIONS=detect_leaks=0" CACHE STRING "") -else () - set(INTERNAL_COMPILER_ENV "" CACHE STRING "") -endif () - -set (INTERNAL_COMPILER_NO_WARNING OFF CACHE INTERNAL "") -set (INTERNAL_COMPILER_HEADERS_DIR "headers" CACHE STRING "") -set (INTERNAL_COMPILER_HEADERS_RELATIVE "${INTERNAL_COMPILER_HEADERS_DIR}/${VERSION_STRING}" CACHE STRING "") -set (INTERNAL_COMPILER_HEADERS "${PATH_SHARE}/clickhouse/${INTERNAL_COMPILER_HEADERS_RELATIVE}" CACHE STRING "") - -if(OS_FREEBSD) - set(INTERNAL_COMPILER_HEADERS_ROOT "" CACHE STRING "") -else() - set(INTERNAL_COMPILER_HEADERS_ROOT "${INTERNAL_COMPILER_HEADERS}" CACHE STRING "") - set(INTERNAL_COMPILER_CUSTOM_ROOT ON CACHE INTERNAL "") -endif() - -if(NOT INTERNAL_COMPILER_FLAGS) - set(INTERNAL_COMPILER_FLAGS "${CMAKE_CXX_FLAGS} ${CMAKE_CXX_FLAGS_${CMAKE_BUILD_TYPE_UC}} ${CXX_FLAGS_INTERNAL_COMPILER} -x c++ -march=native -fPIC -fvisibility=hidden -fno-implement-inlines -Wno-unused-command-line-argument -Bprefix=${PATH_SHARE}/clickhouse" CACHE STRING "") - if(INTERNAL_COMPILER_CUSTOM_ROOT) - set(INTERNAL_COMPILER_FLAGS "${INTERNAL_COMPILER_FLAGS} -nostdinc -nostdinc++") - if(INTERNAL_COMPILER_HEADERS_ROOT) - set(INTERNAL_COMPILER_FLAGS "${INTERNAL_COMPILER_FLAGS} -isysroot=${INTERNAL_COMPILER_HEADERS_ROOT}") - endif() - endif() -endif() -# TODO: use libs from package: -nodefaultlibs -lm -lc -lgcc_s -lgcc -lc++ -lc++abi - -string(REPLACE "${INCLUDE_DEBUG_HELPERS}" "" INTERNAL_COMPILER_FLAGS ${INTERNAL_COMPILER_FLAGS}) -string(REPLACE "-no-pie" "" INTERNAL_COMPILER_FLAGS ${INTERNAL_COMPILER_FLAGS}) -if (INTERNAL_COMPILER_NO_WARNING) - string (REPLACE "-Wall" "" INTERNAL_COMPILER_FLAGS ${INTERNAL_COMPILER_FLAGS}) - string (REPLACE "-Wextra" "" INTERNAL_COMPILER_FLAGS ${INTERNAL_COMPILER_FLAGS}) - string (REPLACE "-Werror" "" INTERNAL_COMPILER_FLAGS ${INTERNAL_COMPILER_FLAGS}) -endif () - -list(GET Poco_INCLUDE_DIRS 0 Poco_Foundation_INCLUDE_DIR) -list(GET Poco_INCLUDE_DIRS 1 Poco_Util_INCLUDE_DIR) - -if (NOT DOUBLE_CONVERSION_INCLUDE_DIR) - get_target_property(DOUBLE_CONVERSION_INCLUDE_DIR ${DOUBLE_CONVERSION_LIBRARIES} INTERFACE_INCLUDE_DIRECTORIES) -endif () - -string (REPLACE ${ClickHouse_SOURCE_DIR} "" INTERNAL_DOUBLE_CONVERSION_INCLUDE_DIR ${DOUBLE_CONVERSION_INCLUDE_DIR}) -string (REPLACE ${ClickHouse_SOURCE_DIR} "" INTERNAL_Boost_INCLUDE_DIRS ${Boost_INCLUDE_DIRS}) -string (REPLACE ${ClickHouse_SOURCE_DIR} "" INTERNAL_Poco_Foundation_INCLUDE_DIR ${Poco_Foundation_INCLUDE_DIR}) -string (REPLACE ${ClickHouse_SOURCE_DIR} "" INTERNAL_Poco_Util_INCLUDE_DIR ${Poco_Util_INCLUDE_DIR}) - -message (STATUS "Using internal=${USE_INTERNAL_LLVM_LIBRARY} compiler=${USE_EMBEDDED_COMPILER}: headers=${INTERNAL_COMPILER_HEADERS} root=${INTERNAL_COMPILER_HEADERS_ROOT}: ${INTERNAL_COMPILER_ENV} ${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE} ${INTERNAL_COMPILER_FLAGS}; ${INTERNAL_LINKER_EXECUTABLE}") - -set (CONFIG_COMPILE ${ClickHouse_BINARY_DIR}/dbms/src/Interpreters/config_compile.h) -configure_file (${ClickHouse_SOURCE_DIR}/dbms/src/Interpreters/config_compile.h.in ${CONFIG_COMPILE}) - if (ENABLE_TESTS) add_subdirectory (tests) endif () diff --git a/dbms/src/Interpreters/Compiler.cpp b/dbms/src/Interpreters/Compiler.cpp deleted file mode 100644 index 3b420b6acce..00000000000 --- a/dbms/src/Interpreters/Compiler.cpp +++ /dev/null @@ -1,326 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if __has_include() -#include -#endif - -namespace ProfileEvents -{ - extern const Event CompileAttempt; - extern const Event CompileSuccess; -} - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_DLOPEN; - extern const int CANNOT_COMPILE_CODE; -} - -Compiler::Compiler(const std::string & path_, size_t threads) - : path(path_), pool(threads) -{ - Poco::File(path).createDirectory(); - - Poco::DirectoryIterator dir_end; - for (Poco::DirectoryIterator dir_it(path); dir_end != dir_it; ++dir_it) - { - const std::string & name = dir_it.name(); - if (endsWith(name, ".so")) - { - files.insert(name.substr(0, name.size() - 3)); - } - } - - LOG_INFO(log, "Having " << files.size() << " compiled files from previous start."); -} - -Compiler::~Compiler() -{ - LOG_DEBUG(log, "Waiting for threads to finish."); - pool.wait(); -} - - -static Compiler::HashedKey getHash(const std::string & key) -{ - SipHash hash; - - auto revision = ClickHouseRevision::get(); - hash.update(revision); - hash.update(key.data(), key.size()); - - Compiler::HashedKey res; - hash.get128(res.low, res.high); - return res; -} - - -/// Without .so extension. -static std::string hashedKeyToFileName(Compiler::HashedKey hashed_key) -{ - WriteBufferFromOwnString out; - out << hashed_key.low << '_' << hashed_key.high; - return out.str(); -} - - -SharedLibraryPtr Compiler::getOrCount( - const std::string & key, - UInt32 min_count_to_compile, - const std::string & additional_compiler_flags, - CodeGenerator get_code, - ReadyCallback on_ready) -{ - HashedKey hashed_key = getHash(key); - - std::lock_guard lock(mutex); - - UInt32 count = ++counts[hashed_key]; - - /// Is there a ready open library? Or, if the library is in the process of compiling, there will be nullptr. - Libraries::iterator libraries_it = libraries.find(hashed_key); - if (libraries.end() != libraries_it) - { - if (!libraries_it->second) - LOG_INFO(log, "Library " << hashedKeyToFileName(hashed_key) << " is already compiling or compilation was failed."); - - /// TODO In this case, after the compilation is finished, the callback will not be called. - - return libraries_it->second; - } - - /// Is there a file with the library left over from the previous launch? - std::string file_name = hashedKeyToFileName(hashed_key); - Files::iterator files_it = files.find(file_name); - if (files.end() != files_it) - { - std::string so_file_path = path + '/' + file_name + ".so"; - LOG_INFO(log, "Loading existing library " << so_file_path); - - SharedLibraryPtr lib; - - try - { - lib = std::make_shared(so_file_path); - } - catch (const Exception & e) - { - if (e.code() != ErrorCodes::CANNOT_DLOPEN) - throw; - - /// Found broken .so file (or file cannot be dlopened by whatever reason). - /// This could happen when filesystem is corrupted after server restart. - /// We remove the file - it will be recompiled on next attempt. - - tryLogCurrentException(log); - - files.erase(files_it); - Poco::File(so_file_path).remove(); - return nullptr; - } - - libraries[hashed_key] = lib; - return lib; - } - - /// Has min_count_to_compile been reached? - if (count >= min_count_to_compile) - { - /// The min_count_to_compile value of zero indicates the need for synchronous compilation. - - /// Indicates that the library is in the process of compiling. - libraries[hashed_key] = nullptr; - - LOG_INFO(log, "Compiling code " << file_name << ", key: " << key); - - if (min_count_to_compile == 0) - { - { - ext::unlock_guard unlock(mutex); - compile(hashed_key, file_name, additional_compiler_flags, get_code, on_ready); - } - - return libraries[hashed_key]; - } - else - { - bool res = pool.trySchedule([=] - { - try - { - compile(hashed_key, file_name, additional_compiler_flags, get_code, on_ready); - } - catch (...) - { - tryLogCurrentException("Compiler"); - } - }); - - if (!res) - LOG_INFO(log, "All threads are busy."); - } - } - - return nullptr; -} - - -/// This will guarantee that code will compile only when version of headers match version of running server. -static void addCodeToAssertHeadersMatch(WriteBuffer & out) -{ - out << - "#include \n" - "#if VERSION_REVISION != " << ClickHouseRevision::get() << "\n" - "#define STRING2(x) #x\n" - "#define STRING(x) STRING2(x)\n" - "#pragma message \"ClickHouse headers revision = \" STRING(VERSION_REVISION) \n" - "#error \"ClickHouse headers revision doesn't match runtime revision of the server (" << ClickHouseRevision::get() << ").\"\n" - "#endif\n\n"; -} - - -void Compiler::compile( - HashedKey hashed_key, - std::string file_name, - const std::string & additional_compiler_flags, - CodeGenerator get_code, - ReadyCallback on_ready) -{ - ProfileEvents::increment(ProfileEvents::CompileAttempt); - -#if !defined(INTERNAL_COMPILER_EXECUTABLE) - throw Exception("Cannot compile code: Compiler disabled", ErrorCodes::CANNOT_COMPILE_CODE); -#else - std::string prefix = path + "/" + file_name; - std::string cpp_file_path = prefix + ".cpp"; - std::string so_file_path = prefix + ".so"; - std::string so_tmp_file_path = prefix + ".so.tmp"; - - { - WriteBufferFromFile out(cpp_file_path); - - addCodeToAssertHeadersMatch(out); - out << get_code(); - } - - std::stringstream command; - - auto compiler_executable_root = Poco::Util::Application::instance().config().getString("compiler_executable_root", INTERNAL_COMPILER_BIN_ROOT); - auto compiler_headers = Poco::Util::Application::instance().config().getString("compiler_headers", INTERNAL_COMPILER_HEADERS); - auto compiler_headers_root = Poco::Util::Application::instance().config().getString("compiler_headers_root", INTERNAL_COMPILER_HEADERS_ROOT); - LOG_DEBUG(log, "Using internal compiler: compiler_executable_root=" << compiler_executable_root << "; compiler_headers_root=" << compiler_headers_root << "; compiler_headers=" << compiler_headers); - - /// Slightly unconvenient. - command << - "(" - INTERNAL_COMPILER_ENV - " " << compiler_executable_root << INTERNAL_COMPILER_EXECUTABLE - " " INTERNAL_COMPILER_FLAGS - /// It is hard to correctly call a ld program manually, because it is easy to skip critical flags, which might lead to - /// unhandled exceptions. Therefore pass path to llvm's lld directly to clang. - " -fuse-ld=" << compiler_executable_root << INTERNAL_LINKER_EXECUTABLE - " -fdiagnostics-color=never" - - /// Do not use libgcc and startup files. The library will work nevertheless and we avoid extra dependency. - " -nodefaultlibs -nostartfiles" - - #if INTERNAL_COMPILER_CUSTOM_ROOT - /// To get correct order merge this results carefully: - /// echo | clang -x c++ -E -Wp,-v - - /// echo | g++ -x c++ -E -Wp,-v - - - " -isystem " << compiler_headers_root << "/usr/include/c++/*" - #if defined(CMAKE_LIBRARY_ARCHITECTURE) - " -isystem " << compiler_headers_root << "/usr/include/" CMAKE_LIBRARY_ARCHITECTURE "/c++/*" - #endif - " -isystem " << compiler_headers_root << "/usr/include/c++/*/backward" - " -isystem " << compiler_headers_root << "/usr/include/clang/*/include" /// if compiler is clang (from package) - " -isystem " << compiler_headers_root << "/usr/local/lib/clang/*/include" /// if clang installed manually - " -isystem " << compiler_headers_root << "/usr/lib/clang/*/include" /// if clang build from submodules - #if defined(CMAKE_LIBRARY_ARCHITECTURE) - " -isystem " << compiler_headers_root << "/usr/lib/gcc/" CMAKE_LIBRARY_ARCHITECTURE "/*/include-fixed" - " -isystem " << compiler_headers_root << "/usr/lib/gcc/" CMAKE_LIBRARY_ARCHITECTURE "/*/include" - #endif - " -isystem " << compiler_headers_root << "/usr/local/include" /// if something installed manually - #if defined(CMAKE_LIBRARY_ARCHITECTURE) - " -isystem " << compiler_headers_root << "/usr/include/" CMAKE_LIBRARY_ARCHITECTURE - #endif - " -isystem " << compiler_headers_root << "/usr/include" - #endif - " -I " << compiler_headers << "/dbms/src/" - " -isystem " << compiler_headers << "/contrib/cityhash102/include/" - " -isystem " << compiler_headers << "/contrib/libpcg-random/include/" - #if USE_MIMALLOC - " -isystem " << compiler_headers << "/contrib/mimalloc/include/" - #endif - " -isystem " << compiler_headers << INTERNAL_DOUBLE_CONVERSION_INCLUDE_DIR - " -isystem " << compiler_headers << INTERNAL_Poco_Foundation_INCLUDE_DIR - " -isystem " << compiler_headers << INTERNAL_Boost_INCLUDE_DIRS - " -I " << compiler_headers << "/libs/libcommon/include/" - " " << additional_compiler_flags << - " -shared -o " << so_tmp_file_path << " " << cpp_file_path - << " 2>&1" - ") || echo Return code: $?"; - -#ifndef NDEBUG - LOG_TRACE(log, "Compile command: " << command.str()); -#endif - - std::string compile_result; - - { - auto process = ShellCommand::execute(command.str()); - readStringUntilEOF(compile_result, process->out); - process->wait(); - } - - if (!compile_result.empty()) - { - std::string error_message = "Cannot compile code:\n\n" + command.str() + "\n\n" + compile_result; - - Poco::File so_tmp_file(so_tmp_file_path); - if (so_tmp_file.exists() && so_tmp_file.canExecute()) - { - /// Compiler may emit information messages. This is suspicious, but we still can use compiled result. - LOG_WARNING(log, error_message); - } - else - throw Exception(error_message, ErrorCodes::CANNOT_COMPILE_CODE); - } - - /// If there was an error before, the file with the code remains for viewing. - Poco::File(cpp_file_path).remove(); - - Poco::File(so_tmp_file_path).renameTo(so_file_path); - SharedLibraryPtr lib(new SharedLibrary(so_file_path)); - - { - std::lock_guard lock(mutex); - libraries[hashed_key] = lib; - } - - LOG_INFO(log, "Compiled code " << file_name); - ProfileEvents::increment(ProfileEvents::CompileSuccess); - - on_ready(lib); - -#endif -} - - -} diff --git a/dbms/src/Interpreters/Compiler.h b/dbms/src/Interpreters/Compiler.h deleted file mode 100644 index b79cf26e0f0..00000000000 --- a/dbms/src/Interpreters/Compiler.h +++ /dev/null @@ -1,88 +0,0 @@ -#pragma once - -#include - -#include -#include -#include -#include -#include -#include - -#include - -#include -#include -#include -#include -#include - -namespace DB -{ - -/** Lets you compile a piece of code that uses the server's header files into the dynamic library. - * Conducts statistic of calls, and initiates compilation only on the N-th call for one key. - * Compilation is performed asynchronously, in separate threads, if there are free threads. - * NOTE: There is no cleaning of obsolete and unnecessary results. - */ -class Compiler -{ -public: - /** path - path to the directory with temporary files - the results of the compilation. - * The compilation results are saved when the server is restarted, - * but use the revision number as part of the key. That is, they become obsolete when the server is updated. - */ - Compiler(const std::string & path_, size_t threads); - ~Compiler(); - - using HashedKey = UInt128; - - using CodeGenerator = std::function; - using ReadyCallback = std::function; - - /** Increase the counter for the given key `key` by one. - * If the compilation result already exists (already open, or there is a file with the library), - * then return ready SharedLibrary. - * Otherwise, if min_count_to_compile == 0, then initiate the compilation in the same thread, wait for it, and return the result. - * Otherwise, if the counter has reached min_count_to_compile, - * initiate compilation in a separate thread, if there are free threads, and return nullptr. - * Otherwise, return nullptr. - */ - SharedLibraryPtr getOrCount( - const std::string & key, - UInt32 min_count_to_compile, - const std::string & additional_compiler_flags, - CodeGenerator get_code, - ReadyCallback on_ready); - -private: - using Counts = std::unordered_map; - using Libraries = std::unordered_map; - using Files = std::unordered_set; - - const std::string path; - ThreadPool pool; - - /// Number of calls to `getOrCount`. - Counts counts; - - /// Compiled and open libraries. Or nullptr for libraries in the compilation process. - Libraries libraries; - - /// Compiled files remaining from previous runs, but not yet open. - Files files; - - std::mutex mutex; - - Logger * log = &Logger::get("Compiler"); - - - void compile( - HashedKey hashed_key, - std::string file_name, - const std::string & additional_compiler_flags, - CodeGenerator get_code, - ReadyCallback on_ready); -}; - -} diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 53392f9ad9a..67a81a94b3a 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -35,7 +35,6 @@ #include #include #include -#include #include #include #include @@ -140,7 +139,6 @@ struct ContextShared std::optional background_pool; /// The thread pool for the background work performed by the tables. std::optional schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables) MultiVersion macros; /// Substitutions extracted from config. - std::optional compiler; /// Used for dynamic compilation of queries' parts if it necessary. std::unique_ptr ddl_worker; /// Process ddl commands from zk. /// Rules for selecting the compression settings, depending on the size of the part. mutable std::unique_ptr compression_codec_selector; @@ -1634,17 +1632,6 @@ void Context::setCluster(const String & cluster_name, const std::shared_ptrcompiler) - shared->compiler.emplace(shared->path + "build/", 1); - - return *shared->compiler; -} - - void Context::initializeSystemLogs() { auto lock = getLock(); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index dcc3fa9b3ee..2d583c3c353 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1654,7 +1654,6 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre Aggregator::Params params(header, keys, aggregates, overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode, - settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile, allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, @@ -1721,7 +1720,6 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const Aggregator::Params params(header_before_aggregation, keys, aggregates, overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode, - settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile, allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, @@ -1943,7 +1941,6 @@ void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificato Aggregator::Params params(header, keys, aggregates, false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, - settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile, SettingUInt64(0), SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, context.getTemporaryPath(), settings.max_threads); @@ -1973,7 +1970,6 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPipeline & pipeline, Modif Aggregator::Params params(header_before_transform, keys, aggregates, false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, - settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile, SettingUInt64(0), SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, context.getTemporaryPath(), settings.max_threads); diff --git a/dbms/src/Interpreters/SpecializedAggregator.h b/dbms/src/Interpreters/SpecializedAggregator.h deleted file mode 100644 index 9a238c77032..00000000000 --- a/dbms/src/Interpreters/SpecializedAggregator.h +++ /dev/null @@ -1,215 +0,0 @@ -#include -#include -#include -#include - - -namespace DB -{ - - -/** An aggregation loop template that allows you to generate a custom variant for a specific combination of aggregate functions. - * It differs from the usual one in that calls to aggregate functions should be inlined, and the update loop of the aggregate functions should be unrolled. - * - * Since there are too many possible combinations, it is not possible to generate them all in advance. - * This template is intended to instantiate it in runtime, - * by running the compiler, compiling shared library, and using it with `dlopen`. - */ - - -struct AggregateFunctionsUpdater -{ - AggregateFunctionsUpdater( - const Aggregator::AggregateFunctionsPlainPtrs & aggregate_functions_, - const Sizes & offsets_of_aggregate_states_, - Aggregator::AggregateColumns & aggregate_columns_, - AggregateDataPtr & value_, - size_t row_num_, - Arena * arena_) - : aggregate_functions(aggregate_functions_), - offsets_of_aggregate_states(offsets_of_aggregate_states_), - aggregate_columns(aggregate_columns_), - value(value_), row_num(row_num_), arena(arena_) - { - } - - template - void operator()() ALWAYS_INLINE; - - const Aggregator::AggregateFunctionsPlainPtrs & aggregate_functions; - const Sizes & offsets_of_aggregate_states; - Aggregator::AggregateColumns & aggregate_columns; - AggregateDataPtr & value; - size_t row_num; - Arena * arena; -}; - -template -void AggregateFunctionsUpdater::operator()() -{ - static_cast(aggregate_functions[column_num])->add( - value + offsets_of_aggregate_states[column_num], - aggregate_columns[column_num].data(), - row_num, arena); -} - -struct AggregateFunctionsCreator -{ - AggregateFunctionsCreator( - const Aggregator::AggregateFunctionsPlainPtrs & aggregate_functions_, - const Sizes & offsets_of_aggregate_states_, - AggregateDataPtr & aggregate_data_) - : aggregate_functions(aggregate_functions_), - offsets_of_aggregate_states(offsets_of_aggregate_states_), - aggregate_data(aggregate_data_) - { - } - - template - void operator()() ALWAYS_INLINE; - - const Aggregator::AggregateFunctionsPlainPtrs & aggregate_functions; - const Sizes & offsets_of_aggregate_states; - AggregateDataPtr & aggregate_data; -}; - -template -void AggregateFunctionsCreator::operator()() -{ - AggregateFunction * func = static_cast(aggregate_functions[column_num]); - - try - { - /** An exception may occur if there is a shortage of memory. - * To ensure that everything is properly destroyed, we "roll back" some of the created states. - * The code is not very convenient. - */ - func->create(aggregate_data + offsets_of_aggregate_states[column_num]); - } - catch (...) - { - for (size_t rollback_j = 0; rollback_j < column_num; ++rollback_j) - func->destroy(aggregate_data + offsets_of_aggregate_states[rollback_j]); - - throw; - } -} - - -template -void NO_INLINE Aggregator::executeSpecialized( - Method & method, - Arena * aggregates_pool, - size_t rows, - ColumnRawPtrs & key_columns, - AggregateColumns & aggregate_columns, - bool no_more_keys, - AggregateDataPtr overflow_row) const -{ - typename Method::State state(key_columns, key_sizes, aggregation_state_cache); - - if (!no_more_keys) - executeSpecializedCase( - method, state, aggregates_pool, rows, aggregate_columns, overflow_row); - else - executeSpecializedCase( - method, state, aggregates_pool, rows, aggregate_columns, overflow_row); -} - -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wuninitialized" - -template -void NO_INLINE Aggregator::executeSpecializedCase( - Method & method, - typename Method::State & state, - Arena * aggregates_pool, - size_t rows, - AggregateColumns & aggregate_columns, - AggregateDataPtr overflow_row) const -{ - /// For all rows. - for (size_t i = 0; i < rows; ++i) - { - AggregateDataPtr aggregate_data = nullptr; - - if (!no_more_keys) /// Insert. - { - auto emplace_result = state.emplaceKey(method.data, i, *aggregates_pool); - - /// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key. - if (emplace_result.isInserted()) - { - /// exception-safety - if you can not allocate memory or create states, then destructors will not be called. - emplace_result.setMapped(nullptr); - - aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); - AggregateFunctionsList::forEach(AggregateFunctionsCreator( - aggregate_functions, offsets_of_aggregate_states, aggregate_data)); - - emplace_result.setMapped(aggregate_data); - } - else - aggregate_data = emplace_result.getMapped(); - } - else - { - /// Add only if the key already exists. - auto find_result = state.findKey(method.data, i, *aggregates_pool); - if (find_result.isFound()) - aggregate_data = find_result.getMapped(); - } - - /// If the key does not fit, and the data does not need to be aggregated in a separate row, then there's nothing to do. - if (!aggregate_data && !overflow_row) - continue; - - auto value = aggregate_data ? aggregate_data : overflow_row; - - /// Add values into the aggregate functions. - AggregateFunctionsList::forEach(AggregateFunctionsUpdater( - aggregate_functions, offsets_of_aggregate_states, aggregate_columns, value, i, aggregates_pool)); - } -} - -#pragma GCC diagnostic pop - -template -void NO_INLINE Aggregator::executeSpecializedWithoutKey( - AggregatedDataWithoutKey & res, - size_t rows, - AggregateColumns & aggregate_columns, - Arena * arena) const -{ - for (size_t i = 0; i < rows; ++i) - { - AggregateFunctionsList::forEach(AggregateFunctionsUpdater( - aggregate_functions, offsets_of_aggregate_states, aggregate_columns, res, i, arena)); - } -} - -} - - -/** The main code is compiled with gcc 7. - * But SpecializedAggregator is compiled using clang 6 into the .so file. - * This is done because gcc can not get functions inlined, - * which were de-virtualized, in a particular case, and the performance is lower. - * And also it's easier to distribute clang for deploy to the servers. - * - * After switching from gcc 4.8 and gnu++1x to gcc 4.9 and gnu++1y (and then to gcc 5), - * an error occurred with `dlopen`: undefined symbol: __cxa_pure_virtual - * - * Most likely, this is due to the changed version of this symbol: - * gcc creates a symbol in .so - * U __cxa_pure_virtual@@CXXABI_1.3 - * but clang creates a symbol - * U __cxa_pure_virtual - * - * But it does not matter for us how the __cxa_pure_virtual function will be implemented, - * because it is not called during normal program execution, - * and if called - then the program is guaranteed buggy. - * - * Therefore, we can work around the problem this way - */ -extern "C" void __attribute__((__visibility__("default"), __noreturn__)) __cxa_pure_virtual() { abort(); } diff --git a/dbms/src/Interpreters/config_compile.h.in b/dbms/src/Interpreters/config_compile.h.in deleted file mode 100644 index e8db534a62d..00000000000 --- a/dbms/src/Interpreters/config_compile.h.in +++ /dev/null @@ -1,26 +0,0 @@ -#pragma once - -#cmakedefine CMAKE_LIBRARY_ARCHITECTURE "@CMAKE_LIBRARY_ARCHITECTURE@" -#cmakedefine PATH_SHARE "@PATH_SHARE@" -#cmakedefine INTERNAL_COMPILER_FLAGS "@INTERNAL_COMPILER_FLAGS@" -#cmakedefine INTERNAL_COMPILER_BIN_ROOT "@INTERNAL_COMPILER_BIN_ROOT@" -#cmakedefine INTERNAL_LINKER_EXECUTABLE "@INTERNAL_LINKER_EXECUTABLE@" -#cmakedefine INTERNAL_COMPILER_EXECUTABLE "@INTERNAL_COMPILER_EXECUTABLE@" -#cmakedefine INTERNAL_COMPILER_ENV "@INTERNAL_COMPILER_ENV@" -#if !defined(INTERNAL_COMPILER_ENV) -# define INTERNAL_COMPILER_ENV "" -#endif -#cmakedefine INTERNAL_COMPILER_HEADERS "@INTERNAL_COMPILER_HEADERS@" -#if !defined(INTERNAL_COMPILER_HEADERS) -# define INTERNAL_COMPILER_HEADERS "" -#endif -#cmakedefine INTERNAL_COMPILER_HEADERS_ROOT "@INTERNAL_COMPILER_HEADERS_ROOT@" -#if !defined(INTERNAL_COMPILER_HEADERS_ROOT) -# define INTERNAL_COMPILER_HEADERS_ROOT "" -#endif - -#cmakedefine01 INTERNAL_COMPILER_CUSTOM_ROOT -#cmakedefine INTERNAL_DOUBLE_CONVERSION_INCLUDE_DIR "@INTERNAL_DOUBLE_CONVERSION_INCLUDE_DIR@" -#cmakedefine INTERNAL_Poco_Foundation_INCLUDE_DIR "@INTERNAL_Poco_Foundation_INCLUDE_DIR@" -#cmakedefine INTERNAL_Poco_Util_INCLUDE_DIR "@INTERNAL_Poco_Util_INCLUDE_DIR@" -#cmakedefine INTERNAL_Boost_INCLUDE_DIRS "@INTERNAL_Boost_INCLUDE_DIRS@" diff --git a/dbms/src/Interpreters/tests/CMakeLists.txt b/dbms/src/Interpreters/tests/CMakeLists.txt index b4f9fff1d36..3fac5424c00 100644 --- a/dbms/src/Interpreters/tests/CMakeLists.txt +++ b/dbms/src/Interpreters/tests/CMakeLists.txt @@ -41,9 +41,6 @@ add_executable (two_level_hash_map two_level_hash_map.cpp) target_include_directories (two_level_hash_map SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) target_link_libraries (two_level_hash_map PRIVATE dbms) -add_executable (compiler_test compiler_test.cpp) -target_link_libraries (compiler_test PRIVATE dbms) - add_executable (logical_expressions_optimizer logical_expressions_optimizer.cpp) target_link_libraries (logical_expressions_optimizer PRIVATE dbms clickhouse_parsers) diff --git a/dbms/src/Interpreters/tests/compiler_test.cpp b/dbms/src/Interpreters/tests/compiler_test.cpp deleted file mode 100644 index c56cf5775d6..00000000000 --- a/dbms/src/Interpreters/tests/compiler_test.cpp +++ /dev/null @@ -1,57 +0,0 @@ -#include -#include - -#include - - -int main(int, char **) -{ - using namespace DB; - - Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); - - /// Check exception handling and catching - try - { - Compiler compiler(".", 1); - - auto lib = compiler.getOrCount("catch_me_if_you_can", 0, "", []() -> std::string - { - return - "#include \n" - "void f() __attribute__((__visibility__(\"default\")));\n" - "void f()" - "{" - "try { throw std::runtime_error(\"Catch me if you can\"); }" - "catch (const std::runtime_error & e) { std::cout << \"Caught in .so: \" << e.what() << std::endl; throw; }\n" - "}" - ; - }, [](SharedLibraryPtr &){}); - - auto f = lib->template get("_Z1fv"); - - try - { - f(); - } - catch (const std::exception & e) - { - std::cout << "Caught in main(): " << e.what() << "\n"; - return 0; - } - catch (...) - { - std::cout << "Unknown exception\n"; - return -1; - } - } - catch (...) - { - std::cerr << getCurrentExceptionMessage(true) << "\n"; - return -1; - } - - return 0; -} diff --git a/dbms/tests/queries/0_stateless/00281_compile_sizeof_packed.reference b/dbms/tests/queries/0_stateless/00281_compile_sizeof_packed.reference deleted file mode 100644 index 207dc069e43..00000000000 --- a/dbms/tests/queries/0_stateless/00281_compile_sizeof_packed.reference +++ /dev/null @@ -1,2 +0,0 @@ -1 Hello -2 Hello diff --git a/dbms/tests/queries/0_stateless/00281_compile_sizeof_packed.sql b/dbms/tests/queries/0_stateless/00281_compile_sizeof_packed.sql deleted file mode 100644 index 5902b94b753..00000000000 --- a/dbms/tests/queries/0_stateless/00281_compile_sizeof_packed.sql +++ /dev/null @@ -1,2 +0,0 @@ -SET compile = 1, min_count_to_compile = 0, max_threads = 1, send_logs_level = 'none'; -SELECT arrayJoin([1, 2, 1]) AS UserID, argMax('Hello', today()) AS res GROUP BY UserID; diff --git a/dbms/tests/queries/0_stateless/00568_compile_catch_throw.reference b/dbms/tests/queries/0_stateless/00568_compile_catch_throw.reference deleted file mode 100644 index 6ed281c757a..00000000000 --- a/dbms/tests/queries/0_stateless/00568_compile_catch_throw.reference +++ /dev/null @@ -1,2 +0,0 @@ -1 -1 diff --git a/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh b/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh deleted file mode 100755 index fbf5efcda2c..00000000000 --- a/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh +++ /dev/null @@ -1,14 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. $CURDIR/../shell_config.sh - -SETTINGS="--compile=1 --min_count_to_compile=0 --max_threads=1 --max_memory_usage=8000000 --server_logs_file=/dev/null" -output=$($CLICKHOUSE_CLIENT -q "SELECT length(groupArray(number)) FROM (SELECT * FROM system.numbers LIMIT 1000000)" $SETTINGS 2>&1) - -[[ $? -eq 0 ]] && echo "Expected non-zero RC" -if ! echo "$output" | grep -Fc -e 'Memory limit (for query) exceeded' -e 'Cannot compile code' ; then - echo -e 'There is no expected exception "Memory limit (for query) exceeded: would use..." or "Cannot compile code..."' "Whereas got:\n$output" -fi - -$CLICKHOUSE_CLIENT -q "SELECT 1" From cc0157b29378cc6af380976850a75d9c026f768c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Aug 2019 02:07:05 +0300 Subject: [PATCH 402/509] Added a test --- dbms/src/Functions/trim.cpp | 2 +- .../queries/0_stateless/00997_trim.reference | 0 dbms/tests/queries/0_stateless/00997_trim.sql | 20 +++++++++++++++++++ 3 files changed, 21 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/00997_trim.reference create mode 100644 dbms/tests/queries/0_stateless/00997_trim.sql diff --git a/dbms/src/Functions/trim.cpp b/dbms/src/Functions/trim.cpp index 81916604d63..46f69530005 100644 --- a/dbms/src/Functions/trim.cpp +++ b/dbms/src/Functions/trim.cpp @@ -85,7 +85,7 @@ private: char_data += num_chars; } - if constexpr (mode::trim_left) + if constexpr (mode::trim_right) { const char * found = find_last_not_symbols_or_null<' '>(char_data, char_end); if (found) diff --git a/dbms/tests/queries/0_stateless/00997_trim.reference b/dbms/tests/queries/0_stateless/00997_trim.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00997_trim.sql b/dbms/tests/queries/0_stateless/00997_trim.sql new file mode 100644 index 00000000000..7519877ec5e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00997_trim.sql @@ -0,0 +1,20 @@ +WITH + '0123456789abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ' AS x, + replaceRegexpAll(x, '.', ' ') AS spaces, + concat(substring(spaces, 1, rand(1) % 62), substring(x, 1, rand(2) % 62), substring(spaces, 1, rand(3) % 62)) AS s, + trimLeft(s) AS sl, + trimRight(s) AS sr, + trimBoth(s) AS t, + replaceRegexpOne(s, '^ +', '') AS slr, + replaceRegexpOne(s, ' +$', '') AS srr, + replaceRegexpOne(s, '^ *(.*?) *$', '\\1') AS tr +SELECT + replaceAll(s, ' ', '_'), + replaceAll(sl, ' ', '_'), + replaceAll(slr, ' ', '_'), + replaceAll(sr, ' ', '_'), + replaceAll(srr, ' ', '_'), + replaceAll(t, ' ', '_'), + replaceAll(tr, ' ', '_') +FROM numbers(100000) +WHERE NOT ((sl = slr) AND (sr = srr) AND (t = tr)) From 7703d321138c2854142f10709cb6451e4b5a024e Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 24 Aug 2019 13:53:22 +0800 Subject: [PATCH 403/509] remove symlinks --- docs/zh/database_engines/index.md | 1 - docs/zh/database_engines/mysql.md | 1 - 2 files changed, 2 deletions(-) delete mode 120000 docs/zh/database_engines/index.md delete mode 120000 docs/zh/database_engines/mysql.md diff --git a/docs/zh/database_engines/index.md b/docs/zh/database_engines/index.md deleted file mode 120000 index bbdb762a4ad..00000000000 --- a/docs/zh/database_engines/index.md +++ /dev/null @@ -1 +0,0 @@ -../../en/database_engines/index.md \ No newline at end of file diff --git a/docs/zh/database_engines/mysql.md b/docs/zh/database_engines/mysql.md deleted file mode 120000 index 51ac4126e2d..00000000000 --- a/docs/zh/database_engines/mysql.md +++ /dev/null @@ -1 +0,0 @@ -../../en/database_engines/mysql.md \ No newline at end of file From 7d7c13632cb3c400e3920f06493d888da3b654e0 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 24 Aug 2019 13:54:01 +0800 Subject: [PATCH 404/509] Translate database engine documentation, update table engine documentation. --- docs/zh/database_engines/index.md | 11 +++ docs/zh/database_engines/mysql.md | 124 ++++++++++++++++++++++++++++++ 2 files changed, 135 insertions(+) create mode 100644 docs/zh/database_engines/index.md create mode 100644 docs/zh/database_engines/mysql.md diff --git a/docs/zh/database_engines/index.md b/docs/zh/database_engines/index.md new file mode 100644 index 00000000000..f8ae05e2520 --- /dev/null +++ b/docs/zh/database_engines/index.md @@ -0,0 +1,11 @@ +# 数据库引擎 + +您使用的所有表都是由数据库引擎所提供的 + +默认情况下,ClickHouse使用自己的数据库引擎,该引擎提供可配置的[表引擎](../operations/table_engines/index.md)和[所有支持的SQL语法](../query_language/syntax.md). + +除此之外,您还可以选择使用以下的数据库引擎: + +- [MySQL](mysql.md) + +[来源文章](https://clickhouse.yandex/docs/en/database_engines/) diff --git a/docs/zh/database_engines/mysql.md b/docs/zh/database_engines/mysql.md new file mode 100644 index 00000000000..38dfcb5ef64 --- /dev/null +++ b/docs/zh/database_engines/mysql.md @@ -0,0 +1,124 @@ +# MySQL + +MySQL引擎用于将远程的MySQL服务器中的表映射到ClickHouse中,并允许您对表进行`INSERT`和`SELECT`查询,以方便您在ClickHouse与MySQL之间进行数据交换。 + +`MySQL`数据库引擎会将对其的查询转换为MySQL语法并发送到MySQL服务器中,因此您可以执行诸如`SHOW TABLES`或`SHOW CREATE TABLE`之类的操作。 + +但您无法对其执行以下操作: + +- `ATTACH`/`DETACH` +- `DROP` +- `RENAME` +- `CREATE TABLE` +- `ALTER` + + +## CREATE DATABASE + +``` sql +CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] +ENGINE = MySQL('host:port', 'database', 'user', 'password') +``` + +**MySQL数据库引擎参数** + +- `host:port` — 链接的MySQL地址。 +- `database` — 链接的MySQL数据库。 +- `user` — 链接的MySQL用户。 +- `password` — 链接的MySQL用户密码。 + + +## 支持的类型对应 + +MySQL | ClickHouse +------|------------ +UNSIGNED TINYINT | [UInt8](../data_types/int_uint.md) +TINYINT | [Int8](../data_types/int_uint.md) +UNSIGNED SMALLINT | [UInt16](../data_types/int_uint.md) +SMALLINT | [Int16](../data_types/int_uint.md) +UNSIGNED INT, UNSIGNED MEDIUMINT | [UInt32](../data_types/int_uint.md) +INT, MEDIUMINT | [Int32](../data_types/int_uint.md) +UNSIGNED BIGINT | [UInt64](../data_types/int_uint.md) +BIGINT | [Int64](../data_types/int_uint.md) +FLOAT | [Float32](../data_types/float.md) +DOUBLE | [Float64](../data_types/float.md) +DATE | [Date](../data_types/date.md) +DATETIME, TIMESTAMP | [DateTime](../data_types/datetime.md) +BINARY | [FixedString](../data_types/fixedstring.md) + +其他的MySQL数据类型将全部都转换为[String](../data_types/string.md)。 + +同时以上的所有类型都支持[Nullable](../data_types/nullable.md)。 + + +## 使用示例 + +在MySQL中创建表: + +``` +mysql> USE test; +Database changed + +mysql> CREATE TABLE `mysql_table` ( + -> `int_id` INT NOT NULL AUTO_INCREMENT, + -> `float` FLOAT NOT NULL, + -> PRIMARY KEY (`int_id`)); +Query OK, 0 rows affected (0,09 sec) + +mysql> insert into mysql_table (`int_id`, `float`) VALUES (1,2); +Query OK, 1 row affected (0,00 sec) + +mysql> select * from mysql_table; ++--------+-------+ +| int_id | value | ++--------+-------+ +| 1 | 2 | ++--------+-------+ +1 row in set (0,00 sec) +``` + +在ClickHouse中创建MySQL类型的数据库,同时与MySQL服务器交换数据: + +```sql +CREATE DATABASE mysql_db ENGINE = MySQL('localhost:3306', 'test', 'my_user', 'user_password') +``` +```sql +SHOW DATABASES +``` +```text +┌─name─────┐ +│ default │ +│ mysql_db │ +│ system │ +└──────────┘ +``` +```sql +SHOW TABLES FROM mysql_db +``` +```text +┌─name─────────┐ +│ mysql_table │ +└──────────────┘ +``` +```sql +SELECT * FROM mysql_db.mysql_table +``` +```text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +└────────┴───────┘ +``` +```sql +INSERT INTO mysql_db.mysql_table VALUES (3,4) +``` +```sql +SELECT * FROM mysql_db.mysql_table +``` +```text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +│ 3 │ 4 │ +└────────┴───────┘ +``` + +[来源文章](https://clickhouse.yandex/docs/en/database_engines/mysql/) From 2464dd0b9fc73deb1d265b16f751e884e291af08 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sat, 24 Aug 2019 11:51:02 +0300 Subject: [PATCH 405/509] fix --- .../queries/0_stateless/00943_materialize_index.sh | 6 +++--- .../0_stateless/00944_clear_index_in_partition.sh | 2 +- .../00975_indices_mutation_replicated_zookeeper.sh | 12 ++++++------ 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00943_materialize_index.sh b/dbms/tests/queries/0_stateless/00943_materialize_index.sh index feab59b368e..bc59b41b005 100755 --- a/dbms/tests/queries/0_stateless/00943_materialize_index.sh +++ b/dbms/tests/queries/0_stateless/00943_materialize_index.sh @@ -39,13 +39,13 @@ SET allow_experimental_data_skipping_indices=1; ALTER TABLE test.minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1;" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" -wait_for_mutation "minmax_idx" "mutation_2.txt" "test" +wait_for_mutation "minmax_idx" "mutation_3.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 2;" -wait_for_mutation "minmax_idx" "mutation_3.txt" "test" +wait_for_mutation "minmax_idx" "mutation_4.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" @@ -58,7 +58,7 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx;" -wait_for_mutation "minmax_idx" "mutation_4.txt" "test" +wait_for_mutation "minmax_idx" "mutation_5.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" diff --git a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh index 5a7bdd8e3ae..74f15e63545 100755 --- a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh +++ b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh @@ -43,7 +43,7 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" -wait_for_mutation "minmax_idx" "mutation_2.txt" "test" +wait_for_mutation "minmax_idx" "mutation_3.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" diff --git a/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh index 5e6159475f8..765dfb6abe5 100755 --- a/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh @@ -45,20 +45,20 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.indices_mutaions1 VALUES (9, 1, 2)" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;" -sleep 0.5 +sleep 1 $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 MATERIALIZE INDEX idx IN PARTITION 1;" -wait_for_mutation "indices_mutaions1" "mutation_2.txt" "test" -wait_for_mutation "indices_mutaions2" "mutation_2.txt" "test" +wait_for_mutation "indices_mutaions1" "0000000000" "test" +wait_for_mutation "indices_mutaions2" "0000000000" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" $CLICKHOUSE_CLIENT --query="DROP TABLE test.indices_mutaions1" $CLICKHOUSE_CLIENT --query="DROP TABLE test.indices_mutaions2" From b7fdfcc7976bc31a822c98fc8d9219f626c54817 Mon Sep 17 00:00:00 2001 From: Weiqing Xu Date: Sat, 24 Aug 2019 17:27:36 +0800 Subject: [PATCH 406/509] fix HDFS HA can't work on DEBUG mode The Describe the bug when the using HDFS HA nameserivce as the uri, the port will be 0. hdfsBuilderSetNameNodePort will be called to set the port. hdfsBuilderSetNameNodePort call asset to check if the port is greater than 0. So in Release mode, it works OK. In the Debug mode, the asset will fail. How to reproduce when compiler the Clickhouse, use DEBUG mode, it will throw error when using HDFS HA nameservice url --- dbms/src/IO/HDFSCommon.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/IO/HDFSCommon.cpp b/dbms/src/IO/HDFSCommon.cpp index 0f1a58942d6..a94fbeabd60 100644 --- a/dbms/src/IO/HDFSCommon.cpp +++ b/dbms/src/IO/HDFSCommon.cpp @@ -40,7 +40,10 @@ HDFSBuilderPtr createHDFSBuilder(const Poco::URI & uri) hdfsBuilderSetUserName(builder.get(), user.c_str()); } hdfsBuilderSetNameNode(builder.get(), host.c_str()); - hdfsBuilderSetNameNodePort(builder.get(), port); + if (port != 0) + { + hdfsBuilderSetNameNodePort(builder.get(), port); + } return builder; } From 720bb3ac08f74dbc7fd42738d8835562973d1671 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Aug 2019 02:07:05 +0300 Subject: [PATCH 407/509] Added a test --- libs/libcommon/include/common/find_symbols.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/libs/libcommon/include/common/find_symbols.h b/libs/libcommon/include/common/find_symbols.h index 920a7df04c5..162c73251fa 100644 --- a/libs/libcommon/include/common/find_symbols.h +++ b/libs/libcommon/include/common/find_symbols.h @@ -17,7 +17,7 @@ * but with the following differencies: * - works with any memory ranges, including containing zero bytes; * - doesn't require terminating zero byte: end of memory range is passed explicitly; - * - if not found, returns pointer to end instead of NULL; + * - if not found, returns pointer to end instead of nullptr; * - maximum number of symbols to search is 16. * * Uses SSE 2 in case of small number of symbols for search and SSE 4.2 in the case of large number of symbols, @@ -188,6 +188,7 @@ inline const char * find_first_symbols_sse42_impl(const char * const begin, cons || (num_chars >= 11 && maybe_negate(*pos == c11)) || (num_chars >= 12 && maybe_negate(*pos == c12)) || (num_chars >= 13 && maybe_negate(*pos == c13)) + || (num_chars >= 14 && maybe_negate(*pos == c14)) || (num_chars >= 15 && maybe_negate(*pos == c15)) || (num_chars >= 16 && maybe_negate(*pos == c16))) return pos; From 60fde1d29eb4f33a1abdc4eb273c278cf6a83f15 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Aug 2019 15:03:54 +0300 Subject: [PATCH 408/509] Removed useless statements from debian directory --- debian/clickhouse-common-static.install | 1 - debian/clickhouse-server.docs | 1 + debian/clickhouse-server.install | 2 -- debian/rules | 16 ---------------- 4 files changed, 1 insertion(+), 19 deletions(-) diff --git a/debian/clickhouse-common-static.install b/debian/clickhouse-common-static.install index 6666b090272..81b1dc4eb1b 100644 --- a/debian/clickhouse-common-static.install +++ b/debian/clickhouse-common-static.install @@ -1,4 +1,3 @@ usr/bin/clickhouse usr/bin/clickhouse-odbc-bridge etc/security/limits.d/clickhouse.conf -usr/share/clickhouse/* diff --git a/debian/clickhouse-server.docs b/debian/clickhouse-server.docs index 95969d08c43..e12d6533be2 100644 --- a/debian/clickhouse-server.docs +++ b/debian/clickhouse-server.docs @@ -1,3 +1,4 @@ LICENSE AUTHORS README.md +CHANGELOG.md diff --git a/debian/clickhouse-server.install b/debian/clickhouse-server.install index f69969a6084..b1475fdf162 100644 --- a/debian/clickhouse-server.install +++ b/debian/clickhouse-server.install @@ -1,6 +1,4 @@ usr/bin/clickhouse-server -usr/bin/clickhouse-clang -usr/bin/clickhouse-lld usr/bin/clickhouse-copier usr/bin/clickhouse-report etc/clickhouse-server/config.xml diff --git a/debian/rules b/debian/rules index a49ffc3f66e..c21f0999bbc 100755 --- a/debian/rules +++ b/debian/rules @@ -32,11 +32,6 @@ endif CMAKE_FLAGS += -DENABLE_UTILS=0 -#DEB_CLANG ?= $(shell which clang-6.0 || which clang-5.0 || which clang-4.0 || which clang || which clang-3.9 || which clang-3.8) - -#DEB_CC ?= gcc-7 -#DEB_CXX ?= g++-7 - ifdef DEB_CXX DEB_BUILD_GNU_TYPE := $(shell dpkg-architecture -qDEB_BUILD_GNU_TYPE) DEB_HOST_GNU_TYPE := $(shell dpkg-architecture -qDEB_HOST_GNU_TYPE) @@ -88,12 +83,8 @@ override_dh_auto_configure: override_dh_auto_build: # Fix for ninja. Do not add -O. $(MAKE) $(THREADS_COUNT) -C $(BUILDDIR) $(MAKE_TARGET) -# #cd $(BUILDDIR) && cmake --build . -- -j$(THREADS_COUNT) # cmake return true on error override_dh_auto_test: -# #TODO, use ENABLE_TESTS=1 -# #./debian/tests_wrapper.sh -# cd $(BUILDDIR) && ctest $(THREADS_COUNT) -V -R GLIBC_required_version cd $(BUILDDIR) && ctest $(THREADS_COUNT) -V -E with_server override_dh_clean: @@ -117,11 +108,6 @@ override_dh_install: mkdir -p $(DESTDIR)/etc/systemd/system/ cp debian/clickhouse-server.service $(DESTDIR)/etc/systemd/system/ - # fake metrika files when private dir is empty - mkdir -p $(DESTDIR)/etc/clickhouse-server/metrika - touch $(DESTDIR)/etc/clickhouse-server/metrika/config.xml - touch $(DESTDIR)/etc/clickhouse-server/metrika/users.xml - dh_install --list-missing --sourcedir=$(DESTDIR) override_dh_auto_install: @@ -130,7 +116,5 @@ override_dh_auto_install: override_dh_shlibdeps: true # We depend only on libc and dh_shlibdeps gives us wrong (too strict) dependency. -#TODO: faster packing of non-release builds: ifdef RELEASE_COMPATIBLE override_dh_builddeb: dh_builddeb -- -Z gzip # Older systems don't have "xz", so use "gzip" instead. -#TODO: endif From cd620d2de517acff4943443a25fac71ac063b068 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Aug 2019 15:18:03 +0300 Subject: [PATCH 409/509] Fixed race condition in test (once again) --- .../queries/0_stateless/00600_replace_running_query.reference | 1 - dbms/tests/queries/0_stateless/00600_replace_running_query.sh | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00600_replace_running_query.reference b/dbms/tests/queries/0_stateless/00600_replace_running_query.reference index 804267a1c11..a01672aae85 100644 --- a/dbms/tests/queries/0_stateless/00600_replace_running_query.reference +++ b/dbms/tests/queries/0_stateless/00600_replace_running_query.reference @@ -2,6 +2,5 @@ 1 1 1 -finished 42 readonly SELECT 2, count() FROM system.numbers 1 44 diff --git a/dbms/tests/queries/0_stateless/00600_replace_running_query.sh b/dbms/tests/queries/0_stateless/00600_replace_running_query.sh index 513f6d8440e..dbbf41dd772 100755 --- a/dbms/tests/queries/0_stateless/00600_replace_running_query.sh +++ b/dbms/tests/queries/0_stateless/00600_replace_running_query.sh @@ -30,7 +30,7 @@ ${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 43' 2>&1 | grep -cF 'is alrea # Trying to replace query of a different user $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL?query_id=42&replace_running_query=1" -d 'SELECT 1' | grep -cF 'is already running by user' -$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = '42' SYNC" +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = '42' SYNC" > /dev/null wait ${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 3, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & From a6997aa83fdb8d0c9ef751337e62e6d3189176a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Aug 2019 16:00:04 +0300 Subject: [PATCH 410/509] Modifications after merging constraints support --- .../CheckConstraintsBlockOutputStream.cpp | 93 ++++++++++--------- .../CheckConstraintsBlockOutputStream.h | 24 ++--- docker/builder/build.sh | 4 +- docs/en/query_language/alter.md | 20 ++-- docs/ru/query_language/alter.md | 22 ++--- docs/ru/query_language/create.md | 4 +- 6 files changed, 79 insertions(+), 88 deletions(-) diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index a84885f2833..3ad72515aca 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -1,34 +1,65 @@ +#include #include -#include -#include #include #include +#include + namespace DB { +namespace ErrorCodes +{ + extern const int VIOLATED_CONSTRAINT; +} + + +CheckConstraintsBlockOutputStream::CheckConstraintsBlockOutputStream( + const String & table_, + const BlockOutputStreamPtr & output_, + const Block & header_, + const ConstraintsDescription & constraints_, + const Context & context_) + : table(table_), + output(output_), + header(header_), + constraints(constraints_), + expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())) +{ +} + + void CheckConstraintsBlockOutputStream::write(const Block & block) { - for (size_t i = 0; i < expressions.size(); ++i) + if (block.rows() > 0) { - Block res = block; - auto constraint_expr = expressions[i]; - auto res_column_uint8 = executeOnBlock(res, constraint_expr); - if (!memoryIsByte(res_column_uint8->getRawDataBegin<1>(), res_column_uint8->byteSize(), 0x1)) + for (size_t i = 0; i < expressions.size(); ++i) { - auto indices_wrong = findAllWrong(res_column_uint8->getRawDataBegin<1>(), res_column_uint8->byteSize()); - std::string indices_str = "{"; - for (size_t j = 0; j < indices_wrong.size(); ++j) - { - indices_str += std::to_string(indices_wrong[j]); - indices_str += (j != indices_wrong.size() - 1) ? ", " : "}"; - } + Block block_to_calculate = block; + auto constraint_expr = expressions[i]; - throw Exception{"Violated constraint " + constraints.constraints[i]->name + - " in table " + table + " at indices " + indices_str + ", constraint expression: " + - serializeAST(*(constraints.constraints[i]->expr), true), ErrorCodes::VIOLATED_CONSTRAINT}; + constraint_expr->execute(block_to_calculate); + ColumnWithTypeAndName res_column = block_to_calculate.getByPosition(block_to_calculate.columns() - 1); + const ColumnUInt8 & res_column_uint8 = assert_cast(*res_column.column); + + const UInt8 * data = res_column_uint8.getData().data(); + size_t size = res_column_uint8.size(); + + /// Is violated. + if (!memoryIsByte(data, size, 1)) + { + size_t row_idx = 0; + for (; row_idx < size; ++row_idx) + if (data[row_idx] != 1) + break; + + throw Exception{"Violated constraint " + constraints.constraints[i]->name + + " in table " + table + " at row " + std::to_string(row_idx) + ", constraint expression: " + + serializeAST(*(constraints.constraints[i]->expr), true), ErrorCodes::VIOLATED_CONSTRAINT}; + } } } + output->write(block); rows_written += block.rows(); } @@ -48,32 +79,4 @@ void CheckConstraintsBlockOutputStream::writeSuffix() output->writeSuffix(); } -const ColumnUInt8 *CheckConstraintsBlockOutputStream::executeOnBlock( - Block & block, - const ExpressionActionsPtr & constraint) -{ - constraint->execute(block); - ColumnWithTypeAndName res_column = block.safeGetByPosition(block.columns() - 1); - return checkAndGetColumn(res_column.column.get()); -} - -std::vector CheckConstraintsBlockOutputStream::findAllWrong(const void *data, size_t size) -{ - std::vector res; - - if (size == 0) - return res; - - auto ptr = reinterpret_cast(data); - - for (size_t i = 0; i < size; ++i) - { - if (*(ptr + i) == 0x0) - { - res.push_back(i); - } - } - - return res; -} } diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h index 1ce8d002062..17b30a0ec4b 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h @@ -2,16 +2,14 @@ #include #include -#include -#include + namespace DB { -namespace ErrorCodes -{ - extern const int VIOLATED_CONSTRAINT; -} +/** Check for constraints violation. If anything is found - throw an exception with detailed error message. + * Otherwise just pass block to output unchanged. + */ class CheckConstraintsBlockOutputStream : public IBlockOutputStream { @@ -21,14 +19,7 @@ public: const BlockOutputStreamPtr & output_, const Block & header_, const ConstraintsDescription & constraints_, - const Context & context_) - : table(table_), - output(output_), - header(header_), - constraints(constraints_), - expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())), - rows_written(0) - { } + const Context & context_); Block getHeader() const override { return header; } void write(const Block & block) override; @@ -39,14 +30,11 @@ public: void writeSuffix() override; private: - const ColumnUInt8* executeOnBlock(Block & block, const ExpressionActionsPtr & constraint); - std::vector findAllWrong(const void *data, size_t size); - String table; BlockOutputStreamPtr output; Block header; const ConstraintsDescription constraints; const ConstraintsExpressions expressions; - size_t rows_written; + size_t rows_written = 0; }; } diff --git a/docker/builder/build.sh b/docker/builder/build.sh index 01841f46295..8af31cb3d43 100755 --- a/docker/builder/build.sh +++ b/docker/builder/build.sh @@ -1,9 +1,9 @@ #!/usr/bin/env bash -#ccache -s +#ccache -s # uncomment to display CCache statistics mkdir -p /server/build_docker cd /server/build_docker -cmake -G Ninja /server -DCMAKE_C_COMPILER=`which clang-8` -DCMAKE_CXX_COMPILER=`which clang++-8` -DCMAKE_BUILD_TYPE=Debug +cmake -G Ninja /server -DCMAKE_C_COMPILER=`which gcc-8` -DCMAKE_CXX_COMPILER=`which g++-8` # Set the number of build jobs to the half of number of virtual CPU cores (rounded up). # By default, ninja use all virtual CPU cores, that leads to very high memory consumption without much improvement in build time. diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index 7aa4ef89e21..84ff4f390a8 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -29,7 +29,7 @@ These actions are described in detail below. ADD COLUMN [IF NOT EXISTS] name [type] [default_expr] [AFTER name_after] ``` -Adds a new column to the table with the specified `name`, `type`, and `default_expr` (see the section [Default expressions](create.md#create-default-values)). +Adds a new column to the table with the specified `name`, `type`, and `default_expr` (see the section [Default expressions](create.md#create-default-values)). If the `IF NOT EXISTS` clause is included, the query won't return an error if the column already exists. If you specify `AFTER name_after` (the name of another column), the column is added after the specified one in the list of table columns. Otherwise, the column is added to the end of the table. Note that there is no way to add a column to the beginning of a table. For a chain of actions, `name_after` can be the name of a column that is added in one of the previous actions. @@ -66,7 +66,7 @@ CLEAR COLUMN [IF EXISTS] name IN PARTITION partition_name ``` Resets all data in a column for a specified partition. Read more about setting the partition name in the section [How to specify the partition expression](#alter-how-to-specify-part-expr). - + If the `IF EXISTS` clause is specified, the query won't return an error if the column doesn't exist. Example: @@ -85,7 +85,7 @@ Adds a comment to the column. If the `IF EXISTS` clause is specified, the query Each column can have one comment. If a comment already exists for the column, a new comment overwrites the previous comment. -Comments are stored in the `comment_expression` column returned by the [DESCRIBE TABLE](misc.md#misc-describe-table) query. +Comments are stored in the `comment_expression` column returned by the [DESCRIBE TABLE](misc.md#misc-describe-table) query. Example: @@ -129,7 +129,7 @@ The `ALTER` query lets you create and delete separate elements (columns) in nest There is no support for deleting columns in the primary key or the sampling key (columns that are used in the `ENGINE` expression). Changing the type for columns that are included in the primary key is only possible if this change does not cause the data to be modified (for example, you are allowed to add values to an Enum or to change a type from `DateTime` to `UInt32`). -If the `ALTER` query is not sufficient to make the table changes you need, you can create a new table, copy the data to it using the [INSERT SELECT](insert_into.md#insert_query_insert-select) query, then switch the tables using the [RENAME](misc.md#misc_operations-rename) query and delete the old table. You can use the [clickhouse-copier](../operations/utils/clickhouse-copier.md) as an alternative to the `INSERT SELECT` query. +If the `ALTER` query is not sufficient to make the table changes you need, you can create a new table, copy the data to it using the [INSERT SELECT](insert_into.md#insert_query_insert-select) query, then switch the tables using the [RENAME](misc.md#misc_operations-rename) query and delete the old table. You can use the [clickhouse-copier](../operations/utils/clickhouse-copier.md) as an alternative to the `INSERT SELECT` query. The `ALTER` query blocks all reads and writes for the table. In other words, if a long `SELECT` is running at the time of the `ALTER` query, the `ALTER` query will wait for it to complete. At the same time, all new queries to the same table will wait while this `ALTER` is running. @@ -178,9 +178,9 @@ ALTER TABLE [db].name DROP CONSTRAINT constraint_name; Queries will add or remove metadata about constraints from table so they are processed immediately. -Constraint check *will not be executed* on existing table if it was added. For now, we recommend to create new table and use `INSERT SELECT` query to fill new table. +Constraint check *will not be executed* on existing data if it was added. -All changes on distributed tables are broadcasting to ZooKeeper so will be applied on other replicas. +All changes on replicated tables are broadcasting to ZooKeeper so will be applied on other replicas. ### Manipulations With Partitions and Parts {#alter_manipulations-with-partitions} @@ -267,7 +267,7 @@ This query copies the data partition from the `table1` to `table2`. Note that da For the query to run successfully, the following conditions must be met: - Both tables must have the same structure. -- Both tables must have the same partition key. +- Both tables must have the same partition key. #### CLEAR COLUMN IN PARTITION {#alter_clear-column-partition} @@ -289,13 +289,13 @@ ALTER TABLE visits CLEAR COLUMN hour in PARTITION 201902 ALTER TABLE table_name FREEZE [PARTITION partition_expr] ``` -This query creates a local backup of a specified partition. If the `PARTITION` clause is omitted, the query creates the backup of all partitions at once. +This query creates a local backup of a specified partition. If the `PARTITION` clause is omitted, the query creates the backup of all partitions at once. Note that for old-styled tables you can specify the prefix of the partition name (for example, '2019') - then the query creates the backup for all the corresponding partitions. Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). !!! note The entire backup process is performed without stopping the server. - + At the time of execution, for a data snapshot, the query creates hardlinks to a table data. Hardlinks are placed in the directory `/var/lib/clickhouse/shadow/N/...`, where: - `/var/lib/clickhouse/` is the working ClickHouse directory specified in the config. @@ -348,7 +348,7 @@ ALTER TABLE users ATTACH PARTITION 201902; ``` Note that: -- The `ALTER ... FETCH PARTITION` query isn't replicated. It places the partition to the `detached` directory only on the local server. +- The `ALTER ... FETCH PARTITION` query isn't replicated. It places the partition to the `detached` directory only on the local server. - The `ALTER TABLE ... ATTACH` query is replicated. It adds the data to all replicas. The data is added to one of the replicas from the `detached` directory, and to the others - from neighboring replicas. Before downloading, the system checks if the partition exists and the table structure matches. The most appropriate replica is selected automatically from the healthy replicas. diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index 30fbcbc81ac..c078d35f805 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -29,7 +29,7 @@ ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN ADD COLUMN [IF NOT EXISTS] name [type] [default_expr] [AFTER name_after] ``` -Добавляет в таблицу новый столбец с именем `name`, типом `type` и выражением для умолчания `default_expr` (смотрите раздел [Значения по умолчанию](create.md#create-default-values)). +Добавляет в таблицу новый столбец с именем `name`, типом `type` и выражением для умолчания `default_expr` (смотрите раздел [Значения по умолчанию](create.md#create-default-values)). Если указано `IF NOT EXISTS`, запрос не будет возвращать ошибку, если столбец уже существует. Если указано `AFTER name_after` (имя другого столбца), то столбец добавляется (в список столбцов таблицы) после указанного. Иначе, столбец добавляется в конец таблицы. Обратите внимание, ClickHouse не позволяет добавлять столбцы в начало таблицы. Для цепочки действий, `name_after` может быть именем столбца, который добавляется в одном из предыдущих действий. @@ -84,7 +84,7 @@ COMMENT COLUMN [IF EXISTS] name 'Text comment' Каждый столбец может содержать только один комментарий. При выполнении запроса существующий комментарий заменяется на новый. -Посмотреть комментарии можно в столбце `comment_expression` из запроса [DESCRIBE TABLE](misc.md#misc-describe-table). +Посмотреть комментарии можно в столбце `comment_expression` из запроса [DESCRIBE TABLE](misc.md#misc-describe-table). Пример: @@ -177,9 +177,9 @@ ALTER TABLE [db].name DROP CONSTRAINT constraint_name; Запросы выполняют добавление или удаление метаданных об ограничениях таблицы `[db].name`, поэтому выполняются мнгновенно. -Если ограничение появилось для непустой таблицы, то *проверка ограничения вызвана не будет*. Если же важно добавить ограничение на существующую таблицу, то рекомендуется создать новую таблицу с нужным ограничением и выполнить `INSERT SELECT` запрос для перекачки данных из одной таблицы в другую. +Если ограничение появилось для непустой таблицы, то *проверка ограничения для имеющихся данных не производится*. -Запрос на изменение ограничений так же, как и с индексами, реплицируется через ZooKeeper. +Запрос на изменение ограничений для Replicated таблиц реплицируется, сохраняя новые метаданные в ZooKeeper и применяя изменения на всех репликах. ### Манипуляции с партициями и кусками {#alter_manipulations-with-partitions} @@ -260,7 +260,7 @@ ALTER TABLE visits ATTACH PART 201901_2_2_0; ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1 ``` -Копирует партицию из таблицы `table1` в таблицу `table2`. Данные из `table1` не удаляются. +Копирует партицию из таблицы `table1` в таблицу `table2`. Данные из `table1` не удаляются. Следует иметь в виду: @@ -297,19 +297,19 @@ ALTER TABLE table_name CLEAR INDEX index_name IN PARTITION partition_expr ALTER TABLE table_name FREEZE [PARTITION partition_expr] ``` -Создаёт резервную копию для заданной партиции. Если выражение `PARTITION` опущено, резервные копии будут созданы для всех партиций. +Создаёт резервную копию для заданной партиции. Если выражение `PARTITION` опущено, резервные копии будут созданы для всех партиций. !!! note - Создание резервной копии не требует остановки сервера. - + Создание резервной копии не требует остановки сервера. + Для таблиц старого стиля имя партиций можно задавать в виде префикса (например, '2019'). В этом случае резервные копии будут созданы для всех соответствующих партиций. Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr). - + Запрос делает следующее — для текущего состояния таблицы он формирует жесткие ссылки на данные в этой таблице. Ссылки размещаются в директории `/var/lib/clickhouse/shadow/N/...`, где: - `/var/lib/clickhouse/` — рабочая директория ClickHouse, заданная в конфигурационном файле; - `N` — инкрементальный номер резервной копии. -Структура директорий внутри резервной копии такая же, как внутри `/var/lib/clickhouse/`. Запрос выполнит 'chmod' для всех файлов, запрещая запись в них. +Структура директорий внутри резервной копии такая же, как внутри `/var/lib/clickhouse/`. Запрос выполнит 'chmod' для всех файлов, запрещая запись в них. Обратите внимание, запрос `ALTER TABLE t FREEZE PARTITION` не реплицируется. Он создает резервную копию только на локальном сервере. После создания резервной копии данные из `/var/lib/clickhouse/shadow/` можно скопировать на удалённый сервер, а локальную копию удалить. @@ -357,7 +357,7 @@ ALTER TABLE users ATTACH PARTITION 201902; #### Как задавать имя партиции в запросах ALTER {#alter-how-to-specify-part-expr} -Чтобы задать нужную партицию в запросах `ALTER ... PARTITION`, можно использовать: +Чтобы задать нужную партицию в запросах `ALTER ... PARTITION`, можно использовать: - Имя партиции. Посмотреть имя партиции можно в столбце `partition` системной таблицы [system.parts](../operations/system_tables.md#system_tables-parts). Например, `ALTER TABLE visits DETACH PARTITION 201901`. - Произвольное выражение из столбцов исходной таблицы. Также поддерживаются константы и константные выражения. Например, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))`. diff --git a/docs/ru/query_language/create.md b/docs/ru/query_language/create.md index e3e1c901204..2e0a21cda0b 100644 --- a/docs/ru/query_language/create.md +++ b/docs/ru/query_language/create.md @@ -119,9 +119,9 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ) ENGINE = engine ``` -`boolean_expr_1` может быть любым булевым выражением, состоящим из операторов сравнения или функций. При наличии одного или нескольких ограничений в момент вставки данных выражения ограничений будут проверяться на истинность для каждой вставляемой строки данных. В случае, если в теле INSERT запроса придут некорректные данные — клиентов будет выкинуто исключение с нарушенным ограничением. +`boolean_expr_1` может быть любым булевым выражением, состоящим из операторов сравнения или функций. При наличии одного или нескольких ограничений в момент вставки данных выражения ограничений будут проверяться на истинность для каждой вставляемой строки данных. В случае, если в теле INSERT запроса придут некорректные данные — клиент получит исключение с описанием нарушенного ограничения. -Добавление большого числа ограничений может негативно повлиять на производительность объёмных `INSERT` запросов. +Добавление большого числа ограничений может негативно повлиять на производительность `INSERT` запросов. ### Выражение для TTL From 3fef8e92ad72c3d510e077c24838c2f103342a22 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Aug 2019 00:20:20 +0300 Subject: [PATCH 411/509] Merging CONSTRAINTS support --- dbms/programs/server/TCPHandler.cpp | 2 +- dbms/src/Core/ExternalTable.cpp | 2 +- .../CheckConstraintsBlockOutputStream.cpp | 5 +- dbms/src/Databases/DatabaseMySQL.cpp | 2 +- dbms/src/Databases/DatabasesCommon.cpp | 3 +- .../Interpreters/GlobalSubqueriesVisitor.h | 2 +- .../Interpreters/InterpreterCreateQuery.cpp | 22 ++++-- .../src/Interpreters/InterpreterCreateQuery.h | 6 +- .../Interpreters/InterpreterInsertQuery.cpp | 7 +- .../Interpreters/InterpreterSystemQuery.cpp | 2 + dbms/src/Storages/IStorage.cpp | 14 +--- dbms/src/Storages/IStorage.h | 4 +- dbms/src/Storages/Kafka/StorageKafka.cpp | 2 +- .../src/Storages/LiveView/StorageLiveView.cpp | 4 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 1 - .../MergeTree/StorageFromMergeTreeDataPart.h | 7 +- .../MergeTree/registerStorageMergeTree.cpp | 41 +++++------ dbms/src/Storages/StorageBuffer.cpp | 9 ++- dbms/src/Storages/StorageBuffer.h | 3 +- dbms/src/Storages/StorageDictionary.cpp | 4 +- dbms/src/Storages/StorageDistributed.cpp | 17 +++-- dbms/src/Storages/StorageDistributed.h | 2 + dbms/src/Storages/StorageFactory.cpp | 2 + dbms/src/Storages/StorageFactory.h | 3 + dbms/src/Storages/StorageFile.cpp | 8 ++- dbms/src/Storages/StorageFile.h | 1 + dbms/src/Storages/StorageHDFS.cpp | 8 ++- dbms/src/Storages/StorageHDFS.h | 1 + dbms/src/Storages/StorageJoin.cpp | 4 +- dbms/src/Storages/StorageJoin.h | 1 + dbms/src/Storages/StorageLog.cpp | 9 ++- dbms/src/Storages/StorageLog.h | 1 + dbms/src/Storages/StorageMaterializedView.cpp | 4 +- dbms/src/Storages/StorageMaterializedView.h | 1 + dbms/src/Storages/StorageMemory.cpp | 8 ++- dbms/src/Storages/StorageMemory.h | 2 +- dbms/src/Storages/StorageMerge.cpp | 3 +- dbms/src/Storages/StorageMySQL.cpp | 7 +- dbms/src/Storages/StorageMySQL.h | 2 +- dbms/src/Storages/StorageNull.cpp | 2 +- dbms/src/Storages/StorageNull.h | 6 +- dbms/src/Storages/StorageSet.cpp | 15 ++-- dbms/src/Storages/StorageSet.h | 6 +- dbms/src/Storages/StorageStripeLog.cpp | 9 ++- dbms/src/Storages/StorageStripeLog.h | 1 + dbms/src/Storages/StorageTinyLog.cpp | 9 ++- dbms/src/Storages/StorageTinyLog.h | 1 + dbms/src/Storages/StorageURL.cpp | 12 ++-- dbms/src/Storages/StorageURL.h | 24 ++++--- dbms/src/Storages/StorageView.cpp | 4 +- dbms/src/Storages/StorageXDBC.cpp | 3 +- dbms/src/Storages/StorageXDBC.h | 72 ++++++++++--------- dbms/src/TableFunctions/TableFunctionFile.cpp | 1 + dbms/src/TableFunctions/TableFunctionHDFS.cpp | 1 + .../src/TableFunctions/TableFunctionMySQL.cpp | 1 + dbms/src/TableFunctions/TableFunctionURL.cpp | 3 +- 56 files changed, 243 insertions(+), 153 deletions(-) diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 5091258acaf..5e916ccbddd 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -865,7 +865,7 @@ bool TCPHandler::receiveData() if (!(storage = query_context->tryGetExternalTable(external_table_name))) { NamesAndTypesList columns = block.getNamesAndTypesList(); - storage = StorageMemory::create("_external", external_table_name, ColumnsDescription{columns}); + storage = StorageMemory::create("_external", external_table_name, ColumnsDescription{columns}, ConstraintsDescription{}); storage->startup(); query_context->addExternalTable(external_table_name, storage); } diff --git a/dbms/src/Core/ExternalTable.cpp b/dbms/src/Core/ExternalTable.cpp index e1e059a3b63..8befb4e054c 100644 --- a/dbms/src/Core/ExternalTable.cpp +++ b/dbms/src/Core/ExternalTable.cpp @@ -160,7 +160,7 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, /// Create table NamesAndTypesList columns = sample_block.getNamesAndTypesList(); - StoragePtr storage = StorageMemory::create("_external", data.second, ColumnsDescription{columns}); + StoragePtr storage = StorageMemory::create("_external", data.second, ColumnsDescription{columns}, ConstraintsDescription{}); storage->startup(); context.addExternalTable(data.second, storage); BlockOutputStreamPtr output = storage->write(ASTPtr(), context); diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 3ad72515aca..8fbb5916372 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -33,8 +33,11 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) { if (block.rows() > 0) { + std::cerr << "Checking " << expressions.size() << " constraints\n"; for (size_t i = 0; i < expressions.size(); ++i) { + std::cerr << serializeAST(*(constraints.constraints[i]->expr), true) << "\n"; + Block block_to_calculate = block; auto constraint_expr = expressions[i]; @@ -54,7 +57,7 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) break; throw Exception{"Violated constraint " + constraints.constraints[i]->name + - " in table " + table + " at row " + std::to_string(row_idx) + ", constraint expression: " + + " in table " + table + " at row " + std::to_string(rows_written + row_idx + 1) + ", constraint expression: " + serializeAST(*(constraints.constraints[i]->expr), true), ErrorCodes::VIOLATED_CONSTRAINT}; } } diff --git a/dbms/src/Databases/DatabaseMySQL.cpp b/dbms/src/Databases/DatabaseMySQL.cpp index e9dd5c0dacc..9fd16cc94a8 100644 --- a/dbms/src/Databases/DatabaseMySQL.cpp +++ b/dbms/src/Databases/DatabaseMySQL.cpp @@ -231,7 +231,7 @@ DatabaseMySQL::MySQLStorageInfo DatabaseMySQL::createStorageInfo( { const auto & mysql_table = StorageMySQL::create( database_name, table_name, std::move(mysql_pool), mysql_database_name, table_name, - false, "", ColumnsDescription{columns_name_and_type}, global_context); + false, "", ColumnsDescription{columns_name_and_type}, ConstraintsDescription{}, global_context); const auto & create_table_query = std::make_shared(); diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index af43b25fbdb..1a278fbb03d 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -83,6 +83,7 @@ std::pair createTableFromDefinition( throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context); + ConstraintsDescription constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints); return { @@ -90,7 +91,7 @@ std::pair createTableFromDefinition( StorageFactory::instance().get( ast_create_query, database_data_path, ast_create_query.table, database_name, context, context.getGlobalContext(), - columns, + columns, constraints, true, has_force_restore_data_flag) }; } diff --git a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h index 583ec026af0..926e6afd1c2 100644 --- a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h @@ -91,7 +91,7 @@ public: Block sample = interpreter->getSampleBlock(); NamesAndTypesList columns = sample.getNamesAndTypesList(); - StoragePtr external_storage = StorageMemory::create("_external", external_table_name, ColumnsDescription{columns}); + StoragePtr external_storage = StorageMemory::create("_external", external_table_name, ColumnsDescription{columns}, ConstraintsDescription{}); external_storage->startup(); /** We replace the subquery with the name of the temporary table. diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 555a0951f58..cb5e657bcea 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -381,26 +381,37 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpres } -ColumnsDescription InterpreterCreateQuery::setColumns( +ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const ASTExpressionList * constraints) +{ + ConstraintsDescription res; + if (constraints) + for (const auto & constraint : constraints->children) + res.constraints.push_back(std::dynamic_pointer_cast(constraint->clone())); + return res; +} + + +ColumnsDescription InterpreterCreateQuery::setProperties( ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const { ColumnsDescription columns; IndicesDescription indices; ConstraintsDescription constraints; - if (create.columns_list) { if (create.columns_list->columns) columns = getColumnsDescription(*create.columns_list->columns, context); + if (create.columns_list->indices) for (const auto & index : create.columns_list->indices->children) indices.indices.push_back( std::dynamic_pointer_cast(index->clone())); + if (create.columns_list->constraints) for (const auto & constraint : create.columns_list->constraints->children) constraints.constraints.push_back( - std::dynamic_pointer_cast(constraint->clone())); + std::dynamic_pointer_cast(constraint->clone())); } else if (!create.as_table.empty()) { @@ -567,7 +578,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) else { /// Set and retrieve list of columns. - columns = setColumns(create, as_select_sample, as_storage); + columns = setProperties(create, as_select_sample, as_storage); /// Check low cardinality types in creating table if it was not allowed in setting if (!create.attach && !context.getSettingsRef().allow_suspicious_low_cardinality_types && !create.is_materialized_view) @@ -587,6 +598,8 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) setEngine(create); } + ConstraintsDescription constraints = getConstraintsDescription(create.columns_list->constraints); + { std::unique_ptr guard; @@ -635,6 +648,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) context, context.getGlobalContext(), columns, + constraints, create.attach, false); } diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.h b/dbms/src/Interpreters/InterpreterCreateQuery.h index 23b1e3cdabc..2472f0321e1 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.h +++ b/dbms/src/Interpreters/InterpreterCreateQuery.h @@ -14,6 +14,7 @@ namespace DB class Context; class ASTCreateQuery; class ASTExpressionList; +class ASTConstraintDeclaration; /** Allows to create new table or database, @@ -45,13 +46,14 @@ public: /// Obtain information about columns, their types, default values and column comments, for case when columns in CREATE query is specified explicitly. static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, const Context & context); + static ConstraintsDescription getConstraintsDescription(const ASTExpressionList * constraints); private: BlockIO createDatabase(ASTCreateQuery & create); BlockIO createTable(ASTCreateQuery & create); - /// Calculate list of columns of table and return it. - ColumnsDescription setColumns(ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const; + /// Calculate list of columns, constraints, indices, etc... of table and return columns. + ColumnsDescription setProperties(ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const; void setEngine(ASTCreateQuery & create) const; void checkAccess(const ASTCreateQuery & create); diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 4033bea7f2a..549e06e0155 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -121,8 +121,11 @@ BlockIO InterpreterInsertQuery::execute() out = std::make_shared( out, query_sample_block, out->getHeader(), table->getColumns().getDefaults(), context); - out = std::make_shared( - query.table, out, query_sample_block, table->getConstraints(), context); + std::cerr << table->getConstraints().toString() << "\n"; + + if (const auto & constraints = table->getConstraints(); !constraints.empty()) + out = std::make_shared(query.table, + out, query_sample_block, table->getConstraints(), context); auto out_wrapper = std::make_shared(out); out_wrapper->setProcessListElement(context.getProcessListElement()); diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 0797a3deab4..6e434189c66 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -278,6 +278,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam std::string data_path = database->getDataPath(); auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns_list->columns, system_context); + auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints); StoragePtr table = StorageFactory::instance().get(create, data_path, @@ -286,6 +287,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam system_context, system_context.getGlobalContext(), columns, + constraints, create.attach, false); diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 099462a66cd..b28a3b823cb 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -20,20 +20,8 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; } -IStorage::IStorage(ColumnsDescription columns_) +IStorage::IStorage(ColumnsDescription virtuals_) : virtuals(std::move(virtuals_)) { - setColumns(std::move(columns_)); -} - -IStorage::IStorage(ColumnsDescription columns_, ColumnsDescription virtuals_) : virtuals(std::move(virtuals_)) -{ - setColumns(std::move(columns_)); -} - -IStorage::IStorage(ColumnsDescription columns_, ColumnsDescription virtuals_, IndicesDescription indices_) : virtuals(std::move(virtuals_)) -{ - setColumns(std::move(columns_)); - setIndices(std::move(indices_)); } const ColumnsDescription & IStorage::getColumns() const diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 552b234eea2..201107ec62e 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -64,9 +64,7 @@ class IStorage : public std::enable_shared_from_this { public: IStorage() = default; - explicit IStorage(ColumnsDescription columns_); - IStorage(ColumnsDescription columns_, ColumnsDescription virtuals_); - IStorage(ColumnsDescription columns_, ColumnsDescription virtuals_, IndicesDescription indices_); + explicit IStorage(ColumnsDescription virtuals_); virtual ~IStorage() = default; IStorage(const IStorage &) = delete; diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index e429ef505de..db8e49ca7a9 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -85,7 +85,6 @@ StorageKafka::StorageKafka( size_t skip_broken_, bool intermediate_commit_) : IStorage( - columns_, ColumnsDescription({{"_topic", std::make_shared()}, {"_key", std::make_shared()}, {"_offset", std::make_shared()}, @@ -107,6 +106,7 @@ StorageKafka::StorageKafka( , skip_broken(skip_broken_) , intermediate_commit(intermediate_commit_) { + setColumns(columns_); task = global_context.getSchedulePool().createTask(log->name(), [this]{ threadFunc(); }); task->deactivate(); } diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 3c0d205fa3f..7704f421517 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -218,9 +218,11 @@ StorageLiveView::StorageLiveView( Context & local_context, const ASTCreateQuery & query, const ColumnsDescription & columns_) - : IStorage(columns_), table_name(table_name_), + : table_name(table_name_), database_name(database_name_), global_context(local_context.getGlobalContext()) { + setColumns(columns_); + if (!query.select) throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 3f22e6afaeb..2e191dacfd5 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -124,7 +124,6 @@ MergeTreeData::MergeTreeData( data_parts_by_state_and_info(data_parts_indexes.get()) { setProperties(order_by_ast_, primary_key_ast_, columns_, indices_, constraints_); - setConstraints(constraints_); /// NOTE: using the same columns list as is read when performing actual merges. merging_params.check(getColumns().getAllPhysical()); diff --git a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 83582c9779f..32268499a01 100644 --- a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -40,8 +40,11 @@ public: protected: StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) - : IStorage(part_->storage.getColumns(), part_->storage.getVirtuals(), part_->storage.getIndices()), part(part_) - {} + : IStorage(part_->storage.getVirtuals()), part(part_) + { + setColumns(part_->storage.getColumns()); + setIndices(part_->storage.getIndices()); + } private: MergeTreeData::DataPartPtr part; diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 9e45f3352c6..41aae981a5f 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -422,21 +422,21 @@ static StoragePtr create(const StorageFactory::Arguments & args) switch (merging_params.mode) { - default: - break; - case MergeTreeData::MergingParams::Summing: - add_optional_param("list of columns to sum"); - break; - case MergeTreeData::MergingParams::Replacing: - add_optional_param("version"); - break; - case MergeTreeData::MergingParams::Collapsing: - add_mandatory_param("sign column"); - break; - case MergeTreeData::MergingParams::Graphite: - add_mandatory_param("'config_element_for_graphite_schema'"); - break; - case MergeTreeData::MergingParams::VersionedCollapsing: + default: + break; + case MergeTreeData::MergingParams::Summing: + add_optional_param("list of columns to sum"); + break; + case MergeTreeData::MergingParams::Replacing: + add_optional_param("version"); + break; + case MergeTreeData::MergingParams::Collapsing: + add_mandatory_param("sign column"); + break; + case MergeTreeData::MergingParams::Graphite: + add_mandatory_param("'config_element_for_graphite_schema'"); + break; + case MergeTreeData::MergingParams::VersionedCollapsing: { add_mandatory_param("sign column"); add_mandatory_param("version"); @@ -574,7 +574,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) ASTPtr sample_by_ast; ASTPtr ttl_table_ast; IndicesDescription indices_description; - ConstraintsDescription constraints_description; MergeTreeSettings storage_settings = args.context.getMergeTreeSettings(); if (is_extended_storage_def) @@ -601,12 +600,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.query.columns_list && args.query.columns_list->indices) for (const auto & index : args.query.columns_list->indices->children) indices_description.indices.push_back( - std::dynamic_pointer_cast(index->clone())); + std::dynamic_pointer_cast(index->clone())); - if (args.query.columns_list && args.query.columns_list->constraints) - for (const auto & constraint : args.query.columns_list->constraints->children) - constraints_description.constraints.push_back( - std::dynamic_pointer_cast(constraint->clone())); storage_settings.loadFromQuery(*args.storage_def); } else @@ -643,14 +638,14 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) return StorageReplicatedMergeTree::create( zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name, - args.columns, indices_description, constraints_description, + args.columns, indices_description, args.constraints, args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast, sample_by_ast, ttl_table_ast, merging_params, storage_settings, args.has_force_restore_data_flag); else return StorageMergeTree::create( args.data_path, args.database_name, args.table_name, args.columns, indices_description, - constraints_description, args.attach, args.context, date_column_name, partition_by_ast, order_by_ast, + args.constraints, args.attach, args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast, sample_by_ast, ttl_table_ast, merging_params, storage_settings, args.has_force_restore_data_flag); } diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index cef1c966951..b78cfdf3468 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -55,11 +55,12 @@ namespace ErrorCodes } -StorageBuffer::StorageBuffer(const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, +StorageBuffer::StorageBuffer(const std::string & database_name_, const std::string & table_name_, + const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, Context & context_, size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_, const String & destination_database_, const String & destination_table_, bool allow_materialized_) - : IStorage{columns_}, + : table_name(table_name_), database_name(database_name_), global_context(context_), num_shards(num_shards_), buffers(num_shards_), min_thresholds(min_thresholds_), max_thresholds(max_thresholds_), @@ -67,6 +68,8 @@ StorageBuffer::StorageBuffer(const std::string & database_name_, const std::stri no_destination(destination_database.empty() && destination_table.empty()), allow_materialized(allow_materialized_), log(&Logger::get("StorageBuffer (" + table_name + ")")) { + setColumns(columns_); + setConstraints(constraints_); } StorageBuffer::~StorageBuffer() @@ -743,7 +746,7 @@ void registerStorageBuffer(StorageFactory & factory) return StorageBuffer::create( args.database_name, - args.table_name, args.columns, + args.table_name, args.columns, args.constraints, args.context, num_buckets, StorageBuffer::Thresholds{min_time, min_rows, min_bytes}, diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index 4c317a7a102..8060199d20d 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -140,7 +140,8 @@ protected: /** num_shards - the level of internal parallelism (the number of independent buffers) * The buffer is flushed if all minimum thresholds or at least one of the maximum thresholds are exceeded. */ - StorageBuffer(const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, + StorageBuffer(const std::string & database_name_, const std::string & table_name_, + const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, Context & context_, size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_, const String & destination_database_, const String & destination_table_, bool allow_materialized_); diff --git a/dbms/src/Storages/StorageDictionary.cpp b/dbms/src/Storages/StorageDictionary.cpp index 923a309f1f6..ced0025e36c 100644 --- a/dbms/src/Storages/StorageDictionary.cpp +++ b/dbms/src/Storages/StorageDictionary.cpp @@ -30,11 +30,13 @@ StorageDictionary::StorageDictionary( const Context & context, bool attach, const String & dictionary_name_) - : IStorage{columns_}, table_name(table_name_), + : table_name(table_name_), database_name(database_name_), dictionary_name(dictionary_name_), logger(&Poco::Logger::get("StorageDictionary")) { + setColumns(columns_); + if (!attach) { const auto & dictionary = context.getExternalDictionaries().getDictionary(dictionary_name); diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 7c6300bd1a6..7c20622d89e 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -209,6 +209,7 @@ StorageDistributed::StorageDistributed( const String & database_name_, const String & table_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, const String & remote_database_, const String & remote_table_, const String & cluster_name_, @@ -216,13 +217,16 @@ StorageDistributed::StorageDistributed( const ASTPtr & sharding_key_, const String & data_path_, bool attach_) - : IStorage{columns_}, table_name(table_name_), database_name(database_name_), + : table_name(table_name_), database_name(database_name_), remote_database(remote_database_), remote_table(remote_table_), global_context(context_), cluster_name(global_context.getMacros()->expand(cluster_name_)), has_sharding_key(sharding_key_), sharding_key_expr(sharding_key_ ? buildShardingKeyExpression(sharding_key_, global_context, getColumns().getAllPhysical(), false) : nullptr), sharding_key_column_name(sharding_key_ ? sharding_key_->getColumnName() : String{}), path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(table_name) + '/')) { + setColumns(columns_); + setConstraints(constraints_); + /// Sanity check. Skip check if the table is already created to allow the server to start. if (!attach_ && !cluster_name.empty()) { @@ -237,15 +241,16 @@ StorageDistributed::StorageDistributed( const String & database_name_, const String & table_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, ASTPtr remote_table_function_ptr_, const String & cluster_name_, const Context & context_, const ASTPtr & sharding_key_, const String & data_path_, bool attach) - : StorageDistributed(database_name_, table_name_, columns_, String{}, String{}, cluster_name_, context_, sharding_key_, data_path_, attach) + : StorageDistributed(database_name_, table_name_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, data_path_, attach) { - remote_table_function_ptr = remote_table_function_ptr_; + remote_table_function_ptr = remote_table_function_ptr_; } @@ -258,7 +263,7 @@ StoragePtr StorageDistributed::createWithOwnCluster( const Context & context_) { auto res = ext::shared_ptr_helper::create( - String{}, table_name_, columns_, remote_database_, remote_table_, String{}, context_, ASTPtr(), String(), false); + String{}, table_name_, columns_, ConstraintsDescription{}, remote_database_, remote_table_, String{}, context_, ASTPtr(), String(), false); res->owned_cluster = owned_cluster_; @@ -274,7 +279,7 @@ StoragePtr StorageDistributed::createWithOwnCluster( const Context & context_) { auto res = ext::shared_ptr_helper::create( - String{}, table_name_, columns_, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), false); + String{}, table_name_, columns_, ConstraintsDescription{}, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), false); res->owned_cluster = owned_cluster_; @@ -634,7 +639,7 @@ void registerStorageDistributed(StorageFactory & factory) } return StorageDistributed::create( - args.database_name, args.table_name, args.columns, + args.database_name, args.table_name, args.columns, args.constraints, remote_database, remote_table, cluster_name, args.context, sharding_key, args.data_path, args.attach); diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index 6885a758e9e..e8da6dca4a7 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -161,6 +161,7 @@ protected: const String & database_name_, const String & table_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, const String & remote_database_, const String & remote_table_, const String & cluster_name_, @@ -173,6 +174,7 @@ protected: const String & database_name, const String & table_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, ASTPtr remote_table_function_ptr_, const String & cluster_name_, const Context & context_, diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index 862f76bc3ce..dc73e8a53b8 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -46,6 +46,7 @@ StoragePtr StorageFactory::get( Context & local_context, Context & context, const ColumnsDescription & columns, + const ConstraintsDescription & constraints, bool attach, bool has_force_restore_data_flag) const { @@ -154,6 +155,7 @@ StoragePtr StorageFactory::get( .local_context = local_context, .context = context, .columns = columns, + .constraints = constraints, .attach = attach, .has_force_restore_data_flag = has_force_restore_data_flag }; diff --git a/dbms/src/Storages/StorageFactory.h b/dbms/src/Storages/StorageFactory.h index 2c97fb798ee..43ffc5a638e 100644 --- a/dbms/src/Storages/StorageFactory.h +++ b/dbms/src/Storages/StorageFactory.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -35,6 +36,7 @@ public: Context & local_context; Context & context; const ColumnsDescription & columns; + const ConstraintsDescription & constraints; bool attach; bool has_force_restore_data_flag; }; @@ -49,6 +51,7 @@ public: Context & local_context, Context & context, const ColumnsDescription & columns, + const ConstraintsDescription & constraints, bool attach, bool has_force_restore_data_flag) const; diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 5162e667133..2db24bbd610 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -72,10 +72,14 @@ StorageFile::StorageFile( const std::string & table_name_, const std::string & format_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, Context & context_) - : IStorage(columns_), + : table_name(table_name_), database_name(database_name_), format_name(format_name_), context_global(context_), table_fd(table_fd_) { + setColumns(columns_); + setConstraints(constraints_); + if (table_fd < 0) /// Will use file { use_table_fd = false; @@ -330,7 +334,7 @@ void registerStorageFile(StorageFactory & factory) return StorageFile::create( source_path, source_fd, args.data_path, - args.database_name, args.table_name, format_name, args.columns, + args.database_name, args.table_name, format_name, args.columns, args.constraints, args.context); }); } diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index cc5878520ce..237e9a6989e 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -60,6 +60,7 @@ protected: const std::string & table_name_, const std::string & format_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, Context & context_); private: diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 8c87f4ccd6a..aa055f7d907 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -30,14 +30,16 @@ StorageHDFS::StorageHDFS(const String & uri_, const std::string & table_name_, const String & format_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, Context & context_) - : IStorage(columns_) - , uri(uri_) + : uri(uri_) , format_name(format_name_) , table_name(table_name_) , database_name(database_name_) , context(context_) { + setColumns(columns_); + setConstraints(constraints_); } namespace @@ -175,7 +177,7 @@ void registerStorageHDFS(StorageFactory & factory) String format_name = engine_args[1]->as().value.safeGet(); - return StorageHDFS::create(url, args.database_name, args.table_name, format_name, args.columns, args.context); + return StorageHDFS::create(url, args.database_name, args.table_name, format_name, args.columns, args.constraints, args.context); }); } diff --git a/dbms/src/Storages/StorageHDFS.h b/dbms/src/Storages/StorageHDFS.h index 44f0286f97e..dae7000dd8a 100644 --- a/dbms/src/Storages/StorageHDFS.h +++ b/dbms/src/Storages/StorageHDFS.h @@ -37,6 +37,7 @@ protected: const String & table_name_, const String & format_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, Context & context_); private: diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index b9a6e28afb7..6577846efba 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -36,8 +36,9 @@ StorageJoin::StorageJoin( ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, bool overwrite) - : StorageSetOrJoinBase{path_, database_name_, table_name_, columns_} + : StorageSetOrJoinBase{path_, database_name_, table_name_, columns_, constraints_} , key_names(key_names_) , use_nulls(use_nulls_) , limits(limits_) @@ -170,6 +171,7 @@ void registerStorageJoin(StorageFactory & factory) kind, strictness, args.columns, + args.constraints, join_any_take_last_row); }); } diff --git a/dbms/src/Storages/StorageJoin.h b/dbms/src/Storages/StorageJoin.h index 356f4b0e7d9..c9e1283b90f 100644 --- a/dbms/src/Storages/StorageJoin.h +++ b/dbms/src/Storages/StorageJoin.h @@ -64,6 +64,7 @@ protected: SizeLimits limits_, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, bool overwrite); }; diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index d17caeb0046..69e37ce2305 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -422,12 +422,15 @@ StorageLog::StorageLog( const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, size_t max_compress_block_size_) - : IStorage{columns_}, - path(path_), table_name(table_name_), database_name(database_name_), + : path(path_), table_name(table_name_), database_name(database_name_), max_compress_block_size(max_compress_block_size_), file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json") { + setColumns(columns_); + setConstraints(constraints_); + if (path.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); @@ -644,7 +647,7 @@ void registerStorageLog(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return StorageLog::create( - args.data_path, args.database_name, args.table_name, args.columns, + args.data_path, args.database_name, args.table_name, args.columns, args.constraints, args.context.getSettings().max_compress_block_size); }); } diff --git a/dbms/src/Storages/StorageLog.h b/dbms/src/Storages/StorageLog.h index f612d0ab29a..e646ecd8c46 100644 --- a/dbms/src/Storages/StorageLog.h +++ b/dbms/src/Storages/StorageLog.h @@ -57,6 +57,7 @@ protected: const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, size_t max_compress_block_size_); private: diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index 8a33e8af030..df39f711ab3 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -99,9 +99,11 @@ StorageMaterializedView::StorageMaterializedView( const ASTCreateQuery & query, const ColumnsDescription & columns_, bool attach_) - : IStorage{columns_}, table_name(table_name_), + : table_name(table_name_), database_name(database_name_), global_context(local_context.getGlobalContext()) { + setColumns(columns_); + if (!query.select) throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index 452ee7d51c0..03c93d8d29f 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -31,6 +31,7 @@ public: } BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + void drop() override; void truncate(const ASTPtr &, const Context &) override; diff --git a/dbms/src/Storages/StorageMemory.cpp b/dbms/src/Storages/StorageMemory.cpp index 5e8f100f2f6..560da7dc2d8 100644 --- a/dbms/src/Storages/StorageMemory.cpp +++ b/dbms/src/Storages/StorageMemory.cpp @@ -74,9 +74,11 @@ private: }; -StorageMemory::StorageMemory(String database_name_, String table_name_, ColumnsDescription columns_description_) - : IStorage{std::move(columns_description_)}, database_name(std::move(database_name_)), table_name(std::move(table_name_)) +StorageMemory::StorageMemory(String database_name_, String table_name_, ColumnsDescription columns_description_, ConstraintsDescription constraints_) + : database_name(std::move(database_name_)), table_name(std::move(table_name_)) { + setColumns(std::move(columns_description_)); + setConstraints(std::move(constraints_)); } @@ -143,7 +145,7 @@ void registerStorageMemory(StorageFactory & factory) "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageMemory::create(args.database_name, args.table_name, args.columns); + return StorageMemory::create(args.database_name, args.table_name, args.columns, args.constraints); }); } diff --git a/dbms/src/Storages/StorageMemory.h b/dbms/src/Storages/StorageMemory.h index e4a0f0cf9d4..88e9e5dcf04 100644 --- a/dbms/src/Storages/StorageMemory.h +++ b/dbms/src/Storages/StorageMemory.h @@ -55,7 +55,7 @@ private: std::mutex mutex; protected: - StorageMemory(String database_name_, String table_name_, ColumnsDescription columns_description_); + StorageMemory(String database_name_, String table_name_, ColumnsDescription columns_description_, ConstraintsDescription constraints_); }; } diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index f362b34a515..05ff3d97943 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -53,13 +53,14 @@ StorageMerge::StorageMerge( const String & source_database_, const String & table_name_regexp_, const Context & context_) - : IStorage(columns_, ColumnsDescription({{"_table", std::make_shared()}}, true)) + : IStorage(ColumnsDescription({{"_table", std::make_shared()}}, true)) , table_name(table_name_) , database_name(database_name_) , source_database(source_database_) , table_name_regexp(table_name_regexp_) , global_context(context_) { + setColumns(columns_); } diff --git a/dbms/src/Storages/StorageMySQL.cpp b/dbms/src/Storages/StorageMySQL.cpp index 4ad00338793..ca5313e6ba8 100644 --- a/dbms/src/Storages/StorageMySQL.cpp +++ b/dbms/src/Storages/StorageMySQL.cpp @@ -45,9 +45,9 @@ StorageMySQL::StorageMySQL( const bool replace_query_, const std::string & on_duplicate_clause_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, const Context & context_) - : IStorage{columns_} - , table_name(table_name_) + : table_name(table_name_) , database_name(database_name_) , remote_database_name(remote_database_name_) , remote_table_name(remote_table_name_) @@ -56,6 +56,8 @@ StorageMySQL::StorageMySQL( , pool(std::move(pool_)) , global_context(context_) { + setColumns(columns_); + setConstraints(constraints_); } @@ -241,6 +243,7 @@ void registerStorageMySQL(StorageFactory & factory) replace_query, on_duplicate_clause, args.columns, + args.constraints, args.context); }); } diff --git a/dbms/src/Storages/StorageMySQL.h b/dbms/src/Storages/StorageMySQL.h index 3b891566ed0..320eab9d4a9 100644 --- a/dbms/src/Storages/StorageMySQL.h +++ b/dbms/src/Storages/StorageMySQL.h @@ -28,6 +28,7 @@ public: const bool replace_query_, const std::string & on_duplicate_clause_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, const Context & context_); std::string getName() const override { return "MySQL"; } @@ -54,7 +55,6 @@ private: bool replace_query; std::string on_duplicate_clause; - mysqlxx::Pool pool; Context global_context; }; diff --git a/dbms/src/Storages/StorageNull.cpp b/dbms/src/Storages/StorageNull.cpp index 7894f8804fc..4a54995c544 100644 --- a/dbms/src/Storages/StorageNull.cpp +++ b/dbms/src/Storages/StorageNull.cpp @@ -26,7 +26,7 @@ void registerStorageNull(StorageFactory & factory) "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageNull::create(args.database_name, args.table_name, args.columns); + return StorageNull::create(args.database_name, args.table_name, args.columns, args.constraints); }); } diff --git a/dbms/src/Storages/StorageNull.h b/dbms/src/Storages/StorageNull.h index fec7638706f..d858f738b24 100644 --- a/dbms/src/Storages/StorageNull.h +++ b/dbms/src/Storages/StorageNull.h @@ -52,9 +52,11 @@ private: String database_name; protected: - StorageNull(String database_name_, String table_name_, ColumnsDescription columns_description_) - : IStorage{std::move(columns_description_)}, table_name(std::move(table_name_)), database_name(std::move(database_name_)) + StorageNull(String database_name_, String table_name_, ColumnsDescription columns_description_, ConstraintsDescription constraints_) + : table_name(std::move(table_name_)), database_name(std::move(database_name_)) { + setColumns(std::move(columns_description_)); + setConstraints(std::move(constraints_)); } }; diff --git a/dbms/src/Storages/StorageSet.cpp b/dbms/src/Storages/StorageSet.cpp index 0d649728096..26af630ca62 100644 --- a/dbms/src/Storages/StorageSet.cpp +++ b/dbms/src/Storages/StorageSet.cpp @@ -90,9 +90,13 @@ StorageSetOrJoinBase::StorageSetOrJoinBase( const String & path_, const String & database_name_, const String & table_name_, - const ColumnsDescription & columns_) - : IStorage{columns_}, table_name(table_name_), database_name(database_name_) + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_) + : table_name(table_name_), database_name(database_name_) { + setColumns(columns_); + setConstraints(constraints_); + if (path_.empty()) throw Exception("Join and Set storages require data path", ErrorCodes::INCORRECT_FILE_NAME); @@ -105,8 +109,9 @@ StorageSet::StorageSet( const String & path_, const String & database_name_, const String & table_name_, - const ColumnsDescription & columns_) - : StorageSetOrJoinBase{path_, database_name_, table_name_, columns_}, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_) + : StorageSetOrJoinBase{path_, database_name_, table_name_, columns_, constraints_}, set(std::make_shared(SizeLimits(), false)) { Block header = getSampleBlock(); @@ -209,7 +214,7 @@ void registerStorageSet(StorageFactory & factory) "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageSet::create(args.data_path, args.database_name, args.table_name, args.columns); + return StorageSet::create(args.data_path, args.database_name, args.table_name, args.columns, args.constraints); }); } diff --git a/dbms/src/Storages/StorageSet.h b/dbms/src/Storages/StorageSet.h index 1c2891b78e1..fe6cd332ed8 100644 --- a/dbms/src/Storages/StorageSet.h +++ b/dbms/src/Storages/StorageSet.h @@ -33,7 +33,8 @@ protected: const String & path_, const String & database_name_, const String & table_name_, - const ColumnsDescription & columns_); + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_); String path; String table_name; @@ -81,7 +82,8 @@ protected: const String & path_, const String & database_name_, const String & table_name_, - const ColumnsDescription & columns_); + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_); }; } diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index 42745e11971..447e325095d 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -198,14 +198,17 @@ StorageStripeLog::StorageStripeLog( const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, bool attach, size_t max_compress_block_size_) - : IStorage{columns_}, - path(path_), table_name(table_name_), database_name(database_name_), + : path(path_), table_name(table_name_), database_name(database_name_), max_compress_block_size(max_compress_block_size_), file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json"), log(&Logger::get("StorageStripeLog")) { + setColumns(columns_); + setConstraints(constraints_); + if (path.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); @@ -316,7 +319,7 @@ void registerStorageStripeLog(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return StorageStripeLog::create( - args.data_path, args.database_name, args.table_name, args.columns, + args.data_path, args.database_name, args.table_name, args.columns, args.constraints, args.attach, args.context.getSettings().max_compress_block_size); }); } diff --git a/dbms/src/Storages/StorageStripeLog.h b/dbms/src/Storages/StorageStripeLog.h index 6d528353dbf..22be356713b 100644 --- a/dbms/src/Storages/StorageStripeLog.h +++ b/dbms/src/Storages/StorageStripeLog.h @@ -74,6 +74,7 @@ protected: const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, bool attach, size_t max_compress_block_size_); }; diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 778b07503a8..45d9771822e 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -327,14 +327,17 @@ StorageTinyLog::StorageTinyLog( const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, bool attach, size_t max_compress_block_size_) - : IStorage{columns_}, - path(path_), table_name(table_name_), database_name(database_name_), + : path(path_), table_name(table_name_), database_name(database_name_), max_compress_block_size(max_compress_block_size_), file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json"), log(&Logger::get("StorageTinyLog")) { + setColumns(columns_); + setConstraints(constraints_); + if (path.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); @@ -450,7 +453,7 @@ void registerStorageTinyLog(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return StorageTinyLog::create( - args.data_path, args.database_name, args.table_name, args.columns, + args.data_path, args.database_name, args.table_name, args.columns, args.constraints, args.attach, args.context.getSettings().max_compress_block_size); }); } diff --git a/dbms/src/Storages/StorageTinyLog.h b/dbms/src/Storages/StorageTinyLog.h index 9ed53f12962..7a37a42ec63 100644 --- a/dbms/src/Storages/StorageTinyLog.h +++ b/dbms/src/Storages/StorageTinyLog.h @@ -78,6 +78,7 @@ protected: const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, bool attach, size_t max_compress_block_size_); }; diff --git a/dbms/src/Storages/StorageURL.cpp b/dbms/src/Storages/StorageURL.cpp index 8a36c008be9..ee385af0fe8 100644 --- a/dbms/src/Storages/StorageURL.cpp +++ b/dbms/src/Storages/StorageURL.cpp @@ -24,14 +24,18 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -IStorageURLBase::IStorageURLBase(const Poco::URI & uri_, +IStorageURLBase::IStorageURLBase( + const Poco::URI & uri_, const Context & context_, const std::string & database_name_, const std::string & table_name_, const String & format_name_, - const ColumnsDescription & columns_) - : IStorage(columns_), uri(uri_), context_global(context_), format_name(format_name_), table_name(table_name_), database_name(database_name_) + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_) + : uri(uri_), context_global(context_), format_name(format_name_), table_name(table_name_), database_name(database_name_) { + setColumns(columns_); + setConstraints(constraints_); } namespace @@ -214,7 +218,7 @@ void registerStorageURL(StorageFactory & factory) String format_name = engine_args[1]->as().value.safeGet(); - return StorageURL::create(uri, args.database_name, args.table_name, format_name, args.columns, args.context); + return StorageURL::create(uri, args.database_name, args.table_name, format_name, args.columns, args.constraints, args.context); }); } } diff --git a/dbms/src/Storages/StorageURL.h b/dbms/src/Storages/StorageURL.h index 361b0f6c35e..0c4b4648223 100644 --- a/dbms/src/Storages/StorageURL.h +++ b/dbms/src/Storages/StorageURL.h @@ -2,9 +2,9 @@ #include #include -#include #include + namespace DB { /** @@ -19,7 +19,8 @@ public: String getTableName() const override { return table_name; } String getDatabaseName() const override { return database_name; } - BlockInputStreams read(const Names & column_names, + BlockInputStreams read( + const Names & column_names, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -31,12 +32,14 @@ public: void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; protected: - IStorageURLBase(const Poco::URI & uri_, + IStorageURLBase( + const Poco::URI & uri_, const Context & context_, const std::string & database_name_, const std::string & table_name_, const String & format_name_, - const ColumnsDescription & columns_); + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_); Poco::URI uri; const Context & context_global; @@ -48,13 +51,15 @@ private: virtual std::string getReadMethod() const; - virtual std::vector> getReadURIParams(const Names & column_names, + virtual std::vector> getReadURIParams( + const Names & column_names, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum & processed_stage, size_t max_block_size) const; - virtual std::function getReadPOSTDataCallback(const Names & column_names, + virtual std::function getReadPOSTDataCallback( + const Names & column_names, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum & processed_stage, @@ -63,16 +68,19 @@ private: virtual Block getHeaderBlock(const Names & column_names) const = 0; }; + class StorageURL : public ext::shared_ptr_helper, public IStorageURLBase { public: - StorageURL(const Poco::URI & uri_, + StorageURL( + const Poco::URI & uri_, const std::string & database_name_, const std::string & table_name_, const String & format_name_, const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, Context & context_) - : IStorageURLBase(uri_, context_, database_name_, table_name_, format_name_, columns_) + : IStorageURLBase(uri_, context_, database_name_, table_name_, format_name_, columns_, constraints_) { } diff --git a/dbms/src/Storages/StorageView.cpp b/dbms/src/Storages/StorageView.cpp index 153de1ab176..912e358e3ae 100644 --- a/dbms/src/Storages/StorageView.cpp +++ b/dbms/src/Storages/StorageView.cpp @@ -31,8 +31,10 @@ StorageView::StorageView( const String & table_name_, const ASTCreateQuery & query, const ColumnsDescription & columns_) - : IStorage{columns_}, table_name(table_name_), database_name(database_name_) + : table_name(table_name_), database_name(database_name_) { + setColumns(columns_); + if (!query.select) throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); diff --git a/dbms/src/Storages/StorageXDBC.cpp b/dbms/src/Storages/StorageXDBC.cpp index 57327fc4dcd..915146e1409 100644 --- a/dbms/src/Storages/StorageXDBC.cpp +++ b/dbms/src/Storages/StorageXDBC.cpp @@ -30,7 +30,8 @@ StorageXDBC::StorageXDBC( const ColumnsDescription & columns_, const Context & context_, const BridgeHelperPtr bridge_helper_) - : IStorageURLBase(Poco::URI(), context_, database_name_, table_name_, IXDBCBridgeHelper::DEFAULT_FORMAT, columns_) + /// Please add support for constraints as soon as StorageODBC or JDBC will support insertion. + : IStorageURLBase(Poco::URI(), context_, database_name_, table_name_, IXDBCBridgeHelper::DEFAULT_FORMAT, columns_, ConstraintsDescription{}) , bridge_helper(bridge_helper_) , remote_database_name(remote_database_name_) , remote_table_name(remote_table_name_) diff --git a/dbms/src/Storages/StorageXDBC.h b/dbms/src/Storages/StorageXDBC.h index 23e108519b4..c658fc5c8a1 100644 --- a/dbms/src/Storages/StorageXDBC.h +++ b/dbms/src/Storages/StorageXDBC.h @@ -1,59 +1,61 @@ #pragma once #include -#include #include + namespace DB { + /** Implements storage in the XDBC database. * Use ENGINE = xdbc(connection_string, table_name) * Example ENGINE = odbc('dsn=test', table) * Read only. */ - class StorageXDBC : public IStorageURLBase - { - public: +class StorageXDBC : public IStorageURLBase +{ +public: - BlockInputStreams read(const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) override; + BlockInputStreams read(const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; - StorageXDBC(const std::string & database_name_, - const std::string & table_name_, - const std::string & remote_database_name, - const std::string & remote_table_name, - const ColumnsDescription & columns_, - const Context & context_, BridgeHelperPtr bridge_helper_); + StorageXDBC(const std::string & database_name_, + const std::string & table_name_, + const std::string & remote_database_name, + const std::string & remote_table_name, + const ColumnsDescription & columns_, + const Context & context_, BridgeHelperPtr bridge_helper_); - private: +private: - BridgeHelperPtr bridge_helper; - std::string remote_database_name; - std::string remote_table_name; + BridgeHelperPtr bridge_helper; + std::string remote_database_name; + std::string remote_table_name; - Poco::Logger * log; + Poco::Logger * log; - std::string getReadMethod() const override; + std::string getReadMethod() const override; - std::vector> getReadURIParams(const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size) const override; + std::vector> getReadURIParams(const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size) const override; - std::function getReadPOSTDataCallback(const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size) const override; + std::function getReadPOSTDataCallback(const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size) const override; - Block getHeaderBlock(const Names & column_names) const override; + Block getHeaderBlock(const Names & column_names) const override; + + std::string getName() const override; +}; - std::string getName() const override; - }; } diff --git a/dbms/src/TableFunctions/TableFunctionFile.cpp b/dbms/src/TableFunctions/TableFunctionFile.cpp index 9bd85a5bbb5..7cf2c500f1e 100644 --- a/dbms/src/TableFunctions/TableFunctionFile.cpp +++ b/dbms/src/TableFunctions/TableFunctionFile.cpp @@ -15,6 +15,7 @@ StoragePtr TableFunctionFile::getStorage( table_name, format, columns, + ConstraintsDescription{}, global_context); } diff --git a/dbms/src/TableFunctions/TableFunctionHDFS.cpp b/dbms/src/TableFunctions/TableFunctionHDFS.cpp index c4b61e178d6..4ff999d31dd 100644 --- a/dbms/src/TableFunctions/TableFunctionHDFS.cpp +++ b/dbms/src/TableFunctions/TableFunctionHDFS.cpp @@ -16,6 +16,7 @@ StoragePtr TableFunctionHDFS::getStorage( table_name, format, columns, + ConstraintsDescription{}, global_context); } diff --git a/dbms/src/TableFunctions/TableFunctionMySQL.cpp b/dbms/src/TableFunctions/TableFunctionMySQL.cpp index cee08cf2a1d..3cb9b8dea60 100644 --- a/dbms/src/TableFunctions/TableFunctionMySQL.cpp +++ b/dbms/src/TableFunctions/TableFunctionMySQL.cpp @@ -127,6 +127,7 @@ StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Co replace_query, on_duplicate_clause, ColumnsDescription{columns}, + ConstraintsDescription{}, context); res->startup(); diff --git a/dbms/src/TableFunctions/TableFunctionURL.cpp b/dbms/src/TableFunctions/TableFunctionURL.cpp index 9991bc2f527..1f89264a422 100644 --- a/dbms/src/TableFunctions/TableFunctionURL.cpp +++ b/dbms/src/TableFunctions/TableFunctionURL.cpp @@ -4,13 +4,14 @@ #include #include + namespace DB { StoragePtr TableFunctionURL::getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const { Poco::URI uri(source); - return StorageURL::create(uri, getDatabaseName(), table_name, format, columns, global_context); + return StorageURL::create(uri, getDatabaseName(), table_name, format, columns, ConstraintsDescription{}, global_context); } void registerTableFunctionURL(TableFunctionFactory & factory) From 9660a4db8000981891c58cc7f9999f7c20b74fdf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Aug 2019 00:35:07 +0300 Subject: [PATCH 412/509] Better exception message --- .../CheckConstraintsBlockOutputStream.cpp | 27 ++++++++++++++++--- 1 file changed, 24 insertions(+), 3 deletions(-) diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 8fbb5916372..b50bc3da028 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -56,9 +57,29 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) if (data[row_idx] != 1) break; - throw Exception{"Violated constraint " + constraints.constraints[i]->name + - " in table " + table + " at row " + std::to_string(rows_written + row_idx + 1) + ", constraint expression: " + - serializeAST(*(constraints.constraints[i]->expr), true), ErrorCodes::VIOLATED_CONSTRAINT}; + Names related_columns = constraint_expr->getRequiredColumns(); + + std::stringstream exception_message; + + exception_message << "Constraint " << backQuote(constraints.constraints[i]->name) + << " for table " << backQuote(table) + << " is violated at row " << (rows_written + row_idx + 1) + << ". Expression: (" << serializeAST(*(constraints.constraints[i]->expr), true) << ")" + << ". Column values"; + + bool first = true; + for (const auto & name : related_columns) + { + const IColumn & column = *block.getByName(name).column; + assert(row_idx < column.size()); + + exception_message << (first ? ": " : ", ") + << backQuoteIfNeed(name) << " = " << applyVisitor(FieldVisitorToString(), column[row_idx]); + + first = false; + } + + throw Exception{exception_message.str(), ErrorCodes::VIOLATED_CONSTRAINT}; } } } From 2fc177830c620a2f1926863954d8771879a7c50c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Aug 2019 01:01:01 +0300 Subject: [PATCH 413/509] Added a test --- .../00998_constraints_all_tables.reference | 14 +++++ .../00998_constraints_all_tables.sql | 53 +++++++++++++++++++ 2 files changed, 67 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00998_constraints_all_tables.reference create mode 100644 dbms/tests/queries/0_stateless/00998_constraints_all_tables.sql diff --git a/dbms/tests/queries/0_stateless/00998_constraints_all_tables.reference b/dbms/tests/queries/0_stateless/00998_constraints_all_tables.reference new file mode 100644 index 00000000000..730df555af3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00998_constraints_all_tables.reference @@ -0,0 +1,14 @@ +0 +0 +3 +0 +0 +3 +0 +0 +3 +0 +0 +3 +CREATE TABLE default.constrained (`URL` String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Log +CREATE TABLE default.constrained2 (`URL` String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Log diff --git a/dbms/tests/queries/0_stateless/00998_constraints_all_tables.sql b/dbms/tests/queries/0_stateless/00998_constraints_all_tables.sql new file mode 100644 index 00000000000..66b93fca97b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00998_constraints_all_tables.sql @@ -0,0 +1,53 @@ +DROP TABLE IF EXISTS constrained; +CREATE TABLE constrained (URL String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Null; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 } +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), ('https://yandex.ru/te\xFFst'); -- { serverError 469 } +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), (toValidUTF8('https://yandex.ru/te\xFFst')); +DROP TABLE constrained; + +CREATE TABLE constrained (URL String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Memory; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 } +SELECT count() FROM constrained; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), ('https://yandex.ru/te\xFFst'); -- { serverError 469 } +SELECT count() FROM constrained; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), (toValidUTF8('https://yandex.ru/te\xFFst')); +SELECT count() FROM constrained; +DROP TABLE constrained; + +CREATE TABLE constrained (URL String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = StripeLog; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 } +SELECT count() FROM constrained; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), ('https://yandex.ru/te\xFFst'); -- { serverError 469 } +SELECT count() FROM constrained; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), (toValidUTF8('https://yandex.ru/te\xFFst')); +SELECT count() FROM constrained; +DROP TABLE constrained; + +CREATE TABLE constrained (URL String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = TinyLog; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 } +SELECT count() FROM constrained; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), ('https://yandex.ru/te\xFFst'); -- { serverError 469 } +SELECT count() FROM constrained; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), (toValidUTF8('https://yandex.ru/te\xFFst')); +SELECT count() FROM constrained; +DROP TABLE constrained; + +CREATE TABLE constrained (URL String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Log; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 } +SELECT count() FROM constrained; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), ('https://yandex.ru/te\xFFst'); -- { serverError 469 } +SELECT count() FROM constrained; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), (toValidUTF8('https://yandex.ru/te\xFFst')); +SELECT count() FROM constrained; +DROP TABLE constrained; + + +DROP TABLE IF EXISTS constrained2; +CREATE TABLE constrained (URL String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Log; +CREATE TABLE constrained2 AS constrained; +SHOW CREATE TABLE constrained FORMAT TSVRaw; +SHOW CREATE TABLE constrained2 FORMAT TSVRaw; +INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 } +INSERT INTO constrained2 VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 } +DROP TABLE constrained; +DROP TABLE constrained2; From 2dbfabd08c839a54a27ed99dead67b71ed8660ff Mon Sep 17 00:00:00 2001 From: Nikita Vasilev <31595000+nikvas0@users.noreply.github.com> Date: Sun, 25 Aug 2019 01:01:36 +0300 Subject: [PATCH 414/509] fix Set index check useless (#6651) * fixed useless detection * fixed useless detection * fix * fix * fix --- .../Storages/MergeTree/MergeTreeIndexSet.cpp | 25 ++----------------- .../00997_set_index_array.reference | 1 + .../0_stateless/00997_set_index_array.sql | 24 ++++++++++++++++++ 3 files changed, 27 insertions(+), 23 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00997_set_index_array.reference create mode 100644 dbms/tests/queries/0_stateless/00997_set_index_array.sql diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/dbms/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 40aba822353..954ac774583 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -405,25 +405,6 @@ bool MergeTreeIndexConditionSet::operatorFromAST(ASTPtr & node) const return true; } -static bool checkAtomName(const String & name) -{ - static std::set atoms = { - "notEquals", - "equals", - "less", - "greater", - "lessOrEquals", - "greaterOrEquals", - "in", - "notIn", - "like", - "startsWith", - "endsWith", - "multiSearchAny" - }; - return atoms.find(name) != atoms.end(); -} - bool MergeTreeIndexConditionSet::checkASTUseless(const ASTPtr &node, bool atomic) const { if (const auto * func = node->as()) @@ -439,16 +420,14 @@ bool MergeTreeIndexConditionSet::checkASTUseless(const ASTPtr &node, bool atomic return checkASTUseless(args[0], atomic) || checkASTUseless(args[1], atomic); else if (func->name == "not") return checkASTUseless(args[0], atomic); - else if (!atomic && checkAtomName(func->name)) - return checkASTUseless(node, true); else return std::any_of(args.begin(), args.end(), - [this, &atomic](const auto & arg) { return checkASTUseless(arg, atomic); }); + [this](const auto & arg) { return checkASTUseless(arg, true); }); } else if (const auto * literal = node->as()) return !atomic && literal->value.get(); else if (const auto * identifier = node->as()) - return key_columns.find(identifier->getColumnName()) == key_columns.end(); + return key_columns.find(identifier->getColumnName()) == std::end(key_columns); else return true; } diff --git a/dbms/tests/queries/0_stateless/00997_set_index_array.reference b/dbms/tests/queries/0_stateless/00997_set_index_array.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00997_set_index_array.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00997_set_index_array.sql b/dbms/tests/queries/0_stateless/00997_set_index_array.sql new file mode 100644 index 00000000000..c57507ce22d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00997_set_index_array.sql @@ -0,0 +1,24 @@ +SET allow_experimental_data_skipping_indices = 1; + +DROP TABLE IF EXISTS test.set_array; + +CREATE TABLE test.set_array +( + primary_key String, + index_array Array(UInt64), + INDEX additional_index_array (index_array) TYPE set(10000) GRANULARITY 1 +) ENGINE = MergeTree() +ORDER BY (primary_key); + +INSERT INTO test.set_array +select + toString(intDiv(number, 1000000)) as primary_key, + array(number) as index_array +from system.numbers +limit 10000000; + +SET max_rows_to_read = 8192; + +select count() from test.set_array where has(index_array, 333); + +DROP TABLE test.set_array; \ No newline at end of file From 17a1938f0f1297d02a3cd438237e7ade59273810 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Aug 2019 01:04:36 +0300 Subject: [PATCH 415/509] Addition to prev. revision --- .../src/DataStreams/CheckConstraintsBlockOutputStream.cpp | 8 ++++---- dbms/src/Interpreters/InterpreterInsertQuery.cpp | 2 -- dbms/tests/queries/0_stateless/00952_basic_constraints.sh | 5 ++--- .../queries/0_stateless/00953_constraints_operations.sh | 3 ++- 4 files changed, 8 insertions(+), 10 deletions(-) diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index b50bc3da028..6c04558144a 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -34,12 +34,9 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) { if (block.rows() > 0) { - std::cerr << "Checking " << expressions.size() << " constraints\n"; + Block block_to_calculate = block; for (size_t i = 0; i < expressions.size(); ++i) { - std::cerr << serializeAST(*(constraints.constraints[i]->expr), true) << "\n"; - - Block block_to_calculate = block; auto constraint_expr = expressions[i]; constraint_expr->execute(block_to_calculate); @@ -54,8 +51,11 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) { size_t row_idx = 0; for (; row_idx < size; ++row_idx) + { + std::cerr << row_idx << ": " << int(data[row_idx]) << "\n"; if (data[row_idx] != 1) break; + } Names related_columns = constraint_expr->getRequiredColumns(); diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 549e06e0155..45d528b0e74 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -121,8 +121,6 @@ BlockIO InterpreterInsertQuery::execute() out = std::make_shared( out, query_sample_block, out->getHeader(), table->getColumns().getDefaults(), context); - std::cerr << table->getConstraints().toString() << "\n"; - if (const auto & constraints = table->getConstraints(); !constraints.empty()) out = std::make_shared(query.table, out, query_sample_block, table->getConstraints(), context); diff --git a/dbms/tests/queries/0_stateless/00952_basic_constraints.sh b/dbms/tests/queries/0_stateless/00952_basic_constraints.sh index b6aa28c46bf..8ba3b082f12 100755 --- a/dbms/tests/queries/0_stateless/00952_basic_constraints.sh +++ b/dbms/tests/queries/0_stateless/00952_basic_constraints.sh @@ -3,6 +3,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh +EXCEPTION_TEXT=violated EXCEPTION_SUCCESS_TEXT=ok $CLICKHOUSE_CLIENT --query="CREATE DATABASE IF NOT EXISTS test;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_constraints;" @@ -20,7 +21,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must throw and exception -EXCEPTION_TEXT="Violated constraint b_constraint in table test_constraints at indices {1, 3}" + $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (3, 4), (1, 0), (3, 4), (6, 0);" 2>&1 \ | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" @@ -38,13 +39,11 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE test_constraints ENGINE = MergeTree ORDER BY (a);" # This one must throw an exception -EXCEPTION_TEXT="Violated constraint b_constraint in table test_constraints at indices {0}" $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" 2>&1 \ | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must throw an exception -EXCEPTION_TEXT="Violated constraint a_constraint in table test_constraints at indices {1}" $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (5, 16), (10, 11), (9, 11), (8, 12);" 2>&1 \ | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" diff --git a/dbms/tests/queries/0_stateless/00953_constraints_operations.sh b/dbms/tests/queries/0_stateless/00953_constraints_operations.sh index 8a563a21e02..3b415848501 100755 --- a/dbms/tests/queries/0_stateless/00953_constraints_operations.sh +++ b/dbms/tests/queries/0_stateless/00953_constraints_operations.sh @@ -3,6 +3,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh +EXCEPTION_TEXT=violated EXCEPTION_SUCCESS_TEXT=ok $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_constraints;" @@ -20,7 +21,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" # This one must throw and exception -EXCEPTION_TEXT="Violated constraint b_constraint in table test_constraints at indices" + $CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 0);" 2>&1 \ | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception" $CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;" From ffbbb173c5a59c26988a0d6441ae7f41994ca501 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Aug 2019 01:04:58 +0300 Subject: [PATCH 416/509] Addition to prev. revision --- dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 6c04558144a..4bdd1a73295 100644 --- a/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -51,11 +51,8 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) { size_t row_idx = 0; for (; row_idx < size; ++row_idx) - { - std::cerr << row_idx << ": " << int(data[row_idx]) << "\n"; if (data[row_idx] != 1) break; - } Names related_columns = constraint_expr->getRequiredColumns(); From 7144a3f827a34a30f50c7818681c9fe21d40ac93 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Aug 2019 01:06:13 +0300 Subject: [PATCH 417/509] Speed up MemoryTracker by function inlining --- dbms/src/Common/CurrentThread.cpp | 6 ------ dbms/src/Common/CurrentThread.h | 7 ++++++- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index 446772f218d..ca39bec414c 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -51,12 +51,6 @@ MemoryTracker * CurrentThread::getMemoryTracker() return ¤t_thread->memory_tracker; } -Int64 & CurrentThread::getUntrackedMemory() -{ - /// It assumes that (current_thread != nullptr) is already checked with getMemoryTracker() - return current_thread->untracked_memory; -} - void CurrentThread::updateProgressIn(const Progress & value) { if (unlikely(!current_thread)) diff --git a/dbms/src/Common/CurrentThread.h b/dbms/src/Common/CurrentThread.h index 01e46fbeadc..1e0140c6330 100644 --- a/dbms/src/Common/CurrentThread.h +++ b/dbms/src/Common/CurrentThread.h @@ -52,7 +52,12 @@ public: static ProfileEvents::Counters & getProfileEvents(); static MemoryTracker * getMemoryTracker(); - static Int64 & getUntrackedMemory(); + + static inline Int64 & getUntrackedMemory() + { + /// It assumes that (current_thread != nullptr) is already checked with getMemoryTracker() + return current_thread->untracked_memory; + } /// Update read and write rows (bytes) statistics (used in system.query_thread_log) static void updateProgressIn(const Progress & value); From 6d3250867c1151ebeb8fb3a078fb73a6e1f6509a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Aug 2019 01:08:45 +0300 Subject: [PATCH 418/509] Fixed build of tests --- dbms/src/Interpreters/tests/aggregate.cpp | 2 +- dbms/src/Processors/tests/processors_test_aggregation.cpp | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/tests/aggregate.cpp b/dbms/src/Interpreters/tests/aggregate.cpp index e0068e9a56b..73e71d178ea 100644 --- a/dbms/src/Interpreters/tests/aggregate.cpp +++ b/dbms/src/Interpreters/tests/aggregate.cpp @@ -79,7 +79,7 @@ int main(int argc, char ** argv) Aggregator::Params params( stream->getHeader(), {0, 1}, aggregate_descriptions, - false, 0, OverflowMode::THROW, nullptr, 0, 0, 0, 0, false, "", 1); + false, 0, OverflowMode::THROW, 0, 0, 0, false, "", 1); Aggregator aggregator(params); diff --git a/dbms/src/Processors/tests/processors_test_aggregation.cpp b/dbms/src/Processors/tests/processors_test_aggregation.cpp index a645804eba8..2306de4edc0 100644 --- a/dbms/src/Processors/tests/processors_test_aggregation.cpp +++ b/dbms/src/Processors/tests/processors_test_aggregation.cpp @@ -224,8 +224,6 @@ try overflow_row, max_rows_to_group_by, OverflowMode::THROW, - nullptr, /// No compiler - 0, /// min_count_to_compile group_by_two_level_threshold, group_by_two_level_threshold_bytes, max_bytes_before_external_group_by, @@ -298,8 +296,6 @@ try overflow_row, max_rows_to_group_by, OverflowMode::THROW, - nullptr, /// No compiler - 0, /// min_count_to_compile group_by_two_level_threshold, group_by_two_level_threshold_bytes, max_bytes_before_external_group_by, From d798cbde2fd48be636517c6e444157557d1f80c3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Aug 2019 01:28:06 +0300 Subject: [PATCH 419/509] Fixed double whitespaces --- dbms/src/Storages/StorageDistributed.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 7c20622d89e..4a54b4c5317 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -279,7 +279,7 @@ StoragePtr StorageDistributed::createWithOwnCluster( const Context & context_) { auto res = ext::shared_ptr_helper::create( - String{}, table_name_, columns_, ConstraintsDescription{}, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), false); + String{}, table_name_, columns_, ConstraintsDescription{}, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), false); res->owned_cluster = owned_cluster_; From f959c29be60b9a70eef34ecff4135e871c45201e Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sat, 24 Aug 2019 21:38:50 -0400 Subject: [PATCH 420/509] Revert "Temporarily disable all LIVE VIEW tests because this feature has subtle bugs that manifestate under TSan" This reverts commit 96869d405fc132a7443cfbc13d1e583079503db8. --- .../00960_live_view_watch_events_live.py | 42 ++++ ...0960_live_view_watch_events_live.reference | 0 .../00961_temporary_live_view_watch.reference | 3 + .../00961_temporary_live_view_watch.sql | 18 ++ .../00962_temporary_live_view_watch_live.py | 42 ++++ ...2_temporary_live_view_watch_live.reference | 0 ...y_live_view_watch_live_timeout.py.disabled | 49 +++++ ...ary_live_view_watch_live_timeout.reference | 0 .../00964_live_view_watch_events_heartbeat.py | 44 ++++ ...live_view_watch_events_heartbeat.reference | 0 .../00965_live_view_watch_heartbeat.py | 45 ++++ .../00965_live_view_watch_heartbeat.reference | 0 .../00966_live_view_watch_events_http.py | 37 ++++ ...0966_live_view_watch_events_http.reference | 0 .../0_stateless/00967_live_view_watch_http.py | 37 ++++ .../00967_live_view_watch_http.reference | 0 ...t_format_jsoneachrowwithprogress.reference | 4 + ..._select_format_jsoneachrowwithprogress.sql | 12 + ...h_format_jsoneachrowwithprogress.reference | 6 + ...w_watch_format_jsoneachrowwithprogress.sql | 18 ++ ...0_live_view_watch_events_http_heartbeat.py | 43 ++++ ...view_watch_events_http_heartbeat.reference | 0 .../00971_live_view_watch_http_heartbeat.py | 43 ++++ ...1_live_view_watch_http_heartbeat.reference | 0 .../00972_live_view_select_1.reference | 1 + .../0_stateless/00972_live_view_select_1.sql | 7 + .../00973_live_view_select.reference | 4 + .../0_stateless/00973_live_view_select.sql | 18 ++ ...ive_view_select_with_aggregation.reference | 2 + ...0974_live_view_select_with_aggregation.sql | 16 ++ .../00975_live_view_create.reference | 0 .../0_stateless/00975_live_view_create.sql | 7 + .../00976_live_view_select_version.reference | 3 + .../00976_live_view_select_version.sql | 12 + .../00977_live_view_watch_events.reference | 3 + .../00977_live_view_watch_events.sql | 18 ++ .../00978_live_view_watch.reference | 3 + .../0_stateless/00978_live_view_watch.sql | 18 ++ .../0_stateless/00979_live_view_watch_live.py | 48 ++++ .../00979_live_view_watch_live.reference | 0 ...00980_create_temporary_live_view.reference | 3 + .../00980_create_temporary_live_view.sql | 15 ++ .../00991_live_view_watch_event_live.python | 81 +++++++ ...00991_live_view_watch_event_live.reference | 7 + ...991_live_view_watch_event_live.sh.disabled | 6 + .../00991_live_view_watch_http.python | 63 ++++++ .../00991_live_view_watch_http.reference | 4 + .../00991_live_view_watch_http.sh.disabled | 6 + ...ry_live_view_watch_events_heartbeat.python | 83 +++++++ ...live_view_watch_events_heartbeat.reference | 0 ...ve_view_watch_events_heartbeat.sh.disabled | 6 + ...0991_temporary_live_view_watch_live.python | 81 +++++++ ...1_temporary_live_view_watch_live.reference | 7 + ...temporary_live_view_watch_live.sh.disabled | 6 + .../queries/0_stateless/helpers/client.py | 36 +++ .../queries/0_stateless/helpers/httpclient.py | 14 ++ .../queries/0_stateless/helpers/httpexpect.py | 73 +++++++ .../queries/0_stateless/helpers/uexpect.py | 206 ++++++++++++++++++ 58 files changed, 1300 insertions(+) create mode 100755 dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py create mode 100644 dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.reference create mode 100644 dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference create mode 100644 dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql create mode 100755 dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py create mode 100644 dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.reference create mode 100755 dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled create mode 100644 dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.reference create mode 100755 dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py create mode 100644 dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.reference create mode 100755 dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py create mode 100644 dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.reference create mode 100755 dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py create mode 100644 dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.reference create mode 100755 dbms/tests/queries/0_stateless/00967_live_view_watch_http.py create mode 100644 dbms/tests/queries/0_stateless/00967_live_view_watch_http.reference create mode 100644 dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference create mode 100644 dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql create mode 100644 dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference create mode 100644 dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql create mode 100755 dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py create mode 100644 dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.reference create mode 100755 dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py create mode 100644 dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.reference create mode 100644 dbms/tests/queries/0_stateless/00972_live_view_select_1.reference create mode 100644 dbms/tests/queries/0_stateless/00972_live_view_select_1.sql create mode 100644 dbms/tests/queries/0_stateless/00973_live_view_select.reference create mode 100644 dbms/tests/queries/0_stateless/00973_live_view_select.sql create mode 100644 dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.reference create mode 100644 dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql create mode 100644 dbms/tests/queries/0_stateless/00975_live_view_create.reference create mode 100644 dbms/tests/queries/0_stateless/00975_live_view_create.sql create mode 100644 dbms/tests/queries/0_stateless/00976_live_view_select_version.reference create mode 100644 dbms/tests/queries/0_stateless/00976_live_view_select_version.sql create mode 100644 dbms/tests/queries/0_stateless/00977_live_view_watch_events.reference create mode 100644 dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql create mode 100644 dbms/tests/queries/0_stateless/00978_live_view_watch.reference create mode 100644 dbms/tests/queries/0_stateless/00978_live_view_watch.sql create mode 100755 dbms/tests/queries/0_stateless/00979_live_view_watch_live.py create mode 100644 dbms/tests/queries/0_stateless/00979_live_view_watch_live.reference create mode 100644 dbms/tests/queries/0_stateless/00980_create_temporary_live_view.reference create mode 100644 dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql create mode 100644 dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python create mode 100644 dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference create mode 100755 dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled create mode 100755 dbms/tests/queries/0_stateless/00991_live_view_watch_http.python create mode 100644 dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference create mode 100755 dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled create mode 100644 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python create mode 100644 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.reference create mode 100755 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled create mode 100644 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python create mode 100644 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference create mode 100755 dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled create mode 100644 dbms/tests/queries/0_stateless/helpers/client.py create mode 100644 dbms/tests/queries/0_stateless/helpers/httpclient.py create mode 100644 dbms/tests/queries/0_stateless/helpers/httpexpect.py create mode 100644 dbms/tests/queries/0_stateless/helpers/uexpect.py diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py new file mode 100755 index 00000000000..b7fc3f4e3a6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py @@ -0,0 +1,42 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv EVENTS') + client1.expect('1.*' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect('2.*' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') + client1.expect('3.*' + end_of_block) + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.reference b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference new file mode 100644 index 00000000000..6fbbedf1b21 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference @@ -0,0 +1,3 @@ +0 1 +6 2 +21 3 diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql new file mode 100644 index 00000000000..c3e2ab8d102 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +WATCH test.lv LIMIT 0; + +INSERT INTO test.mt VALUES (1),(2),(3); + +WATCH test.lv LIMIT 0; + +INSERT INTO test.mt VALUES (4),(5),(6); + +WATCH test.lv LIMIT 0; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py new file mode 100755 index 00000000000..f27b1213c70 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py @@ -0,0 +1,42 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv') + client1.expect(r'0.*1' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(r'6.*2' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') + client1.expect(r'21.*3' + end_of_block) + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled new file mode 100755 index 00000000000..df627c84e49 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled @@ -0,0 +1,49 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('SET temporary_live_view_timeout=1') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv') + client1.expect(r'0.*1' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client2.expect(prompt) + client1.expect(r'6.*2' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') + client2.expect(prompt) + client1.expect(r'21.*3' + end_of_block) + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send('SELECT sleep(1)') + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect('Table test.lv doesn\'t exist') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.reference b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py new file mode 100755 index 00000000000..5664c0e6c6d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -0,0 +1,44 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('SET live_view_heartbeat_interval=1') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv EVENTS') + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect('2.*' + end_of_block) + client1.expect('Progress: 2.00 rows.*\)') + # wait for heartbeat + client1.expect('Progress: 2.00 rows.*\)') + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.reference b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py new file mode 100755 index 00000000000..03e22175dff --- /dev/null +++ b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py @@ -0,0 +1,45 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('SET live_view_heartbeat_interval=1') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv') + client1.expect(r'0.*1' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(r'6.*2' + end_of_block) + client1.expect('Progress: 2.00 rows.*\)') + # wait for heartbeat + client1.expect('Progress: 2.00 rows.*\)') + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.reference b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py new file mode 100755 index 00000000000..bb9d6152200 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py @@ -0,0 +1,37 @@ +#!/usr/bin/env python +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block +from httpclient import client as http_client + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1: + client1.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + + + with http_client({'method':'GET', 'url': '/?query=WATCH%20test.lv%20EVENTS'}, name='client2>', log=log) as client2: + client2.expect('.*1\n') + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) + client2.expect('.*2\n') + + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.reference b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py new file mode 100755 index 00000000000..d3439431eb3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py @@ -0,0 +1,37 @@ +#!/usr/bin/env python +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block +from httpclient import client as http_client + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1: + client1.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + + + with http_client({'method':'GET', 'url':'/?query=WATCH%20test.lv'}, name='client2>', log=log) as client2: + client2.expect('.*0\t1\n') + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) + client2.expect('.*6\t2\n') + + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.reference b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference new file mode 100644 index 00000000000..5ae423d90d1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference @@ -0,0 +1,4 @@ +{"row":{"a":1}} +{"row":{"a":2}} +{"row":{"a":3}} +{"progress":{"read_rows":"3","read_bytes":"36","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} diff --git a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql new file mode 100644 index 00000000000..8c6f4197d54 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT * FROM test.lv FORMAT JSONEachRowWithProgress; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference new file mode 100644 index 00000000000..287a1ced92d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference @@ -0,0 +1,6 @@ +{"row":{"sum(a)":"0","_version":"1"}} +{"progress":{"read_rows":"1","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} +{"row":{"sum(a)":"6","_version":"2"}} +{"progress":{"read_rows":"1","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} +{"row":{"sum(a)":"21","_version":"3"}} +{"progress":{"read_rows":"1","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} diff --git a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql new file mode 100644 index 00000000000..725a4ad00ed --- /dev/null +++ b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; + +INSERT INTO test.mt VALUES (1),(2),(3); + +WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; + +INSERT INTO test.mt VALUES (4),(5),(6); + +WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py new file mode 100755 index 00000000000..63628c4a76f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py @@ -0,0 +1,43 @@ +#!/usr/bin/env python +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block +from httpclient import client as http_client + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1: + client1.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + + + with http_client({'method':'GET', 'url': '/?live_view_heartbeat_interval=1&query=WATCH%20test.lv%20EVENTS%20FORMAT%20JSONEachRowWithProgress'}, name='client2>', log=log) as client2: + client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}\n', escape=True) + client2.expect('{"row":{"version":"1"}', escape=True) + client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) + # heartbeat is provided by progress message + client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) + + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) + + client2.expect('{"row":{"version":"2"}}\n', escape=True) + + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.reference b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py new file mode 100755 index 00000000000..7bdb47b7caa --- /dev/null +++ b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py @@ -0,0 +1,43 @@ +#!/usr/bin/env python +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block +from httpclient import client as http_client + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1: + client1.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + + with http_client({'method':'GET', 'url':'/?live_view_heartbeat_interval=1&query=WATCH%20test.lv%20FORMAT%20JSONEachRowWithProgress'}, name='client2>', log=log) as client2: + client2.expect('"progress".*',) + client2.expect('{"row":{"sum(a)":"0","_version":"1"}}\n', escape=True) + client2.expect('"progress".*\n') + # heartbeat is provided by progress message + client2.expect('"progress".*\n') + + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) + + client2.expect('"progress".*"read_rows":"2".*\n') + client2.expect('{"row":{"sum(a)":"6","_version":"2"}}\n', escape=True) + + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.reference b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference b/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql b/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql new file mode 100644 index 00000000000..661080b577b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS test.lv; + +CREATE LIVE VIEW test.lv AS SELECT 1; + +SELECT * FROM test.lv; + +DROP TABLE test.lv; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_select.reference b/dbms/tests/queries/0_stateless/00973_live_view_select.reference new file mode 100644 index 00000000000..75236c0daf7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00973_live_view_select.reference @@ -0,0 +1,4 @@ +6 1 +6 1 +12 2 +12 2 diff --git a/dbms/tests/queries/0_stateless/00973_live_view_select.sql b/dbms/tests/queries/0_stateless/00973_live_view_select.sql new file mode 100644 index 00000000000..ff4a45ffcc1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00973_live_view_select.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT *,_version FROM test.lv; +SELECT *,_version FROM test.lv; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT *,_version FROM test.lv; +SELECT *,_version FROM test.lv; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.reference b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.reference new file mode 100644 index 00000000000..6d50f0e9c3a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.reference @@ -0,0 +1,2 @@ +6 +21 diff --git a/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql new file mode 100644 index 00000000000..3c11f855c9d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT sum(a) FROM test.lv; + +INSERT INTO test.mt VALUES (4),(5),(6); + +SELECT sum(a) FROM test.lv; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00975_live_view_create.reference b/dbms/tests/queries/0_stateless/00975_live_view_create.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00975_live_view_create.sql b/dbms/tests/queries/0_stateless/00975_live_view_create.sql new file mode 100644 index 00000000000..1c929b15b00 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00975_live_view_create.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00976_live_view_select_version.reference b/dbms/tests/queries/0_stateless/00976_live_view_select_version.reference new file mode 100644 index 00000000000..453bd800469 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00976_live_view_select_version.reference @@ -0,0 +1,3 @@ +1 1 +2 1 +3 1 diff --git a/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql b/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql new file mode 100644 index 00000000000..5f3ab1f7546 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT *,_version FROM test.lv; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00977_live_view_watch_events.reference b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.reference new file mode 100644 index 00000000000..01e79c32a8c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.reference @@ -0,0 +1,3 @@ +1 +2 +3 diff --git a/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql new file mode 100644 index 00000000000..a3b84e8d4c1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +WATCH test.lv EVENTS LIMIT 0; + +INSERT INTO test.mt VALUES (1),(2),(3); + +WATCH test.lv EVENTS LIMIT 0; + +INSERT INTO test.mt VALUES (4),(5),(6); + +WATCH test.lv EVENTS LIMIT 0; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00978_live_view_watch.reference b/dbms/tests/queries/0_stateless/00978_live_view_watch.reference new file mode 100644 index 00000000000..6fbbedf1b21 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00978_live_view_watch.reference @@ -0,0 +1,3 @@ +0 1 +6 2 +21 3 diff --git a/dbms/tests/queries/0_stateless/00978_live_view_watch.sql b/dbms/tests/queries/0_stateless/00978_live_view_watch.sql new file mode 100644 index 00000000000..abe4a6c32ae --- /dev/null +++ b/dbms/tests/queries/0_stateless/00978_live_view_watch.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +WATCH test.lv LIMIT 0; + +INSERT INTO test.mt VALUES (1),(2),(3); + +WATCH test.lv LIMIT 0; + +INSERT INTO test.mt VALUES (4),(5),(6); + +WATCH test.lv LIMIT 0; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py new file mode 100755 index 00000000000..948e4c93662 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py @@ -0,0 +1,48 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv') + client1.expect(r'0.*1' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(r'6.*2' + end_of_block) + client2.expect(prompt) + client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') + client1.expect(r'21.*3' + end_of_block) + client2.expect(prompt) + for i in range(1,129): + client2.send('INSERT INTO test.mt VALUES (1)') + client1.expect(r'%d.*%d' % (21+i, 3+i) + end_of_block) + client2.expect(prompt) + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00979_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.reference b/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.reference new file mode 100644 index 00000000000..7f9fcbb2e9c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.reference @@ -0,0 +1,3 @@ +temporary_live_view_timeout 5 +live_view_heartbeat_interval 15 +0 diff --git a/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql b/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql new file mode 100644 index 00000000000..8cd6ee06ace --- /dev/null +++ b/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +SELECT name, value from system.settings WHERE name = 'temporary_live_view_timeout'; +SELECT name, value from system.settings WHERE name = 'live_view_heartbeat_interval'; + +SET temporary_live_view_timeout=1; +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +SHOW TABLES LIKE 'lv'; +SELECT sleep(2); +SHOW TABLES LIKE 'lv'; + +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python new file mode 100644 index 00000000000..782671cdfaf --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python @@ -0,0 +1,81 @@ +#!/usr/bin/env python + +import subprocess +import threading +import Queue as queue +import os +import sys +import signal + + +CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') +CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') +CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') + + +def send_query(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + + +def send_query_in_process_group(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) + + +def read_lines_and_push_to_queue(pipe, queue): + try: + for line in iter(pipe.readline, ''): + line = line.strip() + print(line) + sys.stdout.flush() + queue.put(line) + except KeyboardInterrupt: + pass + + queue.put(None) + + +def test(): + send_query('DROP TABLE IF EXISTS test.lv').read() + send_query('DROP TABLE IF EXISTS test.mt').read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() + send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + + q = queue.Queue() + p = send_query_in_process_group('WATCH test.lv') + thread = threading.Thread(target=read_lines_and_push_to_queue, args=(p.stdout, q)) + thread.start() + + line = q.get() + print(line) + assert (line == '0\t1') + + send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + line = q.get() + print(line) + assert (line == '6\t2') + + send_query('INSERT INTO test.mt VALUES (4),(5),(6)').read() + line = q.get() + print(line) + assert (line == '21\t3') + + # Send Ctrl+C to client. + os.killpg(os.getpgid(p.pid), signal.SIGINT) + # This insert shouldn't affect lv. + send_query('INSERT INTO test.mt VALUES (7),(8),(9)').read() + line = q.get() + print(line) + assert (line is None) + + send_query('DROP TABLE if exists test.lv').read() + send_query('DROP TABLE if exists test.lv').read() + + thread.join() + +test() diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference new file mode 100644 index 00000000000..1e94cdade41 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference @@ -0,0 +1,7 @@ +0 1 +0 1 +6 2 +6 2 +21 3 +21 3 +None diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled new file mode 100755 index 00000000000..10e4e98b2e3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +python $CURDIR/00991_live_view_watch_event_live.python diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python new file mode 100755 index 00000000000..938547ca0cb --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python @@ -0,0 +1,63 @@ +#!/usr/bin/env python + +import subprocess +import threading +import Queue as queue +import os +import sys + + +CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') +CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') +CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') + + +def send_query(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + + +def send_http_query(query): + cmd = list(CLICKHOUSE_CURL.split()) # list(['curl', '-sSN', '--max-time', '10']) + cmd += ['-sSN', CLICKHOUSE_URL, '-d', query] + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + + +def read_lines_and_push_to_queue(pipe, queue): + for line in iter(pipe.readline, ''): + line = line.strip() + print(line) + sys.stdout.flush() + queue.put(line) + + queue.put(None) + + +def test(): + send_query('DROP TABLE IF EXISTS test.lv').read() + send_query('DROP TABLE IF EXISTS test.mt').read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() + send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + + q = queue.Queue() + pipe = send_http_query('WATCH test.lv') + thread = threading.Thread(target=read_lines_and_push_to_queue, args=(pipe, q)) + thread.start() + + line = q.get() + print(line) + assert (line == '0\t1') + + send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + line = q.get() + print(line) + assert (line == '6\t2') + + send_query('DROP TABLE if exists test.lv').read() + send_query('DROP TABLE if exists test.lv').read() + + thread.join() + +test() diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference new file mode 100644 index 00000000000..489457d751b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference @@ -0,0 +1,4 @@ +0 1 +0 1 +6 2 +6 2 diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled new file mode 100755 index 00000000000..88cce77f595 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +python $CURDIR/00991_live_view_watch_http.python diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python new file mode 100644 index 00000000000..70063adc6e3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python @@ -0,0 +1,83 @@ +#!/usr/bin/env python + +import subprocess +import threading +import Queue as queue +import os +import sys +import signal + + +CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') +CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') +CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') + + +def send_query(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + + +def send_query_in_process_group(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query, '--live_view_heartbeat_interval=1', '--progress'] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) + + +def read_lines_and_push_to_queue(pipe, queue): + try: + for line in iter(pipe.readline, ''): + line = line.strip() + # print(line) + sys.stdout.flush() + queue.put(line) + except KeyboardInterrupt: + pass + + queue.put(None) + + +def test(): + send_query('DROP TABLE IF EXISTS test.lv').read() + send_query('DROP TABLE IF EXISTS test.mt').read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() + send_query('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + + q = queue.Queue() + p = send_query_in_process_group('WATCH test.lv') + thread = threading.Thread(target=read_lines_and_push_to_queue, args=(p.stdout, q)) + thread.start() + + line = q.get() + # print(line) + assert (line.endswith('0\t1')) + assert ('Progress: 0.00 rows' in line) + + send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + line = q.get() + assert (line.endswith('6\t2')) + assert ('Progress: 1.00 rows' in line) + + # send_query('INSERT INTO test.mt VALUES (4),(5),(6)').read() + # line = q.get() + # print(line) + # assert (line.endswith('6\t2')) + # assert ('Progress: 1.00 rows' in line) + + # Send Ctrl+C to client. + os.killpg(os.getpgid(p.pid), signal.SIGINT) + # This insert shouldn't affect lv. + send_query('INSERT INTO test.mt VALUES (7),(8),(9)').read() + line = q.get() + # print(line) + # assert (line is None) + + send_query('DROP TABLE if exists test.lv').read() + send_query('DROP TABLE if exists test.lv').read() + + thread.join() + +test() diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.reference b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled new file mode 100755 index 00000000000..f7aa13d52b3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +python $CURDIR/00991_temporary_live_view_watch_events_heartbeat.python diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python new file mode 100644 index 00000000000..d290018a02c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python @@ -0,0 +1,81 @@ +#!/usr/bin/env python + +import subprocess +import threading +import Queue as queue +import os +import sys +import signal + + +CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') +CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') +CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') + + +def send_query(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + + +def send_query_in_process_group(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) + + +def read_lines_and_push_to_queue(pipe, queue): + try: + for line in iter(pipe.readline, ''): + line = line.strip() + print(line) + sys.stdout.flush() + queue.put(line) + except KeyboardInterrupt: + pass + + queue.put(None) + + +def test(): + send_query('DROP TABLE IF EXISTS test.lv').read() + send_query('DROP TABLE IF EXISTS test.mt').read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() + send_query('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + + q = queue.Queue() + p = send_query_in_process_group('WATCH test.lv') + thread = threading.Thread(target=read_lines_and_push_to_queue, args=(p.stdout, q)) + thread.start() + + line = q.get() + print(line) + assert (line == '0\t1') + + send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + line = q.get() + print(line) + assert (line == '6\t2') + + send_query('INSERT INTO test.mt VALUES (4),(5),(6)').read() + line = q.get() + print(line) + assert (line == '21\t3') + + # Send Ctrl+C to client. + os.killpg(os.getpgid(p.pid), signal.SIGINT) + # This insert shouldn't affect lv. + send_query('INSERT INTO test.mt VALUES (7),(8),(9)').read() + line = q.get() + print(line) + assert (line is None) + + send_query('DROP TABLE if exists test.lv').read() + send_query('DROP TABLE if exists test.lv').read() + + thread.join() + +test() diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference new file mode 100644 index 00000000000..1e94cdade41 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference @@ -0,0 +1,7 @@ +0 1 +0 1 +6 2 +6 2 +21 3 +21 3 +None diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled new file mode 100755 index 00000000000..4d01d1c3a8e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +python $CURDIR/00991_temporary_live_view_watch_live.python diff --git a/dbms/tests/queries/0_stateless/helpers/client.py b/dbms/tests/queries/0_stateless/helpers/client.py new file mode 100644 index 00000000000..f3938d3bf63 --- /dev/null +++ b/dbms/tests/queries/0_stateless/helpers/client.py @@ -0,0 +1,36 @@ +import os +import sys +import time + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +sys.path.insert(0, os.path.join(CURDIR)) + +import uexpect + +prompt = ':\) ' +end_of_block = r'.*\r\n.*\r\n' + +class client(object): + def __init__(self, command=None, name='', log=None): + self.client = uexpect.spawn(['/bin/bash','--noediting']) + if command is None: + command = os.environ.get('CLICKHOUSE_BINARY', 'clickhouse') + '-client' + self.client.command = command + self.client.eol('\r') + self.client.logger(log, prefix=name) + self.client.timeout(20) + self.client.expect('[#\$] ', timeout=2) + self.client.send(command) + + def __enter__(self): + return self.client.__enter__() + + def __exit__(self, type, value, traceback): + self.client.reader['kill_event'].set() + # send Ctrl-C + self.client.send('\x03', eol='') + time.sleep(0.3) + self.client.send('quit', eol='\r') + self.client.send('\x03', eol='') + return self.client.__exit__(type, value, traceback) diff --git a/dbms/tests/queries/0_stateless/helpers/httpclient.py b/dbms/tests/queries/0_stateless/helpers/httpclient.py new file mode 100644 index 00000000000..a42fad2cbc3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/helpers/httpclient.py @@ -0,0 +1,14 @@ +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +sys.path.insert(0, os.path.join(CURDIR)) + +import httpexpect + +def client(request, name='', log=None): + client = httpexpect.spawn({'host':'localhost','port':8123}, request) + client.logger(log, prefix=name) + client.timeout(20) + return client diff --git a/dbms/tests/queries/0_stateless/helpers/httpexpect.py b/dbms/tests/queries/0_stateless/helpers/httpexpect.py new file mode 100644 index 00000000000..e440dafce4e --- /dev/null +++ b/dbms/tests/queries/0_stateless/helpers/httpexpect.py @@ -0,0 +1,73 @@ +# Copyright (c) 2019 Vitaliy Zakaznikov +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +import os +import sys +import httplib + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, CURDIR) + +import uexpect + +from threading import Thread, Event +from Queue import Queue, Empty + +class IO(uexpect.IO): + def __init__(self, connection, response, queue, reader): + self.connection = connection + self.response = response + super(IO, self).__init__(None, None, queue, reader) + + def write(self, data): + raise NotImplementedError + + def close(self, force=True): + self.reader['kill_event'].set() + self.connection.close() + if self._logger: + self._logger.write('\n') + self._logger.flush() + + +def reader(response, queue, kill_event): + while True: + try: + if kill_event.is_set(): + break + data = response.read(1) + queue.put(data) + except Exception, e: + if kill_event.is_set(): + break + raise + +def spawn(connection, request): + connection = httplib.HTTPConnection(**connection) + connection.request(**request) + response = connection.getresponse() + + queue = Queue() + reader_kill_event = Event() + thread = Thread(target=reader, args=(response, queue, reader_kill_event)) + thread.daemon = True + thread.start() + + return IO(connection, response, queue, reader={'thread':thread, 'kill_event':reader_kill_event}) + +if __name__ == '__main__': + with http({'host':'localhost','port':8123},{'method':'GET', 'url':'?query=SELECT%201'}) as client: + client.logger(sys.stdout) + client.timeout(2) + print client.response.status, client.response.reason + client.expect('1\n') diff --git a/dbms/tests/queries/0_stateless/helpers/uexpect.py b/dbms/tests/queries/0_stateless/helpers/uexpect.py new file mode 100644 index 00000000000..f71b32a53e1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/helpers/uexpect.py @@ -0,0 +1,206 @@ +# Copyright (c) 2019 Vitaliy Zakaznikov +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +import os +import pty +import time +import sys +import re + +from threading import Thread, Event +from subprocess import Popen +from Queue import Queue, Empty + +class TimeoutError(Exception): + def __init__(self, timeout): + self.timeout = timeout + + def __str__(self): + return 'Timeout %.3fs' % float(self.timeout) + +class ExpectTimeoutError(Exception): + def __init__(self, pattern, timeout, buffer): + self.pattern = pattern + self.timeout = timeout + self.buffer = buffer + + def __str__(self): + s = 'Timeout %.3fs ' % float(self.timeout) + if self.pattern: + s += 'for %s ' % repr(self.pattern.pattern) + if self.buffer: + s += 'buffer %s ' % repr(self.buffer[:]) + s += 'or \'%s\'' % ','.join(['%x' % ord(c) for c in self.buffer[:]]) + return s + +class IO(object): + class EOF(object): + pass + + class Timeout(object): + pass + + EOF = EOF + TIMEOUT = Timeout + + class Logger(object): + def __init__(self, logger, prefix=''): + self._logger = logger + self._prefix = prefix + + def write(self, data): + self._logger.write(('\n' + data).replace('\n','\n' + self._prefix)) + + def flush(self): + self._logger.flush() + + def __init__(self, process, master, queue, reader): + self.process = process + self.master = master + self.queue = queue + self.buffer = None + self.before = None + self.after = None + self.match = None + self.pattern = None + self.reader = reader + self._timeout = None + self._logger = None + self._eol = '' + + def __enter__(self): + return self + + def __exit__(self, type, value, traceback): + self.close() + + def logger(self, logger=None, prefix=''): + if logger: + self._logger = self.Logger(logger, prefix=prefix) + return self._logger + + def timeout(self, timeout=None): + if timeout: + self._timeout = timeout + return self._timeout + + def eol(self, eol=None): + if eol: + self._eol = eol + return self._eol + + def close(self, force=True): + self.reader['kill_event'].set() + os.system('pkill -TERM -P %d' % self.process.pid) + if force: + self.process.kill() + else: + self.process.terminate() + os.close(self.master) + if self._logger: + self._logger.write('\n') + self._logger.flush() + + def send(self, data, eol=None): + if eol is None: + eol = self._eol + return self.write(data + eol) + + def write(self, data): + return os.write(self.master, data) + + def expect(self, pattern, timeout=None, escape=False): + self.match = None + self.before = None + self.after = None + if escape: + pattern = re.escape(pattern) + pattern = re.compile(pattern) + if timeout is None: + timeout = self._timeout + timeleft = timeout + while True: + start_time = time.time() + if self.buffer is not None: + self.match = pattern.search(self.buffer, 0) + if self.match is not None: + self.after = self.buffer[self.match.start():self.match.end()] + self.before = self.buffer[:self.match.start()] + self.buffer = self.buffer[self.match.end():] + break + if timeleft < 0: + break + try: + data = self.read(timeout=timeleft, raise_exception=True) + except TimeoutError: + if self._logger: + self._logger.write((self.buffer or '') + '\n') + self._logger.flush() + exception = ExpectTimeoutError(pattern, timeout, self.buffer) + self.buffer = None + raise exception + timeleft -= (time.time() - start_time) + if data: + self.buffer = (self.buffer + data) if self.buffer else data + if self._logger: + self._logger.write((self.before or '') + (self.after or '')) + self._logger.flush() + if self.match is None: + exception = ExpectTimeoutError(pattern, timeout, self.buffer) + self.buffer = None + raise exception + return self.match + + def read(self, timeout=0, raise_exception=False): + data = '' + timeleft = timeout + try: + while timeleft >= 0 : + start_time = time.time() + data += self.queue.get(timeout=timeleft) + if data: + break + timeleft -= (time.time() - start_time) + except Empty: + if data: + return data + if raise_exception: + raise TimeoutError(timeout) + pass + if not data and raise_exception: + raise TimeoutError(timeout) + + return data + +def spawn(command): + master, slave = pty.openpty() + process = Popen(command, preexec_fn=os.setsid, stdout=slave, stdin=slave, stderr=slave, bufsize=1) + os.close(slave) + + queue = Queue() + reader_kill_event = Event() + thread = Thread(target=reader, args=(process, master, queue, reader_kill_event)) + thread.daemon = True + thread.start() + + return IO(process, master, queue, reader={'thread':thread, 'kill_event':reader_kill_event}) + +def reader(process, out, queue, kill_event): + while True: + try: + data = os.read(out, 65536) + queue.put(data) + except: + if kill_event.is_set(): + break + raise From 62988800e61042c131bec717b36a369f8a419ca0 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sat, 24 Aug 2019 21:40:24 -0400 Subject: [PATCH 421/509] Rewriting implementation of LIVE VIEW no users thread. --- .../LiveView/LiveViewBlockInputStream.h | 21 +++--- .../LiveView/LiveViewEventsBlockInputStream.h | 20 +++--- .../src/Storages/LiveView/StorageLiveView.cpp | 66 +++++++++---------- dbms/src/Storages/LiveView/StorageLiveView.h | 16 ++--- 4 files changed, 63 insertions(+), 60 deletions(-) diff --git a/dbms/src/Storages/LiveView/LiveViewBlockInputStream.h b/dbms/src/Storages/LiveView/LiveViewBlockInputStream.h index 345fceaf095..60839f3e66f 100644 --- a/dbms/src/Storages/LiveView/LiveViewBlockInputStream.h +++ b/dbms/src/Storages/LiveView/LiveViewBlockInputStream.h @@ -18,10 +18,13 @@ using NonBlockingResult = std::pair; public: ~LiveViewBlockInputStream() override { - /// Start storage no users thread - /// if we are the last active user - if (!storage->is_dropped && blocks_ptr.use_count() < 3) - storage->startNoUsersThread(temporary_live_view_timeout_sec); + /// Wake up no users thread + { + std::cerr << "DEBUG: live view block input stream ... send wake up thread\n"; + std::lock_guard lock(storage->no_users_thread_mutex); + storage->no_users_thread_wakeup = true; + storage->no_users_thread_condition.notify_one(); + } } LiveViewBlockInputStream(std::shared_ptr storage_, @@ -29,9 +32,12 @@ public: std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, const bool has_limit_, const UInt64 limit_, - const UInt64 heartbeat_interval_sec_, - const UInt64 temporary_live_view_timeout_sec_) - : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000), temporary_live_view_timeout_sec(temporary_live_view_timeout_sec_) + const UInt64 heartbeat_interval_sec_) + : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), + blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), + active_ptr(std::move(active_ptr_)), + has_limit(has_limit_), limit(limit_), + heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000) { /// grab active pointer active = active_ptr.lock(); @@ -200,7 +206,6 @@ private: Int64 num_updates = -1; bool end_of_blocks = false; UInt64 heartbeat_interval_usec; - UInt64 temporary_live_view_timeout_sec; UInt64 last_event_timestamp_usec = 0; Poco::Timestamp timestamp; }; diff --git a/dbms/src/Storages/LiveView/LiveViewEventsBlockInputStream.h b/dbms/src/Storages/LiveView/LiveViewEventsBlockInputStream.h index 120d0098536..e0e6ff78d21 100644 --- a/dbms/src/Storages/LiveView/LiveViewEventsBlockInputStream.h +++ b/dbms/src/Storages/LiveView/LiveViewEventsBlockInputStream.h @@ -37,10 +37,12 @@ using NonBlockingResult = std::pair; public: ~LiveViewEventsBlockInputStream() override { - /// Start storage no users thread - /// if we are the last active user - if (!storage->is_dropped && blocks_ptr.use_count() < 3) - storage->startNoUsersThread(temporary_live_view_timeout_sec); + /// Wake up no users thread + { + std::lock_guard lock(storage->no_users_thread_mutex); + storage->no_users_thread_wakeup = true; + storage->no_users_thread_condition.notify_one(); + } } /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update /// and LIMIT 0 just returns data without waiting for any updates @@ -49,9 +51,12 @@ public: std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, const bool has_limit_, const UInt64 limit_, - const UInt64 heartbeat_interval_sec_, - const UInt64 temporary_live_view_timeout_sec_) - : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000), temporary_live_view_timeout_sec(temporary_live_view_timeout_sec_) + const UInt64 heartbeat_interval_sec_) + : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), + blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), + active_ptr(std::move(active_ptr_)), has_limit(has_limit_), + limit(limit_), + heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000) { /// grab active pointer active = active_ptr.lock(); @@ -236,7 +241,6 @@ private: Int64 num_updates = -1; bool end_of_blocks = false; UInt64 heartbeat_interval_usec; - UInt64 temporary_live_view_timeout_sec; UInt64 last_event_timestamp_usec = 0; Poco::Timestamp timestamp; }; diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 3c0d205fa3f..d5de3b4a914 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -363,45 +363,49 @@ void StorageLiveView::checkTableCanBeDropped() const } } -void StorageLiveView::noUsersThread(const UInt64 & timeout) +void StorageLiveView::noUsersThread(std::shared_ptr storage, const UInt64 & timeout) { - if (shutdown_called) - return; - bool drop_table = false; + if (storage->shutdown_called || storage->is_dropped) + return; + { while (1) { - std::unique_lock lock(no_users_thread_mutex); - if (!no_users_thread_condition.wait_for(lock, std::chrono::seconds(timeout), [&] { return no_users_thread_wakeup; })) + std::unique_lock lock(storage->no_users_thread_mutex); + if(!storage->no_users_thread_condition.wait_for(lock, std::chrono::seconds(timeout), [&] { return storage->no_users_thread_wakeup; })) { - no_users_thread_wakeup = false; - if (shutdown_called) + storage->no_users_thread_wakeup = false; + if (storage->shutdown_called || storage->is_dropped) return; - if (hasUsers()) - return; - if (!global_context.getDependencies(database_name, table_name).empty()) + if (storage->hasUsers()) + continue; + if (!storage->global_context.getDependencies(storage->database_name, storage->table_name).empty()) continue; drop_table = true; } + else { + storage->no_users_thread_wakeup = false; + continue; + } break; } } if (drop_table) { - if (global_context.tryGetTable(database_name, table_name)) + if (storage->global_context.tryGetTable(storage->database_name, storage->table_name)) { try { /// We create and execute `drop` query for this table auto drop_query = std::make_shared(); - drop_query->database = database_name; - drop_query->table = table_name; + drop_query->database = storage->database_name; + drop_query->table = storage->table_name; drop_query->kind = ASTDropQuery::Kind::Drop; ASTPtr ast_drop_query = drop_query; - InterpreterDropQuery drop_interpreter(ast_drop_query, global_context); + InterpreterDropQuery drop_interpreter(ast_drop_query, storage->global_context); drop_interpreter.execute(); } catch (...) @@ -413,13 +417,6 @@ void StorageLiveView::noUsersThread(const UInt64 & timeout) void StorageLiveView::startNoUsersThread(const UInt64 & timeout) { - bool expected = false; - if (!start_no_users_thread_called.compare_exchange_strong(expected, true)) - return; - - if (is_dropped) - return; - if (is_temporary) { if (no_users_thread.joinable()) @@ -435,10 +432,9 @@ void StorageLiveView::startNoUsersThread(const UInt64 & timeout) std::lock_guard lock(no_users_thread_mutex); no_users_thread_wakeup = false; } - if (!is_dropped) - no_users_thread = std::thread(&StorageLiveView::noUsersThread, this, timeout); + no_users_thread = std::thread(&StorageLiveView::noUsersThread, + std::static_pointer_cast(shared_from_this()), timeout); } - start_no_users_thread_called = false; } void StorageLiveView::startup() @@ -454,19 +450,19 @@ void StorageLiveView::shutdown() if (no_users_thread.joinable()) { - std::lock_guard lock(no_users_thread_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - /// Must detach the no users thread - /// as we can't join it as it will result - /// in a deadlock - no_users_thread.detach(); /// TODO Not viable at all. + { + std::lock_guard lock(no_users_thread_mutex); + no_users_thread_wakeup = true; + no_users_thread_condition.notify_one(); + } } } StorageLiveView::~StorageLiveView() { shutdown(); + if (no_users_thread.joinable()) + no_users_thread.detach(); } void StorageLiveView::drop() @@ -532,8 +528,7 @@ BlockInputStreams StorageLiveView::watch( if (query.is_watch_events) { - auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), - context.getSettingsRef().temporary_live_view_timeout.totalSeconds()); + auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds()); if (no_users_thread.joinable()) { @@ -557,8 +552,7 @@ BlockInputStreams StorageLiveView::watch( } else { - auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), - context.getSettingsRef().temporary_live_view_timeout.totalSeconds()); + auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds()); if (no_users_thread.joinable()) { diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index 9930d8d6154..a9a8985b4f8 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -71,11 +71,9 @@ public: { return active_ptr.use_count() > 1; } - /// Background thread for temporary tables - /// which drops this table if there are no users - void startNoUsersThread(const UInt64 & timeout); + /// No users thread mutex, predicate and wake up condition std::mutex no_users_thread_mutex; - bool no_users_thread_wakeup{false}; + bool no_users_thread_wakeup = false; std::condition_variable no_users_thread_condition; /// Get blocks hash /// must be called with mutex locked @@ -149,7 +147,7 @@ private: String database_name; ASTPtr inner_query; Context & global_context; - bool is_temporary {false}; + bool is_temporary = false; mutable Block sample_block; /// Mutex for the blocks and ready condition @@ -166,10 +164,12 @@ private: std::shared_ptr blocks_metadata_ptr; BlocksPtrs mergeable_blocks; - void noUsersThread(const UInt64 & timeout); + /// Background thread for temporary tables + /// which drops this table if there are no users + void startNoUsersThread(const UInt64 & timeout); + static void noUsersThread(std::shared_ptr storage, const UInt64 & timeout); std::thread no_users_thread; - std::atomic shutdown_called{false}; - std::atomic start_no_users_thread_called{false}; + std::atomic shutdown_called = false; UInt64 temporary_live_view_timeout; StorageLiveView( From 2342d64d1b8f2c2b7fa2946d65664432a6a890b9 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sun, 25 Aug 2019 07:36:08 -0400 Subject: [PATCH 422/509] * Updating no users thread to sleep 3 times longer when users are present to reduce the number of times thread wakes up. * Updating startNoUsersThread to return if the thread is already running. --- .../src/Storages/LiveView/StorageLiveView.cpp | 33 +++++++++++++------ 1 file changed, 23 insertions(+), 10 deletions(-) diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index d5de3b4a914..6ceabf3478c 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -366,27 +366,40 @@ void StorageLiveView::checkTableCanBeDropped() const void StorageLiveView::noUsersThread(std::shared_ptr storage, const UInt64 & timeout) { bool drop_table = false; + UInt64 next_timeout = timeout; - if (storage->shutdown_called || storage->is_dropped) + if (storage->shutdown_called) return; { while (1) { std::unique_lock lock(storage->no_users_thread_mutex); - if(!storage->no_users_thread_condition.wait_for(lock, std::chrono::seconds(timeout), [&] { return storage->no_users_thread_wakeup; })) + if (!storage->no_users_thread_condition.wait_for(lock, std::chrono::seconds(next_timeout), [&] { return storage->no_users_thread_wakeup; })) { storage->no_users_thread_wakeup = false; - if (storage->shutdown_called || storage->is_dropped) + if (storage->shutdown_called) return; if (storage->hasUsers()) + { + /// Thread woke up but there are still users so sleep for 3 times longer than + /// the original timeout to reduce the number of times thread wakes up. + /// Wait until we are explicitely woken up when a user goes away to + /// reset wait time to the original timeout. + next_timeout = timeout * 3; continue; + } if (!storage->global_context.getDependencies(storage->database_name, storage->table_name).empty()) continue; drop_table = true; } - else { + else + { + /// Thread was explicitly awaken so reset timeout to the original + next_timeout = timeout; storage->no_users_thread_wakeup = false; + if (storage->shutdown_called) + return; continue; } break; @@ -421,12 +434,12 @@ void StorageLiveView::startNoUsersThread(const UInt64 & timeout) { if (no_users_thread.joinable()) { - { - std::lock_guard lock(no_users_thread_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - } - no_users_thread.join(); + /// If the thread is already running then + /// wake it up and just return + std::lock_guard lock(no_users_thread_mutex); + no_users_thread_wakeup = true; + no_users_thread_condition.notify_one(); + return; } { std::lock_guard lock(no_users_thread_mutex); From 7fb13b12f9e15eb01ecac959c4e9b0e9a5f6eb53 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sun, 25 Aug 2019 08:27:47 -0400 Subject: [PATCH 423/509] Reverting to previous no users thread functionality to avoid keeping no users thread always alive for each live view. --- .../LiveView/LiveViewBlockInputStream.h | 18 +++--- .../LiveView/LiveViewEventsBlockInputStream.h | 17 +++--- .../src/Storages/LiveView/StorageLiveView.cpp | 56 +++++++++---------- dbms/src/Storages/LiveView/StorageLiveView.h | 3 +- 4 files changed, 47 insertions(+), 47 deletions(-) diff --git a/dbms/src/Storages/LiveView/LiveViewBlockInputStream.h b/dbms/src/Storages/LiveView/LiveViewBlockInputStream.h index 60839f3e66f..f73991ddb77 100644 --- a/dbms/src/Storages/LiveView/LiveViewBlockInputStream.h +++ b/dbms/src/Storages/LiveView/LiveViewBlockInputStream.h @@ -18,13 +18,10 @@ using NonBlockingResult = std::pair; public: ~LiveViewBlockInputStream() override { - /// Wake up no users thread - { - std::cerr << "DEBUG: live view block input stream ... send wake up thread\n"; - std::lock_guard lock(storage->no_users_thread_mutex); - storage->no_users_thread_wakeup = true; - storage->no_users_thread_condition.notify_one(); - } + /// Start storage no users thread + /// if we are the last active user + if (!storage->is_dropped && blocks_ptr.use_count() < 3) + storage->startNoUsersThread(temporary_live_view_timeout_sec); } LiveViewBlockInputStream(std::shared_ptr storage_, @@ -32,12 +29,14 @@ public: std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, const bool has_limit_, const UInt64 limit_, - const UInt64 heartbeat_interval_sec_) + const UInt64 heartbeat_interval_sec_, + const UInt64 temporary_live_view_timeout_sec_) : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), - heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000) + heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000), + temporary_live_view_timeout_sec(temporary_live_view_timeout_sec_) { /// grab active pointer active = active_ptr.lock(); @@ -206,6 +205,7 @@ private: Int64 num_updates = -1; bool end_of_blocks = false; UInt64 heartbeat_interval_usec; + UInt64 temporary_live_view_timeout_sec; UInt64 last_event_timestamp_usec = 0; Poco::Timestamp timestamp; }; diff --git a/dbms/src/Storages/LiveView/LiveViewEventsBlockInputStream.h b/dbms/src/Storages/LiveView/LiveViewEventsBlockInputStream.h index e0e6ff78d21..3308ff2858b 100644 --- a/dbms/src/Storages/LiveView/LiveViewEventsBlockInputStream.h +++ b/dbms/src/Storages/LiveView/LiveViewEventsBlockInputStream.h @@ -37,12 +37,10 @@ using NonBlockingResult = std::pair; public: ~LiveViewEventsBlockInputStream() override { - /// Wake up no users thread - { - std::lock_guard lock(storage->no_users_thread_mutex); - storage->no_users_thread_wakeup = true; - storage->no_users_thread_condition.notify_one(); - } + /// Start storage no users thread + /// if we are the last active user + if (!storage->is_dropped && blocks_ptr.use_count() < 3) + storage->startNoUsersThread(temporary_live_view_timeout_sec); } /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update /// and LIMIT 0 just returns data without waiting for any updates @@ -51,12 +49,14 @@ public: std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, const bool has_limit_, const UInt64 limit_, - const UInt64 heartbeat_interval_sec_) + const UInt64 heartbeat_interval_sec_, + const UInt64 temporary_live_view_timeout_sec_) : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), - heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000) + heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000), + temporary_live_view_timeout_sec(temporary_live_view_timeout_sec_) { /// grab active pointer active = active_ptr.lock(); @@ -241,6 +241,7 @@ private: Int64 num_updates = -1; bool end_of_blocks = false; UInt64 heartbeat_interval_usec; + UInt64 temporary_live_view_timeout_sec; UInt64 last_event_timestamp_usec = 0; Poco::Timestamp timestamp; }; diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 6ceabf3478c..6c949424f68 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -366,7 +366,6 @@ void StorageLiveView::checkTableCanBeDropped() const void StorageLiveView::noUsersThread(std::shared_ptr storage, const UInt64 & timeout) { bool drop_table = false; - UInt64 next_timeout = timeout; if (storage->shutdown_called) return; @@ -375,33 +374,17 @@ void StorageLiveView::noUsersThread(std::shared_ptr storage, co while (1) { std::unique_lock lock(storage->no_users_thread_mutex); - if (!storage->no_users_thread_condition.wait_for(lock, std::chrono::seconds(next_timeout), [&] { return storage->no_users_thread_wakeup; })) + if (!storage->no_users_thread_condition.wait_for(lock, std::chrono::seconds(timeout), [&] { return storage->no_users_thread_wakeup; })) { storage->no_users_thread_wakeup = false; if (storage->shutdown_called) return; if (storage->hasUsers()) - { - /// Thread woke up but there are still users so sleep for 3 times longer than - /// the original timeout to reduce the number of times thread wakes up. - /// Wait until we are explicitely woken up when a user goes away to - /// reset wait time to the original timeout. - next_timeout = timeout * 3; - continue; - } + return; if (!storage->global_context.getDependencies(storage->database_name, storage->table_name).empty()) continue; drop_table = true; } - else - { - /// Thread was explicitly awaken so reset timeout to the original - next_timeout = timeout; - storage->no_users_thread_wakeup = false; - if (storage->shutdown_called) - return; - continue; - } break; } } @@ -430,24 +413,31 @@ void StorageLiveView::noUsersThread(std::shared_ptr storage, co void StorageLiveView::startNoUsersThread(const UInt64 & timeout) { + bool expected = false; + if (!start_no_users_thread_called.compare_exchange_strong(expected, true)) + return; + if (is_temporary) { if (no_users_thread.joinable()) { - /// If the thread is already running then - /// wake it up and just return - std::lock_guard lock(no_users_thread_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - return; + { + std::lock_guard lock(no_users_thread_mutex); + no_users_thread_wakeup = true; + no_users_thread_condition.notify_one(); + } + no_users_thread.join(); } { std::lock_guard lock(no_users_thread_mutex); no_users_thread_wakeup = false; } - no_users_thread = std::thread(&StorageLiveView::noUsersThread, - std::static_pointer_cast(shared_from_this()), timeout); + if (!is_dropped) + no_users_thread = std::thread(&StorageLiveView::noUsersThread, + std::static_pointer_cast(shared_from_this()), timeout); } + + start_no_users_thread_called = false; } void StorageLiveView::startup() @@ -541,7 +531,11 @@ BlockInputStreams StorageLiveView::watch( if (query.is_watch_events) { - auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds()); + auto reader = std::make_shared( + std::static_pointer_cast(shared_from_this()), + blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, + context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), + context.getSettingsRef().temporary_live_view_timeout.totalSeconds()); if (no_users_thread.joinable()) { @@ -565,7 +559,11 @@ BlockInputStreams StorageLiveView::watch( } else { - auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds()); + auto reader = std::make_shared( + std::static_pointer_cast(shared_from_this()), + blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, + context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), + context.getSettingsRef().temporary_live_view_timeout.totalSeconds()); if (no_users_thread.joinable()) { diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index a9a8985b4f8..f71d0758f03 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -72,6 +72,7 @@ public: return active_ptr.use_count() > 1; } /// No users thread mutex, predicate and wake up condition + void startNoUsersThread(const UInt64 & timeout); std::mutex no_users_thread_mutex; bool no_users_thread_wakeup = false; std::condition_variable no_users_thread_condition; @@ -166,10 +167,10 @@ private: /// Background thread for temporary tables /// which drops this table if there are no users - void startNoUsersThread(const UInt64 & timeout); static void noUsersThread(std::shared_ptr storage, const UInt64 & timeout); std::thread no_users_thread; std::atomic shutdown_called = false; + std::atomic start_no_users_thread_called = false; UInt64 temporary_live_view_timeout; StorageLiveView( From 12cb72175b8e5ad156a03ea9f8d45dd453d62a1d Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sun, 25 Aug 2019 20:44:03 -0400 Subject: [PATCH 424/509] Updating test server config to enable live views using the allow_experimental_live_view option. --- dbms/tests/server-test.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/tests/server-test.xml b/dbms/tests/server-test.xml index d68cbca53c1..4a4e0a333ff 100644 --- a/dbms/tests/server-test.xml +++ b/dbms/tests/server-test.xml @@ -43,6 +43,7 @@ + 1 3 /tmp/clickhouse/data/ /tmp/clickhouse/tmp/ From 0599bdba25b0f0e9828634554869e2bb14492c09 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sun, 25 Aug 2019 22:34:16 -0300 Subject: [PATCH 425/509] Described arbitrary datatype for range_hashed --- .../dicts/external_dicts_dict_layout.md | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/docs/ru/query_language/dicts/external_dicts_dict_layout.md b/docs/ru/query_language/dicts/external_dicts_dict_layout.md index aafcf531860..826d9b78ae9 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_layout.md @@ -95,7 +95,7 @@ Словарь хранится в оперативной памяти в виде хэш-таблицы с упорядоченным массивом диапазонов и соответствующих им значений. -Этот способ размещения работает также как и hashed и позволяет дополнительно к ключу использовать дипазоны по дате/времени, если они указаны в словаре. +Этот способ размещения работает также как и hashed и позволяет дополнительно к ключу использовать дипазоны по дате/времени (произвольному числовому типу). Пример: таблица содержит скидки для каждого рекламодателя в виде: @@ -111,7 +111,7 @@ +---------------+---------------------+-------------------+--------+ ``` -Чтобы использовать выборку по диапазонам дат, необходимо в [structure](external_dicts_dict_structure.md) определить элементы `range_min`, `range_max`. +Чтобы использовать выборку по диапазонам дат, необходимо в [structure](external_dicts_dict_structure.md) определить элементы `range_min`, `range_max`. В этих элементах должны присутствовать элементы `name` и `type` (если `type` не указан, будет использован тип по умолчанию -- Date). `type` может быть любым численным типом (Date/DateTime/UInt64/Int32/др.). Пример: @@ -122,14 +122,16 @@ first + Date last + Date ... ``` -Для работы с такими словарями в функцию `dictGetT` необходимо передавать дополнительный аргумент - дату: : +Для работы с такими словарями в функцию `dictGetT` необходимо передавать дополнительный аргумент, для которого подбирается диапазон: dictGetT('dict_name', 'attr_name', id, date) @@ -158,10 +160,12 @@ Abcdef - StartDate + StartTimeStamp + UInt64 - EndDate + EndTimeStamp + UInt64 XXXType From 0a4364370a0a2c54ca9ba12d3b97ef4ca960c79b Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sun, 25 Aug 2019 22:44:13 -0300 Subject: [PATCH 426/509] Described arbitrary datatype for range_hashed --- .../dicts/external_dicts_dict_layout.md | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/docs/en/query_language/dicts/external_dicts_dict_layout.md b/docs/en/query_language/dicts/external_dicts_dict_layout.md index 03279688d6c..c3096544d25 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/en/query_language/dicts/external_dicts_dict_layout.md @@ -95,7 +95,7 @@ Configuration example: The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values. -This storage method works the same way as hashed and allows using date/time ranges in addition to the key, if they appear in the dictionary. +This storage method works the same way as hashed and allows using date/time (arbitrary numeric type) ranges in addition to the key. Example: The table contains discounts for each advertiser in the format: @@ -111,7 +111,7 @@ Example: The table contains discounts for each advertiser in the format: +---------------+---------------------+-------------------+--------+ ``` -To use a sample for date ranges, define the `range_min` and `range_max` elements in the [structure](external_dicts_dict_structure.md). +To use a sample for date ranges, define the `range_min` and `range_max` elements in the [structure](external_dicts_dict_structure.md). These elements must contain elements `name` and` type` (if `type` is not specified, the default type will be used - Date). `type` can be any numeric type (Date / DateTime / UInt64 / Int32 / others). Example: @@ -122,14 +122,16 @@ Example: first + Date last + Date ... ``` -To work with these dictionaries, you need to pass an additional date argument to the `dictGetT` function: +To work with these dictionaries, you need to pass an additional argument to the `dictGetT` function, for which a range is selected: ``` dictGetT('dict_name', 'attr_name', id, date) @@ -160,10 +162,12 @@ Configuration example: Abcdef - StartDate + StartTimeStamp + UInt64 - EndDate + EndTimeStamp + UInt64 XXXType From c70f656d5d6e6c83f6b3e3720d14bfc3277372b8 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sun, 25 Aug 2019 21:57:32 -0400 Subject: [PATCH 427/509] Another attempt to enable allow_experimental_live_view options during tests. --- dbms/tests/clickhouse-client.xml | 1 + dbms/tests/server-test.xml | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/clickhouse-client.xml b/dbms/tests/clickhouse-client.xml index b6003ca2d09..ebce35127e5 100644 --- a/dbms/tests/clickhouse-client.xml +++ b/dbms/tests/clickhouse-client.xml @@ -1,3 +1,4 @@ + 1 100000 diff --git a/dbms/tests/server-test.xml b/dbms/tests/server-test.xml index 4a4e0a333ff..d68cbca53c1 100644 --- a/dbms/tests/server-test.xml +++ b/dbms/tests/server-test.xml @@ -43,7 +43,6 @@ - 1 3 /tmp/clickhouse/data/ /tmp/clickhouse/tmp/ From 25635d1ab0e24762a8be06908f827f2bb7dbc3b8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Aug 2019 05:46:21 +0300 Subject: [PATCH 428/509] Fixed build of tests --- .../tests/gtest_transform_query_for_external_database.cpp | 2 +- dbms/src/Storages/tests/storage_log.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp index e61ab6279a2..a7d79cd23d7 100644 --- a/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -23,7 +23,7 @@ struct State { registerFunctions(); DatabasePtr database = std::make_shared("test"); - database->attachTable("table", StorageMemory::create("test", "table", ColumnsDescription{columns})); + database->attachTable("table", StorageMemory::create("test", "table", ColumnsDescription{columns}, ConstraintsDescription{})); context.makeGlobalContext(); context.addDatabase("test", database); context.setCurrentDatabase("test"); diff --git a/dbms/src/Storages/tests/storage_log.cpp b/dbms/src/Storages/tests/storage_log.cpp index bcf7b1d4d03..02a845cf61c 100644 --- a/dbms/src/Storages/tests/storage_log.cpp +++ b/dbms/src/Storages/tests/storage_log.cpp @@ -25,7 +25,7 @@ try names_and_types.emplace_back("a", std::make_shared()); names_and_types.emplace_back("b", std::make_shared()); - StoragePtr table = StorageLog::create("./", "test", "test", ColumnsDescription{names_and_types}, 1048576); + StoragePtr table = StorageLog::create("./", "test", "test", ColumnsDescription{names_and_types}, ConstraintsDescription{}, 1048576); table->startup(); auto context = Context::createGlobal(); From 6e8b7b9c3babde48b33916a1aace2ba079b7f0fd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Aug 2019 05:53:42 +0300 Subject: [PATCH 429/509] Fixed error --- dbms/src/Interpreters/InterpreterCreateQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index cb5e657bcea..59ae2fc2b72 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -567,6 +567,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) } ColumnsDescription columns; + ConstraintsDescription constraints; StoragePtr res; if (create.as_table_function) @@ -579,6 +580,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) { /// Set and retrieve list of columns. columns = setProperties(create, as_select_sample, as_storage); + constraints = getConstraintsDescription(create.columns_list->constraints); /// Check low cardinality types in creating table if it was not allowed in setting if (!create.attach && !context.getSettingsRef().allow_suspicious_low_cardinality_types && !create.is_materialized_view) @@ -598,8 +600,6 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) setEngine(create); } - ConstraintsDescription constraints = getConstraintsDescription(create.columns_list->constraints); - { std::unique_ptr guard; From d0d63d769efe7ea01cb01676dc5e3ab95098db2c Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Mon, 26 Aug 2019 11:00:48 +0300 Subject: [PATCH 430/509] Implemented hasTokenCaseInsensitive function And tests; --- dbms/src/Common/StringSearcher.h | 1 + dbms/src/Common/Volnitsky.h | 3 +- dbms/src/Functions/FunctionsStringSearch.cpp | 15 ++- .../queries/0_stateless/00990_hasToken.python | 100 ++++++++++++------ .../0_stateless/00990_hasToken.reference | 41 +++++++ 5 files changed, 123 insertions(+), 37 deletions(-) diff --git a/dbms/src/Common/StringSearcher.h b/dbms/src/Common/StringSearcher.h index 25287db11f5..d395e6f254a 100644 --- a/dbms/src/Common/StringSearcher.h +++ b/dbms/src/Common/StringSearcher.h @@ -775,6 +775,7 @@ using ASCIICaseInsensitiveStringSearcher = StringSearcher; using UTF8CaseSensitiveStringSearcher = StringSearcher; using UTF8CaseInsensitiveStringSearcher = StringSearcher; using ASCIICaseSensitiveTokenSearcher = TokenSearcher; +using ASCIICaseInsensitiveTokenSearcher = TokenSearcher; /** Uses functions from libc. diff --git a/dbms/src/Common/Volnitsky.h b/dbms/src/Common/Volnitsky.h index c87bdd79dab..a0273a1d858 100644 --- a/dbms/src/Common/Volnitsky.h +++ b/dbms/src/Common/Volnitsky.h @@ -652,7 +652,8 @@ using VolnitskyUTF8 = VolnitskyBase; /// ignores non-ASCII bytes using VolnitskyCaseInsensitiveUTF8 = VolnitskyBase; -using VolnitskyToken = VolnitskyBase; +using VolnitskyCaseSensitiveToken = VolnitskyBase; +using VolnitskyCaseInsensitiveToken = VolnitskyBase; using MultiVolnitsky = MultiVolnitskyBase; using MultiVolnitskyUTF8 = MultiVolnitskyBase; diff --git a/dbms/src/Functions/FunctionsStringSearch.cpp b/dbms/src/Functions/FunctionsStringSearch.cpp index 5d688232bd4..e44138bb482 100644 --- a/dbms/src/Functions/FunctionsStringSearch.cpp +++ b/dbms/src/Functions/FunctionsStringSearch.cpp @@ -436,7 +436,7 @@ struct MultiSearchFirstIndexImpl /** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation. */ -template +template struct HasTokenImpl { using ResultType = UInt8; @@ -454,7 +454,7 @@ struct HasTokenImpl /// The current index in the array of strings. size_t i = 0; - VolnitskyToken searcher(pattern.data(), pattern.size(), end - pos); + TokenSearcher searcher(pattern.data(), pattern.size(), end - pos); /// We will search for the next occurrence in all rows at once. while (pos < end && end != (pos = searcher.search(pos, end - pos))) @@ -483,7 +483,7 @@ struct HasTokenImpl static void constant_constant(const std::string & data, const std::string & pattern, UInt8 & res) { - VolnitskyToken searcher(pattern.data(), pattern.size(), data.size()); + TokenSearcher searcher(pattern.data(), pattern.size(), data.size()); const auto found = searcher.search(data.c_str(), data.size()) != data.end().base(); res = negate_result ^ found; } @@ -589,6 +589,11 @@ struct NameHasToken static constexpr auto name = "hasToken"; }; +struct NameHasTokenCaseInsensitive +{ + static constexpr auto name = "hasTokenCaseInsensitive"; +}; + using FunctionPosition = FunctionsStringSearch, NamePosition>; using FunctionPositionUTF8 = FunctionsStringSearch, NamePositionUTF8>; @@ -615,7 +620,8 @@ using FunctionMultiSearchFirstPositionUTF8 = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitive>; using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitiveUTF8>; -using FunctionHasToken = FunctionsStringSearch, NameHasToken>; +using FunctionHasToken = FunctionsStringSearch, NameHasToken>; +using FunctionHasTokenCaseInsensitive = FunctionsStringSearch, NameHasTokenCaseInsensitive>; void registerFunctionsStringSearch(FunctionFactory & factory) { @@ -645,6 +651,7 @@ void registerFunctionsStringSearch(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive); } diff --git a/dbms/tests/queries/0_stateless/00990_hasToken.python b/dbms/tests/queries/0_stateless/00990_hasToken.python index 217d96dfe52..cd2a284655f 100755 --- a/dbms/tests/queries/0_stateless/00990_hasToken.python +++ b/dbms/tests/queries/0_stateless/00990_hasToken.python @@ -2,11 +2,12 @@ # encoding: utf-8 import re +from string import Template HAYSTACKS = [ + "hay hay hay hay hay hay hay hay hay needle hay hay hay hay hay hay hay hay hay", "hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay needle", "needle hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay", - "hay hay hay hay hay hay hay hay hay needle hay hay hay hay hay hay hay hay hay", ] NEEDLE = "needle" @@ -48,47 +49,77 @@ def transform_needle(query, string_transformation_func): return NEEDLE_RE.sub(replace_with_transformation, query) - -def create_cases(table_row_template, table_query_template, const_query_template): +def create_cases(case_sensitive_func, case_insensitive_func, table_row_template, table_query_template, const_query_template): const_queries = [] table_rows = [] table_queries = set() - def add_case(haystack, needle, match): + def add_case(func, haystack, needle, match): match = int(match) - const_queries.append(const_query_template.format(haystack=haystack, needle=needle, match=match)) - table_queries.add(table_query_template.format(haystack=haystack, needle=needle, match=match)) - table_rows.append(table_row_template.format(haystack=haystack, needle=needle, match=match)) + args = dict( + func = func, + haystack = haystack, + needle = needle, + match = match + ) + const_queries.append(const_query_template.substitute(args)) + table_queries.add(table_query_template.substitute(args)) + table_rows.append(table_row_template.substitute(args)) + + def add_case_sensitive(haystack, needle, match): + add_case(case_sensitive_func, haystack, needle, match) + if match: + add_case(case_sensitive_func, transform_needle(haystack, str.swapcase), transform_needle(needle, str.swapcase), match) + + def add_case_insensitive(haystack, needle, match): + add_case(case_insensitive_func, haystack, needle, match) + if match: + add_case(case_insensitive_func, transform_needle(haystack, str.swapcase), needle, match) + add_case(case_insensitive_func, haystack, transform_needle(needle, str.swapcase), match) + # Negative cases - add_case(remove_needle(HAYSTACKS[0]), NEEDLE, False) + add_case_sensitive(remove_needle(HAYSTACKS[0]), NEEDLE, False) + add_case_insensitive(remove_needle(HAYSTACKS[0]), NEEDLE, False) + for haystack in HAYSTACKS: - add_case(transform_needle(haystack, str.title), NEEDLE, False) + add_case_sensitive(transform_needle(haystack, str.swapcase), NEEDLE, False) + sep = '' h = replace_separators(haystack, sep) - add_case(h, NEEDLE, False) - add_case(small_needle(h), small_needle(NEEDLE), False) - add_case(enlarge_haystack(h, 10, sep), NEEDLE, False) + + add_case_sensitive(h, NEEDLE, False) + add_case_insensitive(h, NEEDLE, False) + + add_case_sensitive(small_needle(h), small_needle(NEEDLE), False) + add_case_insensitive(small_needle(h), small_needle(NEEDLE), False) + + add_case_sensitive(enlarge_haystack(h, 10, sep), NEEDLE, False) + add_case_insensitive(enlarge_haystack(h, 10, sep), NEEDLE, False) # positive cases for haystack in HAYSTACKS: - add_case(transform_needle(haystack, str.title), transform_needle(NEEDLE, str.title), True) - add_case(transform_needle(haystack, str.upper), transform_needle(NEEDLE, str.upper), True) + add_case_sensitive(haystack, NEEDLE, True) + add_case_insensitive(haystack, NEEDLE, True) - # Not checking all separators since some (like ' and \n) cause issues when coupled with - # re-based replacement and quoting in query - # other are rare in practice and checking all separators makes this test too lengthy. - # r'\\\\' turns into a single '\' in query - #separators = list(''' \t`~!@#$%^&*()-=+|]}[{";:/?.>,<''') + [r'\\\\'] - separators = list(''' \t;:?.,''') + [r'\\\\'] - for sep in separators: + for sep in list(''' ,'''): h = replace_separators(haystack, sep) - add_case(h, NEEDLE, True) - add_case(small_needle(h), small_needle(NEEDLE), True) - add_case(enlarge_haystack(h, 200, sep), NEEDLE, True) - add_case(replace_needle(h, 'иголка'), replace_needle(NEEDLE, 'иголка'), True) - add_case(replace_needle(h, '指针'), replace_needle(NEEDLE, '指针'), True) + add_case_sensitive(h, NEEDLE, True) + add_case_sensitive(small_needle(h), small_needle(NEEDLE), True) + add_case_sensitive(enlarge_haystack(h, 200, sep), NEEDLE, True) + + add_case_insensitive(h, NEEDLE, True) + add_case_insensitive(small_needle(h), small_needle(NEEDLE), True) + add_case_insensitive(enlarge_haystack(h, 200, sep), NEEDLE, True) + + # case insesitivity works only on ASCII strings + add_case_sensitive(replace_needle(h, 'иголка'), replace_needle(NEEDLE, 'иголка'), True) + add_case_sensitive(replace_needle(h, '指针'), replace_needle(NEEDLE, '指针'), True) + + for sep in list('''~!@$%^&*()-=+|]}[{";:/?.><\t''') + [r'\\\\']: + h = replace_separators(HAYSTACKS[0], sep) + add_case(case_sensitive_func, h, NEEDLE, True) return table_rows, table_queries, const_queries @@ -97,12 +128,14 @@ def main(): def query(x): print x - CONST_QUERY = """SELECT hasToken('{haystack}', '{needle}'), ' expecting ', {match};""" - #SELECT hasToken(haystack, '{needle}') FROM ht WHERE needle = '{needle}' AND match = {match};""" - TABLE_QUERY = """WITH '{needle}' as n SELECT haystack, needle, hasToken(haystack, n) as result FROM ht WHERE needle = n AND result != match;""" - TABLE_ROW = """('{haystack}', '{needle}', {match})""" + CONST_QUERY = Template("""SELECT ${func}('${haystack}', '${needle}'), ' expecting ', ${match};""") + TABLE_QUERY = Template("""WITH '${needle}' as n + SELECT haystack, needle, ${func}(haystack, n) as result + FROM ht + WHERE func = '${func}' AND needle = n AND result != match;""") + TABLE_ROW = Template("""('${haystack}', '${needle}', ${match}, '${func}')""") - rows, table_queries, const_queries = create_cases(TABLE_ROW, TABLE_QUERY, CONST_QUERY) + rows, table_queries, const_queries = create_cases('hasToken', 'hasTokenCaseInsensitive', TABLE_ROW, TABLE_QUERY, CONST_QUERY) for q in const_queries: query(q) @@ -112,7 +145,8 @@ def main(): ( haystack String, needle String, - match UInt8 + match UInt8, + func String ) ENGINE MergeTree() ORDER BY haystack; @@ -120,5 +154,7 @@ INSERT INTO ht VALUES {values};""".format(values=", ".join(rows))) for q in sorted(table_queries): query(q) + query("""DROP TABLE ht""") + if __name__ == '__main__': main() diff --git a/dbms/tests/queries/0_stateless/00990_hasToken.reference b/dbms/tests/queries/0_stateless/00990_hasToken.reference index 867c0c1c691..1e8c067362c 100644 --- a/dbms/tests/queries/0_stateless/00990_hasToken.reference +++ b/dbms/tests/queries/0_stateless/00990_hasToken.reference @@ -11,6 +11,47 @@ 0 expecting 0 0 expecting 0 0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 1 expecting 1 1 expecting 1 1 expecting 1 From 8047aab684477c0a3bf7f78e30bdb954b011433f Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 26 Aug 2019 13:13:56 +0300 Subject: [PATCH 431/509] Add const --- dbms/src/Core/SettingsCommon.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index 8fc557f1768..b8c56d50caa 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -325,7 +325,7 @@ private: /// Can be updated after first load for config/definition. /// Non updatable settings can be `changed`, /// if they were overwritten in config/definition. - bool updateable; + const bool updateable; GetStringFunction get_string; GetFieldFunction get_field; SetStringFunction set_string; From c2b6cffabc709afd1862ceb7348564d54c9f74a4 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 26 Aug 2019 15:39:35 +0300 Subject: [PATCH 432/509] Update IStorage.cpp --- dbms/src/Storages/IStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index eb9b4adc4f4..855336c06e5 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -399,7 +399,7 @@ void IStorage::alterSettings( { if (hasSetting(change.name)) { - auto finder = [&change] (const SettingChange & c) { return c.name == change.name;}; + auto finder = [&change] (const SettingChange & c) { return c.name == change.name; }; if (auto it = std::find_if(storage_changes.begin(), storage_changes.end(), finder); it != storage_changes.end()) it->value = change.value; else From 4f89dcc80db1b3309f07c3828a72c88c49bb0675 Mon Sep 17 00:00:00 2001 From: Dmitry Rubashkin Date: Mon, 26 Aug 2019 15:47:19 +0300 Subject: [PATCH 433/509] Try to fix --- dbms/programs/client/Client.cpp | 2 +- dbms/src/Common/TerminalSize.cpp | 8 +++----- dbms/src/Common/TerminalSize.h | 3 +-- 3 files changed, 5 insertions(+), 8 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index e1f1bd92840..6caf57a28bd 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -131,7 +131,7 @@ private: bool print_time_to_stderr = false; /// Output execution time to stderr in batch mode. bool stdin_is_not_tty = false; /// stdin is not a terminal. - uint16_t terminal_width{}; /// Terminal width is needed to render progress bar. + uint16_t terminal_width = 0; /// Terminal width is needed to render progress bar. std::unique_ptr connection; /// Connection to DB. String query_id; /// Current query_id. diff --git a/dbms/src/Common/TerminalSize.cpp b/dbms/src/Common/TerminalSize.cpp index ffc301cf77c..7f6bb93119f 100644 --- a/dbms/src/Common/TerminalSize.cpp +++ b/dbms/src/Common/TerminalSize.cpp @@ -3,15 +3,13 @@ #include #include -namespace po = boost::program_options; - namespace DB::ErrorCodes { extern const int SYSTEM_ERROR; } -unsigned short int getTerminalWidth() +uint16_t getTerminalWidth() { if (isatty(STDIN_FILENO)) { @@ -25,13 +23,13 @@ unsigned short int getTerminalWidth() return 0; } -po::options_description createOptionsDescription(const std::string &caption, unsigned short terminal_width) +po::options_description createOptionsDescription(const std::string &caption, uint16_t terminal_width) { unsigned line_length = po::options_description::m_default_line_length; unsigned min_description_length = line_length / 2; std::string longest_option_desc = "--http_native_compression_disable_checksumming_on_decompress"; - line_length = std::max(static_cast(longest_option_desc.size()), terminal_width); + line_length = std::max(static_cast(longest_option_desc.size()), terminal_width); min_description_length = std::min(min_description_length, line_length - 2); return po::options_description(caption, line_length, min_description_length); diff --git a/dbms/src/Common/TerminalSize.h b/dbms/src/Common/TerminalSize.h index 4dd58b5c8ca..d7eee417f46 100644 --- a/dbms/src/Common/TerminalSize.h +++ b/dbms/src/Common/TerminalSize.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include @@ -8,7 +7,7 @@ namespace po = boost::program_options; -unsigned short int getTerminalWidth(); +uint16_t getTerminalWidth(); /** Creates po::options_description with name and an appropriate size for option displaying * when program is called with option --help From 2f5e6158e7a158e84e57317d8de7b66103ab3716 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 26 Aug 2019 16:27:12 +0300 Subject: [PATCH 434/509] Update TerminalSize.cpp --- dbms/src/Common/TerminalSize.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/TerminalSize.cpp b/dbms/src/Common/TerminalSize.cpp index 7f6bb93119f..ea8607b93ae 100644 --- a/dbms/src/Common/TerminalSize.cpp +++ b/dbms/src/Common/TerminalSize.cpp @@ -23,7 +23,7 @@ uint16_t getTerminalWidth() return 0; } -po::options_description createOptionsDescription(const std::string &caption, uint16_t terminal_width) +po::options_description createOptionsDescription(const std::string & caption, uint16_t terminal_width) { unsigned line_length = po::options_description::m_default_line_length; unsigned min_description_length = line_length / 2; From 479b4d81f314e11badd048052fe32e4ed226e646 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Aug 2019 16:46:07 +0300 Subject: [PATCH 435/509] Fixed error --- dbms/src/Storages/StorageDistributed.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index e3952634c00..90a973ae3b9 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -220,13 +220,17 @@ StorageDistributed::StorageDistributed( : table_name(table_name_), database_name(database_name_), remote_database(remote_database_), remote_table(remote_table_), global_context(context_), cluster_name(global_context.getMacros()->expand(cluster_name_)), has_sharding_key(sharding_key_), - sharding_key_expr(sharding_key_ ? buildShardingKeyExpression(sharding_key_, global_context, getColumns().getAllPhysical(), false) : nullptr), - sharding_key_column_name(sharding_key_ ? sharding_key_->getColumnName() : String{}), path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(table_name) + '/')) { setColumns(columns_); setConstraints(constraints_); + if (sharding_key_) + { + sharding_key_expr = buildShardingKeyExpression(sharding_key_, global_context, getColumns().getAllPhysical(), false); + sharding_key_column_name = sharding_key_->getColumnName(); + } + /// Sanity check. Skip check if the table is already created to allow the server to start. if (!attach_ && !cluster_name.empty()) { From dff0cc1e1cf9f87efaac8ba734bd140bbe80d88f Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 26 Aug 2019 17:24:29 +0300 Subject: [PATCH 436/509] Remove COW and add MultiVersion --- dbms/src/Interpreters/Context.cpp | 8 ++--- .../Storages/MergeTree/DataPartsExchange.cpp | 4 +-- .../MergeTree/IMergedBlockOutputStream.cpp | 4 +-- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 31 ++++++++-------- dbms/src/Storages/MergeTree/MergeTreeData.h | 35 +++++-------------- .../MergeTree/MergeTreeDataMergerMutator.cpp | 12 +++---- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 ++-- .../MergeTreeIndexGranularityInfo.cpp | 2 +- .../Storages/MergeTree/MergeTreeSettings.cpp | 5 --- .../Storages/MergeTree/MergeTreeSettings.h | 13 ++----- .../MergeTreeThreadSelectBlockInputStream.cpp | 2 +- .../ReplicatedMergeTreeCleanupThread.cpp | 8 ++--- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- .../ReplicatedMergeTreeRestartingThread.cpp | 6 ++-- .../ReplicatedMergeTreeTableMetadata.cpp | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 6 ++-- dbms/src/Storages/StorageMergeTree.h | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 32 ++++++++--------- .../src/Storages/StorageReplicatedMergeTree.h | 2 +- 20 files changed, 75 insertions(+), 109 deletions(-) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 39b9fd40c9a..016f1fa0e49 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -142,7 +142,7 @@ struct ContextShared std::unique_ptr ddl_worker; /// Process ddl commands from zk. /// Rules for selecting the compression settings, depending on the size of the part. mutable std::unique_ptr compression_codec_selector; - MergeTreeSettingsPtr merge_tree_settings; /// Settings of MergeTree* engines. + std::optional merge_tree_settings; /// Settings of MergeTree* engines. size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default) size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default) String format_schema_path; /// Path to a directory that contains schema files used by input formats. @@ -1759,9 +1759,9 @@ const MergeTreeSettings & Context::getMergeTreeSettings() const if (!shared->merge_tree_settings) { auto & config = getConfigRef(); - MutableMergeTreeSettingsPtr settings_ptr = MergeTreeSettings::create(); - settings_ptr->loadFromConfig("merge_tree", config); - shared->merge_tree_settings = std::move(settings_ptr); + MergeTreeSettings mt_settings; + mt_settings.loadFromConfig("merge_tree", config); + shared->merge_tree_settings.emplace(mt_settings); } return *shared->merge_tree_settings; diff --git a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp index 795cc68f1ea..8a6ceec509a 100644 --- a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp @@ -54,7 +54,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED); String part_name = params.get("part"); - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); /// Validation of the input that may come from malicious replica. MergeTreePartInfo::fromPartName(part_name, data.format_version); @@ -175,7 +175,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( { /// Validation of the input that may come from malicious replica. MergeTreePartInfo::fromPartName(part_name, data.format_version); - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); Poco::URI uri; uri.setScheme(interserver_scheme); diff --git a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 11b64fd9788..255fb998446 100644 --- a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -39,7 +39,7 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( , compute_granularity(index_granularity.empty()) , codec(std::move(codec_)) , skip_indices(indices_to_recalc) - , with_final_mark(storage.getCOWSettings()->write_final_mark && can_use_adaptive_granularity) + , with_final_mark(storage.getSettings()->write_final_mark && can_use_adaptive_granularity) { if (blocks_are_granules_size && !index_granularity.empty()) throw Exception("Can't take information about index granularity from blocks, when non empty index_granularity array specified", ErrorCodes::LOGICAL_ERROR); @@ -139,7 +139,7 @@ void fillIndexGranularityImpl( void IMergedBlockOutputStream::fillIndexGranularity(const Block & block) { - const auto storage_settings = storage.getCOWSettings(); + const auto storage_settings = storage.getSettings(); fillIndexGranularityImpl( block, storage_settings->index_granularity_bytes, diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 271ccaf527e..f10c2067e44 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -107,7 +107,7 @@ MergeTreeData::MergeTreeData( const ASTPtr & sample_by_ast_, const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - MergeTreeSettingsPtr settings_, + std::unique_ptr storage_settings_, bool require_part_metadata_, bool attach, BrokenPartCallback broken_part_callback_) @@ -121,11 +121,11 @@ MergeTreeData::MergeTreeData( full_path(full_path_), broken_part_callback(broken_part_callback_), log_name(database_name + "." + table_name), log(&Logger::get(log_name)), - guarded_settings(settings_), + storage_settings(std::move(storage_settings_)), data_parts_by_info(data_parts_indexes.get()), data_parts_by_state_and_info(data_parts_indexes.get()) { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); setProperties(order_by_ast_, primary_key_ast_, columns_, indices_, constraints_); setConstraints(constraints_); @@ -733,7 +733,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) { LOG_DEBUG(log, "Loading data parts"); - const auto settings = getCOWSettings(); + const auto settings = getSettings(); Strings part_file_names; Poco::DirectoryIterator end; for (Poco::DirectoryIterator it(full_path); it != end; ++it) @@ -966,7 +966,7 @@ void MergeTreeData::clearOldTemporaryDirectories(ssize_t custom_directories_life if (!lock.try_lock()) return; - const auto settings = getCOWSettings(); + const auto settings = getSettings(); time_t current_time = time(nullptr); ssize_t deadline = (custom_directories_lifetime_seconds >= 0) ? current_time - custom_directories_lifetime_seconds @@ -1021,7 +1021,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts() if (part.unique() && /// Grab only parts that are not used by anyone (SELECTs for example). part_remove_time < now && - now - part_remove_time > getCOWSettings()->old_parts_lifetime.totalSeconds()) + now - part_remove_time > getSettings()->old_parts_lifetime.totalSeconds()) { parts_to_delete.emplace_back(it); } @@ -1105,7 +1105,7 @@ void MergeTreeData::clearOldPartsFromFilesystem() void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_remove) { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); if (parts_to_remove.size() > 1 && settings->max_part_removal_threads > 1 && parts_to_remove.size() > settings->concurrent_part_removal_threshold) { /// Parallel parts removal. @@ -1342,7 +1342,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name const IndicesASTs & old_indices, const IndicesASTs & new_indices, ExpressionActionsPtr & out_expression, NameToNameMap & out_rename_map, bool & out_force_update_metadata) const { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); out_expression = nullptr; out_rename_map = {}; out_force_update_metadata = false; @@ -1508,7 +1508,7 @@ void MergeTreeData::alterDataPart( bool skip_sanity_checks, AlterDataPartTransactionPtr & transaction) { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); ExpressionActionsPtr expression; const auto & part = transaction->getDataPart(); bool force_update_metadata; @@ -1653,11 +1653,10 @@ void MergeTreeData::alterSettings( const Context & context, TableStructureWriteLockHolder & table_lock_holder) { - std::lock_guard lock(settings_mutex); - MutableMergeTreeSettingsPtr settings = std::move(*guarded_settings.getPtr()).mutate(); - settings->updateFromChanges(new_changes); + MergeTreeSettings copy = *getSettings(); + copy.updateFromChanges(new_changes); IStorage::alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); - guarded_settings.setPtr(std::move(settings)); + storage_settings.set(std::make_unique(copy)); } bool MergeTreeData::hasSetting(const String & setting_name) const @@ -2343,7 +2342,7 @@ std::optional MergeTreeData::getMinPartDataVersion() const void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); const size_t parts_count_in_total = getPartsCount(); if (parts_count_in_total >= settings->max_parts_in_total) { @@ -2381,7 +2380,7 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const void MergeTreeData::throwInsertIfNeeded() const { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); const size_t parts_count_in_total = getPartsCount(); if (parts_count_in_total >= settings->max_parts_in_total) { @@ -3076,7 +3075,7 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); if (!settings->enable_mixed_granularity_parts || settings->index_granularity_bytes == 0) { diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 9f3d1127f2c..8babd723fae 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -331,7 +331,7 @@ public: const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - MergeTreeSettingsPtr settings_, + std::unique_ptr settings_, bool require_part_metadata_, bool attach, BrokenPartCallback broken_part_callback_ = [](const String &){}); @@ -620,7 +620,7 @@ public: /// Has additional constraint in replicated version virtual bool canUseAdaptiveGranularity() const { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); return settings->index_granularity_bytes != 0 && (settings->enable_mixed_granularity_parts || !has_non_adaptive_index_granularity_parts); } @@ -683,13 +683,12 @@ public: bool has_non_adaptive_index_granularity_parts = false; - /// Get copy-on-write pointer to storage settings. + /// Get constant pointer to storage settings. /// Copy this pointer into your scope and you will /// get consistent settings. - const MergeTreeSettingsPtr getCOWSettings() const + MergeTreeSettingsPtr getSettings() const { - std::shared_lock lock(settings_mutex); - return guarded_settings.copyPtr(); + return storage_settings.get(); } protected: @@ -721,26 +720,9 @@ protected: String log_name; Logger * log; - /// Just hides settings pointer from direct usage - class MergeTreeSettingsGuard - { - private: - /// Settings COW pointer. Data maybe changed at any point of time. - /// If you need consistent settings, just copy pointer to your scope. - MergeTreeSettingsPtr settings_ptr; - public: - MergeTreeSettingsGuard(MergeTreeSettingsPtr settings_ptr_) - : settings_ptr(settings_ptr_) - {} - - const MergeTreeSettingsPtr copyPtr() const { return settings_ptr; } - MergeTreeSettingsPtr getPtr() { return settings_ptr; } - void setPtr(MergeTreeSettingsPtr ptr) { settings_ptr = ptr; } - }; - - /// Storage settings. Don't use this field directly, if you - /// want readonly settings. Prefer getCOWSettings() method. - MergeTreeSettingsGuard guarded_settings; + /// Storage settings. + /// Use get and set to receive readonly versions. + MultiVersion storage_settings; /// Work with data parts @@ -829,7 +811,6 @@ protected: /// The same for clearOldTemporaryDirectories. std::mutex clear_old_temporary_directories_mutex; /// Mutex for settings usage - mutable std::shared_mutex settings_mutex; void setProperties(const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast, const ColumnsDescription & new_columns, diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 312c5a195d8..ad489a91603 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -141,7 +141,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); size_t free_entries = pool_size - pool_used; - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); UInt64 max_size = 0; if (free_entries >= data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge) @@ -159,7 +159,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() { - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); size_t total_threads_in_pool = pool.getNumberOfThreads(); size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); @@ -179,7 +179,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( String * out_disable_reason) { MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector(); - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); if (data_parts.empty()) { @@ -556,7 +556,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor Names all_column_names = data.getColumns().getNamesOfPhysical(); NamesAndTypesList all_columns = data.getColumns().getAllPhysical(); - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); NamesAndTypesList gathering_columns, merging_columns; Names gathering_column_names, merging_column_names; @@ -965,7 +965,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor const auto & updated_header = mutations_interpreter.getUpdatedHeader(); NamesAndTypesList all_columns = data.getColumns().getAllPhysical(); - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); Block in_header = in->getHeader(); @@ -1145,7 +1145,7 @@ MergeTreeDataMergerMutator::MergeAlgorithm MergeTreeDataMergerMutator::chooseMer const MergeTreeData::DataPartsVector & parts, size_t sum_rows_upper_bound, const NamesAndTypesList & gathering_columns, bool deduplicate, bool need_remove_expired_values) const { - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); if (deduplicate) return MergeAlgorithm::Horizontal; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 8cbb0819d20..40dc0bf6b52 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -673,7 +673,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( size_t sum_marks = 0; size_t total_rows = 0; - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); size_t adaptive_parts = 0; for (size_t i = 0; i < parts.size(); ++i) { @@ -832,7 +832,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO SortingInfoPtr sorting_info = query_info.sorting_info; size_t adaptive_parts = 0; std::vector sum_marks_in_parts(parts.size()); - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); for (size_t i = 0; i < parts.size(); ++i) { @@ -1035,7 +1035,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal const Names & virt_columns, const Settings & settings) const { - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); size_t sum_marks = 0; size_t adaptive_parts = 0; for (size_t i = 0; i < parts.size(); ++i) diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/dbms/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index 143af37c10d..c25968480c8 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -25,7 +25,7 @@ std::optional MergeTreeIndexGranularityInfo::getMrkExtensionFromFS( MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo( const MergeTreeData & storage) { - const auto storage_settings = storage.getCOWSettings(); + const auto storage_settings = storage.getSettings(); fixed_index_granularity = storage_settings->index_granularity; /// Granularity is fixed if (!storage.canUseAdaptiveGranularity()) diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp index 5bc44eee842..9eee33554ab 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -75,9 +75,4 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) #undef ADD_IF_ABSENT } - -MergeTreeSettings::MutablePtr MergeTreeSettings::clone() const -{ - return COW::create(*this); -} } diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index 68d3e1ec66c..6ba08fed5da 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -2,7 +2,6 @@ #include #include -#include namespace Poco @@ -22,11 +21,9 @@ class ASTStorage; /** Settings for the MergeTree family of engines. * Could be loaded from config or from a CREATE TABLE query (SETTINGS clause). */ -struct MergeTreeSettings : public SettingsCollection, public COW +struct MergeTreeSettings : public SettingsCollection { - friend class COW; - /// M (mutable) for normal settings, IM (immutable) for not updateable settings. #define LIST_OF_MERGE_TREE_SETTINGS(M, IM) \ IM(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.") \ @@ -101,14 +98,8 @@ struct MergeTreeSettings : public SettingsCollection, public /// NOTE: will rewrite the AST to add immutable settings. void loadFromQuery(ASTStorage & storage_def); - - MutablePtr clone() const; -private: - MergeTreeSettings() = default; - MergeTreeSettings(const MergeTreeSettings & o) = default; }; -using MergeTreeSettingsPtr = MergeTreeSettings::Ptr; -using MutableMergeTreeSettingsPtr = MergeTreeSettings::MutablePtr; +using MergeTreeSettingsPtr = std::shared_ptr; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp index 7a09bde0998..69cf173212d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp @@ -31,7 +31,7 @@ MergeTreeThreadSelectBlockInputStream::MergeTreeThreadSelectBlockInputStream( /// Maybe it will make sence to add settings `max_block_size_bytes` if (max_block_size_rows && !storage.canUseAdaptiveGranularity()) { - size_t fixed_index_granularity = storage.getCOWSettings()->index_granularity; + size_t fixed_index_granularity = storage.getSettings()->index_granularity; min_marks_to_read = (min_marks_to_read_ * fixed_index_granularity + max_block_size_rows - 1) / max_block_size_rows * max_block_size_rows / fixed_index_granularity; } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 2b03ed86895..b89fd9629c0 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -27,7 +27,7 @@ ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplic void ReplicatedMergeTreeCleanupThread::run() { - auto storage_settings = storage.getCOWSettings(); + auto storage_settings = storage.getSettings(); const auto CLEANUP_SLEEP_MS = storage_settings->cleanup_delay_period * 1000 + std::uniform_int_distribution(0, storage_settings->cleanup_delay_period_random_add * 1000)(rng); @@ -75,7 +75,7 @@ void ReplicatedMergeTreeCleanupThread::iterate() void ReplicatedMergeTreeCleanupThread::clearOldLogs() { auto zookeeper = storage.getZooKeeper(); - auto storage_settings = storage.getCOWSettings(); + auto storage_settings = storage.getSettings(); Coordination::Stat stat; if (!zookeeper->exists(storage.zookeeper_path + "/log", &stat)) @@ -287,7 +287,7 @@ struct ReplicatedMergeTreeCleanupThread::NodeWithStat void ReplicatedMergeTreeCleanupThread::clearOldBlocks() { auto zookeeper = storage.getZooKeeper(); - auto storage_settings = storage.getCOWSettings(); + auto storage_settings = storage.getSettings(); std::vector timed_blocks; getBlocksSortedByTime(*zookeeper, timed_blocks); @@ -404,7 +404,7 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper & void ReplicatedMergeTreeCleanupThread::clearOldMutations() { - auto storage_settings = storage.getCOWSettings(); + auto storage_settings = storage.getSettings(); if (!storage_settings->finished_mutations_to_keep) return; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index d8255add55a..e8cfeb78963 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -964,7 +964,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( * Setting max_bytes_to_merge_at_max_space_in_pool still working for regular merges, * because the leader replica does not assign merges of greater size (except OPTIMIZE PARTITION and OPTIMIZE FINAL). */ - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); bool ignore_max_size = (entry.type == LogEntry::MERGE_PARTS) && (max_source_parts_size == data_settings->max_bytes_to_merge_at_max_space_in_pool); if (!ignore_max_size && sum_parts_size_in_bytes > max_source_parts_size) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 6145713492f..c766219a349 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -44,7 +44,7 @@ ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(Storage , log(&Logger::get(log_name)) , active_node_identifier(generateActiveNodeIdentifier()) { - const auto storage_settings = storage.getCOWSettings(); + const auto storage_settings = storage.getSettings(); check_period_ms = storage_settings->zookeeper_session_expiration_check_period.totalSeconds() * 1000; /// Periodicity of checking lag of replica. @@ -122,7 +122,7 @@ void ReplicatedMergeTreeRestartingThread::run() } time_t current_time = time(nullptr); - const auto storage_settings = storage.getCOWSettings(); + const auto storage_settings = storage.getSettings(); if (current_time >= prev_time_of_check_delay + static_cast(storage_settings->check_delay_period)) { /// Find out lag of replicas. @@ -171,7 +171,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() activateReplica(); const auto & zookeeper = storage.getZooKeeper(); - const auto storage_settings = storage.getCOWSettings(); + const auto storage_settings = storage.getSettings(); storage.cloneReplicaIfNeeded(zookeeper); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 6b12df91546..5ada5d50b23 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -27,7 +27,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos]; - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); sampling_expression = formattedAST(data.sample_by_ast); index_granularity = data_settings->index_granularity; merging_params_mode = static_cast(data.merging_params.mode); diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 8afca855dba..cebe6c3ca44 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -575,7 +575,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) ASTPtr ttl_table_ast; IndicesDescription indices_description; ConstraintsDescription constraints_description; - MutableMergeTreeSettingsPtr storage_settings = MergeTreeSettings::create(args.context.getMergeTreeSettings()); + std::unique_ptr storage_settings = std::make_unique(args.context.getMergeTreeSettings()); if (is_extended_storage_def) { diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index e7be6a35cb9..c0caa7ff1e0 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -63,14 +63,14 @@ StorageMergeTree::StorageMergeTree( const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - MergeTreeSettingsPtr settings_, + std::unique_ptr storage_settings_, bool has_force_restore_data_flag) : MergeTreeData(database_name_, table_name_, path_ + escapeForFileName(table_name_) + '/', columns_, indices_, constraints_, context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, sample_by_ast_, ttl_table_ast_, merging_params_, - settings_, false, attach), + std::move(storage_settings_), false, attach), path(path_), background_pool(context_.getBackgroundPool()), reader(*this), writer(*this), merger_mutator(*this, global_context.getBackgroundPool()) @@ -804,7 +804,7 @@ Int64 StorageMergeTree::getCurrentMutationVersion( void StorageMergeTree::clearOldMutations() { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); if (!settings->finished_mutations_to_keep) return; diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 9f51a9357ce..df1cd1682b9 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -153,7 +153,7 @@ protected: const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - MergeTreeSettingsPtr settings_, + std::unique_ptr settings_, bool has_force_restore_data_flag); }; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index ce9448dd0e7..2bf8e26112f 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -202,14 +202,14 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const ASTPtr & sample_by_ast_, const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - MergeTreeSettingsPtr settings_, + std::unique_ptr settings_, bool has_force_restore_data_flag) : MergeTreeData(database_name_, table_name_, path_ + escapeForFileName(table_name_) + '/', columns_, indices_, constraints_, context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, sample_by_ast_, ttl_table_ast_, merging_params_, - settings_, true, attach, + std::move(settings_), true, attach, [this] (const std::string & name) { enqueuePartForCheck(name); }), zookeeper_path(global_context.getMacros()->expand(zookeeper_path_, database_name_, table_name_)), replica_name(global_context.getMacros()->expand(replica_name_, database_name_, table_name_)), @@ -637,7 +637,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) for (const auto & part : parts) total_rows_on_filesystem += part->rows_count; - const auto storage_settings = getCOWSettings(); + const auto storage_settings = getSettings(); bool insane = unexpected_parts_rows > total_rows_on_filesystem * storage_settings->replicated_max_ratio_of_wrong_parts; if (insane && !skip_sanity_checks) @@ -781,7 +781,7 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: if (!has_been_already_added) { - const auto storage_settings = getCOWSettings(); + const auto storage_settings = getSettings(); String part_path = replica_path + "/parts/" + part_name; ops.emplace_back(zkutil::makeCheckRequest( @@ -864,7 +864,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd String StorageReplicatedMergeTree::getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums) const { return MinimalisticDataPartChecksums::getSerializedString(checksums, - getCOWSettings()->use_minimalistic_checksums_in_zookeeper); + getSettings()->use_minimalistic_checksums_in_zookeeper); } @@ -1035,7 +1035,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) parts.push_back(part); } - const auto storage_settings = getCOWSettings(); + const auto storage_settings = getSettings(); if (!have_all_parts) { /// If you do not have all the necessary parts, try to take some already merged part from someone. @@ -1161,7 +1161,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedMergeTree::LogEntry & entry) { const String & source_part_name = entry.source_parts.at(0); - const auto storage_settings = getCOWSettings(); + const auto storage_settings = getSettings(); LOG_TRACE(log, "Executing log entry to mutate part " << source_part_name << " to " << entry.new_part_name); DataPartPtr source_part = getActiveContainingPart(source_part_name); @@ -1276,7 +1276,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) { String replica = findReplicaHavingCoveringPart(entry, true); - const auto storage_settings = getCOWSettings(); + const auto storage_settings = getSettings(); static std::atomic_uint total_fetches {0}; if (storage_settings->replicated_max_parallel_fetches && total_fetches >= storage_settings->replicated_max_parallel_fetches) @@ -2213,7 +2213,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() if (!is_leader) return; - const auto storage_settings = getCOWSettings(); + const auto storage_settings = getSettings(); const bool deduplicate = false; /// TODO: read deduplicate option from table config const bool force_ttl = false; @@ -3029,7 +3029,7 @@ void StorageReplicatedMergeTree::assertNotReadonly() const BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const Context & context) { - const auto storage_settings = getCOWSettings(); + const auto storage_settings = getSettings(); assertNotReadonly(); const Settings & query_settings = context.getSettingsRef(); @@ -3067,7 +3067,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p }; bool force_ttl = (final && (hasTableTTL() || hasAnyColumnTTL())); - const auto storage_settings = getCOWSettings(); + const auto storage_settings = getSettings(); if (!partition && final) { @@ -3948,7 +3948,7 @@ void StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(const String & void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) { auto zookeeper = tryGetZooKeeper(); - const auto storage_settings = getCOWSettings(); + const auto storage_settings = getSettings(); res.is_leader = is_leader; res.can_become_leader = storage_settings->replicated_can_become_leader; @@ -4141,7 +4141,7 @@ void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, t out_absolute_delay = getAbsoluteDelay(); out_relative_delay = 0; - const auto storage_settings = getCOWSettings(); + const auto storage_settings = getSettings(); /** Relative delay is the maximum difference of absolute delay from any other replica, * (if this replica lags behind any other live replica, or zero, otherwise). @@ -4969,7 +4969,7 @@ void StorageReplicatedMergeTree::getCommitPartOps( const String & block_id_path) const { const String & part_name = part->name; - const auto storage_settings = getCOWSettings(); + const auto storage_settings = getSettings(); if (!block_id_path.empty()) { @@ -5016,7 +5016,7 @@ void StorageReplicatedMergeTree::updatePartHeaderInZooKeeperAndCommit( AlterDataPartTransaction & transaction) { String part_path = replica_path + "/parts/" + transaction.getPartName(); - const auto storage_settings = getCOWSettings(); + const auto storage_settings = getSettings(); bool need_delete_columns_and_checksums_nodes = false; try @@ -5201,7 +5201,7 @@ CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, const C bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const { - const auto storage_settings = getCOWSettings(); + const auto storage_settings = getSettings(); return storage_settings->index_granularity_bytes != 0 && (storage_settings->enable_mixed_granularity_parts || (!has_non_adaptive_index_granularity_parts && !other_replicas_fixed_granularity)); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 5d8090a8cbc..20af4aeb19c 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -540,7 +540,7 @@ protected: const ASTPtr & sample_by_ast_, const ASTPtr & table_ttl_ast_, const MergingParams & merging_params_, - MergeTreeSettingsPtr settings_, + std::unique_ptr settings_, bool has_force_restore_data_flag); }; From d17a64c28d26cb32f84f8e62a68614c81ba88fcd Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 26 Aug 2019 17:39:49 +0300 Subject: [PATCH 437/509] fix vulnerabilities --- dbms/src/Compression/CompressionCodecDelta.cpp | 12 +++++++++--- dbms/src/Compression/CompressionCodecDoubleDelta.cpp | 12 +++++++++--- dbms/src/Compression/CompressionCodecGorilla.cpp | 12 +++++++++--- dbms/src/Compression/CompressionCodecMultiple.cpp | 6 +++++- dbms/src/Compression/ICompressionCodec.cpp | 2 +- 5 files changed, 33 insertions(+), 11 deletions(-) diff --git a/dbms/src/Compression/CompressionCodecDelta.cpp b/dbms/src/Compression/CompressionCodecDelta.cpp index 9f2397f8e59..849485312c6 100644 --- a/dbms/src/Compression/CompressionCodecDelta.cpp +++ b/dbms/src/Compression/CompressionCodecDelta.cpp @@ -80,7 +80,7 @@ UInt32 CompressionCodecDelta::doCompressData(const char * source, UInt32 source_ { UInt8 bytes_to_skip = source_size % delta_bytes_size; dest[0] = delta_bytes_size; - dest[1] = bytes_to_skip; + dest[1] = bytes_to_skip; /// unused (backward compatibility) memcpy(&dest[2], source, bytes_to_skip); size_t start_pos = 2 + bytes_to_skip; switch (delta_bytes_size) @@ -101,10 +101,16 @@ UInt32 CompressionCodecDelta::doCompressData(const char * source, UInt32 source_ return 1 + 1 + source_size; } -void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 /* uncompressed_size */) const +void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { + if (source_size < 2) + throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); + UInt8 bytes_size = source[0]; - UInt8 bytes_to_skip = source[1]; + UInt8 bytes_to_skip = uncompressed_size % bytes_size; + + if (2 + bytes_to_skip > source_size) + throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); memcpy(dest, &source[2], bytes_to_skip); UInt32 source_size_no_header = source_size - bytes_to_skip - 2; diff --git a/dbms/src/Compression/CompressionCodecDoubleDelta.cpp b/dbms/src/Compression/CompressionCodecDoubleDelta.cpp index d3830dc9fdb..d0e948f821c 100644 --- a/dbms/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/dbms/src/Compression/CompressionCodecDoubleDelta.cpp @@ -304,7 +304,7 @@ UInt32 CompressionCodecDoubleDelta::doCompressData(const char * source, UInt32 s { UInt8 bytes_to_skip = source_size % data_bytes_size; dest[0] = data_bytes_size; - dest[1] = bytes_to_skip; + dest[1] = bytes_to_skip; /// unused (backward compatibility) memcpy(&dest[2], source, bytes_to_skip); size_t start_pos = 2 + bytes_to_skip; UInt32 compressed_size = 0; @@ -328,10 +328,16 @@ UInt32 CompressionCodecDoubleDelta::doCompressData(const char * source, UInt32 s return 1 + 1 + compressed_size; } -void CompressionCodecDoubleDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 /* uncompressed_size */) const +void CompressionCodecDoubleDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { + if (source_size < 2) + throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); + UInt8 bytes_size = source[0]; - UInt8 bytes_to_skip = source[1]; + UInt8 bytes_to_skip = uncompressed_size % bytes_size; + + if (2 + bytes_to_skip > source_size) + throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); memcpy(dest, &source[2], bytes_to_skip); UInt32 source_size_no_header = source_size - bytes_to_skip - 2; diff --git a/dbms/src/Compression/CompressionCodecGorilla.cpp b/dbms/src/Compression/CompressionCodecGorilla.cpp index 8af6c8bfd39..f0d15804422 100644 --- a/dbms/src/Compression/CompressionCodecGorilla.cpp +++ b/dbms/src/Compression/CompressionCodecGorilla.cpp @@ -264,7 +264,7 @@ UInt32 CompressionCodecGorilla::doCompressData(const char * source, UInt32 sourc { UInt8 bytes_to_skip = source_size % data_bytes_size; dest[0] = data_bytes_size; - dest[1] = bytes_to_skip; + dest[1] = bytes_to_skip; /// unused (backward compatibility) memcpy(&dest[2], source, bytes_to_skip); size_t start_pos = 2 + bytes_to_skip; UInt32 result_size = 0; @@ -289,10 +289,16 @@ UInt32 CompressionCodecGorilla::doCompressData(const char * source, UInt32 sourc return 1 + 1 + result_size; } -void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 /* uncompressed_size */) const +void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { + if (source_size < 2) + throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); + UInt8 bytes_size = source[0]; - UInt8 bytes_to_skip = source[1]; + UInt8 bytes_to_skip = uncompressed_size % bytes_size; + + if (2 + bytes_to_skip > source_size) + throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); memcpy(dest, &source[2], bytes_to_skip); UInt32 source_size_no_header = source_size - bytes_to_skip - 2; diff --git a/dbms/src/Compression/CompressionCodecMultiple.cpp b/dbms/src/Compression/CompressionCodecMultiple.cpp index 23c244f4dcb..fad72c932c3 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.cpp +++ b/dbms/src/Compression/CompressionCodecMultiple.cpp @@ -88,6 +88,9 @@ void CompressionCodecMultiple::useInfoAboutType(DataTypePtr data_type) void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 decompressed_size) const { + if (source_size < 1 || !source[0]) + throw Exception("Wrong compression methods list", ErrorCodes::CORRUPTED_DATA); + UInt8 compression_methods_size = source[0]; PODArray compressed_buf(&source[compression_methods_size + 1], &source[source_size]); @@ -103,7 +106,8 @@ void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 sour UInt32 uncompressed_size = ICompressionCodec::readDecompressedBlockSize(compressed_buf.data()); if (idx == 0 && uncompressed_size != decompressed_size) - throw Exception("Wrong final decompressed size in codec Multiple, got " + toString(uncompressed_size) + ", expected " + toString(decompressed_size), ErrorCodes::CORRUPTED_DATA); + throw Exception("Wrong final decompressed size in codec Multiple, got " + toString(uncompressed_size) + + ", expected " + toString(decompressed_size), ErrorCodes::CORRUPTED_DATA); uncompressed_buf.resize(uncompressed_size + codec->getAdditionalSizeAtTheEndOfBuffer()); codec->decompress(compressed_buf.data(), source_size, uncompressed_buf.data()); diff --git a/dbms/src/Compression/ICompressionCodec.cpp b/dbms/src/Compression/ICompressionCodec.cpp index aafca2f5eb3..f9d392e5d6d 100644 --- a/dbms/src/Compression/ICompressionCodec.cpp +++ b/dbms/src/Compression/ICompressionCodec.cpp @@ -47,7 +47,7 @@ UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, ch throw Exception("Can't decompress data with codec byte " + toString(method) + " from codec with byte " + toString(method), ErrorCodes::CANNOT_DECOMPRESS); UInt8 header_size = getHeaderSize(); - UInt32 decompressed_size = unalignedLoad(&source[5]); + UInt32 decompressed_size = readDecompressedBlockSize(source); doDecompressData(&source[header_size], source_size - header_size, dest, decompressed_size); return decompressed_size; From 0d0ebf885f1e75d5a129d2a63119281313f862c0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 26 Aug 2019 17:50:34 +0300 Subject: [PATCH 438/509] Remove two redundant arguments from alter --- dbms/src/Interpreters/InterpreterAlterQuery.cpp | 2 +- dbms/src/Storages/IStorage.cpp | 11 ++++++----- dbms/src/Storages/IStorage.h | 4 +--- dbms/src/Storages/Kafka/StorageKafka.cpp | 2 -- dbms/src/Storages/Kafka/StorageKafka.h | 2 -- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 7 ++++--- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 -- dbms/src/Storages/StorageBuffer.cpp | 5 ++++- dbms/src/Storages/StorageBuffer.h | 3 +-- dbms/src/Storages/StorageDistributed.cpp | 6 ++++-- dbms/src/Storages/StorageDistributed.h | 3 +-- dbms/src/Storages/StorageMerge.cpp | 5 ++--- dbms/src/Storages/StorageMerge.h | 3 +-- dbms/src/Storages/StorageMergeTree.cpp | 7 ++++--- dbms/src/Storages/StorageMergeTree.h | 4 +--- dbms/src/Storages/StorageNull.cpp | 6 ++++-- dbms/src/Storages/StorageNull.h | 3 +-- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 8 +++++--- dbms/src/Storages/StorageReplicatedMergeTree.h | 4 +--- 19 files changed, 41 insertions(+), 46 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index a32609bd53d..5798e182b69 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -104,7 +104,7 @@ BlockIO InterpreterAlterQuery::execute() { auto table_lock_holder = table->lockAlterIntention(context.getCurrentQueryId()); alter_commands.validate(*table, context); - table->alter(alter_commands, database_name, table_name, context, table_lock_holder); + table->alter(alter_commands, context, table_lock_holder); } return {}; diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 855336c06e5..b56c20f7372 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -384,11 +384,12 @@ TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id) void IStorage::alterSettings( const SettingsChanges & new_changes, - const String & current_database_name, - const String & current_table_name, const Context & context, TableStructureWriteLockHolder & /* table_lock_holder */) { + const String current_database_name = getDatabaseName(); + const String current_table_name = getTableName(); + IDatabase::ASTModifier storage_modifier = [&] (IAST & ast) { if (!new_changes.empty()) @@ -416,16 +417,16 @@ void IStorage::alterSettings( void IStorage::alter( const AlterCommands & params, - const String & database_name, - const String & table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { + const String database_name = getDatabaseName(); + const String table_name = getTableName(); if (params.isSettingsAlter()) { SettingsChanges new_changes; params.applyForSettingsOnly(new_changes); - alterSettings(new_changes, database_name, table_name, context, table_lock_holder); + alterSettings(new_changes, context, table_lock_holder); return; } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index b27ab89dcbc..30aa2b41d0d 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -283,7 +283,7 @@ public: * This method must fully execute the ALTER query, taking care of the locks itself. * To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata. */ - virtual void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder); + virtual void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder); /** ALTER tables with regard to its partitions. * Should handle locks for each command on its own. @@ -297,8 +297,6 @@ public: */ virtual void alterSettings( const SettingsChanges & new_changes, - const String & current_database_name, - const String & current_table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder); diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index f00d273c6a4..45ca4aa6cf7 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -414,8 +414,6 @@ bool StorageKafka::hasSetting(const String & setting_name) const void StorageKafka::alterSettings( const SettingsChanges & /* new_changes */, - const String & /* current_database_name */, - const String & /* current_table_name */, const Context & /* context */, TableStructureWriteLockHolder & /* table_lock_holder */) { diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index a6039eac254..27f1e7e0ec7 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -61,8 +61,6 @@ public: void alterSettings( const SettingsChanges & new_changes, - const String & current_database_name, - const String & current_table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index f10c2067e44..448f5afcc86 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1648,14 +1648,15 @@ void MergeTreeData::alterDataPart( void MergeTreeData::alterSettings( const SettingsChanges & new_changes, - const String & current_database_name, - const String & current_table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { + const String current_database_name = getDatabaseName(); + const String current_table_name = getTableName(); + MergeTreeSettings copy = *getSettings(); copy.updateFromChanges(new_changes); - IStorage::alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); + IStorage::alterSettings(new_changes, context, table_lock_holder); storage_settings.set(std::make_unique(copy)); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 8babd723fae..f759b87f986 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -541,8 +541,6 @@ public: /// Not atomic, have to be done with alter intention lock. void alterSettings( const SettingsChanges & new_changes, - const String & current_database_name, - const String & current_table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index dd6ecf7f261..1cbf557af9b 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -692,10 +692,13 @@ void StorageBuffer::flushThread() } -void StorageBuffer::alter(const AlterCommands & params, const String & database_name_, const String & table_name_, const Context & context, TableStructureWriteLockHolder & table_lock_holder) +void StorageBuffer::alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); + const String database_name_ = getDatabaseName(); + const String table_name_ = getTableName(); + /// So that no blocks of the old structure remain. optimize({} /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context); diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index 4c317a7a102..8eecbaa17ec 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -91,8 +91,7 @@ public: /// The structure of the subordinate table is not checked and does not change. void alter( - const AlterCommands & params, const String & database_name, const String & table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; private: String table_name; diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 9bb68a6b13f..83d4e4fc554 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -383,11 +383,13 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const Context & c void StorageDistributed::alter( - const AlterCommands & params, const String & current_database_name, const String & current_table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); + const String current_database_name = getDatabaseName(); + const String current_table_name = getTableName(); + auto new_columns = getColumns(); auto new_indices = getIndices(); auto new_constraints = getConstraints(); diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index 6885a758e9e..86cb0fef92c 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -85,8 +85,7 @@ public: /// in the sub-tables, you need to manually add and delete columns /// the structure of the sub-table is not checked void alter( - const AlterCommands & params, const String & database_name, const String & table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; void startup() override; void shutdown() override; diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 3005bf7dd52..2b0e5413c26 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -396,8 +396,7 @@ DatabaseIteratorPtr StorageMerge::getDatabaseIterator(const Context & context) c void StorageMerge::alter( - const AlterCommands & params, const String & database_name_, const String & table_name_, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); @@ -405,7 +404,7 @@ void StorageMerge::alter( auto new_indices = getIndices(); auto new_constraints = getConstraints(); params.applyForColumnsOnly(new_columns); - context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, new_constraints, {}); + context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {}); setColumns(new_columns); } diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index 6d02ad029cc..7ccf072ac12 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -47,8 +47,7 @@ public: /// you need to add and remove columns in the sub-tables manually /// the structure of sub-tables is not checked void alter( - const AlterCommands & params, const String & database_name, const String & table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index c0caa7ff1e0..1504413801c 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -245,11 +245,12 @@ std::vector StorageMergeTree::prepar void StorageMergeTree::alter( const AlterCommands & params, - const String & current_database_name, - const String & current_table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { + const String current_database_name = getDatabaseName(); + const String current_table_name = getTableName(); + if (!params.isMutable()) { SettingsChanges new_changes; @@ -257,7 +258,7 @@ void StorageMergeTree::alter( if (params.isSettingsAlter()) { params.applyForSettingsOnly(new_changes); - alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); + alterSettings(new_changes, context, table_lock_holder); return; } diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index df1cd1682b9..bf11a9a0335 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -59,9 +59,7 @@ public: void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; - void alter( - const AlterCommands & params, const String & database_name, const String & table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; void checkTableCanBeDropped() const override; diff --git a/dbms/src/Storages/StorageNull.cpp b/dbms/src/Storages/StorageNull.cpp index d610ff45212..b419c35f964 100644 --- a/dbms/src/Storages/StorageNull.cpp +++ b/dbms/src/Storages/StorageNull.cpp @@ -31,11 +31,13 @@ void registerStorageNull(StorageFactory & factory) } void StorageNull::alter( - const AlterCommands & params, const String & current_database_name, const String & current_table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); + const String current_database_name = getDatabaseName(); + const String current_table_name = getTableName(); + ColumnsDescription new_columns = getColumns(); IndicesDescription new_indices = getIndices(); ConstraintsDescription new_constraints = getConstraints(); diff --git a/dbms/src/Storages/StorageNull.h b/dbms/src/Storages/StorageNull.h index fec7638706f..e4fcd7991e5 100644 --- a/dbms/src/Storages/StorageNull.h +++ b/dbms/src/Storages/StorageNull.h @@ -44,8 +44,7 @@ public: } void alter( - const AlterCommands & params, const String & database_name, const String & table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; private: String table_name; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 2bf8e26112f..fa7ab961ff9 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3142,13 +3142,15 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p void StorageReplicatedMergeTree::alter( - const AlterCommands & params, const String & current_database_name, const String & current_table_name, - const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) + const AlterCommands & params, const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) { assertNotReadonly(); LOG_DEBUG(log, "Doing ALTER"); + const String current_database_name = getDatabaseName(); + const String current_table_name = getTableName(); + if (params.isSettingsAlter()) { /// We don't replicate storage_settings ALTER. It's local operation. @@ -3156,7 +3158,7 @@ void StorageReplicatedMergeTree::alter( LOG_DEBUG(log, "ALTER storage_settings only"); SettingsChanges new_changes; params.applyForSettingsOnly(new_changes); - alterSettings(new_changes, current_database_name, current_table_name, query_context, table_lock_holder); + alterSettings(new_changes, query_context, table_lock_holder); return; } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 20af4aeb19c..60ce6623b1a 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -98,9 +98,7 @@ public: bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & query_context) override; - void alter( - const AlterCommands & params, const String & database_name, const String & table_name, - const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) override; void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & query_context) override; From 305f3ba47955670ebae21dcd44fd48799122fbe2 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Mon, 26 Aug 2019 19:09:30 +0300 Subject: [PATCH 439/509] DOCAPI-7428: domain function docs. EN review, RU translation. (#6624) * Update url_functions.md * DOCAPI-7428: RU translation * DOCAPI-7428: Fix. * Update docs/en/query_language/functions/url_functions.md Co-Authored-By: Ivan Blinkov * Update docs/en/query_language/functions/url_functions.md Co-Authored-By: Ivan Blinkov * Update docs/ru/query_language/functions/url_functions.md Co-Authored-By: Ivan Blinkov * Update docs/ru/query_language/functions/url_functions.md Co-Authored-By: Ivan Blinkov * Update docs/ru/query_language/functions/url_functions.md Co-Authored-By: Ivan Blinkov * DOCAPI-7428: Fixes. --- .../query_language/functions/url_functions.md | 32 ++++--- .../query_language/functions/url_functions.md | 87 ++++++++++++++++++- 2 files changed, 105 insertions(+), 14 deletions(-) diff --git a/docs/en/query_language/functions/url_functions.md b/docs/en/query_language/functions/url_functions.md index a21c2cfa0bf..f4ff63ce021 100644 --- a/docs/en/query_language/functions/url_functions.md +++ b/docs/en/query_language/functions/url_functions.md @@ -2,17 +2,19 @@ All these functions don't follow the RFC. They are maximally simplified for improved performance. -## Functions that extract part of a URL +## Functions that Extract Parts of a URL -If there isn't anything similar in a URL, an empty string is returned. +If the relevant part isn't present in a URL, an empty string is returned. ### protocol -Returns the protocol. Examples: http, ftp, mailto, magnet... +Extracts the protocol from a URL. + +Examples of typical returned values: http, https, ftp, mailto, tel, magnet... ### domain -Extracts the host part from URL. +Extracts the hostname from a URL. ``` domain(url) @@ -23,7 +25,7 @@ domain(url) - `url` — URL. Type: [String](../../data_types/string.md). -URL can be specified with or without scheme. Examples: +The URL can be specified with or without a scheme. Examples: ``` svn+ssh://some.svn-hosting.com:80/repo/trunk @@ -31,10 +33,18 @@ some.svn-hosting.com:80/repo/trunk https://yandex.com/time/ ``` +For these examples, the `domain` function returns the following results: + +``` +some.svn-hosting.com +some.svn-hosting.com +yandex.com +``` + **Returned values** -- Host name. If ClickHouse can parse input string as URL. -- Empty string. If ClickHouse cannot parse input string as URL. +- Host name. If ClickHouse can parse the input string as a URL. +- Empty string. If ClickHouse can't parse the input string as a URL. Type: `String`. @@ -55,7 +65,7 @@ Returns the domain and removes no more than one 'www.' from the beginning of it, ### topLevelDomain -Extracts the the top-level domain from URL. +Extracts the the top-level domain from a URL. ``` topLevelDomain(url) @@ -65,7 +75,7 @@ topLevelDomain(url) - `url` — URL. Type: [String](../../data_types/string.md). -URL can be specified with or without scheme. Examples: +The URL can be specified with or without a scheme. Examples: ``` svn+ssh://some.svn-hosting.com:80/repo/trunk @@ -75,8 +85,8 @@ https://yandex.com/time/ **Returned values** -- Domain name. If ClickHouse can parse input string as URL. -- Empty string. If ClickHouse cannot parse input string as URL. +- Domain name. If ClickHouse can parse the input string as a URL. +- Empty string. If ClickHouse cannot parse the input string as a URL. Type: `String`. diff --git a/docs/ru/query_language/functions/url_functions.md b/docs/ru/query_language/functions/url_functions.md index 1897d1b28a3..7002273d5cb 100644 --- a/docs/ru/query_language/functions/url_functions.md +++ b/docs/ru/query_language/functions/url_functions.md @@ -10,13 +10,94 @@ Возвращает протокол. Примеры: http, ftp, mailto, magnet... ### domain -Возвращает домен. Отсекает схему размером не более 16 байт. + +Извлекает имя хоста из URL. + +``` +domain(url) +``` + +**Параметры** + +- `url` — URL. Тип — [String](../../data_types/string.md). + +URL может быть указан со схемой или без неё. Примеры: + +``` +svn+ssh://some.svn-hosting.com:80/repo/trunk +some.svn-hosting.com:80/repo/trunk +https://yandex.com/time/ +``` + +Для указанных примеров функция `domain` возвращает следующие результаты: + +``` +some.svn-hosting.com +some.svn-hosting.com +yandex.com +``` + +**Возвращаемые значения** + +- Имя хоста. Если ClickHouse может распарсить входную строку как URL. +- Пустая строка. Если ClickHouse не может распарсить входную строку как URL. + +Тип — `String`. + +**Пример** + +```sql +SELECT domain('svn+ssh://some.svn-hosting.com:80/repo/trunk') +``` + +```text +┌─domain('svn+ssh://some.svn-hosting.com:80/repo/trunk')─┐ +│ some.svn-hosting.com │ +└────────────────────────────────────────────────────────┘ +``` ### domainWithoutWWW -Возвращает домен, удалив не более одного 'www.' с начала, если есть. + +Возвращает домен, удалив префикс 'www.', если он присутствовал. ### topLevelDomain -Возвращает домен верхнего уровня. Пример: .ru. + +Извлекает домен верхнего уровня из URL. + +``` +topLevelDomain(url) +``` + +**Параметры** + +- `url` — URL. Тип — [String](../../data_types/string.md). + +URL может быть указан со схемой или без неё. Примеры: + +``` +svn+ssh://some.svn-hosting.com:80/repo/trunk +some.svn-hosting.com:80/repo/trunk +https://yandex.com/time/ +``` + +**Возвращаемые значения** + +- Имя домена. Если ClickHouse может распарсить входную строку как URL. +- Пустая строка. Если ClickHouse не может распарсить входную строку как URL. + +Тип — `String`. + +**Пример** + +```sql +SELECT topLevelDomain('svn+ssh://www.some.svn-hosting.com:80/repo/trunk') +``` + +```text +┌─topLevelDomain('svn+ssh://www.some.svn-hosting.com:80/repo/trunk')─┐ +│ com │ +└────────────────────────────────────────────────────────────────────┘ +``` ### firstSignificantSubdomain Возвращает "первый существенный поддомен". Это понятие является нестандартным и специфично для Яндекс.Метрики. Первый существенный поддомен - это домен второго уровня, если он не равен одному из com, net, org, co, или домен третьего уровня, иначе. Например, firstSignificantSubdomain('') = 'yandex', firstSignificantSubdomain('') = 'yandex'. Список "несущественных" доменов второго уровня и другие детали реализации могут изменяться в будущем. From 1da261576e3d36393e8fab1a59ff21a7fd67d96c Mon Sep 17 00:00:00 2001 From: Dmitry Rubashkin Date: Mon, 26 Aug 2019 19:42:20 +0300 Subject: [PATCH 440/509] Try fix2 --- dbms/src/Common/TerminalSize.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Common/TerminalSize.cpp b/dbms/src/Common/TerminalSize.cpp index 7f6bb93119f..5f0036b86ce 100644 --- a/dbms/src/Common/TerminalSize.cpp +++ b/dbms/src/Common/TerminalSize.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB::ErrorCodes From 12084270dedecbc6e9856eaa0ae5e2f9a7e1bded Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 26 Aug 2019 19:58:40 +0300 Subject: [PATCH 441/509] fix build --- dbms/src/Compression/CompressionCodecDelta.cpp | 2 +- dbms/src/Compression/CompressionCodecDoubleDelta.cpp | 2 +- dbms/src/Compression/CompressionCodecGorilla.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Compression/CompressionCodecDelta.cpp b/dbms/src/Compression/CompressionCodecDelta.cpp index 849485312c6..abf0b79f9c8 100644 --- a/dbms/src/Compression/CompressionCodecDelta.cpp +++ b/dbms/src/Compression/CompressionCodecDelta.cpp @@ -109,7 +109,7 @@ void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_ UInt8 bytes_size = source[0]; UInt8 bytes_to_skip = uncompressed_size % bytes_size; - if (2 + bytes_to_skip > source_size) + if (UInt32(2 + bytes_to_skip) > source_size) throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); memcpy(dest, &source[2], bytes_to_skip); diff --git a/dbms/src/Compression/CompressionCodecDoubleDelta.cpp b/dbms/src/Compression/CompressionCodecDoubleDelta.cpp index d0e948f821c..9a6b551f159 100644 --- a/dbms/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/dbms/src/Compression/CompressionCodecDoubleDelta.cpp @@ -336,7 +336,7 @@ void CompressionCodecDoubleDelta::doDecompressData(const char * source, UInt32 s UInt8 bytes_size = source[0]; UInt8 bytes_to_skip = uncompressed_size % bytes_size; - if (2 + bytes_to_skip > source_size) + if (UInt32(2 + bytes_to_skip) > source_size) throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); memcpy(dest, &source[2], bytes_to_skip); diff --git a/dbms/src/Compression/CompressionCodecGorilla.cpp b/dbms/src/Compression/CompressionCodecGorilla.cpp index f0d15804422..574e40b06bf 100644 --- a/dbms/src/Compression/CompressionCodecGorilla.cpp +++ b/dbms/src/Compression/CompressionCodecGorilla.cpp @@ -297,7 +297,7 @@ void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 sourc UInt8 bytes_size = source[0]; UInt8 bytes_to_skip = uncompressed_size % bytes_size; - if (2 + bytes_to_skip > source_size) + if (UInt32(2 + bytes_to_skip) > source_size) throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); memcpy(dest, &source[2], bytes_to_skip); From a98b91c521b1ee2c5c826df98cb10fdbfc4f2424 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Mon, 26 Aug 2019 20:48:48 +0300 Subject: [PATCH 442/509] DOCAPI-7436: Enum type docs. EN review, RU translation. (#6630) * Update enum.md (#33) * DOCAPI-7436: RU translation. * Update docs/en/data_types/enum.md Co-Authored-By: Ivan Blinkov * Update docs/ru/data_types/enum.md Co-Authored-By: Ivan Blinkov * DOCAPI-7436: Fixes. --- docs/en/data_types/enum.md | 12 ++++++---- docs/ru/data_types/enum.md | 48 +++++++++++++++++++++----------------- 2 files changed, 34 insertions(+), 26 deletions(-) diff --git a/docs/en/data_types/enum.md b/docs/en/data_types/enum.md index 3fbe5b3708b..692cbad39b1 100644 --- a/docs/en/data_types/enum.md +++ b/docs/en/data_types/enum.md @@ -1,13 +1,15 @@ # Enum -Enumerated type storing pairs of the `'string' = integer` format. +Enumerated type consisting of named values. + +Named values must be delcared as `'string' = integer` pairs. ClickHouse stores only numbers, but supports operations with the values through their names. ClickHouse supports: -- 8-bit `Enum`. It can contain up to 256 values with enumeration of `[-128, 127]`. -- 16-bit `Enum`. It can contain up to 65536 values with enumeration of `[-32768, 32767]`. +- 8-bit `Enum`. It can contain up to 256 values enumerated in the `[-128, 127]` range. +- 16-bit `Enum`. It can contain up to 65536 values enumerated in the `[-32768, 32767]` range. -ClickHouse automatically chooses a type for `Enum` at data insertion. Also, you can use `Enum8` or `Enum16` types to be sure in size of storage. +ClickHouse automatically chooses the type of `Enum` when data is inserted. You can also use `Enum8` or `Enum16` types to be sure in the size of storage. ## Usage examples @@ -21,7 +23,7 @@ CREATE TABLE t_enum ENGINE = TinyLog ``` -This column `x` can only store the values that are listed in the type definition: `'hello'` or `'world'`. If you try to save any other value, ClickHouse will generate an exception. ClickHouse automatically chooses the 8-bit size for enumeration of this `Enum`. +Column `x` can only store values that are listed in the type definition: `'hello'` or `'world'`. If you try to save any other value, ClickHouse will raise an exception. 8-bit size for this `Enum` is chosen automatically. ```sql :) INSERT INTO t_enum VALUES ('hello'), ('world'), ('hello') diff --git a/docs/ru/data_types/enum.md b/docs/ru/data_types/enum.md index 9191dc5d2b0..8d32cce1648 100644 --- a/docs/ru/data_types/enum.md +++ b/docs/ru/data_types/enum.md @@ -1,27 +1,32 @@ +# Enum -# Enum8, Enum16 +Перечисляемый тип данных, содержащий именованные значения. -Включает в себя типы `Enum8` и `Enum16`. `Enum` сохраняет конечный набор пар `'строка' = целое число`. Все операции с данными типа `Enum` ClickHouse выполняет как с числами, однако пользователь при этом работает со строковыми константами. Это более эффективно с точки зрения производительности, чем работа с типом данных `String`. +Именованные значения задаются парами `'string' = integer`. ClickHouse хранить только числа, но допускает операции над ними с помощью заданных имён. -- `Enum8` описывается парами `'String' = Int8`. -- `Enum16` описывается парами `'String' = Int16`. +ClickHouse поддерживает: -## Примеры применения +- 8-битный `Enum`. Может содержать до 256 значений, пронумерованных в диапазоне `[-128, 127]`. +- 16-битный `Enum`. Может содержать до 65536 значений, пронумерованных в диапазоне `[-32768, 32767]`. + +ClickHouse автоматически выбирает размерность `Enum` при вставке данных. Чтобы точно понимать размер хранимых данных можно использовать типы `Enum8` или `Enum16`. + +## Примеры использования Создадим таблицу со столбцом типа `Enum8('hello' = 1, 'world' = 2)`. -``` +```sql CREATE TABLE t_enum ( - x Enum8('hello' = 1, 'world' = 2) + x Enum('hello' = 1, 'world' = 2) ) ENGINE = TinyLog ``` -В столбец `x` можно сохранять только значения, перечисленные при определении типа, т.е. `'hello'` или `'world'`. Если попытаться сохранить другое значение, ClickHouse сгенерирует исключение. +В столбец `x` можно сохранять только значения, перечисленные при определении типа, т.е. `'hello'` или `'world'`. Если вы попытаетесь сохранить любое другое значение, ClickHouse сгенерирует исключение. ClickHouse автоматически выберет размерность 8-bit для этого `Enum`. -``` -:) INSERT INTO t_enum Values('hello'),('world'),('hello') +```sql +:) INSERT INTO t_enum VALUES ('hello'), ('world'), ('hello') INSERT INTO t_enum VALUES @@ -35,12 +40,12 @@ INSERT INTO t_enum VALUES Exception on client: -Code: 49. DB::Exception: Unknown element 'a' for type Enum8('hello' = 1, 'world' = 2) +Code: 49. DB::Exception: Unknown element 'a' for type Enum('hello' = 1, 'world' = 2) ``` При запросе данных из таблицы ClickHouse выдаст строковые значения из `Enum`. -``` +```sql SELECT * FROM t_enum ┌─x─────┐ @@ -49,10 +54,11 @@ SELECT * FROM t_enum │ hello │ └───────┘ ``` -Если необходимо увидеть цифровые эквиваленты строкам, то необходимо привести тип. -``` -SELECT CAST(x, 'Int8') FROM t_enum +Если необходимо увидеть цифровые эквиваленты строкам, то необходимо привести тип `Enum` к целочисленному. + +```sql +SELECT CAST(x AS Int8) FROM t_enum ┌─CAST(x, 'Int8')─┐ │ 1 │ @@ -61,14 +67,14 @@ SELECT CAST(x, 'Int8') FROM t_enum └─────────────────┘ ``` -Чтобы создать значение типа Enum в запросе, также необходима функция `CAST`. +Чтобы создать значение типа `Enum` в запросе, также необходимо использовать функцию `CAST`. -``` -SELECT toTypeName(CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)')) +```sql +SELECT toTypeName(CAST('a', 'Enum(\'a\' = 1, \'b\' = 2)')) -┌─toTypeName(CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)'))─┐ -│ Enum8('a' = 1, 'b' = 2) │ -└──────────────────────────────────────────────────────┘ +┌─toTypeName(CAST('a', 'Enum(\'a\' = 1, \'b\' = 2)'))─┐ +│ Enum8('a' = 1, 'b' = 2) │ +└─────────────────────────────────────────────────────┘ ``` ## Общие правила и особенности использования From 4cd376f5876c528820ff8b143422da74b64291ea Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 26 Aug 2019 21:00:13 +0300 Subject: [PATCH 443/509] fix table ttl after bad merge with master --- dbms/src/DataStreams/TTLBlockInputStream.cpp | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/dbms/src/DataStreams/TTLBlockInputStream.cpp b/dbms/src/DataStreams/TTLBlockInputStream.cpp index 752f9800aa3..e2a3a7ca03b 100644 --- a/dbms/src/DataStreams/TTLBlockInputStream.cpp +++ b/dbms/src/DataStreams/TTLBlockInputStream.cpp @@ -70,7 +70,7 @@ bool TTLBlockInputStream::isTTLExpired(time_t ttl) Block TTLBlockInputStream::readImpl() { /// Skip all data if table ttl is expired for part - if (storage.hasTableTTL() && old_ttl_infos.table_ttl.max <= current_time) + if (storage.hasTableTTL() && isTTLExpired(old_ttl_infos.table_ttl.max)) { rows_removed = data_part->rows_count; return {}; @@ -80,18 +80,8 @@ Block TTLBlockInputStream::readImpl() if (!block) return block; - if (storage.hasTableTTL()) - { - /// Skip all data if table ttl is expired for part - if (isTTLExpired(old_ttl_infos.table_ttl.max)) - { - rows_removed = data_part->rows_count; - return {}; - } - - if (force || isTTLExpired(old_ttl_infos.table_ttl.min)) - removeRowsWithExpiredTableTTL(block); - } + if (storage.hasTableTTL() && (force || isTTLExpired(old_ttl_infos.table_ttl.min))) + removeRowsWithExpiredTableTTL(block); removeValuesWithExpiredColumnTTL(block); From c0bb52ca6d3af6d4e6513ed1c9df167998535b6c Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 19 Aug 2019 20:03:57 +0300 Subject: [PATCH 444/509] Separate the hash table interface for ReverseIndex. It is significantly different from other uses of hash tables, and poses the main obstacle to changing the hash table interface to the one that can be easily supported by compound hash tables. Make it explicitly separate, implement it only for a particular kind of HashTable, and move this implementation to the ReverseIndex module. --- dbms/src/Columns/ReverseIndex.h | 87 ++++++++++++++++++++-- dbms/src/Common/HashTable/FixedHashTable.h | 12 +-- dbms/src/Common/HashTable/HashTable.h | 31 ++------ 3 files changed, 88 insertions(+), 42 deletions(-) diff --git a/dbms/src/Columns/ReverseIndex.h b/dbms/src/Columns/ReverseIndex.h index 8fa4e87680b..09f4f0d7f34 100644 --- a/dbms/src/Columns/ReverseIndex.h +++ b/dbms/src/Columns/ReverseIndex.h @@ -81,6 +81,8 @@ namespace template static bool isZero(const T &, const State & /*state*/) { + /// Careful: apparently this uses SFINAE to redefine isZero for all types + /// except the IndexType, for which the default ZeroTraits::isZero is used. static_assert(!std::is_same_v::type, typename std::decay::type>); return false; } @@ -121,19 +123,88 @@ namespace }; + /** + * ReverseIndexHashTableBase implements a special hash table interface for + * reverse index. + * + * The following requirements are different compared to a plain hash table: + * + * 1) Provide public access to 'hash table state' that contains + * additional data needed to calculate cell hashes. + * + * 2) Support emplace() and find() with a Key different from the resulting + * hash table key. This means emplace() accepts a different kind of object + * as a key, and then the real key can be read from the returned cell iterator. + * + * These requirements are unique to ReverseIndex and are in conflict with + * supporting hash tables that use alternative key storage, such as FixedHashMap + * or StringHashMap. Therefore, we implement an interface for ReverseIndex + * separately. + */ template - class HashTableWithPublicState : public HashTable, HashTableAllocator> + class ReverseIndexHashTableBase : public HashTable, HashTableAllocator> { using State = typename Cell::State; using Base = HashTable, HashTableAllocator>; public: using Base::Base; + using iterator = typename Base::iterator; State & getState() { return *this; } + + + template + size_t ALWAYS_INLINE reverseIndexFindCell(const ObjectToCompareWith & x, + size_t hash_value, size_t place_value) const + { + while (!this->buf[place_value].isZero(*this) + && !this->buf[place_value].keyEquals(x, hash_value, *this)) + { + place_value = this->grower.next(place_value); + } + + return place_value; + } + + template + void ALWAYS_INLINE reverseIndexEmplaceNonZero(const Key & key, iterator & it, + bool & inserted, size_t hash_value, const ObjectToCompareWith & object) + { + size_t place_value = reverseIndexFindCell(object, hash_value, + this->grower.place(hash_value)); + // emplaceNonZeroImpl() might need to re-find the cell if the table grows, + // but it will find it correctly by the key alone, so we don't have to + // pass it the 'object'. + this->emplaceNonZeroImpl(place_value, key, it, inserted, hash_value); + } + + /// Searches position by object. + template + void ALWAYS_INLINE reverseIndexEmplace(Key key, iterator & it, bool & inserted, + size_t hash_value, const ObjectToCompareWith& object) + { + if (!this->emplaceIfZero(key, it, inserted, hash_value)) + { + reverseIndexEmplaceNonZero(key, it, inserted, hash_value, object); + } + } + + template + iterator ALWAYS_INLINE reverseIndexFind(ObjectToCompareWith x, size_t hash_value) + { + if (Cell::isZero(x, *this)) + return this->hasZero() ? this->iteratorToZero() : this->end(); + + size_t place_value = reverseIndexFindCell(x, hash_value, + this->grower.place(hash_value)); + return !this->buf[place_value].isZero(*this) + ? iterator(this, &this->buf[place_value]) + : this->end(); + } }; template - class ReverseIndexStringHashTable : public HashTableWithPublicState< + class ReverseIndexStringHashTable : public ReverseIndexHashTableBase< IndexType, ReverseIndexHashTableCell< IndexType, @@ -144,7 +215,7 @@ namespace has_base_index>, ReverseIndexHash> { - using Base = HashTableWithPublicState< + using Base = ReverseIndexHashTableBase< IndexType, ReverseIndexHashTableCell< IndexType, @@ -166,7 +237,7 @@ namespace }; template - class ReverseIndexNumberHashTable : public HashTableWithPublicState< + class ReverseIndexNumberHashTable : public ReverseIndexHashTableBase< IndexType, ReverseIndexHashTableCell< IndexType, @@ -177,7 +248,7 @@ namespace has_base_index>, ReverseIndexHash> { - using Base = HashTableWithPublicState< + using Base = ReverseIndexHashTableBase< IndexType, ReverseIndexHashTableCell< IndexType, @@ -356,7 +427,7 @@ void ReverseIndex::buildIndex() else hash = getHash(column->getDataAt(row)); - index->emplace(row + base_index, iterator, inserted, hash, column->getDataAt(row)); + index->reverseIndexEmplace(row + base_index, iterator, inserted, hash, column->getDataAt(row)); if (!inserted) throw Exception("Duplicating keys found in ReverseIndex.", ErrorCodes::LOGICAL_ERROR); @@ -401,7 +472,7 @@ UInt64 ReverseIndex::insert(const StringRef & data) else column->insertData(data.data, data.size); - index->emplace(num_rows + base_index, iterator, inserted, hash, data); + index->reverseIndexEmplace(num_rows + base_index, iterator, inserted, hash, data); if constexpr (use_saved_hash) { @@ -427,7 +498,7 @@ UInt64 ReverseIndex::getInsertionPoint(const StringRef & IteratorType iterator; auto hash = getHash(data); - iterator = index->find(data, hash); + iterator = index->reverseIndexFind(data, hash); return iterator == index->end() ? size() + base_index : iterator->getValue(); } diff --git a/dbms/src/Common/HashTable/FixedHashTable.h b/dbms/src/Common/HashTable/FixedHashTable.h index 663848865da..0c3a2c1237a 100644 --- a/dbms/src/Common/HashTable/FixedHashTable.h +++ b/dbms/src/Common/HashTable/FixedHashTable.h @@ -294,26 +294,22 @@ public: void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted) { emplaceImpl(x, it, inserted); } void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t) { emplaceImpl(x, it, inserted); } - template - iterator ALWAYS_INLINE find(ObjectToCompareWith x) + iterator ALWAYS_INLINE find(Key x) { return !buf[x].isZero(*this) ? iterator(this, &buf[x]) : end(); } - template - const_iterator ALWAYS_INLINE find(ObjectToCompareWith x) const + const_iterator ALWAYS_INLINE find(Key x) const { return !buf[x].isZero(*this) ? const_iterator(this, &buf[x]) : end(); } - template - iterator ALWAYS_INLINE find(ObjectToCompareWith, size_t hash_value) + iterator ALWAYS_INLINE find(Key, size_t hash_value) { return !buf[hash_value].isZero(*this) ? iterator(this, &buf[hash_value]) : end(); } - template - const_iterator ALWAYS_INLINE find(ObjectToCompareWith, size_t hash_value) const + const_iterator ALWAYS_INLINE find(Key, size_t hash_value) const { return !buf[hash_value].isZero(*this) ? const_iterator(this, &buf[hash_value]) : end(); } diff --git a/dbms/src/Common/HashTable/HashTable.h b/dbms/src/Common/HashTable/HashTable.h index f15fe22f8fd..c5a0c812ee2 100644 --- a/dbms/src/Common/HashTable/HashTable.h +++ b/dbms/src/Common/HashTable/HashTable.h @@ -280,8 +280,7 @@ protected: #endif /// Find a cell with the same key or an empty cell, starting from the specified position and further along the collision resolution chain. - template - size_t ALWAYS_INLINE findCell(const ObjectToCompareWith & x, size_t hash_value, size_t place_value) const + size_t ALWAYS_INLINE findCell(const Key & x, size_t hash_value, size_t place_value) const { while (!buf[place_value].isZero(*this) && !buf[place_value].keyEquals(x, hash_value, *this)) { @@ -700,13 +699,6 @@ protected: emplaceNonZeroImpl(place_value, x, it, inserted, hash_value); } - /// Same but find place using object. Hack for ReverseIndex. - template - void ALWAYS_INLINE emplaceNonZero(Key x, iterator & it, bool & inserted, size_t hash_value, const ObjectToCompareWith & object) - { - size_t place_value = findCell(object, hash_value, grower.place(hash_value)); - emplaceNonZeroImpl(place_value, x, it, inserted, hash_value); - } public: @@ -763,14 +755,6 @@ public: emplaceNonZero(x, it, inserted, hash_value); } - /// Same, but search position by object. Hack for ReverseIndex. - template - void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t hash_value, const ObjectToCompareWith & object) - { - if (!emplaceIfZero(x, it, inserted, hash_value)) - emplaceNonZero(x, it, inserted, hash_value, object); - } - /// Copy the cell from another hash table. It is assumed that the cell is not zero, and also that there was no such key in the table yet. void ALWAYS_INLINE insertUniqueNonZero(const Cell * cell, size_t hash_value) { @@ -783,9 +767,7 @@ public: resize(); } - - template - iterator ALWAYS_INLINE find(ObjectToCompareWith x) + iterator ALWAYS_INLINE find(Key x) { if (Cell::isZero(x, *this)) return this->hasZero() ? iteratorToZero() : end(); @@ -796,8 +778,7 @@ public: } - template - const_iterator ALWAYS_INLINE find(ObjectToCompareWith x) const + const_iterator ALWAYS_INLINE find(Key x) const { if (Cell::isZero(x, *this)) return this->hasZero() ? iteratorToZero() : end(); @@ -808,8 +789,7 @@ public: } - template - iterator ALWAYS_INLINE find(ObjectToCompareWith x, size_t hash_value) + iterator ALWAYS_INLINE find(Key x, size_t hash_value) { if (Cell::isZero(x, *this)) return this->hasZero() ? iteratorToZero() : end(); @@ -819,8 +799,7 @@ public: } - template - const_iterator ALWAYS_INLINE find(ObjectToCompareWith x, size_t hash_value) const + const_iterator ALWAYS_INLINE find(Key x, size_t hash_value) const { if (Cell::isZero(x, *this)) return this->hasZero() ? iteratorToZero() : end(); From 0c75d5f42b3bff0da65cfee2e1d5aac6ad682eb7 Mon Sep 17 00:00:00 2001 From: Dmitry Rubashkin Date: Mon, 26 Aug 2019 22:03:47 +0300 Subject: [PATCH 445/509] Function transformQueryForExternalDatabase() fixed --- dbms/src/Storages/transformQueryForExternalDatabase.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/transformQueryForExternalDatabase.cpp b/dbms/src/Storages/transformQueryForExternalDatabase.cpp index 55a0ef95200..7cbffc46e67 100644 --- a/dbms/src/Storages/transformQueryForExternalDatabase.cpp +++ b/dbms/src/Storages/transformQueryForExternalDatabase.cpp @@ -141,16 +141,12 @@ String transformQueryForExternalDatabase( if (function->name == "and") { bool compatible_found = false; - auto new_function_and = std::make_shared(); - auto new_function_and_arguments = std::make_shared(); - new_function_and->arguments = new_function_and_arguments; - new_function_and->children.push_back(new_function_and_arguments); - + auto new_function_and = makeASTFunction("and"); for (const auto & elem : function->arguments->children) { if (isCompatible(*elem)) { - new_function_and_arguments->children.push_back(elem); + new_function_and->arguments->children.push_back(elem); compatible_found = true; } } From 0eec226c9a163ae536a0733f3a852988c90c1032 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Aug 2019 22:07:29 +0300 Subject: [PATCH 446/509] Simplify shared_ptr_helper --- .../tests/in_join_subqueries_preprocessor.cpp | 1 + dbms/src/Storages/Kafka/StorageKafka.h | 1 + .../MergeTree/StorageFromMergeTreeDataPart.h | 1 + dbms/src/Storages/StorageBuffer.h | 5 +++-- dbms/src/Storages/StorageDictionary.h | 1 + dbms/src/Storages/StorageDistributed.cpp | 10 ++------- dbms/src/Storages/StorageDistributed.h | 1 + dbms/src/Storages/StorageFile.h | 1 + dbms/src/Storages/StorageHDFS.h | 1 + dbms/src/Storages/StorageJoin.h | 1 + dbms/src/Storages/StorageLog.h | 1 + dbms/src/Storages/StorageMaterializedView.h | 1 + dbms/src/Storages/StorageMemory.h | 1 + dbms/src/Storages/StorageMerge.h | 1 + dbms/src/Storages/StorageMergeTree.h | 1 + dbms/src/Storages/StorageMySQL.h | 1 + dbms/src/Storages/StorageNull.h | 1 + .../src/Storages/StorageReplicatedMergeTree.h | 1 + dbms/src/Storages/StorageStripeLog.h | 1 + dbms/src/Storages/StorageTinyLog.h | 1 + dbms/src/Storages/StorageURL.h | 1 + dbms/src/Storages/StorageValues.h | 1 + dbms/src/Storages/StorageView.h | 1 + ...torageSystemAggregateFunctionCombinators.h | 2 ++ .../System/StorageSystemAsynchronousMetrics.h | 4 +++- .../System/StorageSystemBuildOptions.h | 1 + .../Storages/System/StorageSystemClusters.h | 1 + .../Storages/System/StorageSystemCollations.h | 1 + .../Storages/System/StorageSystemColumns.h | 1 + .../System/StorageSystemContributors.h | 1 + .../System/StorageSystemDataTypeFamilies.h | 1 + .../Storages/System/StorageSystemDatabases.h | 1 + .../System/StorageSystemDetachedParts.cpp | 1 + .../System/StorageSystemDictionaries.h | 1 + .../src/Storages/System/StorageSystemEvents.h | 1 + .../Storages/System/StorageSystemFormats.h | 1 + .../Storages/System/StorageSystemFunctions.h | 1 + .../Storages/System/StorageSystemGraphite.h | 1 + .../src/Storages/System/StorageSystemMacros.h | 1 + .../System/StorageSystemMergeTreeSettings.h | 1 + .../src/Storages/System/StorageSystemMerges.h | 1 + .../Storages/System/StorageSystemMetrics.h | 1 + .../src/Storages/System/StorageSystemModels.h | 1 + .../Storages/System/StorageSystemMutations.h | 1 + .../Storages/System/StorageSystemNumbers.h | 1 + dbms/src/Storages/System/StorageSystemOne.h | 1 + dbms/src/Storages/System/StorageSystemParts.h | 1 + .../System/StorageSystemPartsColumns.h | 1 + .../Storages/System/StorageSystemProcesses.h | 1 + .../Storages/System/StorageSystemReplicas.h | 1 + .../System/StorageSystemReplicationQueue.h | 1 + .../Storages/System/StorageSystemSettings.h | 1 + .../System/StorageSystemTableEngines.h | 1 + .../System/StorageSystemTableFunctions.h | 1 + .../src/Storages/System/StorageSystemTables.h | 1 + .../Storages/System/StorageSystemZooKeeper.h | 1 + .../libcommon/include/ext/shared_ptr_helper.h | 22 +++---------------- 57 files changed, 65 insertions(+), 30 deletions(-) diff --git a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp index 7c90769b37b..43624bf16b8 100644 --- a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp +++ b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp @@ -30,6 +30,7 @@ namespace DB /// Simplified version of the StorageDistributed class. class StorageDistributedFake : public ext::shared_ptr_helper, public DB::IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "DistributedFake"; } bool isRemote() const override { return true; } diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index a6039eac254..7dc71cbc5e4 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -20,6 +20,7 @@ namespace DB */ class StorageKafka : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Kafka"; } std::string getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 32268499a01..0e15a5660a9 100644 --- a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -14,6 +14,7 @@ namespace DB /// A Storage that allows reading from a single MergeTree data part. class StorageFromMergeTreeDataPart : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: String getName() const override { return "FromMergeTreeDataPart"; } String getTableName() const override { return part->storage.getTableName() + " (part " + part->name + ")"; } diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index 8060199d20d..a7683463ef5 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -39,6 +39,7 @@ class Context; */ class StorageBuffer : public ext::shared_ptr_helper, public IStorage { +friend struct ext::shared_ptr_helper; friend class BufferBlockInputStream; friend class BufferBlockOutputStream; @@ -94,6 +95,8 @@ public: const AlterCommands & params, const String & database_name, const String & table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + ~StorageBuffer() override; + private: String table_name; String database_name; @@ -145,8 +148,6 @@ protected: Context & context_, size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_, const String & destination_database_, const String & destination_table_, bool allow_materialized_); - - ~StorageBuffer() override; }; } diff --git a/dbms/src/Storages/StorageDictionary.h b/dbms/src/Storages/StorageDictionary.h index cf5fd647e74..097e81d15c6 100644 --- a/dbms/src/Storages/StorageDictionary.h +++ b/dbms/src/Storages/StorageDictionary.h @@ -21,6 +21,7 @@ class ExternalDictionaries; class StorageDictionary : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Dictionary"; } std::string getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 90a973ae3b9..add3de7cfae 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -266,11 +266,8 @@ StoragePtr StorageDistributed::createWithOwnCluster( ClusterPtr owned_cluster_, const Context & context_) { - auto res = ext::shared_ptr_helper::create( - String{}, table_name_, columns_, ConstraintsDescription{}, remote_database_, remote_table_, String{}, context_, ASTPtr(), String(), false); - + auto res = create(String{}, table_name_, columns_, ConstraintsDescription{}, remote_database_, remote_table_, String{}, context_, ASTPtr(), String(), false); res->owned_cluster = owned_cluster_; - return res; } @@ -282,11 +279,8 @@ StoragePtr StorageDistributed::createWithOwnCluster( ClusterPtr & owned_cluster_, const Context & context_) { - auto res = ext::shared_ptr_helper::create( - String{}, table_name_, columns_, ConstraintsDescription{}, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), false); - + auto res = create(String{}, table_name_, columns_, ConstraintsDescription{}, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), false); res->owned_cluster = owned_cluster_; - return res; } diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index e8da6dca4a7..4b0e939cb4b 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -29,6 +29,7 @@ class StorageDistributedDirectoryMonitor; */ class StorageDistributed : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; friend class DistributedBlockOutputStream; friend class StorageDistributedDirectoryMonitor; diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index 237e9a6989e..7268c8ddff0 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -20,6 +20,7 @@ class StorageFileBlockOutputStream; class StorageFile : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "File"; } std::string getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/StorageHDFS.h b/dbms/src/Storages/StorageHDFS.h index dae7000dd8a..30a99c9de70 100644 --- a/dbms/src/Storages/StorageHDFS.h +++ b/dbms/src/Storages/StorageHDFS.h @@ -15,6 +15,7 @@ namespace DB */ class StorageHDFS : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: String getName() const override { return "HDFS"; } String getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/StorageJoin.h b/dbms/src/Storages/StorageJoin.h index c9e1283b90f..760e9eb1815 100644 --- a/dbms/src/Storages/StorageJoin.h +++ b/dbms/src/Storages/StorageJoin.h @@ -22,6 +22,7 @@ using JoinPtr = std::shared_ptr; */ class StorageJoin : public ext::shared_ptr_helper, public StorageSetOrJoinBase { + friend struct ext::shared_ptr_helper; public: String getName() const override { return "Join"; } diff --git a/dbms/src/Storages/StorageLog.h b/dbms/src/Storages/StorageLog.h index e646ecd8c46..70de62cb47b 100644 --- a/dbms/src/Storages/StorageLog.h +++ b/dbms/src/Storages/StorageLog.h @@ -21,6 +21,7 @@ class StorageLog : public ext::shared_ptr_helper, public IStorage { friend class LogBlockInputStream; friend class LogBlockOutputStream; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Log"; } diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index 03c93d8d29f..b635634addf 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -11,6 +11,7 @@ namespace DB class StorageMaterializedView : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "MaterializedView"; } std::string getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/StorageMemory.h b/dbms/src/Storages/StorageMemory.h index 88e9e5dcf04..92dcd4be18f 100644 --- a/dbms/src/Storages/StorageMemory.h +++ b/dbms/src/Storages/StorageMemory.h @@ -21,6 +21,7 @@ class StorageMemory : public ext::shared_ptr_helper, public IStor { friend class MemoryBlockInputStream; friend class MemoryBlockOutputStream; +friend struct ext::shared_ptr_helper; public: String getName() const override { return "Memory"; } diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index 6d02ad029cc..10aa90d398b 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -14,6 +14,7 @@ namespace DB */ class StorageMerge : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Merge"; } std::string getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 9f51a9357ce..050ef2982d0 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -23,6 +23,7 @@ namespace DB */ class StorageMergeTree : public ext::shared_ptr_helper, public MergeTreeData { + friend struct ext::shared_ptr_helper; public: void startup() override; void shutdown() override; diff --git a/dbms/src/Storages/StorageMySQL.h b/dbms/src/Storages/StorageMySQL.h index 320eab9d4a9..5f369da4d25 100644 --- a/dbms/src/Storages/StorageMySQL.h +++ b/dbms/src/Storages/StorageMySQL.h @@ -18,6 +18,7 @@ namespace DB */ class StorageMySQL : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: StorageMySQL( const std::string & database_name_, diff --git a/dbms/src/Storages/StorageNull.h b/dbms/src/Storages/StorageNull.h index d858f738b24..4444604113e 100644 --- a/dbms/src/Storages/StorageNull.h +++ b/dbms/src/Storages/StorageNull.h @@ -16,6 +16,7 @@ namespace DB */ class StorageNull : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Null"; } std::string getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 5d8090a8cbc..b201d7fa3c9 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -74,6 +74,7 @@ namespace DB class StorageReplicatedMergeTree : public ext::shared_ptr_helper, public MergeTreeData { + friend struct ext::shared_ptr_helper; public: void startup() override; void shutdown() override; diff --git a/dbms/src/Storages/StorageStripeLog.h b/dbms/src/Storages/StorageStripeLog.h index 22be356713b..d53f000f277 100644 --- a/dbms/src/Storages/StorageStripeLog.h +++ b/dbms/src/Storages/StorageStripeLog.h @@ -23,6 +23,7 @@ class StorageStripeLog : public ext::shared_ptr_helper, public { friend class StripeLogBlockInputStream; friend class StripeLogBlockOutputStream; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "StripeLog"; } diff --git a/dbms/src/Storages/StorageTinyLog.h b/dbms/src/Storages/StorageTinyLog.h index 7a37a42ec63..1c148acf957 100644 --- a/dbms/src/Storages/StorageTinyLog.h +++ b/dbms/src/Storages/StorageTinyLog.h @@ -22,6 +22,7 @@ class StorageTinyLog : public ext::shared_ptr_helper, public ISt { friend class TinyLogBlockInputStream; friend class TinyLogBlockOutputStream; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "TinyLog"; } diff --git a/dbms/src/Storages/StorageURL.h b/dbms/src/Storages/StorageURL.h index 0c4b4648223..2facca8ce38 100644 --- a/dbms/src/Storages/StorageURL.h +++ b/dbms/src/Storages/StorageURL.h @@ -71,6 +71,7 @@ private: class StorageURL : public ext::shared_ptr_helper, public IStorageURLBase { + friend struct ext::shared_ptr_helper; public: StorageURL( const Poco::URI & uri_, diff --git a/dbms/src/Storages/StorageValues.h b/dbms/src/Storages/StorageValues.h index 36c3bc15301..c07cf7cbb63 100644 --- a/dbms/src/Storages/StorageValues.h +++ b/dbms/src/Storages/StorageValues.h @@ -11,6 +11,7 @@ namespace DB */ class StorageValues : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Values"; } std::string getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/StorageView.h b/dbms/src/Storages/StorageView.h index cda128027c2..de56f120fa1 100644 --- a/dbms/src/Storages/StorageView.h +++ b/dbms/src/Storages/StorageView.h @@ -12,6 +12,7 @@ namespace DB class StorageView : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "View"; } std::string getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h index 1d7226eda8b..86585f0ea73 100644 --- a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h +++ b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h @@ -4,11 +4,13 @@ #include #include #include + namespace DB { class StorageSystemAggregateFunctionCombinators : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h index 853cb97c974..77f247f89eb 100644 --- a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h +++ b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h @@ -12,8 +12,10 @@ class Context; /** Implements system table asynchronous_metrics, which allows to get values of periodically (asynchronously) updated metrics. */ -class StorageSystemAsynchronousMetrics : public ext::shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemAsynchronousMetrics : public ext::shared_ptr_helper, + public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemAsynchronousMetrics"; } diff --git a/dbms/src/Storages/System/StorageSystemBuildOptions.h b/dbms/src/Storages/System/StorageSystemBuildOptions.h index 749ffbddbaf..d81682765a2 100644 --- a/dbms/src/Storages/System/StorageSystemBuildOptions.h +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemBuildOptions : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemClusters.h b/dbms/src/Storages/System/StorageSystemClusters.h index dde9e53b626..8efb148cdb9 100644 --- a/dbms/src/Storages/System/StorageSystemClusters.h +++ b/dbms/src/Storages/System/StorageSystemClusters.h @@ -17,6 +17,7 @@ class Context; */ class StorageSystemClusters : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemClusters"; } diff --git a/dbms/src/Storages/System/StorageSystemCollations.h b/dbms/src/Storages/System/StorageSystemCollations.h index f8b7b6ee3af..eccce154047 100644 --- a/dbms/src/Storages/System/StorageSystemCollations.h +++ b/dbms/src/Storages/System/StorageSystemCollations.h @@ -8,6 +8,7 @@ namespace DB class StorageSystemCollations : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemColumns.h b/dbms/src/Storages/System/StorageSystemColumns.h index b9aa04b0b25..b569c2bafff 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.h +++ b/dbms/src/Storages/System/StorageSystemColumns.h @@ -13,6 +13,7 @@ class Context; */ class StorageSystemColumns : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemColumns"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemContributors.h b/dbms/src/Storages/System/StorageSystemContributors.h index 4e2a47960f3..4fc91ce1fbd 100644 --- a/dbms/src/Storages/System/StorageSystemContributors.h +++ b/dbms/src/Storages/System/StorageSystemContributors.h @@ -14,6 +14,7 @@ class Context; class StorageSystemContributors : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h index 365e2790699..44dd59d471f 100644 --- a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h +++ b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h @@ -9,6 +9,7 @@ namespace DB class StorageSystemDataTypeFamilies : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemDatabases.h b/dbms/src/Storages/System/StorageSystemDatabases.h index c83f5a72efc..957fcae2707 100644 --- a/dbms/src/Storages/System/StorageSystemDatabases.h +++ b/dbms/src/Storages/System/StorageSystemDatabases.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemDatabases : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { diff --git a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp index ce1bfc6fc21..b95a299af68 100644 --- a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp @@ -21,6 +21,7 @@ class StorageSystemDetachedParts : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemDetachedParts"; } std::string getTableName() const override { return "detached_parts"; } diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.h b/dbms/src/Storages/System/StorageSystemDictionaries.h index 87df9ceada7..6b28f03d917 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.h +++ b/dbms/src/Storages/System/StorageSystemDictionaries.h @@ -12,6 +12,7 @@ class Context; class StorageSystemDictionaries : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemDictionaries"; } diff --git a/dbms/src/Storages/System/StorageSystemEvents.h b/dbms/src/Storages/System/StorageSystemEvents.h index 5b02b7739f1..88753d3ea4f 100644 --- a/dbms/src/Storages/System/StorageSystemEvents.h +++ b/dbms/src/Storages/System/StorageSystemEvents.h @@ -13,6 +13,7 @@ class Context; */ class StorageSystemEvents : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemEvents"; } diff --git a/dbms/src/Storages/System/StorageSystemFormats.h b/dbms/src/Storages/System/StorageSystemFormats.h index 82f8303b5b0..61bd9ebeb8f 100644 --- a/dbms/src/Storages/System/StorageSystemFormats.h +++ b/dbms/src/Storages/System/StorageSystemFormats.h @@ -7,6 +7,7 @@ namespace DB { class StorageSystemFormats : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemFunctions.h b/dbms/src/Storages/System/StorageSystemFunctions.h index baead3d8186..1ae4483583f 100644 --- a/dbms/src/Storages/System/StorageSystemFunctions.h +++ b/dbms/src/Storages/System/StorageSystemFunctions.h @@ -15,6 +15,7 @@ class Context; */ class StorageSystemFunctions : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemFunctions"; } diff --git a/dbms/src/Storages/System/StorageSystemGraphite.h b/dbms/src/Storages/System/StorageSystemGraphite.h index b874e294782..5ba7b7bda8c 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.h +++ b/dbms/src/Storages/System/StorageSystemGraphite.h @@ -13,6 +13,7 @@ namespace DB /// Provides information about Graphite configuration. class StorageSystemGraphite : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemGraphite"; } diff --git a/dbms/src/Storages/System/StorageSystemMacros.h b/dbms/src/Storages/System/StorageSystemMacros.h index fdc091dfe1b..09cc8e3a2d9 100644 --- a/dbms/src/Storages/System/StorageSystemMacros.h +++ b/dbms/src/Storages/System/StorageSystemMacros.h @@ -15,6 +15,7 @@ class Context; */ class StorageSystemMacros : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemMacros"; } diff --git a/dbms/src/Storages/System/StorageSystemMergeTreeSettings.h b/dbms/src/Storages/System/StorageSystemMergeTreeSettings.h index 780390dd485..cb3668f13ec 100644 --- a/dbms/src/Storages/System/StorageSystemMergeTreeSettings.h +++ b/dbms/src/Storages/System/StorageSystemMergeTreeSettings.h @@ -15,6 +15,7 @@ class Context; */ class SystemMergeTreeSettings : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemMergeTreeSettings"; } diff --git a/dbms/src/Storages/System/StorageSystemMerges.h b/dbms/src/Storages/System/StorageSystemMerges.h index f45f895d661..6d970b8797f 100644 --- a/dbms/src/Storages/System/StorageSystemMerges.h +++ b/dbms/src/Storages/System/StorageSystemMerges.h @@ -15,6 +15,7 @@ class Context; class StorageSystemMerges : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemMerges"; } diff --git a/dbms/src/Storages/System/StorageSystemMetrics.h b/dbms/src/Storages/System/StorageSystemMetrics.h index f74db926126..86d95800a31 100644 --- a/dbms/src/Storages/System/StorageSystemMetrics.h +++ b/dbms/src/Storages/System/StorageSystemMetrics.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemMetrics : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemMetrics"; } diff --git a/dbms/src/Storages/System/StorageSystemModels.h b/dbms/src/Storages/System/StorageSystemModels.h index ef30bd511ea..8bcb70da35e 100644 --- a/dbms/src/Storages/System/StorageSystemModels.h +++ b/dbms/src/Storages/System/StorageSystemModels.h @@ -12,6 +12,7 @@ class Context; class StorageSystemModels : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemModels"; } diff --git a/dbms/src/Storages/System/StorageSystemMutations.h b/dbms/src/Storages/System/StorageSystemMutations.h index d2dcf99aa46..270cef13ef6 100644 --- a/dbms/src/Storages/System/StorageSystemMutations.h +++ b/dbms/src/Storages/System/StorageSystemMutations.h @@ -14,6 +14,7 @@ class Context; /// in the MergeTree tables. class StorageSystemMutations : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: String getName() const override { return "SystemMutations"; } diff --git a/dbms/src/Storages/System/StorageSystemNumbers.h b/dbms/src/Storages/System/StorageSystemNumbers.h index 5efc23a1125..76070839012 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.h +++ b/dbms/src/Storages/System/StorageSystemNumbers.h @@ -25,6 +25,7 @@ class Context; */ class StorageSystemNumbers : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemNumbers"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemOne.h b/dbms/src/Storages/System/StorageSystemOne.h index 974435e99f0..3e35fdb3477 100644 --- a/dbms/src/Storages/System/StorageSystemOne.h +++ b/dbms/src/Storages/System/StorageSystemOne.h @@ -17,6 +17,7 @@ class Context; */ class StorageSystemOne : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemOne"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemParts.h b/dbms/src/Storages/System/StorageSystemParts.h index eb1ded1c5d6..f7f58daad2e 100644 --- a/dbms/src/Storages/System/StorageSystemParts.h +++ b/dbms/src/Storages/System/StorageSystemParts.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemParts : public ext::shared_ptr_helper, public StorageSystemPartsBase { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemParts"; } diff --git a/dbms/src/Storages/System/StorageSystemPartsColumns.h b/dbms/src/Storages/System/StorageSystemPartsColumns.h index bd03e65557c..1ee90c7cde3 100644 --- a/dbms/src/Storages/System/StorageSystemPartsColumns.h +++ b/dbms/src/Storages/System/StorageSystemPartsColumns.h @@ -16,6 +16,7 @@ class Context; class StorageSystemPartsColumns : public ext::shared_ptr_helper, public StorageSystemPartsBase { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemPartsColumns"; } diff --git a/dbms/src/Storages/System/StorageSystemProcesses.h b/dbms/src/Storages/System/StorageSystemProcesses.h index 3cbe0028af3..735315115c4 100644 --- a/dbms/src/Storages/System/StorageSystemProcesses.h +++ b/dbms/src/Storages/System/StorageSystemProcesses.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemProcesses : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemProcesses"; } diff --git a/dbms/src/Storages/System/StorageSystemReplicas.h b/dbms/src/Storages/System/StorageSystemReplicas.h index 49865ad869a..fb006f4cbde 100644 --- a/dbms/src/Storages/System/StorageSystemReplicas.h +++ b/dbms/src/Storages/System/StorageSystemReplicas.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemReplicas : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemReplicas"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemReplicationQueue.h b/dbms/src/Storages/System/StorageSystemReplicationQueue.h index 63dc58118cd..79e015250da 100644 --- a/dbms/src/Storages/System/StorageSystemReplicationQueue.h +++ b/dbms/src/Storages/System/StorageSystemReplicationQueue.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemReplicationQueue : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemReplicationQueue"; } diff --git a/dbms/src/Storages/System/StorageSystemSettings.h b/dbms/src/Storages/System/StorageSystemSettings.h index e44e0abbcd4..f745fe32dcf 100644 --- a/dbms/src/Storages/System/StorageSystemSettings.h +++ b/dbms/src/Storages/System/StorageSystemSettings.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemSettings : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemSettings"; } diff --git a/dbms/src/Storages/System/StorageSystemTableEngines.h b/dbms/src/Storages/System/StorageSystemTableEngines.h index f0f6b62d59d..0bbb03d3898 100644 --- a/dbms/src/Storages/System/StorageSystemTableEngines.h +++ b/dbms/src/Storages/System/StorageSystemTableEngines.h @@ -10,6 +10,7 @@ namespace DB class StorageSystemTableEngines : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemTableFunctions.h b/dbms/src/Storages/System/StorageSystemTableFunctions.h index 413af0f5c66..504ecda4afb 100644 --- a/dbms/src/Storages/System/StorageSystemTableFunctions.h +++ b/dbms/src/Storages/System/StorageSystemTableFunctions.h @@ -9,6 +9,7 @@ namespace DB class StorageSystemTableFunctions : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/dbms/src/Storages/System/StorageSystemTables.h b/dbms/src/Storages/System/StorageSystemTables.h index faf419a6139..fc5a82e9a52 100644 --- a/dbms/src/Storages/System/StorageSystemTables.h +++ b/dbms/src/Storages/System/StorageSystemTables.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemTables : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemTables"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemZooKeeper.h b/dbms/src/Storages/System/StorageSystemZooKeeper.h index 9644fe96162..ac0ddd29d1f 100644 --- a/dbms/src/Storages/System/StorageSystemZooKeeper.h +++ b/dbms/src/Storages/System/StorageSystemZooKeeper.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemZooKeeper : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemZooKeeper"; } diff --git a/libs/libcommon/include/ext/shared_ptr_helper.h b/libs/libcommon/include/ext/shared_ptr_helper.h index f7fd7c38ace..ca7219e6261 100644 --- a/libs/libcommon/include/ext/shared_ptr_helper.h +++ b/libs/libcommon/include/ext/shared_ptr_helper.h @@ -7,32 +7,16 @@ namespace ext /** Allows to make std::shared_ptr from T with protected constructor. * - * Derive your T class from shared_ptr_helper + * Derive your T class from shared_ptr_helper and add shared_ptr_helper as a friend * and you will have static 'create' method in your class. - * - * Downsides: - * - your class cannot be final; - * - awful compilation error messages; - * - bad code navigation. - * - different dynamic type of created object, you cannot use typeid. */ template struct shared_ptr_helper { template - static auto create(TArgs &&... args) + static std::shared_ptr create(TArgs &&... args) { - /** Local struct makes protected constructor to be accessible by std::make_shared function. - * This trick is suggested by Yurii Diachenko, - * inspired by https://habrahabr.ru/company/mailru/blog/341584/ - * that is translation of http://videocortex.io/2017/Bestiary/#-voldemort-types - */ - struct Local : T - { - Local(TArgs &&... args) : T(std::forward(args)...) {} - }; - - return std::make_shared(std::forward(args)...); + return std::shared_ptr(new T(std::forward(args)...)); } }; From 5989b19684e1b01022dc6f2e21cca252fda384bb Mon Sep 17 00:00:00 2001 From: Dmitry Rubashkin Date: Mon, 26 Aug 2019 22:16:28 +0300 Subject: [PATCH 447/509] Final --- dbms/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index b589c398238..1de9536bb56 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -240,6 +240,7 @@ target_link_libraries(clickhouse_common_io ${EXECINFO_LIBRARIES} PUBLIC ${Boost_SYSTEM_LIBRARY} + ${Boost_PROGRAM_OPTIONS_LIBRARY} PRIVATE apple_rt PUBLIC From 5f25c2d7b4b41500df2f4859f7db8faca5851f48 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Mon, 26 Aug 2019 22:49:52 +0300 Subject: [PATCH 448/509] merging with master --- dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 03bf45b4a59..8f07fc430e1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -213,7 +213,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( part_info.min_ttl = part->ttl_infos.part_min_ttl; part_info.max_ttl = part->ttl_infos.part_max_ttl; - time_t ttl = data.settings.ttl_only_drop_parts ? part_info.max_ttl : part_info.min_ttl; + time_t ttl = data_settings->ttl_only_drop_parts ? part_info.max_ttl : part_info.min_ttl; if (ttl && ttl <= current_time) has_part_with_expired_ttl = true; @@ -242,7 +242,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( /// NOTE Could allow selection of different merge strategy. if (can_merge_with_ttl && has_part_with_expired_ttl && !ttl_merges_blocker.isCancelled()) { - merge_selector = std::make_unique(current_time, data.settings.ttl_only_drop_parts); + merge_selector = std::make_unique(current_time, data_settings->ttl_only_drop_parts); last_merge_with_ttl = current_time; } else From 57fa4e1dd72224f36965ef82550978022cc44360 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Aug 2019 00:40:29 +0300 Subject: [PATCH 449/509] Minor lock improvement for ReplicatedMergeTree --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 4 ++++ dbms/src/Storages/TableStructureLockHolder.h | 5 +++++ 2 files changed, 9 insertions(+) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 6984ab07ee4..b3ae13ee208 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4936,7 +4936,11 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ /// If necessary, wait until the operation is performed on all replicas. if (context.getSettingsRef().replication_alter_partitions_sync > 1) + { + lock2.release(); + lock1.release(); waitForAllReplicasToProcessLogEntry(entry); + } } void StorageReplicatedMergeTree::getCommitPartOps( diff --git a/dbms/src/Storages/TableStructureLockHolder.h b/dbms/src/Storages/TableStructureLockHolder.h index 1413ead80a8..17e960503f2 100644 --- a/dbms/src/Storages/TableStructureLockHolder.h +++ b/dbms/src/Storages/TableStructureLockHolder.h @@ -32,6 +32,11 @@ private: /// Order is important. RWLockImpl::LockHolder new_data_structure_lock; RWLockImpl::LockHolder structure_lock; + + void release() + { + *this = TableStructureReadLockHolder(); + } }; } From d654f2507eb0b1a1c60fb266b0b5812c79922330 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Aug 2019 00:40:56 +0300 Subject: [PATCH 450/509] Fixed typo in test --- .../0_stateless/00626_replace_partition_from_table_zookeeper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index 5d1a7338e46..0d0bbae1402 100755 --- a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -101,7 +101,7 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE test.src;" $CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" $CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);" $CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2);" -- trash part to be +$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2); -- trash part to be deleted" # Stop replication at the second replica and remove source table to use fetch instead of copying $CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES test.dst_r2;" From 7985270624e877de15f958f0e3f19a23365f34de Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Aug 2019 00:41:10 +0300 Subject: [PATCH 451/509] Disable processors by default --- dbms/src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 3adae6d9e93..f12efe6548e 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -332,7 +332,7 @@ struct Settings : public SettingsCollection M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only for 'mysql' table function.") \ M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.") \ \ - M(SettingBool, experimental_use_processors, true, "Use processors pipeline.") \ + M(SettingBool, experimental_use_processors, false, "Use processors pipeline.") \ \ M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \ M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \ From 5fb5c8dffcbc773d46d96d3ec43fc4febca4d468 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Mon, 26 Aug 2019 19:50:37 -0400 Subject: [PATCH 452/509] Updating all live view tests to set the allow_experimental_live_view option. --- .../0_stateless/00960_live_view_watch_events_live.py | 5 +++++ .../queries/0_stateless/00961_temporary_live_view_watch.sql | 2 ++ .../0_stateless/00962_temporary_live_view_watch_live.py | 5 +++++ ...00963_temporary_live_view_watch_live_timeout.py.disabled | 5 +++++ .../0_stateless/00964_live_view_watch_events_heartbeat.py | 5 +++++ .../queries/0_stateless/00965_live_view_watch_heartbeat.py | 5 +++++ .../0_stateless/00966_live_view_watch_events_http.py | 5 ++++- .../tests/queries/0_stateless/00967_live_view_watch_http.py | 5 ++++- ...0968_live_view_select_format_jsoneachrowwithprogress.sql | 2 ++ ...00969_live_view_watch_format_jsoneachrowwithprogress.sql | 2 ++ .../00970_live_view_watch_events_http_heartbeat.py | 6 ++++-- .../0_stateless/00971_live_view_watch_http_heartbeat.py | 5 ++++- dbms/tests/queries/0_stateless/00972_live_view_select_1.sql | 2 ++ dbms/tests/queries/0_stateless/00973_live_view_select.sql | 2 ++ .../0_stateless/00974_live_view_select_with_aggregation.sql | 2 ++ dbms/tests/queries/0_stateless/00975_live_view_create.sql | 2 ++ .../queries/0_stateless/00976_live_view_select_version.sql | 2 ++ .../queries/0_stateless/00977_live_view_watch_events.sql | 2 ++ dbms/tests/queries/0_stateless/00978_live_view_watch.sql | 2 ++ .../tests/queries/0_stateless/00979_live_view_watch_live.py | 5 +++++ .../0_stateless/00980_create_temporary_live_view.sql | 2 ++ 21 files changed, 68 insertions(+), 5 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py index b7fc3f4e3a6..2095683720e 100755 --- a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py +++ b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py @@ -16,6 +16,11 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client2.expect(prompt) + client1.send('SET allow_experimental_live_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_live_view = 1') + client2.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.mt') diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql index c3e2ab8d102..7992da92f97 100644 --- a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql +++ b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql @@ -1,3 +1,5 @@ +SET allow_experimental_live_view = 1; + DROP TABLE IF EXISTS test.lv; DROP TABLE IF EXISTS test.mt; diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py index f27b1213c70..3dbec01b29a 100755 --- a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py @@ -16,6 +16,11 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client2.expect(prompt) + client1.send('SET allow_experimental_live_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_live_view = 1') + client2.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.mt') diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled index df627c84e49..b324c1b90cc 100755 --- a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled +++ b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled @@ -16,6 +16,11 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client2.expect(prompt) + client1.send('SET allow_experimental_live_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_live_view = 1') + client2.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.mt') diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py index 5664c0e6c6d..528f18839bb 100755 --- a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -16,6 +16,11 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client2.expect(prompt) + client1.send('SET allow_experimental_live_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_live_view = 1') + client2.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.mt') diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py index 03e22175dff..2723936f876 100755 --- a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py @@ -16,6 +16,11 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client2.expect(prompt) + client1.send('SET allow_experimental_live_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_live_view = 1') + client2.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.mt') diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py index bb9d6152200..72ab3ea8818 100755 --- a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py +++ b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py @@ -15,6 +15,9 @@ log = None with client(name='client1>', log=log) as client1: client1.expect(prompt) + client1.send('SET allow_experimental_live_view = 1') + client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.mt') @@ -25,7 +28,7 @@ with client(name='client1>', log=log) as client1: client1.expect(prompt) - with http_client({'method':'GET', 'url': '/?query=WATCH%20test.lv%20EVENTS'}, name='client2>', log=log) as client2: + with http_client({'method':'GET', 'url': '/?allow_experimental_live_view=1&query=WATCH%20test.lv%20EVENTS'}, name='client2>', log=log) as client2: client2.expect('.*1\n') client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py index d3439431eb3..e2f33971c3d 100755 --- a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py +++ b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py @@ -15,6 +15,9 @@ log = None with client(name='client1>', log=log) as client1: client1.expect(prompt) + client1.send('SET allow_experimental_live_view = 1') + client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.mt') @@ -25,7 +28,7 @@ with client(name='client1>', log=log) as client1: client1.expect(prompt) - with http_client({'method':'GET', 'url':'/?query=WATCH%20test.lv'}, name='client2>', log=log) as client2: + with http_client({'method':'GET', 'url':'/?allow_experimental_live_view=1&query=WATCH%20test.lv'}, name='client2>', log=log) as client2: client2.expect('.*0\t1\n') client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql index 8c6f4197d54..1023cdf6b29 100644 --- a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql +++ b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql @@ -1,3 +1,5 @@ +SET allow_experimental_live_view = 1; + DROP TABLE IF EXISTS test.lv; DROP TABLE IF EXISTS test.mt; diff --git a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql index 725a4ad00ed..3e46d55c014 100644 --- a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql +++ b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql @@ -1,3 +1,5 @@ +SET allow_experimental_live_view = 1; + DROP TABLE IF EXISTS test.lv; DROP TABLE IF EXISTS test.mt; diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py index 63628c4a76f..8435cdc147a 100755 --- a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py @@ -15,6 +15,9 @@ log = None with client(name='client1>', log=log) as client1: client1.expect(prompt) + client1.send('SET allow_experimental_live_view = 1') + client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.mt') @@ -24,8 +27,7 @@ with client(name='client1>', log=log) as client1: client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) - - with http_client({'method':'GET', 'url': '/?live_view_heartbeat_interval=1&query=WATCH%20test.lv%20EVENTS%20FORMAT%20JSONEachRowWithProgress'}, name='client2>', log=log) as client2: + with http_client({'method':'GET', 'url': '/?allow_experimental_live_view=1&live_view_heartbeat_interval=1&query=WATCH%20test.lv%20EVENTS%20FORMAT%20JSONEachRowWithProgress'}, name='client2>', log=log) as client2: client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}\n', escape=True) client2.expect('{"row":{"version":"1"}', escape=True) client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py index 7bdb47b7caa..2317d705efe 100755 --- a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py +++ b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py @@ -15,6 +15,9 @@ log = None with client(name='client1>', log=log) as client1: client1.expect(prompt) + client1.send('SET allow_experimental_live_view = 1') + client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.mt') @@ -24,7 +27,7 @@ with client(name='client1>', log=log) as client1: client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') client1.expect(prompt) - with http_client({'method':'GET', 'url':'/?live_view_heartbeat_interval=1&query=WATCH%20test.lv%20FORMAT%20JSONEachRowWithProgress'}, name='client2>', log=log) as client2: + with http_client({'method':'GET', 'url':'/?allow_experimental_live_view=1&live_view_heartbeat_interval=1&query=WATCH%20test.lv%20FORMAT%20JSONEachRowWithProgress'}, name='client2>', log=log) as client2: client2.expect('"progress".*',) client2.expect('{"row":{"sum(a)":"0","_version":"1"}}\n', escape=True) client2.expect('"progress".*\n') diff --git a/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql b/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql index 661080b577b..135516b0cd3 100644 --- a/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql +++ b/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql @@ -1,3 +1,5 @@ +SET allow_experimental_live_view = 1; + DROP TABLE IF EXISTS test.lv; CREATE LIVE VIEW test.lv AS SELECT 1; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_select.sql b/dbms/tests/queries/0_stateless/00973_live_view_select.sql index ff4a45ffcc1..4b5ca0a2dd7 100644 --- a/dbms/tests/queries/0_stateless/00973_live_view_select.sql +++ b/dbms/tests/queries/0_stateless/00973_live_view_select.sql @@ -1,3 +1,5 @@ +SET allow_experimental_live_view = 1; + DROP TABLE IF EXISTS test.lv; DROP TABLE IF EXISTS test.mt; diff --git a/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql index 3c11f855c9d..3faaec8f623 100644 --- a/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql +++ b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql @@ -1,3 +1,5 @@ +SET allow_experimental_live_view = 1; + DROP TABLE IF EXISTS test.lv; DROP TABLE IF EXISTS test.mt; diff --git a/dbms/tests/queries/0_stateless/00975_live_view_create.sql b/dbms/tests/queries/0_stateless/00975_live_view_create.sql index 1c929b15b00..02c1644d193 100644 --- a/dbms/tests/queries/0_stateless/00975_live_view_create.sql +++ b/dbms/tests/queries/0_stateless/00975_live_view_create.sql @@ -1,3 +1,5 @@ +SET allow_experimental_live_view = 1; + DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); diff --git a/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql b/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql index 5f3ab1f7546..ae1c59a92d7 100644 --- a/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql +++ b/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql @@ -1,3 +1,5 @@ +SET allow_experimental_live_view = 1; + DROP TABLE IF EXISTS test.lv; DROP TABLE IF EXISTS test.mt; diff --git a/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql index a3b84e8d4c1..3e0d066fb8d 100644 --- a/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql +++ b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql @@ -1,3 +1,5 @@ +SET allow_experimental_live_view = 1; + DROP TABLE IF EXISTS test.lv; DROP TABLE IF EXISTS test.mt; diff --git a/dbms/tests/queries/0_stateless/00978_live_view_watch.sql b/dbms/tests/queries/0_stateless/00978_live_view_watch.sql index abe4a6c32ae..b8d0d93ccab 100644 --- a/dbms/tests/queries/0_stateless/00978_live_view_watch.sql +++ b/dbms/tests/queries/0_stateless/00978_live_view_watch.sql @@ -1,3 +1,5 @@ +SET allow_experimental_live_view = 1; + DROP TABLE IF EXISTS test.lv; DROP TABLE IF EXISTS test.mt; diff --git a/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py index 948e4c93662..8c5bc5b8eb2 100755 --- a/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py +++ b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py @@ -16,6 +16,11 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client2.expect(prompt) + client1.send('SET allow_experimental_live_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_live_view = 1') + client2.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.mt') diff --git a/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql b/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql index 8cd6ee06ace..037c2a9e587 100644 --- a/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql +++ b/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql @@ -1,3 +1,5 @@ +SET allow_experimental_live_view = 1; + DROP TABLE IF EXISTS test.lv; DROP TABLE IF EXISTS test.mt; From 2fbcd607f3b10ee49cecd796fc75594b62edca46 Mon Sep 17 00:00:00 2001 From: Weiqing Xu Date: Tue, 27 Aug 2019 08:23:07 +0800 Subject: [PATCH 453/509] check free space when use external sort/aggerator --- dbms/src/Core/Settings.h | 1 + .../src/DataStreams/MergeSortingBlockInputStream.cpp | 10 ++++++++-- dbms/src/DataStreams/MergeSortingBlockInputStream.h | 8 +++++++- dbms/src/Interpreters/Aggregator.cpp | 5 +++++ dbms/src/Interpreters/Aggregator.h | 10 +++++++--- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 12 ++++++------ dbms/src/Interpreters/tests/aggregate.cpp | 2 +- .../Processors/Transforms/MergeSortingTransform.cpp | 9 ++++++++- .../Processors/Transforms/MergeSortingTransform.h | 8 +++++++- .../Processors/tests/processors_test_aggregation.cpp | 6 ++++-- .../processors_test_merge_sorting_transform.cpp | 2 +- 11 files changed, 55 insertions(+), 18 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index cb12a969b76..43e22f78149 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -352,6 +352,7 @@ struct Settings : public SettingsCollection \ M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") \ M(SettingBool, compile, false, "Whether query compilation is enabled. Will be removed after 2020-03-13") \ + M(SettingUInt64, min_free_disk_space, 0, "The minimum disk space to keep") \ DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp index e30dd4ae1de..86fcc4ffff9 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp @@ -21,10 +21,11 @@ namespace DB MergeSortingBlockInputStream::MergeSortingBlockInputStream( const BlockInputStreamPtr & input, SortDescription & description_, size_t max_merged_block_size_, UInt64 limit_, size_t max_bytes_before_remerge_, - size_t max_bytes_before_external_sort_, const std::string & tmp_path_) + size_t max_bytes_before_external_sort_, const std::string & tmp_path_, size_t min_free_disk_space_) : description(description_), max_merged_block_size(max_merged_block_size_), limit(limit_), max_bytes_before_remerge(max_bytes_before_remerge_), - max_bytes_before_external_sort(max_bytes_before_external_sort_), tmp_path(tmp_path_) + max_bytes_before_external_sort(max_bytes_before_external_sort_), tmp_path(tmp_path_), + min_free_disk_space(min_free_disk_space_) { children.push_back(input); header = children.at(0)->getHeader(); @@ -77,6 +78,11 @@ Block MergeSortingBlockInputStream::readImpl() */ if (max_bytes_before_external_sort && sum_bytes_in_blocks > max_bytes_before_external_sort) { + auto freeSpace = Poco::File(tmp_path).freeSpace(); + if (min_free_disk_space > freeSpace - sum_bytes_in_blocks) + { + throw Exception("Not enough space.", ErrorCodes::NOT_ENOUGH_SPACE); + } Poco::File(tmp_path).createDirectories(); temporary_files.emplace_back(std::make_unique(tmp_path)); const std::string & path = temporary_files.back()->path(); diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.h b/dbms/src/DataStreams/MergeSortingBlockInputStream.h index 9f257b82260..4cd9315bc3c 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.h +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.h @@ -18,6 +18,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_ENOUGH_SPACE; +} /** Merges stream of sorted each-separately blocks to sorted as-a-whole stream of blocks. * If data to sort is too much, could use external sorting, with temporary files. */ @@ -73,7 +77,8 @@ public: MergeSortingBlockInputStream(const BlockInputStreamPtr & input, SortDescription & description_, size_t max_merged_block_size_, UInt64 limit_, size_t max_bytes_before_remerge_, - size_t max_bytes_before_external_sort_, const std::string & tmp_path_); + size_t max_bytes_before_external_sort_, const std::string & tmp_path_, + size_t min_free_disk_space_); String getName() const override { return "MergeSorting"; } @@ -93,6 +98,7 @@ private: size_t max_bytes_before_remerge; size_t max_bytes_before_external_sort; const std::string tmp_path; + size_t min_free_disk_space; Logger * log = &Logger::get("MergeSortingBlockInputStream"); diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 373b47f7315..f2018e443f1 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -639,6 +639,11 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re && current_memory_usage > static_cast(params.max_bytes_before_external_group_by) && worth_convert_to_two_level) { + auto freeSpace = Poco::File(params.tmp_path).freeSpace(); + if (params.min_free_disk_space > freeSpace - current_memory_usage) + { + throw Exception("Not enough space.", ErrorCodes::NOT_ENOUGH_SPACE); + } writeToTemporaryFile(result); } diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index b48663ff689..c3d1d5df8fd 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -39,6 +39,7 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_AGGREGATED_DATA_VARIANT; + extern const int NOT_ENOUGH_SPACE; } class IBlockOutputStream; @@ -796,6 +797,7 @@ public: /// Settings is used to determine cache size. No threads are created. size_t max_threads; + const size_t min_free_disk_space; Params( const Block & src_header_, const ColumnNumbers & keys_, const AggregateDescriptions & aggregates_, @@ -803,21 +805,23 @@ public: size_t group_by_two_level_threshold_, size_t group_by_two_level_threshold_bytes_, size_t max_bytes_before_external_group_by_, bool empty_result_for_aggregation_by_empty_set_, - const std::string & tmp_path_, size_t max_threads_) + const std::string & tmp_path_, size_t max_threads_, + size_t min_free_disk_space_) : src_header(src_header_), keys(keys_), aggregates(aggregates_), keys_size(keys.size()), aggregates_size(aggregates.size()), overflow_row(overflow_row_), max_rows_to_group_by(max_rows_to_group_by_), group_by_overflow_mode(group_by_overflow_mode_), group_by_two_level_threshold(group_by_two_level_threshold_), group_by_two_level_threshold_bytes(group_by_two_level_threshold_bytes_), max_bytes_before_external_group_by(max_bytes_before_external_group_by_), empty_result_for_aggregation_by_empty_set(empty_result_for_aggregation_by_empty_set_), - tmp_path(tmp_path_), max_threads(max_threads_) + tmp_path(tmp_path_), max_threads(max_threads_), + min_free_disk_space(min_free_disk_space_) { } /// Only parameters that matter during merge. Params(const Block & intermediate_header_, const ColumnNumbers & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_) - : Params(Block(), keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, 0, 0, 0, false, "", max_threads_) + : Params(Block(), keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, 0, 0, 0, false, "", max_threads_, 0) { intermediate_header = intermediate_header_; } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 2d583c3c353..6c8769aa3d1 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1657,7 +1657,7 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context.getTemporaryPath(), settings.max_threads); + context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space); /// If there are several sources, then we perform parallel aggregation if (pipeline.streams.size() > 1) @@ -1723,7 +1723,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context.getTemporaryPath(), settings.max_threads); + context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space); auto transform_params = std::make_shared(params, final); @@ -1943,7 +1943,7 @@ void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificato false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, SettingUInt64(0), SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context.getTemporaryPath(), settings.max_threads); + context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space); if (modificator == Modificator::ROLLUP) pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); @@ -1972,7 +1972,7 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPipeline & pipeline, Modif false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, SettingUInt64(0), SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context.getTemporaryPath(), settings.max_threads); + context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space); auto transform_params = std::make_shared(params, true); @@ -2073,7 +2073,7 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr so pipeline.firstStream() = std::make_shared( pipeline.firstStream(), order_descr, settings.max_block_size, limit, settings.max_bytes_before_remerge_sort, - settings.max_bytes_before_external_sort, context.getTemporaryPath()); + settings.max_bytes_before_external_sort, context.getTemporaryPath(), settings.min_free_disk_space); } } @@ -2111,7 +2111,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP return std::make_shared( header, order_descr, settings.max_block_size, limit, settings.max_bytes_before_remerge_sort, - settings.max_bytes_before_external_sort, context.getTemporaryPath()); + settings.max_bytes_before_external_sort, context.getTemporaryPath(), settings.min_free_disk_space); }); } diff --git a/dbms/src/Interpreters/tests/aggregate.cpp b/dbms/src/Interpreters/tests/aggregate.cpp index 73e71d178ea..4d4d964aa9a 100644 --- a/dbms/src/Interpreters/tests/aggregate.cpp +++ b/dbms/src/Interpreters/tests/aggregate.cpp @@ -79,7 +79,7 @@ int main(int argc, char ** argv) Aggregator::Params params( stream->getHeader(), {0, 1}, aggregate_descriptions, - false, 0, OverflowMode::THROW, 0, 0, 0, false, "", 1); + false, 0, OverflowMode::THROW, 0, 0, 0, false, "", 1, 0); Aggregator aggregator(params); diff --git a/dbms/src/Processors/Transforms/MergeSortingTransform.cpp b/dbms/src/Processors/Transforms/MergeSortingTransform.cpp index 8591f5447f7..93e9acf9ebc 100644 --- a/dbms/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/dbms/src/Processors/Transforms/MergeSortingTransform.cpp @@ -236,11 +236,13 @@ MergeSortingTransform::MergeSortingTransform( SortDescription & description_, size_t max_merged_block_size_, UInt64 limit_, size_t max_bytes_before_remerge_, - size_t max_bytes_before_external_sort_, const std::string & tmp_path_) + size_t max_bytes_before_external_sort_, const std::string & tmp_path_, + size_t min_free_disk_space_) : IProcessor({header}, {header}) , description(description_), max_merged_block_size(max_merged_block_size_), limit(limit_) , max_bytes_before_remerge(max_bytes_before_remerge_) , max_bytes_before_external_sort(max_bytes_before_external_sort_), tmp_path(tmp_path_) + , min_free_disk_space(min_free_disk_space_) { auto & sample = inputs.front().getHeader(); @@ -504,6 +506,11 @@ void MergeSortingTransform::consume(Chunk chunk) */ if (max_bytes_before_external_sort && sum_bytes_in_blocks > max_bytes_before_external_sort) { + auto freeSpace = Poco::File(tmp_path).freeSpace(); + if (min_free_disk_space > freeSpace - sum_bytes_in_blocks) + { + throw Exception("Not enough space.", ErrorCodes::NOT_ENOUGH_SPACE); + } Poco::File(tmp_path).createDirectories(); temporary_files.emplace_back(std::make_unique(tmp_path)); const std::string & path = temporary_files.back()->path(); diff --git a/dbms/src/Processors/Transforms/MergeSortingTransform.h b/dbms/src/Processors/Transforms/MergeSortingTransform.h index 0ab517fc5d4..eec249296ef 100644 --- a/dbms/src/Processors/Transforms/MergeSortingTransform.h +++ b/dbms/src/Processors/Transforms/MergeSortingTransform.h @@ -14,6 +14,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_ENOUGH_SPACE; +} class MergeSorter; class MergeSortingTransform : public IProcessor @@ -24,7 +28,8 @@ public: SortDescription & description_, size_t max_merged_block_size_, UInt64 limit_, size_t max_bytes_before_remerge_, - size_t max_bytes_before_external_sort_, const std::string & tmp_path_); + size_t max_bytes_before_external_sort_, const std::string & tmp_path_, + size_t min_free_disk_space_); ~MergeSortingTransform() override; @@ -44,6 +49,7 @@ private: size_t max_bytes_before_remerge; size_t max_bytes_before_external_sort; const std::string tmp_path; + size_t min_free_disk_space; Logger * log = &Logger::get("MergeSortingBlockInputStream"); diff --git a/dbms/src/Processors/tests/processors_test_aggregation.cpp b/dbms/src/Processors/tests/processors_test_aggregation.cpp index 2306de4edc0..ed868d08762 100644 --- a/dbms/src/Processors/tests/processors_test_aggregation.cpp +++ b/dbms/src/Processors/tests/processors_test_aggregation.cpp @@ -229,7 +229,8 @@ try max_bytes_before_external_group_by, false, /// empty_result_for_aggregation_by_empty_set cur_path, /// tmp_path - 1 /// max_threads + 1, /// max_threads + 0 ); auto agg_params = std::make_shared(params, /* final =*/ false); @@ -301,7 +302,8 @@ try max_bytes_before_external_group_by, false, /// empty_result_for_aggregation_by_empty_set cur_path, /// tmp_path - 1 /// max_threads + 1, /// max_threads + 0 ); auto agg_params = std::make_shared(params, /* final =*/ false); diff --git a/dbms/src/Processors/tests/processors_test_merge_sorting_transform.cpp b/dbms/src/Processors/tests/processors_test_merge_sorting_transform.cpp index a5059011e9b..8e6b4655127 100644 --- a/dbms/src/Processors/tests/processors_test_merge_sorting_transform.cpp +++ b/dbms/src/Processors/tests/processors_test_merge_sorting_transform.cpp @@ -133,7 +133,7 @@ try SortDescription description = {{0, 1, 1}}; auto transform = std::make_shared( source->getPort().getHeader(), description, - max_merged_block_size, limit, max_bytes_before_remerge, max_bytes_before_external_sort, "."); + max_merged_block_size, limit, max_bytes_before_remerge, max_bytes_before_external_sort, ".", 0); auto sink = std::make_shared(); connect(source->getPort(), transform->getInputs().front()); From 7851d8fe0a6dfc98278fcda40f939a6fd898efd8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 27 Aug 2019 12:34:53 +0300 Subject: [PATCH 454/509] Add ability to alter settings with other types of alter --- dbms/src/Storages/IStorage.cpp | 40 +++++++++---------- dbms/src/Storages/IStorage.h | 11 ++--- dbms/src/Storages/Kafka/KafkaSettings.h | 20 +++++----- dbms/src/Storages/Kafka/StorageKafka.cpp | 9 ----- dbms/src/Storages/Kafka/StorageKafka.h | 5 --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 18 ++++----- dbms/src/Storages/MergeTree/MergeTreeData.h | 8 ++-- dbms/src/Storages/StorageMergeTree.cpp | 27 ++++++++----- .../Storages/StorageReplicatedMergeTree.cpp | 19 ++++++++- .../00980_merge_alter_settings.reference | 1 + .../00980_merge_alter_settings.sql | 4 ++ ...keeper_merge_tree_alter_settings.reference | 2 + ...80_zookeeper_merge_tree_alter_settings.sql | 5 +++ 13 files changed, 90 insertions(+), 79 deletions(-) diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 58974010c27..2f3a48d90b6 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -370,15 +370,9 @@ TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id) } -void IStorage::alterSettings( - const SettingsChanges & new_changes, - const Context & context, - TableStructureWriteLockHolder & /* table_lock_holder */) +IDatabase::ASTModifier IStorage::getSettingsModifier(const SettingsChanges & new_changes) const { - const String current_database_name = getDatabaseName(); - const String current_table_name = getTableName(); - - IDatabase::ASTModifier storage_modifier = [&] (IAST & ast) + return [&] (IAST & ast) { if (!new_changes.empty()) { @@ -399,7 +393,6 @@ void IStorage::alterSettings( } } }; - context.getDatabase(current_database_name)->alterTable(context, current_table_name, getColumns(), getIndices(), getConstraints(), storage_modifier); } @@ -408,26 +401,29 @@ void IStorage::alter( const Context & context, TableStructureWriteLockHolder & table_lock_holder) { + if (params.isMutable()) + throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + const String database_name = getDatabaseName(); const String table_name = getTableName(); + if (params.isSettingsAlter()) { SettingsChanges new_changes; params.applyForSettingsOnly(new_changes); - alterSettings(new_changes, context, table_lock_holder); - return; + IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); + context.getDatabase(database_name)->alterTable(context, table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); + } + else + { + lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); + auto new_columns = getColumns(); + auto new_indices = getIndices(); + auto new_constraints = getConstraints(); + params.applyForColumnsOnly(new_columns); + context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {}); + setColumns(std::move(new_columns)); } - - if (params.isMutable()) - throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); - - lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - auto new_columns = getColumns(); - auto new_indices = getIndices(); - auto new_constraints = getConstraints(); - params.applyForColumnsOnly(new_columns); - context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {}); - setColumns(std::move(new_columns)); } } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 5672a1ba3fb..35c1439964a 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -147,6 +147,10 @@ protected: /// still thread-unsafe part. /// Returns whether the column is virtual - by default all columns are real. /// Initially reserved virtual column name may be shadowed by real column. virtual bool isVirtualColumn(const String & column_name) const; + + /// Returns modifier of settings in storage definition + IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const; + private: ColumnsDescription columns; /// combined real and virtual columns const ColumnsDescription virtuals = {}; @@ -291,13 +295,6 @@ public: throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /** ALTER table settings if possible. Otherwise throws exception. - */ - virtual void alterSettings( - const SettingsChanges & new_changes, - const Context & context, - TableStructureWriteLockHolder & table_lock_holder); - /** Perform any background work. For example, combining parts in a MergeTree type table. * Returns whether any work has been done. */ diff --git a/dbms/src/Storages/Kafka/KafkaSettings.h b/dbms/src/Storages/Kafka/KafkaSettings.h index e43ea7cd70e..bc453238b51 100644 --- a/dbms/src/Storages/Kafka/KafkaSettings.h +++ b/dbms/src/Storages/Kafka/KafkaSettings.h @@ -17,16 +17,16 @@ struct KafkaSettings : public SettingsCollection /// M (mutable) for normal settings, IM (immutable) for not updateable settings. #define LIST_OF_KAFKA_SETTINGS(M, IM) \ - M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.") \ - M(SettingString, kafka_topic_list, "", "A list of Kafka topics.") \ - M(SettingString, kafka_group_name, "", "A group of Kafka consumers.") \ - M(SettingString, kafka_format, "", "The message format for Kafka engine.") \ - M(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.") \ - M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine") \ - M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") \ - M(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") \ - M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block") \ - M(SettingUInt64, kafka_commit_every_batch, 0, "Commit every consumed and handled batch instead of a single commit after writing a whole block") + IM(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.") \ + IM(SettingString, kafka_topic_list, "", "A list of Kafka topics.") \ + IM(SettingString, kafka_group_name, "", "A group of Kafka consumers.") \ + IM(SettingString, kafka_format, "", "The message format for Kafka engine.") \ + IM(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.") \ + IM(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine") \ + IM(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") \ + IM(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") \ + IM(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block") \ + IM(SettingUInt64, kafka_commit_every_batch, 0, "Commit every consumed and handled batch instead of a single commit after writing a whole block") DECLARE_SETTINGS_COLLECTION(LIST_OF_KAFKA_SETTINGS) diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 1694c8e3ade..d53469259ab 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -412,15 +412,6 @@ bool StorageKafka::hasSetting(const String & setting_name) const return KafkaSettings::findIndex(setting_name) != KafkaSettings::npos; } -void StorageKafka::alterSettings( - const SettingsChanges & /* new_changes */, - const Context & /* context */, - TableStructureWriteLockHolder & /* table_lock_holder */) -{ - throw Exception("Storage '" + getName() + "' doesn't support settings alter", ErrorCodes::UNSUPPORTED_METHOD); -} - - void registerStorageKafka(StorageFactory & factory) { factory.registerStorage("Kafka", [](const StorageFactory::Arguments & args) diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index 27f1e7e0ec7..c2ff3562116 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -59,11 +59,6 @@ public: bool hasSetting(const String & setting_name) const override; - void alterSettings( - const SettingsChanges & new_changes, - const Context & context, - TableStructureWriteLockHolder & table_lock_holder) override; - protected: StorageKafka( const std::string & table_name_, diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 0767fb35326..b2d4a4b9d73 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1645,18 +1645,16 @@ void MergeTreeData::alterDataPart( return; } -void MergeTreeData::alterSettings( +void MergeTreeData::changeSettings( const SettingsChanges & new_changes, - const Context & context, - TableStructureWriteLockHolder & table_lock_holder) + TableStructureWriteLockHolder & /* table_lock_holder */) { - const String current_database_name = getDatabaseName(); - const String current_table_name = getTableName(); - - MergeTreeSettings copy = *getSettings(); - copy.updateFromChanges(new_changes); - IStorage::alterSettings(new_changes, context, table_lock_holder); - storage_settings.set(std::make_unique(copy)); + if (!new_changes.empty()) + { + MergeTreeSettings copy = *getSettings(); + copy.updateFromChanges(new_changes); + storage_settings.set(std::make_unique(copy)); + } } bool MergeTreeData::hasSetting(const String & setting_name) const diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index f759b87f986..0440a3181c8 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -537,12 +537,10 @@ public: bool skip_sanity_checks, AlterDataPartTransactionPtr& transaction); - /// Performs ALTER of table settings (MergeTreeSettings). Lightweight operation, affects metadata only. - /// Not atomic, have to be done with alter intention lock. - void alterSettings( + /// Change MergeTreeSettings + void changeSettings( const SettingsChanges & new_changes, - const Context & context, - TableStructureWriteLockHolder & table_lock_holder) override; + TableStructureWriteLockHolder & table_lock_holder); /// All MergeTreeData children have settings. bool hasSetting(const String & setting_name) const override; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 1504413801c..4b7b2c446f6 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -253,15 +253,6 @@ void StorageMergeTree::alter( if (!params.isMutable()) { - SettingsChanges new_changes; - /// We don't need to lock table structure exclusively to ALTER settings. - if (params.isSettingsAlter()) - { - params.applyForSettingsOnly(new_changes); - alterSettings(new_changes, context, table_lock_holder); - return; - } - lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); auto new_columns = getColumns(); auto new_indices = getIndices(); @@ -269,8 +260,14 @@ void StorageMergeTree::alter( ASTPtr new_order_by_ast = order_by_ast; ASTPtr new_primary_key_ast = primary_key_ast; ASTPtr new_ttl_table_ast = ttl_table_ast; + SettingsChanges new_changes; + params.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes); - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {}); + + changeSettings(new_changes, table_lock_holder); + + IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); + context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, settings_modifier); setColumns(std::move(new_columns)); return; } @@ -305,9 +302,19 @@ void StorageMergeTree::alter( if (new_ttl_table_ast.get() != ttl_table_ast.get()) storage_ast.set(storage_ast.ttl_table, new_ttl_table_ast); + + if (!new_changes.empty()) + { + auto settings_modifier = getSettingsModifier(new_changes); + settings_modifier(ast); + } }; + + changeSettings(new_changes, table_lock_holder); + context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, storage_modifier); + /// Reinitialize primary key because primary key column types might have changed. setProperties(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index cb3828f5817..5c1924c322c 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3158,7 +3158,12 @@ void StorageReplicatedMergeTree::alter( LOG_DEBUG(log, "ALTER storage_settings_ptr only"); SettingsChanges new_changes; params.applyForSettingsOnly(new_changes); - alterSettings(new_changes, query_context, table_lock_holder); + + changeSettings(new_changes, table_lock_holder); + + IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); + global_context.getDatabase(current_database_name)->alterTable( + query_context, current_table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); return; } @@ -3231,6 +3236,18 @@ void StorageReplicatedMergeTree::alter( if (new_metadata_str != ReplicatedMergeTreeTableMetadata(*this).toString()) changed_nodes.emplace_back(zookeeper_path, "metadata", new_metadata_str); + /// Perform settings update locally + if (!new_changes.empty()) + { + IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); + + changeSettings(new_changes, table_lock_holder); + + global_context.getDatabase(current_database_name)->alterTable( + query_context, current_table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); + + } + /// Modify shared metadata nodes in ZooKeeper. Coordination::Requests ops; for (const auto & node : changed_nodes) diff --git a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.reference b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.reference index c7f912ddc79..ee3818d25dc 100644 --- a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.reference +++ b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.reference @@ -3,3 +3,4 @@ CREATE TABLE default.table_for_alter (`id` UInt64, `Data` String) ENGINE = Merge CREATE TABLE default.table_for_alter (`id` UInt64, `Data` String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 4096, parts_to_throw_insert = 100, parts_to_delay_insert = 100 2 CREATE TABLE default.table_for_alter (`id` UInt64, `Data` String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 4096, parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 30 +CREATE TABLE default.table_for_alter (`id` UInt64, `Data` String, `Data2` UInt64) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 4096, parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 15 diff --git a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql index 43838b8a727..ed42a79ebbf 100644 --- a/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql +++ b/dbms/tests/queries/0_stateless/00980_merge_alter_settings.sql @@ -47,5 +47,9 @@ ALTER TABLE table_for_alter MODIFY SETTING check_delay_period=10, check_delay_pe SHOW CREATE TABLE table_for_alter; +ALTER TABLE table_for_alter ADD COLUMN Data2 UInt64, MODIFY SETTING check_delay_period=5, check_delay_period=10, check_delay_period=15; + +SHOW CREATE TABLE table_for_alter; + DROP TABLE IF EXISTS table_for_alter; diff --git a/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.reference b/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.reference index e55bfadd538..159102e1ca7 100644 --- a/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.reference +++ b/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.reference @@ -8,3 +8,5 @@ CREATE TABLE default.replicated_table_for_alter1 (`id` UInt64, `Data` String) EN 6 CREATE TABLE default.replicated_table_for_alter1 (`id` UInt64, `Data` String) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/replicated_table_for_alter\', \'1\') ORDER BY id SETTINGS index_granularity = 8192, use_minimalistic_part_header_in_zookeeper = 1 CREATE TABLE default.replicated_table_for_alter2 (`id` UInt64, `Data` String) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/replicated_table_for_alter\', \'2\') ORDER BY id SETTINGS index_granularity = 8192, parts_to_throw_insert = 1, parts_to_delay_insert = 1 +CREATE TABLE default.replicated_table_for_alter1 (`id` UInt64, `Data` String, `Data2` UInt64) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/replicated_table_for_alter\', \'1\') ORDER BY id SETTINGS index_granularity = 8192, use_minimalistic_part_header_in_zookeeper = 1, check_delay_period = 15 +CREATE TABLE default.replicated_table_for_alter2 (`id` UInt64, `Data` String, `Data2` UInt64) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/replicated_table_for_alter\', \'2\') ORDER BY id SETTINGS index_granularity = 8192, parts_to_throw_insert = 1, parts_to_delay_insert = 1 diff --git a/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.sql b/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.sql index 792a704b6a1..f2e453c99d2 100644 --- a/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.sql +++ b/dbms/tests/queries/0_stateless/00980_zookeeper_merge_tree_alter_settings.sql @@ -58,5 +58,10 @@ ATTACH TABLE replicated_table_for_alter1; SHOW CREATE TABLE replicated_table_for_alter1; SHOW CREATE TABLE replicated_table_for_alter2; +ALTER TABLE replicated_table_for_alter1 ADD COLUMN Data2 UInt64, MODIFY SETTING check_delay_period=5, check_delay_period=10, check_delay_period=15; + +SHOW CREATE TABLE replicated_table_for_alter1; +SHOW CREATE TABLE replicated_table_for_alter2; + DROP TABLE IF EXISTS replicated_table_for_alter2; DROP TABLE IF EXISTS replicated_table_for_alter1; From 36472b14f96762a997205d3653adf23ac7ce4353 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 27 Aug 2019 12:48:20 +0300 Subject: [PATCH 455/509] Defend kafka storage from alters --- dbms/src/Storages/IStorage.h | 2 +- dbms/src/Storages/Kafka/StorageKafka.cpp | 5 +++++ dbms/src/Storages/Kafka/StorageKafka.h | 2 ++ 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 35c1439964a..6c23a638ddf 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -149,7 +149,7 @@ protected: /// still thread-unsafe part. virtual bool isVirtualColumn(const String & column_name) const; /// Returns modifier of settings in storage definition - IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const; + virtual IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const; private: ColumnsDescription columns; /// combined real and virtual columns diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index d53469259ab..835ce43b1a4 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -412,6 +412,11 @@ bool StorageKafka::hasSetting(const String & setting_name) const return KafkaSettings::findIndex(setting_name) != KafkaSettings::npos; } +IDatabase::ASTModifier StorageKafka::getSettingsModifier(const SettingsChanges & /* new_changes */) const +{ + throw Exception("Storage '" + getName() + "' doesn't support settings alter", ErrorCodes::UNSUPPORTED_METHOD); +} + void registerStorageKafka(StorageFactory & factory) { factory.registerStorage("Kafka", [](const StorageFactory::Arguments & args) diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index c2ff3562116..b1eac57dca1 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -59,6 +59,7 @@ public: bool hasSetting(const String & setting_name) const override; + protected: StorageKafka( const std::string & table_name_, @@ -70,6 +71,7 @@ protected: size_t num_consumers_, UInt64 max_block_size_, size_t skip_broken, bool intermediate_commit_); + IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const override; private: // Configuration and state String table_name; From 9c0b3778687d8b9bef5bcccadd2b086bd3451909 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 27 Aug 2019 13:24:42 +0300 Subject: [PATCH 456/509] DOCAPI-7431: Queries with parameters docs. EN review. RU translation. (#6632) * Update cli.md (#36) * Update http.md (#37) * DOCAPI-7431: RU Translation. * DOCAPI-7431: Fix. * Update docs/en/interfaces/cli.md Co-Authored-By: Ivan Blinkov * DOCAPI-7431: Fixes. --- docs/en/interfaces/cli.md | 12 ++++--- docs/en/interfaces/http.md | 2 +- docs/ru/interfaces/cli.md | 71 ++++++++++++++++++++++++++------------ docs/ru/interfaces/http.md | 10 ++++++ 4 files changed, 66 insertions(+), 29 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 9f9448f27c8..b6e59c4aa50 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -67,22 +67,22 @@ The command-line client allows passing external data (external temporary tables) ### Queries with Parameters {#cli-queries-with-parameters} -You can create a query with parameters, and pass values for these parameters with the parameters of the client app. For example: +You can create a query with parameters and pass values to them from client application. This allows to avoid formatting query with specific dynamic values on client side. For example: ```bash clickhouse-client --param_parName="[1, 2]" -q "SELECT * FROM table WHERE a = {parName:Array(UInt16)}" ``` -#### Syntax of a Query {#cli-queries-with-parameters-syntax} +#### Query Syntax {#cli-queries-with-parameters-syntax} -Format a query by the standard method. Values that you want to put into the query from the app parameters place in braces and format as follows: +Format a query as usual, then place the values that you want to pass from the app parameters to the query in braces in the following format: ``` {:} ``` -- `name` — Identifier of a placeholder that should be used in app parameter as `--param_name = value`. -- `data type` — A data type of app parameter value. For example, data structure like `(integer, ('string', integer))` can have a data type `Tuple(UInt8, Tuple(String, UInt8))` (also you can use another [integer](../data_types/int_uint.md) types). +- `name` — Placeholder identifier. In the console client it should be used in app parameters as `--param_ = value`. +- `data type` — [Data type](../data_types/index.md) of the app parameter value. For example, a data structure like `(integer, ('string', integer))` can have the `Tuple(UInt8, Tuple(String, UInt8))` data type (you can also use another [integer](../data_types/int_uint.md) types). #### Example @@ -118,6 +118,8 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--stacktrace` – If specified, also print the stack trace if an exception occurs. - `--config-file` – The name of the configuration file. - `--secure` – If specified, will connect to server over secure connection. +- `--param_` — Value for a [query with parameters](#cli-queries-with-parameters). + ### Configuration Files diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index ef32b101b71..80cf72ec0e2 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -246,7 +246,7 @@ Use buffering to avoid situations where a query processing error occurred after ### Queries with Parameters {#cli-queries-with-parameters} -You can create a query with parameters, and pass values for these parameters with the parameters of the HTTP request. For more information, see [CLI Formatted Queries](cli.md#cli-queries-with-parameters). +You can create a query with parameters and pass values for them from the corresponding HTTP request parameters. For more information, see [Queries with Parameters for CLI](cli.md#cli-queries-with-parameters). ### Example diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index 0f3ce84345e..59980109240 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -54,8 +54,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA По умолчанию, в качестве формата, используется формат PrettyCompact (красивые таблички). Вы можете изменить формат с помощью секции FORMAT запроса, или с помощью указания `\G` на конце запроса, с помощью аргумента командной строки `--format` или `--vertical`, или с помощью конфигурационного файла клиента. -Чтобы выйти из клиента, нажмите Ctrl+D (или Ctrl+C), или наберите вместо запроса одно из: -"exit", "quit", "logout", "учше", "йгше", "дщпщге", "exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж", "q", "й", "q", "Q", ":q", "й", "Й", "Жй" +Чтобы выйти из клиента, нажмите Ctrl+D (или Ctrl+C), или наберите вместо запроса одно из: "exit", "quit", "logout", "учше", "йгше", "дщпщге", "exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж", "q", "й", "q", "Q", ":q", "й", "Й", "Жй" При выполнении запроса, клиент показывает: @@ -68,38 +67,64 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA Клиент командной строки позволяет передать внешние данные (внешние временные таблицы) для использования запроса. Подробнее смотрите раздел "Внешние данные для обработки запроса" +### Запросы с параметрами {#cli-queries-with-parameters} + +Вы можете создать запрос с параметрами и передавать в них значения из приложения. Это позволяет избежать форматирования запросов на стороне клиента, если известно, какие из параметров запроса динамически меняются. Например: + +```bash +clickhouse-client --param_parName="[1, 2]" -q "SELECT * FROM table WHERE a = {parName:Array(UInt16)}" +``` + +#### Cинтаксис запроса {#cli-queries-with-parameters-syntax} + +Отформатируйте запрос обычным способом. Представьте значения, которые вы хотите передать из параметров приложения в запрос в следующем формате: + +``` +{:} +``` + +- `name` — идентификатор подстановки. В консольном клиенте его следует использовать как часть имени параметра `--param_ = value`. +- `data type` — [тип данных](../data_types/index.md) значения. Например, структура данных `(integer, ('string', integer))` может иметь тип данных `Tuple(UInt8, Tuple(String, UInt8))` ([целочисленный](../data_types/int_uint.md) тип может быть и другим). + +#### Пример + +```bash +clickhouse-client --param_tuple_in_tuple="(10, ('dt', 10))" -q "SELECT * FROM table WHERE val = {tuple_in_tuple:Tuple(UInt8, Tuple(String, UInt8))}" +``` + ## Конфигурирование {#interfaces_cli_configuration} В `clickhouse-client` можно передавать различные параметры (все параметры имеют значения по умолчанию) с помощью: - - Командной строки. - - Параметры командной строки переопределяют значения по умолчанию и параметры конфигурационных файлов. - - - Конфигурационных файлов. - +- Командной строки. + + Параметры командной строки переопределяют значения по умолчанию и параметры конфигурационных файлов. + +- Конфигурационных файлов. + Параметры в конфигурационных файлах переопределяют значения по умолчанию. ### Параметры командной строки -- `--host, -h` - имя сервера, по умолчанию - localhost. Вы можете использовать как имя, так и IPv4 или IPv6 адрес. -- `--port` - порт, к которому соединяться, по умолчанию - 9000. Замечу, что для HTTP и родного интерфейса используются разные порты. -- `--user, -u` - имя пользователя, по умолчанию - default. -- `--password` - пароль, по умолчанию - пустая строка. -- `--query, -q` - запрос для выполнения, при использовании в неинтерактивном режиме. -- `--database, -d` - выбрать текущую БД, по умолчанию - текущая БД из настроек сервера (по умолчанию - БД default). -- `--multiline, -m` - если указано - разрешить многострочные запросы, не отправлять запрос по нажатию Enter. -- `--multiquery, -n` - если указано - разрешить выполнять несколько запросов, разделённых точкой с запятой. Работает только в неинтерактивном режиме. -- `--format, -f` - использовать указанный формат по умолчанию для вывода результата. -- `--vertical, -E` - если указано, использовать формат Vertical по умолчанию для вывода результата. То же самое, что --format=Vertical. В этом формате каждое значение выводится на отдельной строке, что удобно для отображения широких таблиц. -- `--time, -t` - если указано, в неинтерактивном режиме вывести время выполнения запроса в stderr. -- `--stacktrace` - если указано, в случае исключения, выводить также его стек трейс. -- `--config-file` - имя конфигурационного файла. -- `--secure` - если указано, будет использован безопасный канал. +- `--host, -h` — имя сервера, по умолчанию — localhost. Вы можете использовать как имя, так и IPv4 или IPv6 адрес. +- `--port` — порт, к которому соединяться, по умолчанию — 9000. Замечу, что для HTTP и родного интерфейса используются разные порты. +- `--user, -u` — имя пользователя, по умолчанию — default. +- `--password` — пароль, по умолчанию — пустая строка. +- `--query, -q` — запрос для выполнения, при использовании в неинтерактивном режиме. +- `--database, -d` — выбрать текущую БД, по умолчанию — текущая БД из настроек сервера (по умолчанию — БД default). +- `--multiline, -m` — если указано — разрешить многострочные запросы, не отправлять запрос по нажатию Enter. +- `--multiquery, -n` — если указано — разрешить выполнять несколько запросов, разделённых точкой с запятой. Работает только в неинтерактивном режиме. +- `--format, -f` — использовать указанный формат по умолчанию для вывода результата. +- `--vertical, -E` — если указано, использовать формат Vertical по умолчанию для вывода результата. То же самое, что --format=Vertical. В этом формате каждое значение выводится на отдельной строке, что удобно для отображения широких таблиц. +- `--time, -t` — если указано, в неинтерактивном режиме вывести время выполнения запроса в stderr. +- `--stacktrace` — если указано, в случае исключения, выводить также его стек трейс. +- `--config-file` — имя конфигурационного файла. +- `--secure` — если указано, будет использован безопасный канал. +- `--param_` — значение параметра для [запроса с параметрами](#cli-queries-with-parameters). ### Конфигурационные файлы -`clickhouse-client` использует первый существующий файл из: +`clickhouse—client` использует первый существующий файл из: - Определенного параметром `--config-file`. - `./clickhouse-client.xml` diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index 20067942dd8..77eb984e8f4 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -245,5 +245,15 @@ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wa Буферизация позволяет избежать ситуации когда код ответа и HTTP-заголовки были отправлены клиенту, после чего возникла ошибка выполнения запроса. В такой ситуации сообщение об ошибке записывается в конце тела ответа, и на стороне клиента ошибка может быть обнаружена только на этапе парсинга. +### Запросы с параметрами {#cli-queries-with-parameters} + +Можно создать запрос с параметрами и передать для них значения из соответствующих параметров HTTP-запроса. Дополнительную информацию смотрите в [Запросы с параметрами для консольного клиента](cli.md#cli-queries-with-parameters). + +### Пример + +```bash +curl -sS "
?param_id=2¶m_phrase=test" -d "SELECT * FROM table WHERE int_column = {id:UInt8} and string_column = {phrase:String}" +``` + [Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/http_interface/) From cbc78ffc43a08e94c9a07a62f87da8f01d46db33 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 27 Aug 2019 14:04:52 +0300 Subject: [PATCH 457/509] DOCAPI-7443: Virtual columns docs update. EN review. RU translation. (#6640) * Update index.md (#35) * DOCAPI-7443: RU translation. * Link fix. * DOCAPI-7443: Fixes. * DOCAPI-7443: Fixes. --- docs/en/operations/table_engines/index.md | 30 +++++------ docs/en/query_language/misc.md | 2 +- docs/ru/operations/table_engines/index.md | 65 ++++++++++++++++++++++- docs/ru/operations/table_engines/kafka.md | 13 +++++ docs/ru/operations/table_engines/merge.md | 14 ++--- docs/ru/query_language/misc.md | 2 +- 6 files changed, 96 insertions(+), 30 deletions(-) diff --git a/docs/en/operations/table_engines/index.md b/docs/en/operations/table_engines/index.md index 41680a5b3af..ce8cf5b4678 100644 --- a/docs/en/operations/table_engines/index.md +++ b/docs/en/operations/table_engines/index.md @@ -2,7 +2,7 @@ The table engine (type of table) determines: -- How and where data is stored, where to write it to, and from where to read it. +- How and where data is stored, where to write it to, and where to read it from. - Which queries are supported, and how. - Concurrent data access. - Use of indexes, if present. @@ -11,13 +11,13 @@ The table engine (type of table) determines: ## Engine Families -### *MergeTree +### MergeTree -The most universal and functional table engines for high-load tasks. The common property of these engines is quick data insertion with subsequent data processing in the background. The `*MergeTree` engines support data replication (with [Replicated*](replication.md) versions of engines), partitioning and other features not supported in other engines. +The most universal and functional table engines for high-load tasks. The property shared by these engines is quick data insertion with subsequent background data processing. `MergeTree` family engines support data replication (with [Replicated*](replication.md) versions of engines), partitioning, and other features not supported in other engines. -Engines of the family: +Engines in the family: -- [MergTree](mergetree.md) +- [MergeTree](mergetree.md) - [ReplacingMergeTree](replacingmergetree.md) - [SummingMergeTree](summingmergetree.md) - [AggregatingMergeTree](aggregatingmergetree.md) @@ -25,11 +25,11 @@ Engines of the family: - [VersionedCollapsingMergeTree](versionedcollapsingmergetree.md) - [GraphiteMergeTree](graphitemergetree.md) -### *Log +### Log -Lightweight [engines](log_family.md) with minimum functionality. They are the most effective in scenarios when you need to quickly write many small tables (up to about 1 million rows) and read them later as a whole. +Lightweight [engines](log_family.md) with minimum functionality. They're the most effective when you need to quickly write many small tables (up to approximately 1 million rows) and read them later as a whole. -Engines of the family: +Engines in the family: - [TinyLog](tinylog.md) - [StripeLog](stripelog.md) @@ -39,7 +39,7 @@ Engines of the family: Engines for communicating with other data storage and processing systems. -Engines of the family: +Engines in the family: - [Kafka](kafka.md) - [MySQL](mysql.md) @@ -48,9 +48,7 @@ Engines of the family: ### Special engines -Engines solving special tasks. - -Engines of the family: +Engines in the family: - [Distributed](distributed.md) - [MaterializedView](materializedview.md) @@ -67,12 +65,12 @@ Engines of the family: ## Virtual columns {#table_engines-virtual_columns} -Virtual column is an integral attribute of a table engine that is defined in the source code of the engine. +Virtual column is an integral table engine attribute that is defined in the engine source code. -You should not specify virtual columns in the `CREATE TABLE` query, and you cannot see them in the results of `SHOW CREATE TABLE` and `DESCRIBE TABLE` queries. Also, virtual columns are read-only, so you can't insert data into virtual columns. +You shouldn't specify virtual columns in the `CREATE TABLE` query and you can't see them in `SHOW CREATE TABLE` and `DESCRIBE TABLE` query results. Virtual columns are also read-only, so you can't insert data into virtual columns. -To select data from a virtual column, you must specify its name in the `SELECT` query. The `SELECT *` doesn't return values from virtual columns. +To select data from a virtual column, you must specify its name in the `SELECT` query. `SELECT *` doesn't return values from virtual columns. -If you create a table with a column that has the same name as one of the table virtual columns, the virtual column becomes inaccessible. Doing so is not recommended. To help avoiding conflicts virtual column names are usually prefixed with an underscore. +If you create a table with a column that has the same name as one of the table virtual columns, the virtual column becomes inaccessible. We don't recommend doing this. To help avoid conflicts, virtual column names are usually prefixed with an underscore. [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/) diff --git a/docs/en/query_language/misc.md b/docs/en/query_language/misc.md index 514f5d9f823..08e8f819b8c 100644 --- a/docs/en/query_language/misc.md +++ b/docs/en/query_language/misc.md @@ -201,7 +201,7 @@ All tables are renamed under global locking. Renaming tables is a light operatio SET param = value ``` -Assigns `value` to the `param` configurations settings for the current session. You cannot change [server settings](../operations/server_settings/index.md) this way. +Assigns `value` to the `param` [setting](../operations/settings/index.md) for the current session. You cannot change [server settings](../operations/server_settings/index.md) this way. You can also set all the values from the specified settings profile in a single query. diff --git a/docs/ru/operations/table_engines/index.md b/docs/ru/operations/table_engines/index.md index cf38d90b55f..ce414a3c0d1 100644 --- a/docs/ru/operations/table_engines/index.md +++ b/docs/ru/operations/table_engines/index.md @@ -9,8 +9,69 @@ - Возможно ли многопоточное выполнение запроса. - Параметры репликации данных. -При чтении, движок обязан лишь выдать запрошенные столбцы, но в некоторых случаях движок может частично обрабатывать данные при ответе на запрос. +## Семейства движков -Для большинства серьёзных задач, следует использовать движки семейства `MergeTree`. +### MergeTree + +Наиболее универсальные и функциональные движки таблиц для задач с высокой загрузкой. Общим свойством этих движков является быстрая вставка данных с последующей фоновой обработкой данных. Движки `*MergeTree` поддерживают репликацию данных (в [Replicated*](replication.md) версиях движков), партиционирование, и другие возможности не поддержанные для других движков. + +Движки семейства: + +- [MergeTree](mergetree.md) +- [ReplacingMergeTree](replacingmergetree.md) +- [SummingMergeTree](summingmergetree.md) +- [AggregatingMergeTree](aggregatingmergetree.md) +- [CollapsingMergeTree](collapsingmergetree.md) +- [VersionedCollapsingMergeTree](versionedcollapsingmergetree.md) +- [GraphiteMergeTree](graphitemergetree.md) + +### Log + +Простые [движки](log_family.md) с минимальной функциональностью. Они наиболее эффективны, когда вам нужно быстро записать много небольших таблиц (до примерно 1 миллиона строк) и прочитать их позже целиком. + +Движки семейства: + +- [TinyLog](tinylog.md) +- [StripeLog](stripelog.md) +- [Log](log.md) + +### Движки для интергации + +Движки для связи с другими системами хранения и обработки данных. + +Движки семейства: + +- [Kafka](kafka.md) +- [MySQL](mysql.md) +- [ODBC](odbc.md) +- [JDBC](jdbc.md) + +### Специальные движки + +Движки семейства: + +- [Distributed](distributed.md) +- [MaterializedView](materializedview.md) +- [Dictionary](dictionary.md) +- [Merge](merge.md) +- [File](file.md) +- [Null](null.md) +- [Set](set.md) +- [Join](join.md) +- [URL](url.md) +- [View](view.md) +- [Memory](memory.md) +- [Buffer](buffer.md) + +## Виртуальные столбцы {#table_engines-virtual_columns} + +Виртуальный столбец — это неотъемлемый атрибут движка таблиц, определенный в исходном коде движка. + +Виртуальные столбцы не надо указывать в запросе `CREATE TABLE` и их не отображаются в результатах запросов `SHOW CREATE TABLE` и `DESCRIBE TABLE`. Также виртуальные столбцы доступны только для чтения, поэтому вы не можете вставлять в них данные. + +Чтобы получить данные из виртуального столбца, необходимо указать его название в запросе `SELECT`. `SELECT *` не отображает данные из виртуальных столбцов. + +При создании таблицы со столбцом, имя которого совпадает с именем одного из виртуальных столбцов таблицы, виртуальный столбец становится недоступным. Не делайте так. Чтобы помочь избежать конфликтов, имена виртуальных столбцов обычно предваряются подчеркиванием. [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/) + diff --git a/docs/ru/operations/table_engines/kafka.md b/docs/ru/operations/table_engines/kafka.md index 086d4fb4f08..f2318d824e2 100644 --- a/docs/ru/operations/table_engines/kafka.md +++ b/docs/ru/operations/table_engines/kafka.md @@ -152,4 +152,17 @@ Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format В документе [librdkafka configuration reference](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md) можно увидеть список возможных опций конфигурации. Используйте подчеркивание (`_`) вместо точки в конфигурации ClickHouse. Например, `check.crcs=true` будет соответствовать `true`. +## Виртуальные столбцы + +- `_topic` — топик Kafka. +- `_key` — ключ сообщения. +- `_offset` — оффсет сообщения. +- `_timestamp` — временная метка сообщения. +- `_partition` — секция топика Kafka. + +**Смотрите также** + +- [Виртуальные столбцы](index.md#table_engines-virtual_columns) + [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/kafka/) + diff --git a/docs/ru/operations/table_engines/merge.md b/docs/ru/operations/table_engines/merge.md index 28065e5d120..b87bfdafd75 100644 --- a/docs/ru/operations/table_engines/merge.md +++ b/docs/ru/operations/table_engines/merge.md @@ -52,18 +52,12 @@ FROM WatchLog ## Виртуальные столбцы -Виртуальные столбцы — столбцы, предоставляемые движком таблиц независимо от определения таблицы. То есть, такие столбцы не указываются в `CREATE TABLE`, но доступны для `SELECT`. +- `_table` — содержит имя таблицы, из которой данные были прочитаны. Тип — [String](../../data_types/string.md). -Виртуальные столбцы отличаются от обычных следующими особенностями: + В секции `WHERE/PREWHERE` можно установить константное условие на столбец `_table` (например, `WHERE _table='xyz'`). В этом случае операции чтения выполняются только для тех таблиц, для которых выполняется условие на значение `_table`, таким образом, столбец `_table` работает как индекс. -- они не указываются в определении таблицы; -- в них нельзя вставить данные при `INSERT`; -- при `INSERT` без указания списка столбцов виртуальные столбцы не учитываются; -- они не выбираются при использовании звёздочки (`SELECT *`); -- виртуальные столбцы не показываются в запросах `SHOW CREATE TABLE` и `DESC TABLE`; +**Смотрите также** -Таблица типа `Merge` содержит виртуальный столбец `_table` типа `String`. (Если в таблице уже есть столбец `_table`, то виртуальный столбец называется `_table1`; если уже есть `_table1`, то `_table2` и т. п.) Он содержит имя таблицы, из которой были прочитаны данные. - -Если секция `WHERE/PREWHERE` содержит (в качестве одного из элементов конъюнкции или в качестве всего выражения) условия на столбец `_table`, не зависящие от других столбцов таблицы, то эти условия используются как индекс: условия выполняются над множеством имён таблиц, из которых нужно читать данные, и чтение будет производиться только из тех таблиц, для которых условия сработали. +- [Виртуальные столбцы](index.md#table_engines-virtual_columns) [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/merge/) diff --git a/docs/ru/query_language/misc.md b/docs/ru/query_language/misc.md index e366abf2f3d..ab19e559649 100644 --- a/docs/ru/query_language/misc.md +++ b/docs/ru/query_language/misc.md @@ -199,7 +199,7 @@ RENAME TABLE [db11.]name11 TO [db12.]name12, [db21.]name21 TO [db22.]name22, ... SET param = value ``` -Устанавливает значение `value` для настройки `param` в текущей сессии. [Конфигурационные параметры сервера](../operations/server_settings/index.md) нельзя изменить подобным образом. +Устанавливает значение `value` для [настройки](../operations/settings/index.md) `param` в текущей сессии. [Конфигурационные параметры сервера](../operations/server_settings/index.md) нельзя изменить подобным образом. Можно одним запросом установить все настройки из заданного профиля настроек. From f1f85a5a2452d2d566972054a590593fef68cbb2 Mon Sep 17 00:00:00 2001 From: Dmitry Rubashkin Date: Tue, 27 Aug 2019 16:09:58 +0300 Subject: [PATCH 458/509] Fixed case with single subquery and added tests. --- .../gtest_transform_query_for_external_database.cpp | 11 +++++++++++ .../Storages/transformQueryForExternalDatabase.cpp | 2 ++ 2 files changed, 13 insertions(+) diff --git a/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp index e61ab6279a2..b07605cb7ee 100644 --- a/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -76,3 +76,14 @@ TEST(TransformQueryForExternalDatabase, Substring) "SELECT \"column\" FROM \"test\".\"table\"", state().context, state().columns); } + +TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries) +{ + check("SELECT column FROM test.table WHERE 1 = 1 AND toString(column) = '42' AND column = 42 AND left(column, 10) = RIGHT(column, 10) AND column IN (1, 42) AND SUBSTRING(column FROM 1 FOR 2) = 'Hello' AND column != 4", + "SELECT \"column\" FROM \"test\".\"table\" WHERE 1 AND (\"column\" = 42) AND (\"column\" IN (1, 42)) AND (\"column\" != 4)", + state().context, state().columns); + check("SELECT column FROM test.table WHERE toString(column) = '42' AND left(column, 10) = RIGHT(column, 10) AND column = 42", + "SELECT \"column\" FROM \"test\".\"table\" WHERE (\"column\" = 42)", + state().context, state().columns); + +} \ No newline at end of file diff --git a/dbms/src/Storages/transformQueryForExternalDatabase.cpp b/dbms/src/Storages/transformQueryForExternalDatabase.cpp index 7cbffc46e67..b6e48836efa 100644 --- a/dbms/src/Storages/transformQueryForExternalDatabase.cpp +++ b/dbms/src/Storages/transformQueryForExternalDatabase.cpp @@ -150,6 +150,8 @@ String transformQueryForExternalDatabase( compatible_found = true; } } + if (new_function_and->arguments->children.size() == 1) + new_function_and->name = ""; if (compatible_found) select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(new_function_and)); From 17f18e42c0f03c00746875285142bcfd140a3829 Mon Sep 17 00:00:00 2001 From: Dmitry Rubashkin Date: Tue, 27 Aug 2019 16:13:40 +0300 Subject: [PATCH 459/509] new line --- .../tests/gtest_transform_query_for_external_database.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp index b07605cb7ee..67cd21b3cfb 100644 --- a/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -86,4 +86,4 @@ TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries) "SELECT \"column\" FROM \"test\".\"table\" WHERE (\"column\" = 42)", state().context, state().columns); -} \ No newline at end of file +} From b0769330092c3c047c7ab9a704b14d347c821c88 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 27 Aug 2019 16:49:56 +0300 Subject: [PATCH 460/509] DOCAPI-7400: Gorilla codecs. EN review. RU translation. (#6598) * Update create.md * DOCAPI-7400: RU translation. * DOCAPI-7400: Fixes. * Update docs/ru/query_language/create.md Co-Authored-By: Ivan Blinkov * Update docs/ru/query_language/create.md Co-Authored-By: Ivan Blinkov * DOCAPI-7400: Clarifications. * DOCAPI-7400: Link fix. --- .../agg_functions/parametric_functions.md | 2 +- docs/en/query_language/create.md | 21 +++-- docs/ru/query_language/create.md | 35 ++++---- docs/zh/query_language/create.md | 90 ++++++++++++++++++- .../functions/hash_functions.md | 2 +- .../functions/other_functions.md | 2 +- 6 files changed, 128 insertions(+), 24 deletions(-) diff --git a/docs/en/query_language/agg_functions/parametric_functions.md b/docs/en/query_language/agg_functions/parametric_functions.md index 84898a61133..3ea26de5937 100644 --- a/docs/en/query_language/agg_functions/parametric_functions.md +++ b/docs/en/query_language/agg_functions/parametric_functions.md @@ -45,7 +45,7 @@ FROM ( └─────────────────────────────────────────────────────────────────────────┘ ``` -You can visualize a histogram with the [bar](../other_functions.md#function-bar) function, for example: +You can visualize a histogram with the [bar](../functions/other_functions.md#function-bar) function, for example: ```sql WITH histogram(5)(rand() % 100) AS hist diff --git a/docs/en/query_language/create.md b/docs/en/query_language/create.md index 54db63fdaef..8060dfbe09d 100644 --- a/docs/en/query_language/create.md +++ b/docs/en/query_language/create.md @@ -164,19 +164,30 @@ These codecs are designed to make compression more effective using specifities o Specialized codecs: -- `Delta(delta_bytes)` — Compression approach, when raw values are replaced with the difference of two neighbor values. Up to `delta_bytes` are used for storing delta value, so `delta_bytes` is a maximum size of raw values. Possible `delta_bytes` values: 1, 2, 4, 8. Default value for `delta_bytes` is `sizeof(type)`, if it is equals to 1, 2, 4, 8. Otherwise it equals 1. -- `DoubleDelta` — Compresses values down to 1 bit (in the best case), using deltas calculation. Best compression rates are achieved on monotonic sequences with constant stride, for example, time series data. Can be used with any fixed-width type. Implements the algorithm used in Gorilla TSDB, extending it to support 64 bit types. Uses 1 extra bit for 32 byte deltas: 5 bit prefix instead of 4 bit prefix. For additional information, see the "Compressing time stamps" section of the [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf) document. -- `Gorilla` — Compresses values down to 1 bit (in the best case). The codec is efficient when storing series of floating point values that change slowly, because the best compression rate is achieved when neighboring values are binary equal. Implements the algorithm used in Gorilla TSDB, extending it to support 64 bit types. For additional information, see the "Compressing values" section of the [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf) document. +- `Delta(delta_bytes)` — Compression approach in which raw values are replaced by the difference of two neighboring values, except for the first value that stays unchanged. Up to `delta_bytes` are used for storing delta values, so `delta_bytes` is the maximum size of raw values. Possible `delta_bytes` values: 1, 2, 4, 8. The default value for `delta_bytes` is `sizeof(type)` if equal to 1, 2, 4, or 8. In all other cases, it's 1. +- `DoubleDelta` — Calculates delta of deltas and writes it in compact binary form. Optimal compression rates are achieved for monotonic sequences with a constant stride, such as time series data. Can be used with any fixed-width type. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. Uses 1 extra bit for 32-byte deltas: 5-bit prefixes instead of 4-bit prefixes. For additional information, see Compressing Time Stamps in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). +- `Gorilla` — Calculates XOR between current and previous value and writes it in compact binary form. Efficient when storing a series of floating point values that change slowly, because the best compression rate is achieved when neighboring values are binary equal. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. For additional information, see Compressing Values in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). - `T64` — Compression approach that crops unused high bits of values in integer data types (including `Enum`, `Date` and `DateTime`). At each step of its algorithm, codec takes a block of 64 values, puts them into 64x64 bit matrix, transposes it, crops the unused bits of values and returns the rest as a sequence. Unused bits are the bits, that don't differ between maximum and minimum values in the whole data part for which the compression is used. +`DoubleDelta` and `Gorilla` codecs are used in Gorilla TSDB as the components of its compressing algorithm. Gorilla approach is effective in scenarios when there is a sequence of slowly changing values with their timestamps. Timestamps are effectively compressed by the `DoubleDelta` codec, and values are effectively compressed by the `Gorilla` codec. For example, to get an effectively stored table, you can create it in the following configuration: + +```sql +CREATE TABLE codec_example +( + timestamp DateTime CODEC(DoubleDelta), + slow_values Float32 CODEC(Gorilla) +) +ENGINE = MergeTree() +``` + ### Common purpose codecs {#create-query-common-purpose-codecs} Codecs: - `NONE` — No compression. - `LZ4` — Lossless [data compression algorithm](https://github.com/lz4/lz4) used by default. Applies LZ4 fast compression. -- `LZ4HC[(level)]` — LZ4 CH (high compression) algorithm with configurable level. Default level: 9. If you set `level <= 0`, the default level is applied. Possible levels: [1, 12]. Recommended levels are in range: [4, 9]. -- `ZSTD[(level)]` — [ZSTD compression algorithm](https://en.wikipedia.org/wiki/Zstandard) with configurable `level`. Possible levels: [1, 22]. Default level: 1. +- `LZ4HC[(level)]` — LZ4 HC (high compression) algorithm with configurable level. Default level: 9. Setting `level <= 0` applies the default level. Possible levels: [1, 12]. Recommended level range: [4, 9]. +- `ZSTD[(level)]` — [ZSTD compression algorithm](https://en.wikipedia.org/wiki/Zstandard) with configurable `level`. Possible levels: [1, 22]. Default value: 1. High compression levels useful for asymmetric scenarios, like compress once, decompress a lot of times. Greater levels stands for better compression and higher CPU usage. diff --git a/docs/ru/query_language/create.md b/docs/ru/query_language/create.md index 2e0a21cda0b..d32afb7b9d9 100644 --- a/docs/ru/query_language/create.md +++ b/docs/ru/query_language/create.md @@ -127,24 +127,28 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] Определяет время хранения значений. Может быть указано только для таблиц семейства MergeTree. Подробнее смотрите в [TTL для столбцов и таблиц](../operations/table_engines/mergetree.md#table_engine-mergetree-ttl). -## Форматы сжатия для колонок +## Кодеки сжатия столбцов -Помимо сжатия для колонок по умолчанию, определяемого в [настройках сервера](../operations/server_settings/settings.md#compression), -существует возможность указать формат сжатия индивидуально для каждой колонки. +Помимо сжатия данных по умолчанию, определяемого [конфигурационными параметрами сервера](../operations/server_settings/settings.md#compression), можно задать сжатие для каждого отдельного столбца. -Поддерживаемые форматы: +Поддерживаемые алгоритмы сжатия: -- `NONE` - сжатие отсутствует -- `LZ4` -- `LZ4HC(level)` - алгоритм сжатия LZ4\_HC с указанным уровнем компрессии `level`. -Возможный диапазон значений `level`: \[3, 12\]. Значение по умолчанию: 9. Чем выше уровень, тем лучше сжатие, но тратится больше времени. Рекомендованный диапазон \[4, 9\]. -- `ZSTD(level)` - алгоритм сжатия ZSTD с указанным уровнем компрессии `level`. Возможный диапазон значений `level`: \[1, 22\]. Значение по умолчанию: 1. -Чем выше уровень, тем лучше сжатие, но тратится больше времени. -- `Delta(delta_bytes)` - способ сжатия, при котором вместо числовых значений поля сохраняется разность между двумя соседними значениями. Значение `delta_bytes` - число байт для хранения дельты. -Возможные значения: 1, 2, 4, 8. Значение по умолчанию: если `sizeof(type)` равен 1, 2, 4, 8 - `sizeof(type)`, иначе - 1. +- `NONE` — без сжатия. +- `LZ4` — [алгоритм сжатия данных](https://github.com/lz4/lz4) без потерь, используемый по умолчанию. Применяет быстрое сжатие LZ4. +- `LZ4HC[(level)]` — алгоритм сильного сжатия LZ4 HC с настраиваемым уровнем. Уровень по умолчанию — 9. Настройка `level <= 0` устанавливает уровень по умолчанию. Возможные уровни: [1, 12]. Рекомендуемый диапазон уровней: [4, 9]. +- `ZSTD[(level)]` — [Алгоритм сжатия ZSTD](https://en.wikipedia.org/wiki/Zstandard) с настаиваемым уровнем `level`. Возможные уровни: [1, 22]. Значение по умолчанию — 1. +- `Delta(delta_bytes)` — способ сжатия, при котором исходные значения заменяются разностью двух соседних значений. Для хранение разностей используется до `delta_bytes` байтов, т.е. `delta_bytes` — это максимальный размер исходных значений. Возможные значения `delta_bytes` — 1, 2, 4, 8. Значение `delta_bytes` по умолчанию равно `sizeof(type)`, если вычисленный размер равен 1, 2, 4 или 8. Во всех остальных случаях — 1. +- `DoubleDelta` — Сжимает значения вплоть до размера в 1 бит благодаря сохранению разностей. Оптимальные уровни сжатия достигаются для монотонных последовательностей с постоянным шагом, например, временные ряды. Может использоваться с любым типом данных фиксированного размера. Реализует алгоритм, используемый в Gorilla TSDB, расширяя его для поддержки 64-битных типов. Использует 1 дополнительный бит для 32-байтовых значений: 5-битные префиксы вместо 4-битных префиксов. Подробнее смотрите в разделе "Compressing Time Stamps" в [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). +- `Gorilla` — Сжимает значения вплоть до размера в 1 bit. Эффективен при хранении рядов медленно изменяющихся чисел с плавающей запятой, потому, что лучшее сжатие достигается, когда соседние значения бинарно равны. Реализует алгоритм, используемый в Gorilla TSDB, расширяя его для поддержки 64-битных типов. Подробнее смотрите в разделе "Compressing Values" в [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). + +Высокие уровни сжатия полезны для асимметричных сценариев, например, для таких, в которых требуется однократное сжатие и многократная распаковка. Более высокие уровни обеспечивают лучшее сжатие, но более высокое потребление вычислительных ресурсов. + +!!! warning "Предупреждение" + Базу данных ClickHouse не получится распаковать с помощью внешних утилит типа `lz4`. Используйте специальную программу [clickhouse-compressor](https://github.com/yandex/ClickHouse/tree/master/dbms/programs/compressor). Пример использования: -``` + +```sql CREATE TABLE codec_example ( dt Date CODEC(ZSTD), /* используется уровень сжатия по умолчанию */ @@ -157,9 +161,10 @@ PARTITION BY tuple() ORDER BY dt ``` -Кодеки могут комбинироваться между собой. Если для колонки указана своя последовательность кодеков, то общий табличный кодек не применяется (должен быть указан в последовательности принудительно, если нужен). В примере ниже - оптимизация для хранения timeseries метрик. +Кодеки можно комбинировать. Если для колонки указана своя последовательность кодеков, то общий табличный кодек не применяется (должен быть указан в последовательности принудительно, если нужен). В примере ниже - оптимизация для хранения timeseries метрик. Как правило, значения одной и той же метрики `path` не сильно различаются между собой, и выгоднее использовать дельта-компрессию вместо записи всего числа: -``` + +```sql CREATE TABLE timeseries_example ( dt Date, diff --git a/docs/zh/query_language/create.md b/docs/zh/query_language/create.md index 1b1abef47db..d3a6c2e841b 100644 --- a/docs/zh/query_language/create.md +++ b/docs/zh/query_language/create.md @@ -80,7 +80,95 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ... 不能够为nested类型的列设置默认值。 -### 临时表 +### Constraints {#constraints} + +WARNING: This feature is experimental. Correct work is not guaranteed on non-MergeTree family engines. + +Along with columns descriptions constraints could be defined: + +``sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec] [TTL expr1], + ... + CONSTRAINT constraint_name_1 CHECK boolean_expr_1, + ... +) ENGINE = engine +``` + +`boolean_expr_1` could by any boolean expression. If constraints are defined for the table, each of them will be checked for every row in `INSERT` query. If any constraint is not satisfied — server will raise an exception with constraint name and checking expression. + +Adding large amount of constraints can negatively affect performance of big `INSERT` queries. + +### TTL expression + +Defines storage time for values. Can be specified only for MergeTree-family tables. For the detailed description, see [TTL for columns and tables](../operations/table_engines/mergetree.md#table_engine-mergetree-ttl). + +## Column Compression Codecs + +By default, ClickHouse applies to columns the compression method, defined in [server settings](../operations/server_settings/settings.md#compression). Also, you can define compression method for each individual column in the `CREATE TABLE` query. + +``` +CREATE TABLE codec_example +( + dt Date CODEC(ZSTD), + ts DateTime CODEC(LZ4HC), + float_value Float32 CODEC(NONE), + double_value Float64 CODEC(LZ4HC(9)) + value Float32 CODEC(Delta, ZSTD) +) +ENGINE = +... +``` + +If a codec is specified, the default codec doesn't apply. Codecs can be combined in a pipeline, for example, `CODEC(Delta, ZSTD)`. To select the best codecs combination for you project, pass benchmarks, similar to described in the Altinity [New Encodings to Improve ClickHouse Efficiency](https://www.altinity.com/blog/2019/7/new-encodings-to-improve-clickhouse) article. + +!!!warning + You cannot decompress ClickHouse database files with external utilities, for example, `lz4`. Use the special utility, [clickhouse-compressor](https://github.com/yandex/ClickHouse/tree/master/dbms/programs/compressor). + +Compression is supported for the table engines: + +- [*MergeTree](../operations/table_engines/mergetree.md) family +- [*Log](../operations/table_engines/log_family.md) family +- [Set](../operations/table_engines/set.md) +- [Join](../operations/table_engines/join.md) + +ClickHouse supports common purpose codecs and specialized codecs. + +### Specialized codecs {#create-query-specialized-codecs} + +These codecs are designed to make compression more effective using specifities of the data. Some of this codecs don't compress data by itself, but they prepare data to be compressed better by common purpose codecs. + +Specialized codecs: + +- `Delta(delta_bytes)` — Compression approach in which raw values are replaced by the difference of two neighboring values, except for the first value that stays unchanged. Up to `delta_bytes` are used for storing delta values, so `delta_bytes` is the maximum size of raw values. Possible `delta_bytes` values: 1, 2, 4, 8. The default value for `delta_bytes` is `sizeof(type)` if equal to 1, 2, 4, or 8. In all other cases, it's 1. +- `DoubleDelta` — Calculates delta of deltas and writes it in compact binary form. Optimal compression rates are achieved for monotonic sequences with a constant stride, such as time series data. Can be used with any fixed-width type. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. Uses 1 extra bit for 32-byte deltas: 5-bit prefixes instead of 4-bit prefixes. For additional information, see Compressing Time Stamps in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). +- `Gorilla` — Calculates XOR between current and previous value and writes it in compact binary form. Efficient when storing a series of floating point values that change slowly, because the best compression rate is achieved when neighboring values are binary equal. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. For additional information, see Compressing Values in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). +- `T64` — Compression approach that crops unused high bits of values in integer data types (including `Enum`, `Date` and `DateTime`). At each step of its algorithm, codec takes a block of 64 values, puts them into 64x64 bit matrix, transposes it, crops the unused bits of values and returns the rest as a sequence. Unused bits are the bits, that don't differ between maximum and minimum values in the whole data part for which the compression is used. + +`DoubleDelta` and `Gorilla` codecs are used in Gorilla TSDB as the components of its compressing algorithm. Gorilla approach is effective in scenarios when there is a sequence of slowly changing values with their timestamps. Timestamps are effectively compressed by the `DoubleDelta` codec, and values are effectively compressed by the `Gorilla` codec. For example, to get an effectively stored table, you can create it in the following configuration: + +```sql +CREATE TABLE codec_example +( + timestamp DateTime CODEC(DoubleDelta), + slow_values Float32 CODEC(Gorilla) +) +ENGINE = MergeTree() +``` + +### Common purpose codecs {#create-query-common-purpose-codecs} + +Codecs: + +- `NONE` — No compression. +- `LZ4` — Lossless [data compression algorithm](https://github.com/lz4/lz4) used by default. Applies LZ4 fast compression. +- `LZ4HC[(level)]` — LZ4 HC (high compression) algorithm with configurable level. Default level: 9. Setting `level <= 0` applies the default level. Possible levels: [1, 12]. Recommended level range: [4, 9]. +- `ZSTD[(level)]` — [ZSTD compression algorithm](https://en.wikipedia.org/wiki/Zstandard) with configurable `level`. Possible levels: [1, 22]. Default value: 1. + +High compression levels useful for asymmetric scenarios, like compress once, decompress a lot of times. Greater levels stands for better compression and higher CPU usage. + +## 临时表 ClickHouse支持临时表,其具有以下特征: diff --git a/docs/zh/query_language/functions/hash_functions.md b/docs/zh/query_language/functions/hash_functions.md index 57af83a4a5e..9fc6c79b0b3 100644 --- a/docs/zh/query_language/functions/hash_functions.md +++ b/docs/zh/query_language/functions/hash_functions.md @@ -21,7 +21,7 @@ Hash函数可以用于将元素不可逆的伪随机打乱。 SipHash是一种加密哈希函数。它的处理性能至少比MD5快三倍。 有关详细信息,请参阅链接: -## sipHash128 +## sipHash128 {#hash_functions-siphash128} 计算字符串的SipHash。 接受String类型的参数,返回FixedString(16)。 diff --git a/docs/zh/query_language/functions/other_functions.md b/docs/zh/query_language/functions/other_functions.md index 84fbdaeb3ca..329db222b48 100644 --- a/docs/zh/query_language/functions/other_functions.md +++ b/docs/zh/query_language/functions/other_functions.md @@ -122,7 +122,7 @@ Accepts constant strings: database name, table name, and column name. Returns a The function throws an exception if the table does not exist. For elements in a nested data structure, the function checks for the existence of a column. For the nested data structure itself, the function returns 0. -## bar +## bar {#function-bar} 使用unicode构建图表。 From 8ca00a26f6e2f72a6cae6a94e0c14e9363d71f68 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Aug 2019 17:31:34 +0300 Subject: [PATCH 461/509] Fixed build --- dbms/src/Storages/TableStructureLockHolder.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/TableStructureLockHolder.h b/dbms/src/Storages/TableStructureLockHolder.h index 17e960503f2..3f2f8e1fa32 100644 --- a/dbms/src/Storages/TableStructureLockHolder.h +++ b/dbms/src/Storages/TableStructureLockHolder.h @@ -26,17 +26,17 @@ private: struct TableStructureReadLockHolder { + void release() + { + *this = TableStructureReadLockHolder(); + } + private: friend class IStorage; /// Order is important. RWLockImpl::LockHolder new_data_structure_lock; RWLockImpl::LockHolder structure_lock; - - void release() - { - *this = TableStructureReadLockHolder(); - } }; } From b21fdff77e79c84b9e1672a9ce233201f5904960 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Aug 2019 21:59:21 +0300 Subject: [PATCH 462/509] Merging #6678 --- .../src/DataStreams/MergeSortingBlockInputStream.cpp | 12 +++++++----- dbms/src/Interpreters/Aggregator.cpp | 12 +++++++----- .../Processors/Transforms/MergeSortingTransform.cpp | 12 +++++++----- 3 files changed, 21 insertions(+), 15 deletions(-) diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp index 86fcc4ffff9..377c1fee4e0 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -78,11 +79,12 @@ Block MergeSortingBlockInputStream::readImpl() */ if (max_bytes_before_external_sort && sum_bytes_in_blocks > max_bytes_before_external_sort) { - auto freeSpace = Poco::File(tmp_path).freeSpace(); - if (min_free_disk_space > freeSpace - sum_bytes_in_blocks) - { - throw Exception("Not enough space.", ErrorCodes::NOT_ENOUGH_SPACE); - } +#if !UNBUNDLED + auto free_space = Poco::File(tmp_path).freeSpace(); + if (sum_bytes_in_blocks + min_free_disk_space > free_space) + throw Exception("Not enough space for external sort in " + tmp_path, ErrorCodes::NOT_ENOUGH_SPACE); +#endif + Poco::File(tmp_path).createDirectories(); temporary_files.emplace_back(std::make_unique(tmp_path)); const std::string & path = temporary_files.back()->path(); diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index f2018e443f1..f38573d3d34 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -24,6 +24,7 @@ #include #include #include +#include namespace ProfileEvents @@ -639,11 +640,12 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re && current_memory_usage > static_cast(params.max_bytes_before_external_group_by) && worth_convert_to_two_level) { - auto freeSpace = Poco::File(params.tmp_path).freeSpace(); - if (params.min_free_disk_space > freeSpace - current_memory_usage) - { - throw Exception("Not enough space.", ErrorCodes::NOT_ENOUGH_SPACE); - } +#if !UNBUNDLED + auto free_space = Poco::File(params.tmp_path).freeSpace(); + if (current_memory_usage + params.min_free_disk_space > free_space) + throw Exception("Not enough space for external aggregation in " + params.tmp_path, ErrorCodes::NOT_ENOUGH_SPACE); +#endif + writeToTemporaryFile(result); } diff --git a/dbms/src/Processors/Transforms/MergeSortingTransform.cpp b/dbms/src/Processors/Transforms/MergeSortingTransform.cpp index 93e9acf9ebc..c59fc7cc745 100644 --- a/dbms/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/dbms/src/Processors/Transforms/MergeSortingTransform.cpp @@ -5,6 +5,7 @@ #include #include +#include #include #include @@ -506,11 +507,12 @@ void MergeSortingTransform::consume(Chunk chunk) */ if (max_bytes_before_external_sort && sum_bytes_in_blocks > max_bytes_before_external_sort) { - auto freeSpace = Poco::File(tmp_path).freeSpace(); - if (min_free_disk_space > freeSpace - sum_bytes_in_blocks) - { - throw Exception("Not enough space.", ErrorCodes::NOT_ENOUGH_SPACE); - } +#if !UNBUNDLED + auto free_space = Poco::File(tmp_path).freeSpace(); + if (sum_bytes_in_blocks + min_free_disk_space > free_space) + throw Exception("Not enough space for external sort in " + tmp_path, ErrorCodes::NOT_ENOUGH_SPACE); +#endif + Poco::File(tmp_path).createDirectories(); temporary_files.emplace_back(std::make_unique(tmp_path)); const std::string & path = temporary_files.back()->path(); From 25c53ccd9e315a10b60569cff0d0fb63b45b9c46 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Aug 2019 22:05:35 +0300 Subject: [PATCH 463/509] Rename setting --- dbms/src/Core/Settings.h | 2 +- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 3794548628c..88a81027cb6 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -347,12 +347,12 @@ struct Settings : public SettingsCollection M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.") \ M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.") \ M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.") \ + M(SettingUInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.") \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") \ M(SettingBool, compile, false, "Whether query compilation is enabled. Will be removed after 2020-03-13") \ - M(SettingUInt64, min_free_disk_space, 0, "The minimum disk space to keep") \ DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 6c8769aa3d1..79fbcf44323 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1657,7 +1657,7 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space); + context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); /// If there are several sources, then we perform parallel aggregation if (pipeline.streams.size() > 1) @@ -1723,7 +1723,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space); + context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); auto transform_params = std::make_shared(params, final); @@ -1943,7 +1943,7 @@ void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificato false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, SettingUInt64(0), SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space); + context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); if (modificator == Modificator::ROLLUP) pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); @@ -1972,7 +1972,7 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPipeline & pipeline, Modif false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, SettingUInt64(0), SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space); + context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); auto transform_params = std::make_shared(params, true); @@ -2073,7 +2073,7 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr so pipeline.firstStream() = std::make_shared( pipeline.firstStream(), order_descr, settings.max_block_size, limit, settings.max_bytes_before_remerge_sort, - settings.max_bytes_before_external_sort, context.getTemporaryPath(), settings.min_free_disk_space); + settings.max_bytes_before_external_sort, context.getTemporaryPath(), settings.min_free_disk_space_for_temporary_data); } } @@ -2111,7 +2111,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP return std::make_shared( header, order_descr, settings.max_block_size, limit, settings.max_bytes_before_remerge_sort, - settings.max_bytes_before_external_sort, context.getTemporaryPath(), settings.min_free_disk_space); + settings.max_bytes_before_external_sort, context.getTemporaryPath(), settings.min_free_disk_space_for_temporary_data); }); } From 0add95b7d8f0add0e02b6956ae0d2bcaa87f96c6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Aug 2019 22:21:53 +0300 Subject: [PATCH 464/509] Set default value of "queue_max_wait_ms" to zero, because current value makes no sense --- dbms/src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 881ea83f30b..9f780392466 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -61,7 +61,7 @@ struct Settings : public SettingsCollection M(SettingSeconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "") \ M(SettingSeconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "") \ M(SettingSeconds, tcp_keep_alive_timeout, 0, "") \ - M(SettingMilliseconds, queue_max_wait_ms, 5000, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.") \ + M(SettingMilliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.") \ M(SettingUInt64, poll_interval, DBMS_DEFAULT_POLL_INTERVAL, "Block at the query wait loop on the server for the specified number of seconds.") \ M(SettingUInt64, idle_connection_timeout, 3600, "Close idle TCP connections after specified number of seconds.") \ M(SettingUInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.") \ From 63c0070cd50aaa463cc00eedee4c43868ba8c69f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Aug 2019 22:32:33 +0300 Subject: [PATCH 465/509] Fixed flacky test --- .../00626_replace_partition_from_table_zookeeper.sh | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index 0d0bbae1402..75d6a0dd59b 100755 --- a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -1,5 +1,11 @@ #!/usr/bin/env bash +# Because REPLACE PARTITION does not forces immediate removal of replaced data parts from local filesystem +# (it tries to do it as quick as possible, but it still performed in separate thread asynchronously) +# and when we do DETACH TABLE / ATTACH TABLE or SYSTEM RESTART REPLICA, these files may be discovered +# and discarded after restart with Warning/Error messages in log. This is Ok. +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh From 5208a70203e6acd8593733b9e329fb161f84f05d Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 27 Aug 2019 22:41:51 +0300 Subject: [PATCH 466/509] one less place with changed nullability columns --- dbms/src/Interpreters/AnalyzedJoin.cpp | 17 ---------- dbms/src/Interpreters/AnalyzedJoin.h | 3 -- dbms/src/Interpreters/ExpressionActions.cpp | 4 +-- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 37 +++++++++++++-------- dbms/src/Interpreters/SyntaxAnalyzer.h | 4 +-- 5 files changed, 28 insertions(+), 37 deletions(-) diff --git a/dbms/src/Interpreters/AnalyzedJoin.cpp b/dbms/src/Interpreters/AnalyzedJoin.cpp index 4176b0b8012..6a3b9b8ac1b 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.cpp +++ b/dbms/src/Interpreters/AnalyzedJoin.cpp @@ -7,7 +7,6 @@ #include #include -#include namespace DB { @@ -102,22 +101,6 @@ std::unordered_map AnalyzedJoin::getOriginalColumnsMap(const Nam return out; } -void AnalyzedJoin::calculateAvailableJoinedColumns(bool make_nullable) -{ - if (!make_nullable) - { - available_joined_columns = columns_from_joined_table; - return; - } - - for (auto & column : columns_from_joined_table) - { - auto type = column.type->canBeInsideNullable() ? makeNullable(column.type) : column.type; - available_joined_columns.emplace_back(NameAndTypePair(column.name, std::move(type))); - } -} - - NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context) { NamesAndTypesList names_and_type_list; diff --git a/dbms/src/Interpreters/AnalyzedJoin.h b/dbms/src/Interpreters/AnalyzedJoin.h index af010aaca11..1ce11da95e0 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.h +++ b/dbms/src/Interpreters/AnalyzedJoin.h @@ -42,8 +42,6 @@ private: /// All columns which can be read from joined table. Duplicating names are qualified. NamesAndTypesList columns_from_joined_table; - /// Columns from joined table which may be added to block. It's columns_from_joined_table with possibly modified types. - NamesAndTypesList available_joined_columns; /// Name -> original name. Names are the same as in columns_from_joined_table list. std::unordered_map original_names; /// Original name -> name. Only ranamed columns. @@ -61,7 +59,6 @@ public: std::unordered_map getOriginalColumnsMap(const NameSet & required_columns) const; void deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix); - void calculateAvailableJoinedColumns(bool make_nullable); size_t rightKeyInclusion(const String & name) const; }; diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index c7b510abcf0..5ef05569f91 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -278,8 +278,8 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings, case JOIN: { bool is_null_used_as_default = settings.join_use_nulls; - bool right_or_full_join = join_kind == ASTTableJoin::Kind::Right || join_kind == ASTTableJoin::Kind::Full; - bool left_or_full_join = join_kind == ASTTableJoin::Kind::Left || join_kind == ASTTableJoin::Kind::Full; + bool right_or_full_join = isRightOrFull(join_kind); + bool left_or_full_join = isLeftOrFull(join_kind); for (auto & col : sample_block) { diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index c7f89153b44..0c73beeef16 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -30,6 +30,7 @@ #include #include +#include #include #include @@ -488,13 +489,14 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const } } -void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness) +void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, ASTTableJoin::Kind & join_kind) { const ASTTablesInSelectQueryElement * node = select_query.join(); if (!node) return; auto & table_join = const_cast(node)->table_join->as(); + join_kind = table_join.kind; if (table_join.strictness == ASTTableJoin::Strictness::Unspecified && table_join.kind != ASTTableJoin::Kind::Cross) @@ -511,7 +513,7 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul /// Find the columns that are obtained by JOIN. void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & select_query, const NameSet & source_columns, - const Aliases & aliases, bool join_use_nulls) + const Aliases & aliases) { const ASTTablesInSelectQueryElement * node = select_query.join(); if (!node) @@ -537,10 +539,6 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & s if (is_asof) data.asofToJoinKeys(); } - - bool make_nullable = join_use_nulls && isLeftOrFull(table_join.kind); - - analyzed_join.calculateAvailableJoinedColumns(make_nullable); } void replaceJoinedTable(const ASTTablesInSelectQueryElement* join) @@ -611,7 +609,8 @@ std::vector getAggregates(const ASTPtr & query) /// Calculate which columns are required to execute the expression. /// Then, delete all other columns from the list of available columns. /// After execution, columns will only contain the list of columns needed to read from the table. -void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns) +void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns, + bool make_joined_columns_nullable) { /// We caclulate required_source_columns with source_columns modifications and swap them on exit required_source_columns = source_columns; @@ -639,7 +638,7 @@ void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesA /// Add columns obtained by JOIN (if needed). columns_added_by_join.clear(); - for (const auto & joined_column : analyzed_join.available_joined_columns) + for (const auto & joined_column : analyzed_join.columns_from_joined_table) { auto & name = joined_column.name; if (avaliable_columns.count(name)) @@ -649,7 +648,15 @@ void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesA { /// Optimisation: do not add columns needed only in JOIN ON section. if (columns_context.nameInclusion(name) > analyzed_join.rightKeyInclusion(name)) - columns_added_by_join.push_back(joined_column); + { + if (make_joined_columns_nullable) + { + auto type = joined_column.type->canBeInsideNullable() ? makeNullable(joined_column.type) : joined_column.type; + columns_added_by_join.emplace_back(NameAndTypePair(joined_column.name, std::move(type))); + } + else + columns_added_by_join.push_back(joined_column); + } required.erase(name); } } @@ -759,7 +766,7 @@ void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesA if (columns_context.has_table_join) { ss << ", joined columns:"; - for (const auto & column : analyzed_join.available_joined_columns) + for (const auto & column : analyzed_join.columns_from_joined_table) ss << " '" << column.name << "'"; } @@ -865,6 +872,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( /// Optimize if with constant condition after constants was substituted instead of scalar subqueries. OptimizeIfWithConstantConditionVisitor(result.aliases).visit(query); + bool make_joined_columns_nullable = false; if (select_query) { /// GROUP BY injective function elimination. @@ -885,12 +893,15 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( /// Push the predicate expression down to the subqueries. result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize(); - setJoinStrictness(*select_query, settings.join_default_strictness); - collectJoinedColumns(result.analyzed_join, *select_query, source_columns_set, result.aliases, settings.join_use_nulls); + ASTTableJoin::Kind join_kind = ASTTableJoin::Kind::Comma; + setJoinStrictness(*select_query, settings.join_default_strictness, join_kind); + make_joined_columns_nullable = settings.join_use_nulls && isLeftOrFull(join_kind); + + collectJoinedColumns(result.analyzed_join, *select_query, source_columns_set, result.aliases); } result.aggregates = getAggregates(query); - result.collectUsedColumns(query, additional_source_columns); + result.collectUsedColumns(query, additional_source_columns, make_joined_columns_nullable); return std::make_shared(result); } diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.h b/dbms/src/Interpreters/SyntaxAnalyzer.h index a31dfef7e82..e95d7354e8a 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.h +++ b/dbms/src/Interpreters/SyntaxAnalyzer.h @@ -20,7 +20,7 @@ struct SyntaxAnalyzerResult NamesAndTypesList source_columns; /// Set of columns that are enough to read from the table to evaluate the expression. It does not include joined columns. NamesAndTypesList required_source_columns; - /// Columns will be added to block by JOIN. It's a subset of analyzed_join.available_joined_columns + /// Columns will be added to block by JOIN. It's a subset of analyzed_join.columns_from_joined_table with corrected Nullability NamesAndTypesList columns_added_by_join; Aliases aliases; @@ -42,7 +42,7 @@ struct SyntaxAnalyzerResult /// Predicate optimizer overrides the sub queries bool rewrite_subqueries = false; - void collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns); + void collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns, bool make_joined_columns_nullable); Names requiredSourceColumns() const { return required_source_columns.getNames(); } }; From e80ff65a0f10ab1abe6792a79b56196aad74a661 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Tue, 27 Aug 2019 16:23:12 -0400 Subject: [PATCH 467/509] Removing allow_experimental_live_view option from clickhouse-client.xml. --- dbms/tests/clickhouse-client.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/tests/clickhouse-client.xml b/dbms/tests/clickhouse-client.xml index ebce35127e5..b6003ca2d09 100644 --- a/dbms/tests/clickhouse-client.xml +++ b/dbms/tests/clickhouse-client.xml @@ -1,4 +1,3 @@ - 1 100000 From 285d66cee15e8746129230372d1a3d8444527d3a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Aug 2019 23:43:08 +0300 Subject: [PATCH 468/509] Improve table locks (incomplete) --- dbms/src/Databases/DatabaseDictionary.cpp | 20 -------- dbms/src/Databases/DatabaseDictionary.h | 14 ------ dbms/src/Databases/DatabaseFactory.cpp | 1 + dbms/src/Databases/DatabaseMemory.cpp | 20 -------- dbms/src/Databases/DatabaseMemory.h | 14 ------ dbms/src/Databases/DatabaseMySQL.h | 12 +---- dbms/src/Databases/DatabaseOrdinary.cpp | 5 +- dbms/src/Databases/DatabaseOrdinary.h | 4 +- dbms/src/Databases/IDatabase.h | 46 +++++++++++-------- .../src/Interpreters/InterpreterDropQuery.cpp | 8 ++-- .../Interpreters/InterpreterRenameQuery.cpp | 19 ++++---- dbms/src/Storages/IStorage.h | 12 +++-- .../src/Storages/LiveView/StorageLiveView.cpp | 2 +- dbms/src/Storages/LiveView/StorageLiveView.h | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 4 +- dbms/src/Storages/MergeTree/MergeTreeData.h | 4 +- dbms/src/Storages/StorageDictionary.h | 1 - dbms/src/Storages/StorageDistributed.cpp | 2 +- dbms/src/Storages/StorageDistributed.h | 11 +++-- dbms/src/Storages/StorageJoin.cpp | 2 +- dbms/src/Storages/StorageJoin.h | 2 +- dbms/src/Storages/StorageLog.cpp | 4 +- dbms/src/Storages/StorageLog.h | 4 +- dbms/src/Storages/StorageMaterializedView.cpp | 3 +- dbms/src/Storages/StorageMaterializedView.h | 6 +-- dbms/src/Storages/StorageMemory.cpp | 4 +- dbms/src/Storages/StorageMemory.h | 10 ++-- dbms/src/Storages/StorageMergeTree.cpp | 10 ++-- dbms/src/Storages/StorageMergeTree.h | 6 +-- .../Storages/StorageReplicatedMergeTree.cpp | 14 +++--- .../src/Storages/StorageReplicatedMergeTree.h | 6 +-- dbms/src/Storages/StorageSet.cpp | 5 +- dbms/src/Storages/StorageSet.h | 4 +- dbms/src/Storages/StorageStripeLog.cpp | 4 +- dbms/src/Storages/StorageStripeLog.h | 4 +- dbms/src/Storages/StorageTinyLog.cpp | 4 +- dbms/src/Storages/StorageTinyLog.h | 4 +- 37 files changed, 123 insertions(+), 174 deletions(-) diff --git a/dbms/src/Databases/DatabaseDictionary.cpp b/dbms/src/Databases/DatabaseDictionary.cpp index bfc0f6e89d0..2bb9bd30238 100644 --- a/dbms/src/Databases/DatabaseDictionary.cpp +++ b/dbms/src/Databases/DatabaseDictionary.cpp @@ -115,26 +115,6 @@ void DatabaseDictionary::removeTable( throw Exception("DatabaseDictionary: removeTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); } -void DatabaseDictionary::renameTable( - const Context &, - const String &, - IDatabase &, - const String &) -{ - throw Exception("DatabaseDictionary: renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); -} - -void DatabaseDictionary::alterTable( - const Context &, - const String &, - const ColumnsDescription &, - const IndicesDescription &, - const ConstraintsDescription &, - const ASTModifier &) -{ - throw Exception("DatabaseDictionary: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); -} - time_t DatabaseDictionary::getTableMetadataModificationTime( const Context &, const String &) diff --git a/dbms/src/Databases/DatabaseDictionary.h b/dbms/src/Databases/DatabaseDictionary.h index 650a6986722..1e1af7ef581 100644 --- a/dbms/src/Databases/DatabaseDictionary.h +++ b/dbms/src/Databases/DatabaseDictionary.h @@ -60,20 +60,6 @@ public: void attachTable(const String & table_name, const StoragePtr & table) override; StoragePtr detachTable(const String & table_name) override; - void renameTable( - const Context & context, - const String & table_name, - IDatabase & to_database, - const String & to_table_name) override; - - void alterTable( - const Context & context, - const String & name, - const ColumnsDescription & columns, - const IndicesDescription & indices, - const ConstraintsDescription & constraints, - const ASTModifier & engine_modifier) override; - time_t getTableMetadataModificationTime( const Context & context, const String & table_name) override; diff --git a/dbms/src/Databases/DatabaseFactory.cpp b/dbms/src/Databases/DatabaseFactory.cpp index 1dcf41dc4d6..af2a00830e1 100644 --- a/dbms/src/Databases/DatabaseFactory.cpp +++ b/dbms/src/Databases/DatabaseFactory.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include "config_core.h" #if USE_MYSQL diff --git a/dbms/src/Databases/DatabaseMemory.cpp b/dbms/src/Databases/DatabaseMemory.cpp index a7f8460366c..1356a28d245 100644 --- a/dbms/src/Databases/DatabaseMemory.cpp +++ b/dbms/src/Databases/DatabaseMemory.cpp @@ -39,26 +39,6 @@ void DatabaseMemory::removeTable( detachTable(table_name); } -void DatabaseMemory::renameTable( - const Context &, - const String &, - IDatabase &, - const String &) -{ - throw Exception("DatabaseMemory: renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); -} - -void DatabaseMemory::alterTable( - const Context &, - const String &, - const ColumnsDescription &, - const IndicesDescription &, - const ConstraintsDescription &, - const ASTModifier &) -{ - throw Exception("DatabaseMemory: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); -} - time_t DatabaseMemory::getTableMetadataModificationTime( const Context &, const String &) diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index 32d3045612b..33bb8787168 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -37,20 +37,6 @@ public: const Context & context, const String & table_name) override; - void renameTable( - const Context & context, - const String & table_name, - IDatabase & to_database, - const String & to_table_name) override; - - void alterTable( - const Context & context, - const String & name, - const ColumnsDescription & columns, - const IndicesDescription & indices, - const ConstraintsDescription & constraints, - const ASTModifier & engine_modifier) override; - time_t getTableMetadataModificationTime( const Context & context, const String & table_name) override; diff --git a/dbms/src/Databases/DatabaseMySQL.h b/dbms/src/Databases/DatabaseMySQL.h index 6c277f1e9f8..7044a594b4c 100644 --- a/dbms/src/Databases/DatabaseMySQL.h +++ b/dbms/src/Databases/DatabaseMySQL.h @@ -5,6 +5,8 @@ #include #include +#include + namespace DB { @@ -61,21 +63,11 @@ public: throw Exception("MySQL database engine does not support attach table.", ErrorCodes::NOT_IMPLEMENTED); } - void renameTable(const Context &, const String &, IDatabase &, const String &) override - { - throw Exception("MySQL database engine does not support rename table.", ErrorCodes::NOT_IMPLEMENTED); - } - void createTable(const Context &, const String &, const StoragePtr &, const ASTPtr &) override { throw Exception("MySQL database engine does not support create table.", ErrorCodes::NOT_IMPLEMENTED); } - void alterTable(const Context &, const String &, const ColumnsDescription &, const IndicesDescription &, const ConstraintsDescription &, const ASTModifier &) override - { - throw Exception("MySQL database engine does not support alter table.", ErrorCodes::NOT_IMPLEMENTED); - } - private: struct MySQLStorageInfo { diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 4748bd0d792..06e355892e8 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -355,7 +355,8 @@ void DatabaseOrdinary::renameTable( const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name) + const String & to_table_name, + TableStructureWriteLockHolder & lock) { DatabaseOrdinary * to_database_concrete = typeid_cast(&to_database); @@ -372,7 +373,7 @@ void DatabaseOrdinary::renameTable( { table->rename(context.getPath() + "/data/" + escapeForFileName(to_database_concrete->name) + "/", to_database_concrete->name, - to_table_name); + to_table_name, lock); } catch (const Exception &) { diff --git a/dbms/src/Databases/DatabaseOrdinary.h b/dbms/src/Databases/DatabaseOrdinary.h index a71029f6495..e8895075768 100644 --- a/dbms/src/Databases/DatabaseOrdinary.h +++ b/dbms/src/Databases/DatabaseOrdinary.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -35,7 +36,8 @@ public: const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name) override; + const String & to_table_name, + TableStructureWriteLockHolder &) override; void alterTable( const Context & context, diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index b9d4c6ce4cb..549d795b66d 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -1,16 +1,9 @@ #pragma once -#include #include -#include #include -#include -#include -#include #include -#include -#include -#include +#include #include #include @@ -21,8 +14,16 @@ namespace DB { class Context; - struct Settings; +struct ConstraintsDescription; +class ColumnsDescription; +struct IndicesDescription; +struct TableStructureWriteLockHolder; + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} /** Allows to iterate over tables. @@ -102,22 +103,29 @@ public: /// Rename the table and possibly move the table to another database. virtual void renameTable( - const Context & context, - const String & name, - IDatabase & to_database, - const String & to_name) = 0; + const Context & /*context*/, + const String & /*name*/, + IDatabase & /*to_database*/, + const String & /*to_name*/, + TableStructureWriteLockHolder &) + { + throw Exception(getEngineName() + ": renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); + } using ASTModifier = std::function; /// Change the table structure in metadata. /// You must call under the TableStructureLock of the corresponding table . If engine_modifier is empty, then engine does not change. virtual void alterTable( - const Context & context, - const String & name, - const ColumnsDescription & columns, - const IndicesDescription & indices, - const ConstraintsDescription & constraints, - const ASTModifier & engine_modifier) = 0; + const Context & /*context*/, + const String & /*name*/, + const ColumnsDescription & /*columns*/, + const IndicesDescription & /*indices*/, + const ConstraintsDescription & /*constraints*/, + const ASTModifier & /*engine_modifier*/) + { + throw Exception(getEngineName() + ": renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); + } /// Returns time of table's metadata change, 0 if there is no corresponding metadata file. virtual time_t getTableMetadataModificationTime( diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 91213b6100e..d601580a044 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -80,7 +80,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t /// If table was already dropped by anyone, an exception will be thrown auto table_lock = database_and_table.second->lockExclusively(context.getCurrentQueryId()); /// Drop table data, don't touch metadata - database_and_table.second->truncate(query_ptr, context); + database_and_table.second->truncate(query_ptr, context, table_lock); } else if (kind == ASTDropQuery::Kind::Drop) { @@ -94,7 +94,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t database_and_table.first->removeTable(context, database_and_table.second->getTableName()); /// Delete table data - database_and_table.second->drop(); + database_and_table.second->drop(table_lock); database_and_table.second->is_dropped = true; String database_data_path = database_and_table.first->getDataPath(); @@ -128,7 +128,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDr /// If table was already dropped by anyone, an exception will be thrown auto table_lock = table->lockExclusively(context.getCurrentQueryId()); /// Drop table data, don't touch metadata - table->truncate(query_ptr, context); + table->truncate(query_ptr, context, table_lock); } else if (kind == ASTDropQuery::Kind::Drop) { @@ -137,7 +137,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDr /// If table was already dropped by anyone, an exception will be thrown auto table_lock = table->lockExclusively(context.getCurrentQueryId()); /// Delete table data - table->drop(); + table->drop(table_lock); table->is_dropped = true; } } diff --git a/dbms/src/Interpreters/InterpreterRenameQuery.cpp b/dbms/src/Interpreters/InterpreterRenameQuery.cpp index 360adf45194..97b17c6afd7 100644 --- a/dbms/src/Interpreters/InterpreterRenameQuery.cpp +++ b/dbms/src/Interpreters/InterpreterRenameQuery.cpp @@ -26,6 +26,8 @@ struct RenameDescription to_table_name(elem.to.table) {} + TableStructureWriteLockHolder from_table_lock; + String from_database_name; String from_table_name; @@ -75,7 +77,7 @@ BlockIO InterpreterRenameQuery::execute() } }; - std::set unique_tables_from; + std::map tables_from_locks; /// Don't allow to drop tables (that we are renaming); don't allow to create tables in places where tables will be renamed. std::map> table_guards; @@ -87,7 +89,11 @@ BlockIO InterpreterRenameQuery::execute() UniqueTableName from(descriptions.back().from_database_name, descriptions.back().from_table_name); UniqueTableName to(descriptions.back().to_database_name, descriptions.back().to_table_name); - unique_tables_from.emplace(from); + if (!tables_from_locks.count(from)) + if (auto table = context.tryGetTable(from.database_name, from.table_name)) + tables_from_locks.emplace(from, table->lockExclusively(context.getCurrentQueryId())); + + descriptions.back().table_lock = tables_from_locks[from]; if (!table_guards.count(from)) table_guards.emplace(from, context.getDDLGuard(from.database_name, from.table_name)); @@ -96,13 +102,6 @@ BlockIO InterpreterRenameQuery::execute() table_guards.emplace(to, context.getDDLGuard(to.database_name, to.table_name)); } - std::vector locks; - locks.reserve(unique_tables_from.size()); - - for (const auto & names : unique_tables_from) - if (auto table = context.tryGetTable(names.database_name, names.table_name)) - locks.emplace_back(table->lockExclusively(context.getCurrentQueryId())); - /** All tables are locked. If there are more than one rename in chain, * we need to hold global lock while doing all renames. Order matters to avoid deadlocks. * It provides atomicity of all RENAME chain as a whole, from the point of view of DBMS client, @@ -119,7 +118,7 @@ BlockIO InterpreterRenameQuery::execute() context.assertTableDoesntExist(elem.to_database_name, elem.to_table_name); context.getDatabase(elem.from_database_name)->renameTable( - context, elem.from_table_name, *context.getDatabase(elem.to_database_name), elem.to_table_name); + context, elem.from_table_name, *context.getDatabase(elem.to_database_name), elem.to_table_name, elem.table_lock); } return {}; diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 6c23a638ddf..ffe4ed1b775 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -9,11 +9,13 @@ #include #include #include +#include +#include +#include #include #include #include #include -#include #include #include @@ -24,7 +26,6 @@ namespace DB namespace ErrorCodes { - extern const int TABLE_IS_DROPPED; extern const int NOT_IMPLEMENTED; } @@ -261,12 +262,12 @@ public: * The table is not usable during and after call to this method. * If you do not need any action other than deleting the directory with data, you can leave this method blank. */ - virtual void drop() {} + virtual void drop(TableStructureWriteLockHolder &) {} /** Clear the table data and leave it empty. * Must be called under lockForAlter. */ - virtual void truncate(const ASTPtr & /*query*/, const Context & /* context */) + virtual void truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &) { throw Exception("Truncate is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } @@ -276,7 +277,8 @@ public: * In this function, you need to rename the directory with the data, if any. * Called when the table structure is locked for write. */ - virtual void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/) + virtual void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/, + TableStructureWriteLockHolder &) { throw Exception("Method rename is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 7704f421517..9468a180772 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -471,7 +471,7 @@ StorageLiveView::~StorageLiveView() shutdown(); } -void StorageLiveView::drop() +void StorageLiveView::drop(TableStructureWriteLockHolder &) { global_context.removeDependency( DatabaseAndTableName(select_database_name, select_table_name), diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index 9930d8d6154..79ac259f912 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -105,7 +105,7 @@ public: } void checkTableCanBeDropped() const override; - void drop() override; + void drop(TableStructureWriteLockHolder &) override; void startup() override; void shutdown() override; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index b2d4a4b9d73..70ae2414418 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1690,7 +1690,7 @@ void MergeTreeData::removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr empty_columns.clear(); } -void MergeTreeData::freezeAll(const String & with_name, const Context & context) +void MergeTreeData::freezeAll(const String & with_name, const Context & context, TableStructureReadLockHolder &) { freezePartitionsByMatcher([] (const DataPartPtr &){ return true; }, with_name, context); } @@ -2550,7 +2550,7 @@ void MergeTreeData::removePartContributionToColumnSizes(const DataPartPtr & part } -void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String & with_name, const Context & context) +void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String & with_name, const Context & context, TableStructureReadLockHolder &) { std::optional prefix; String partition_id; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 0440a3181c8..ea9db0920a3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -549,7 +549,7 @@ public: void removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr & data_part); /// Freezes all parts. - void freezeAll(const String & with_name, const Context & context); + void freezeAll(const String & with_name, const Context & context, TableStructureReadLockHolder & table_lock_holder); /// Should be called if part data is suspected to be corrupted. void reportBrokenPart(const String & name) const @@ -577,7 +577,7 @@ public: * Backup is created in directory clickhouse_dir/shadow/i/, where i - incremental number, * or if 'with_name' is specified - backup is created in directory with specified name. */ - void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context); + void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context, TableStructureReadLockHolder & table_lock_holder); size_t getColumnCompressedSize(const std::string & name) const { diff --git a/dbms/src/Storages/StorageDictionary.h b/dbms/src/Storages/StorageDictionary.h index 097e81d15c6..9539240e75d 100644 --- a/dbms/src/Storages/StorageDictionary.h +++ b/dbms/src/Storages/StorageDictionary.h @@ -34,7 +34,6 @@ public: size_t max_block_size = DEFAULT_BLOCK_SIZE, unsigned threads = 1) override; - void drop() override {} static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure); template diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index b2e632f9414..2c289dd714e 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -415,7 +415,7 @@ void StorageDistributed::shutdown() } -void StorageDistributed::truncate(const ASTPtr &, const Context &) +void StorageDistributed::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { std::lock_guard lock(cluster_nodes_mutex); diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index 3261f5a0beb..153ada5d284 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -77,12 +77,17 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void drop() override {} + void drop(TableStructureWriteLockHolder &) override {} /// Removes temporary data in local filesystem. - void truncate(const ASTPtr &, const Context &) override; + void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; + + void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override + { + table_name = new_table_name; + database_name = new_database_name; + } - void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; } /// in the sub-tables, you need to manually add and delete columns /// the structure of the sub-table is not checked void alter( diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index e2cbe542e11..54effdcd4fa 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -55,7 +55,7 @@ StorageJoin::StorageJoin( } -void StorageJoin::truncate(const ASTPtr &, const Context &) +void StorageJoin::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { Poco::File(path).remove(true); Poco::File(path).createDirectories(); diff --git a/dbms/src/Storages/StorageJoin.h b/dbms/src/Storages/StorageJoin.h index 760e9eb1815..bdc50b9d767 100644 --- a/dbms/src/Storages/StorageJoin.h +++ b/dbms/src/Storages/StorageJoin.h @@ -26,7 +26,7 @@ class StorageJoin : public ext::shared_ptr_helper, public StorageSe public: String getName() const override { return "Join"; } - void truncate(const ASTPtr &, const Context &) override; + void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; /// Access the innards. JoinPtr & getJoin() { return join; } diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 69e37ce2305..3811b226357 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -512,7 +512,7 @@ void StorageLog::loadMarks() } -void StorageLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) +void StorageLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { std::unique_lock lock(rwlock); @@ -530,7 +530,7 @@ void StorageLog::rename(const String & new_path_to_db, const String & new_databa marks_file = Poco::File(path + escapeForFileName(table_name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME); } -void StorageLog::truncate(const ASTPtr &, const Context &) +void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { std::shared_lock lock(rwlock); diff --git a/dbms/src/Storages/StorageLog.h b/dbms/src/Storages/StorageLog.h index 70de62cb47b..7f792337abc 100644 --- a/dbms/src/Storages/StorageLog.h +++ b/dbms/src/Storages/StorageLog.h @@ -38,11 +38,11 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; + void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; - void truncate(const ASTPtr &, const Context &) override; + void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; std::string full_path() const { return path + escapeForFileName(table_name) + '/';} diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index df39f711ab3..c3e52f8a32e 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -299,7 +299,8 @@ static void executeRenameQuery(Context & global_context, const String & database } -void StorageMaterializedView::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) +void StorageMaterializedView::rename( + const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { if (has_inner_table && tryGetTargetTable()) { diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index b635634addf..327e8feed79 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -33,9 +33,9 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void drop() override; + void drop(TableStructureWriteLockHolder &) override; - void truncate(const ASTPtr &, const Context &) override; + void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; @@ -43,7 +43,7 @@ public: void mutate(const MutationCommands & commands, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; + void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; void shutdown() override; diff --git a/dbms/src/Storages/StorageMemory.cpp b/dbms/src/Storages/StorageMemory.cpp index 560da7dc2d8..1b820e55c5e 100644 --- a/dbms/src/Storages/StorageMemory.cpp +++ b/dbms/src/Storages/StorageMemory.cpp @@ -123,13 +123,13 @@ BlockOutputStreamPtr StorageMemory::write( } -void StorageMemory::drop() +void StorageMemory::drop(TableStructureWriteLockHolder &) { std::lock_guard lock(mutex); data.clear(); } -void StorageMemory::truncate(const ASTPtr &, const Context &) +void StorageMemory::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { std::lock_guard lock(mutex); data.clear(); diff --git a/dbms/src/Storages/StorageMemory.h b/dbms/src/Storages/StorageMemory.h index 92dcd4be18f..eb2d6ff2e21 100644 --- a/dbms/src/Storages/StorageMemory.h +++ b/dbms/src/Storages/StorageMemory.h @@ -40,11 +40,15 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void drop() override; + void drop(TableStructureWriteLockHolder &) override; - void truncate(const ASTPtr &, const Context &) override; + void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; - void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; } + void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override + { + table_name = new_table_name; + database_name = new_database_name; + } private: String database_name; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 4b7b2c446f6..46fcc977a60 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -157,13 +157,13 @@ void StorageMergeTree::checkPartitionCanBeDropped(const ASTPtr & partition) global_context.checkPartitionCanBeDropped(database_name, table_name, partition_size); } -void StorageMergeTree::drop() +void StorageMergeTree::drop(TableStructureWriteLockHolder &) { shutdown(); dropAllData(); } -void StorageMergeTree::truncate(const ASTPtr &, const Context &) +void StorageMergeTree::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { { /// Asks to complete merges and does not allow them to start. @@ -181,7 +181,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const Context &) clearOldPartsFromFilesystem(); } -void StorageMergeTree::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) +void StorageMergeTree::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { std::string new_full_path = new_path_to_db + escapeForFileName(new_table_name) + '/'; @@ -995,7 +995,7 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma case PartitionCommand::FREEZE_PARTITION: { auto lock = lockStructureForShare(false, context.getCurrentQueryId()); - freezePartition(command.partition, command.with_name, context); + freezePartition(command.partition, command.with_name, context, lock); } break; @@ -1020,7 +1020,7 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma case PartitionCommand::FREEZE_ALL_PARTITIONS: { auto lock = lockStructureForShare(false, context.getCurrentQueryId()); - freezeAll(command.with_name, context); + freezeAll(command.with_name, context, lock); } break; diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index d135ffd6e1a..06ffa94c8ec 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -55,10 +55,10 @@ public: std::vector getMutationsStatus() const override; CancellationCode killMutation(const String & mutation_id) override; - void drop() override; - void truncate(const ASTPtr &, const Context &) override; + void drop(TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; + void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index afcc5d968af..e5821c1bcaf 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3132,7 +3132,6 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p if (query_context.getSettingsRef().replication_alter_partitions_sync != 0) { /// NOTE Table lock must not be held while waiting. Some combination of R-W-R locks from different threads will yield to deadlock. - /// TODO Check all other "wait" places. for (auto & merge_entry : merge_entries) waitForAllReplicasToProcessLogEntry(merge_entry); } @@ -3484,7 +3483,7 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part case PartitionCommand::FREEZE_PARTITION: { auto lock = lockStructureForShare(false, query_context.getCurrentQueryId()); - freezePartition(command.partition, command.with_name, query_context); + freezePartition(command.partition, command.with_name, query_context, lock); } break; @@ -3509,7 +3508,7 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part case PartitionCommand::FREEZE_ALL_PARTITIONS: { auto lock = lockStructureForShare(false, query_context.getCurrentQueryId()); - freezeAll(command.with_name, query_context); + freezeAll(command.with_name, query_context, lock); } break; } @@ -3633,8 +3632,10 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPt } -void StorageReplicatedMergeTree::truncate(const ASTPtr & query, const Context & query_context) +void StorageReplicatedMergeTree::truncate(const ASTPtr & query, const Context & query_context, TableStructureWriteLockHolder & table_lock) { + table_lock.release(); /// Truncate is done asynchronously. + assertNotReadonly(); zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); @@ -3701,7 +3702,7 @@ void StorageReplicatedMergeTree::checkPartitionCanBeDropped(const ASTPtr & parti } -void StorageReplicatedMergeTree::drop() +void StorageReplicatedMergeTree::drop(TableStructureWriteLockHolder &) { { auto zookeeper = tryGetZooKeeper(); @@ -3731,7 +3732,8 @@ void StorageReplicatedMergeTree::drop() } -void StorageReplicatedMergeTree::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) +void StorageReplicatedMergeTree::rename( + const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { std::string new_full_path = new_path_to_db + escapeForFileName(new_table_name) + '/'; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 0fa2e3631e2..c5000944439 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -109,11 +109,11 @@ public: /** Removes a replica from ZooKeeper. If there are no other replicas, it deletes the entire table from ZooKeeper. */ - void drop() override; + void drop(TableStructureWriteLockHolder &) override; - void truncate(const ASTPtr &, const Context &) override; + void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; + void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; bool supportsIndexForIn() const override { return true; } diff --git a/dbms/src/Storages/StorageSet.cpp b/dbms/src/Storages/StorageSet.cpp index 26af630ca62..c76857bf610 100644 --- a/dbms/src/Storages/StorageSet.cpp +++ b/dbms/src/Storages/StorageSet.cpp @@ -126,7 +126,7 @@ void StorageSet::insertBlock(const Block & block) { set->insertFromBlock(block); size_t StorageSet::getSize() const { return set->getTotalRowCount(); } -void StorageSet::truncate(const ASTPtr &, const Context &) +void StorageSet::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { Poco::File(path).remove(true); Poco::File(path).createDirectories(); @@ -193,7 +193,8 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path) } -void StorageSetOrJoinBase::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) +void StorageSetOrJoinBase::rename( + const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { /// Rename directory with data. String new_path = new_path_to_db + escapeForFileName(new_table_name); diff --git a/dbms/src/Storages/StorageSet.h b/dbms/src/Storages/StorageSet.h index fe6cd332ed8..671bc78897b 100644 --- a/dbms/src/Storages/StorageSet.h +++ b/dbms/src/Storages/StorageSet.h @@ -22,7 +22,7 @@ public: String getTableName() const override { return table_name; } String getDatabaseName() const override { return database_name; } - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; + void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; @@ -69,7 +69,7 @@ public: /// Access the insides. SetPtr & getSet() { return set; } - void truncate(const ASTPtr &, const Context &) override; + void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; private: SetPtr set; diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index 447e325095d..00c359f9797 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -223,7 +223,7 @@ StorageStripeLog::StorageStripeLog( } -void StorageStripeLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) +void StorageStripeLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { std::unique_lock lock(rwlock); @@ -294,7 +294,7 @@ CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Conte return file_checker.check(); } -void StorageStripeLog::truncate(const ASTPtr &, const Context &) +void StorageStripeLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { if (table_name.empty()) throw Exception("Logical error: table name is empty", ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Storages/StorageStripeLog.h b/dbms/src/Storages/StorageStripeLog.h index d53f000f277..31da4d9d502 100644 --- a/dbms/src/Storages/StorageStripeLog.h +++ b/dbms/src/Storages/StorageStripeLog.h @@ -40,7 +40,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; + void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; @@ -55,7 +55,7 @@ public: String getDataPath() const override { return full_path(); } - void truncate(const ASTPtr &, const Context &) override; + void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; private: String path; diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 45d9771822e..da7013a62c3 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -378,7 +378,7 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type } -void StorageTinyLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) +void StorageTinyLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { std::unique_lock lock(rwlock); @@ -424,7 +424,7 @@ CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context return file_checker.check(); } -void StorageTinyLog::truncate(const ASTPtr &, const Context &) +void StorageTinyLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { if (table_name.empty()) throw Exception("Logical error: table name is empty", ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Storages/StorageTinyLog.h b/dbms/src/Storages/StorageTinyLog.h index 1c148acf957..505edd7c556 100644 --- a/dbms/src/Storages/StorageTinyLog.h +++ b/dbms/src/Storages/StorageTinyLog.h @@ -39,7 +39,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; + void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; @@ -54,7 +54,7 @@ public: String getDataPath() const override { return full_path(); } - void truncate(const ASTPtr &, const Context &) override; + void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; private: String path; From e7cbd34a29ff31d5861d44d0045d9e6749fe26a5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Aug 2019 23:43:35 +0300 Subject: [PATCH 469/509] Improve table locks (incomplete) --- dbms/src/Interpreters/InterpreterRenameQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterRenameQuery.cpp b/dbms/src/Interpreters/InterpreterRenameQuery.cpp index 97b17c6afd7..a3b21063ad7 100644 --- a/dbms/src/Interpreters/InterpreterRenameQuery.cpp +++ b/dbms/src/Interpreters/InterpreterRenameQuery.cpp @@ -93,7 +93,7 @@ BlockIO InterpreterRenameQuery::execute() if (auto table = context.tryGetTable(from.database_name, from.table_name)) tables_from_locks.emplace(from, table->lockExclusively(context.getCurrentQueryId())); - descriptions.back().table_lock = tables_from_locks[from]; + descriptions.back().from_table_lock = tables_from_locks[from]; if (!table_guards.count(from)) table_guards.emplace(from, context.getDDLGuard(from.database_name, from.table_name)); @@ -118,7 +118,7 @@ BlockIO InterpreterRenameQuery::execute() context.assertTableDoesntExist(elem.to_database_name, elem.to_table_name); context.getDatabase(elem.from_database_name)->renameTable( - context, elem.from_table_name, *context.getDatabase(elem.to_database_name), elem.to_table_name, elem.table_lock); + context, elem.from_table_name, *context.getDatabase(elem.to_database_name), elem.to_table_name, elem.from_table_lock); } return {}; From c8798b2ffac4c8d5c36cb39ae2d80e9f4edf93b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 02:47:30 +0300 Subject: [PATCH 470/509] Slightly improve table locks --- dbms/src/Interpreters/InterpreterRenameQuery.cpp | 2 +- dbms/src/Storages/IStorage.cpp | 1 + dbms/src/Storages/Kafka/StorageKafka.cpp | 2 +- dbms/src/Storages/Kafka/StorageKafka.h | 11 ++++------- dbms/src/Storages/StorageBuffer.h | 6 +++++- dbms/src/Storages/StorageFile.cpp | 8 +------- dbms/src/Storages/StorageFile.h | 4 +--- dbms/src/Storages/StorageHDFS.cpp | 2 +- dbms/src/Storages/StorageHDFS.h | 2 +- dbms/src/Storages/StorageMaterializedView.cpp | 4 ++-- dbms/src/Storages/StorageMerge.h | 7 +++++-- dbms/src/Storages/StorageNull.h | 2 +- dbms/src/Storages/StorageURL.cpp | 2 +- dbms/src/Storages/StorageURL.h | 2 +- dbms/src/Storages/StorageView.h | 2 +- dbms/src/Storages/System/StorageSystemColumns.cpp | 1 + dbms/src/Storages/System/StorageSystemPartsBase.cpp | 5 +++++ 17 files changed, 33 insertions(+), 30 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterRenameQuery.cpp b/dbms/src/Interpreters/InterpreterRenameQuery.cpp index a3b21063ad7..e763c002209 100644 --- a/dbms/src/Interpreters/InterpreterRenameQuery.cpp +++ b/dbms/src/Interpreters/InterpreterRenameQuery.cpp @@ -113,7 +113,7 @@ BlockIO InterpreterRenameQuery::execute() if (descriptions.size() > 1) lock = context.getLock(); - for (const auto & elem : descriptions) + for (auto & elem : descriptions) { context.assertTableDoesntExist(elem.to_database_name, elem.to_table_name); diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 2f3a48d90b6..9091008a38b 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; extern const int SETTINGS_ARE_NOT_SUPPORTED; extern const int UNKNOWN_SETTING; + extern const int TABLE_IS_DROPPED; } IStorage::IStorage(ColumnsDescription virtuals_) : virtuals(std::move(virtuals_)) diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 835ce43b1a4..2d55eb42f1e 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -189,7 +189,7 @@ void StorageKafka::shutdown() } -void StorageKafka::rename(const String & /* new_path_to_db */, const String & new_database_name, const String & new_table_name) +void StorageKafka::rename(const String & /* new_path_to_db */, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { table_name = new_table_name; database_name = new_database_name; diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index cd55f28820e..2353d1abe44 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -1,16 +1,14 @@ #pragma once #include -#include -#include #include -#include -#include #include #include #include +#include + namespace DB { @@ -40,10 +38,9 @@ public: BlockOutputStreamPtr write( const ASTPtr & query, - const Context & context - ) override; + const Context & context) override; - void rename(const String & /* new_path_to_db */, const String & new_database_name, const String & new_table_name) override; + void rename(const String & /* new_path_to_db */, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; void updateDependencies() override; diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index b81ca42a0eb..1c565a7d8f0 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -73,7 +73,11 @@ public: void shutdown() override; bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; - void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; } + void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override + { + table_name = new_table_name; + database_name = new_database_name; + } bool supportsSampling() const override { return true; } bool supportsPrewhere() const override diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 2db24bbd610..cfd14c58a2d 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -264,13 +264,7 @@ BlockOutputStreamPtr StorageFile::write( } -void StorageFile::drop() -{ - /// Extra actions are not required. -} - - -void StorageFile::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) +void StorageFile::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { if (!is_db_table) throw Exception("Can't rename table '" + table_name + "' binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED); diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index 7268c8ddff0..1410cc5f215 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -38,9 +38,7 @@ public: const ASTPtr & query, const Context & context) override; - void drop() override; - - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; + void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; String getDataPath() const override { return path; } diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index aa055f7d907..cb25580248f 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -148,7 +148,7 @@ BlockInputStreams StorageHDFS::read( max_block_size)}; } -void StorageHDFS::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) +void StorageHDFS::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { table_name = new_table_name; database_name = new_database_name; diff --git a/dbms/src/Storages/StorageHDFS.h b/dbms/src/Storages/StorageHDFS.h index 30a99c9de70..8361916e0e2 100644 --- a/dbms/src/Storages/StorageHDFS.h +++ b/dbms/src/Storages/StorageHDFS.h @@ -30,7 +30,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; + void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; protected: StorageHDFS(const String & uri_, diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index c3e52f8a32e..87008fce5bf 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -232,7 +232,7 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, } -void StorageMaterializedView::drop() +void StorageMaterializedView::drop(TableStructureWriteLockHolder &) { global_context.removeDependency( DatabaseAndTableName(select_database_name, select_table_name), @@ -242,7 +242,7 @@ void StorageMaterializedView::drop() executeDropQuery(ASTDropQuery::Kind::Drop, global_context, target_database_name, target_table_name); } -void StorageMaterializedView::truncate(const ASTPtr &, const Context &) +void StorageMaterializedView::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { if (has_inner_table) executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, target_database_name, target_table_name); diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index 6708a92c3b0..dbf5d219957 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -42,8 +42,11 @@ public: size_t max_block_size, unsigned num_streams) override; - void drop() override {} - void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; } + void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override + { + table_name = new_table_name; + database_name = new_database_name; + } /// you need to add and remove columns in the sub-tables manually /// the structure of sub-tables is not checked diff --git a/dbms/src/Storages/StorageNull.h b/dbms/src/Storages/StorageNull.h index 04cd5f25e8f..e1a80f3fbaf 100644 --- a/dbms/src/Storages/StorageNull.h +++ b/dbms/src/Storages/StorageNull.h @@ -38,7 +38,7 @@ public: return std::make_shared(getSampleBlock()); } - void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) override + void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override { table_name = new_table_name; database_name = new_database_name; diff --git a/dbms/src/Storages/StorageURL.cpp b/dbms/src/Storages/StorageURL.cpp index ee385af0fe8..4f3d41604f5 100644 --- a/dbms/src/Storages/StorageURL.cpp +++ b/dbms/src/Storages/StorageURL.cpp @@ -187,7 +187,7 @@ BlockInputStreams IStorageURLBase::read(const Names & column_names, return {std::make_shared(block_input, column_defaults, context)}; } -void IStorageURLBase::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) +void IStorageURLBase::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { table_name = new_table_name; database_name = new_database_name; diff --git a/dbms/src/Storages/StorageURL.h b/dbms/src/Storages/StorageURL.h index 2facca8ce38..cdd78c7b60f 100644 --- a/dbms/src/Storages/StorageURL.h +++ b/dbms/src/Storages/StorageURL.h @@ -29,7 +29,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; + void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; protected: IStorageURLBase( diff --git a/dbms/src/Storages/StorageView.h b/dbms/src/Storages/StorageView.h index de56f120fa1..6d2e1d04e6f 100644 --- a/dbms/src/Storages/StorageView.h +++ b/dbms/src/Storages/StorageView.h @@ -30,7 +30,7 @@ public: size_t max_block_size, unsigned num_streams) override; - void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) override + void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override { table_name = new_table_name; database_name = new_database_name; diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index 30b673ddbbb..e4c84de23da 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -18,6 +18,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int TABLE_IS_DROPPED; } StorageSystemColumns::StorageSystemColumns(const std::string & name_) diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.cpp b/dbms/src/Storages/System/StorageSystemPartsBase.cpp index bced500a072..69d11891198 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsBase.cpp @@ -17,6 +17,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int TABLE_IS_DROPPED; +} + bool StorageSystemPartsBase::hasStateColumn(const Names & column_names) const { bool has_state_column = false; From 3db38c690e5c012afa2a5623c50d4b10cbc8fbdf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 03:15:08 +0300 Subject: [PATCH 471/509] Changes to Benchmark after merge --- dbms/programs/benchmark/Benchmark.cpp | 4 +- .../src/Common/{T_test.h => StudentTTest.cpp} | 145 ++++++------------ dbms/src/Common/StudentTTest.h | 59 +++++++ 3 files changed, 111 insertions(+), 97 deletions(-) rename dbms/src/Common/{T_test.h => StudentTTest.cpp} (64%) create mode 100644 dbms/src/Common/StudentTTest.h diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index cb034a258af..fedb7f778a1 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -32,7 +32,7 @@ #include #include #include -#include +#include /** A tool for evaluating ClickHouse performance. @@ -192,7 +192,7 @@ private: using MultiStats = std::vector>; MultiStats comparison_info_per_interval; MultiStats comparison_info_total; - T_test t_test; + StudentTTest t_test; Stopwatch total_watch; Stopwatch delay_watch; diff --git a/dbms/src/Common/T_test.h b/dbms/src/Common/StudentTTest.cpp similarity index 64% rename from dbms/src/Common/T_test.h rename to dbms/src/Common/StudentTTest.cpp index 1160fd0805e..170f06c2877 100644 --- a/dbms/src/Common/T_test.h +++ b/dbms/src/Common/StudentTTest.cpp @@ -1,55 +1,14 @@ -#pragma once +#include "StudentTTest.h" -#include #include #include #include #include +#include -/** - * About: - * This is implementation of Independent two-sample t-test - * Read about it on https://en.wikipedia.org/wiki/Student%27s_t-test (Equal or unequal sample sizes, equal variance) - * - * Usage: - * It's it used to assume with some level of confidence that two distributions don't differ. - * Values can be added with T_test.add(0/1, value) and after compared and reported with compareAndReport(). - */ -struct T_test + +namespace { - struct DistributionData - { - size_t size = 0; - double sum = 0; - double squares_sum = 0; - - void add(double value) - { - ++size; - sum += value; - squares_sum += value * value; - } - - double avg() const - { - return sum / size; - } - - double var() const - { - return (squares_sum - (sum * sum / size)) / static_cast(size - 1); - } - - void clear() - { - size = 0; - sum = 0; - squares_sum = 0; - } - }; - - std::vector data; - /// First row corresponds to infinity size of distributions case const double students_table[101][6] = { @@ -156,59 +115,55 @@ struct T_test { 1.290, 1.660, 1.984, 2.364, 2.626, 3.174 }, }; - const std::vector confidence_level = { 80, 90, 95, 98, 99, 99.5 }; + const double confidence_level[6] = { 80, 90, 95, 98, 99, 99.5 }; +} - T_test() + +void StudentTTest::clear() +{ + data[0].clear(); + data[1].clear(); +} + +void StudentTTest::add(size_t distribution, double value) +{ + if (distribution > 1) + throw std::logic_error("Distribution number for Student's T-Test must be eigther 0 or 1"); + data[distribution].add(value); +} + +/// Confidence_level_index can be set in range [0, 5]. Corresponding values can be found above. +std::pair StudentTTest::compareAndReport(size_t confidence_level_index) const +{ + if (confidence_level_index > 5) + confidence_level_index = 5; + + if (data[0].size == 0 || data[1].size == 0) + return {true, ""}; + + size_t degrees_of_freedom = (data[0].size - 1) + (data[1].size - 1); + + double table_value = students_table[degrees_of_freedom > 100 ? 0 : degrees_of_freedom][confidence_level_index]; + + double pooled_standard_deviation = sqrt(((data[0].size - 1) * data[0].var() + (data[1].size - 1) * data[1].var()) / degrees_of_freedom); + + double t_statistic = pooled_standard_deviation * sqrt(1.0 / data[0].size + 1.0 / data[1].size); + + double mean_difference = fabs(data[0].avg() - data[1].avg()); + + double mean_confidence_interval = table_value * t_statistic; + + std::stringstream ss; + if (mean_difference > mean_confidence_interval && (mean_difference - mean_confidence_interval > 0.0001)) /// difference must be more than 0.0001, to take into account connection latency. { - data.resize(2); + ss << "Difference at " << confidence_level[confidence_level_index] << "% confidence : "; + ss << std::fixed << std::setprecision(8) << "mean difference is " << mean_difference << ", but confidence interval is " << mean_confidence_interval; + return {false, ss.str()}; } - - void clear() + else { - data[0].clear(); - data[1].clear(); + ss << "No difference proven at " << confidence_level[confidence_level_index] << "% confidence"; + return {true, ss.str()}; } +} - void add(size_t distribution, double value) - { - if (distribution > 1) - return; - data[distribution].add(value); - } - - /// Confidence_level_index can be set in range [0, 5]. Corresponding values can be found above. - std::pair compareAndReport(size_t confidence_level_index = 5) const - { - if (confidence_level_index > 5) - confidence_level_index = 5; - - if (data[0].size == 0 || data[1].size == 0) - return {true, ""}; - - size_t degrees_of_freedom = (data[0].size - 1) + (data[1].size - 1); - - double table_value = students_table[degrees_of_freedom > 100 ? 0 : degrees_of_freedom][confidence_level_index]; - - double pooled_standard_deviation = sqrt(((data[0].size - 1) * data[0].var() + (data[1].size - 1) * data[1].var()) / degrees_of_freedom); - - double t_statistic = pooled_standard_deviation * sqrt(1.0 / data[0].size + 1.0 / data[1].size); - - double mean_difference = fabs(data[0].avg() - data[1].avg()); - - double mean_confidence_interval = table_value * t_statistic; - - std::stringstream ss; - if (mean_difference > mean_confidence_interval && (mean_difference - mean_confidence_interval > 0.0001)) /// difference must be more than 0.0001, to take into account connection latency. - { - ss << "Difference at " << confidence_level[confidence_level_index] << "% confidence : "; - ss << std::fixed << std::setprecision(8) << "mean difference is " << mean_difference << ", but confidence interval is " << mean_confidence_interval; - return {false, ss.str()}; - } - else - { - ss << "No difference proven at " << confidence_level[confidence_level_index] << "% confidence"; - return {true, ss.str()}; - } - } - -}; diff --git a/dbms/src/Common/StudentTTest.h b/dbms/src/Common/StudentTTest.h new file mode 100644 index 00000000000..0816268fba7 --- /dev/null +++ b/dbms/src/Common/StudentTTest.h @@ -0,0 +1,59 @@ +#pragma once + +#include +#include +#include + +/** + * About: + * This is implementation of Independent two-sample t-test + * Read about it on https://en.wikipedia.org/wiki/Student%27s_t-test (Equal or unequal sample sizes, equal variance) + * + * Usage: + * It's it used to assume with some level of confidence that two distributions don't differ. + * Values can be added with t_test.add(0/1, value) and after compared and reported with compareAndReport(). + */ +class StudentTTest +{ +private: + struct DistributionData + { + size_t size = 0; + double sum = 0; + double squares_sum = 0; + + void add(double value) + { + ++size; + sum += value; + squares_sum += value * value; + } + + double avg() const + { + return sum / size; + } + + double var() const + { + return (squares_sum - (sum * sum / size)) / static_cast(size - 1); + } + + void clear() + { + size = 0; + sum = 0; + squares_sum = 0; + } + }; + + std::array data; + +public: + void clear(); + + void add(size_t distribution, double value); + + /// Confidence_level_index can be set in range [0, 5]. Corresponding values can be found above. TODO: Trash - no separation of concepts in code. + std::pair compareAndReport(size_t confidence_level_index = 5) const; +}; From f2d081a7857064e3af74e8e41f076810e302361b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 03:42:45 +0300 Subject: [PATCH 472/509] Addition to prev. revision --- dbms/src/Common/StudentTTest.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/StudentTTest.h b/dbms/src/Common/StudentTTest.h index 0816268fba7..b09190050b5 100644 --- a/dbms/src/Common/StudentTTest.h +++ b/dbms/src/Common/StudentTTest.h @@ -47,7 +47,7 @@ private: } }; - std::array data; + std::array data {}; public: void clear(); From 32c7f9688aa5f6bbc60164c82795bd5aed04c304 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 13:13:22 +0300 Subject: [PATCH 473/509] Removed extra quoting from Settings --- dbms/src/Core/Settings.h | 2 +- dbms/src/Core/SettingsCommon.h | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 88a81027cb6..aa6893d6d85 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -302,7 +302,7 @@ struct Settings : public SettingsCollection M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.") \ M(SettingBool, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.") \ M(SettingBool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.") \ - M(SettingBool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \N") \ + M(SettingBool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \\N") \ \ M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \ M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.") \ diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index b8c56d50caa..97edfbe9934 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -695,7 +695,7 @@ public: #define IMPLEMENT_SETTINGS_COLLECTION_ADD_MUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ add({[](const Derived & d) { return d.NAME.changed; }, \ - StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), true, \ + StringRef(#NAME, strlen(#NAME)), StringRef(DESCRIPTION, strlen(DESCRIPTION)), true, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \ &Functions::NAME##_serialize, &Functions::NAME##_deserialize, \ @@ -703,7 +703,7 @@ public: #define IMPLEMENT_SETTINGS_COLLECTION_ADD_IMMUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ add({[](const Derived & d) { return d.NAME.changed; }, \ - StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), false, \ + StringRef(#NAME, strlen(#NAME)), StringRef(DESCRIPTION, strlen(DESCRIPTION)), false, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \ &Functions::NAME##_serialize, &Functions::NAME##_deserialize, \ From 24a8755b828a42c00aa0bbc501e02bb792c1c548 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 2 Aug 2019 19:30:09 +0300 Subject: [PATCH 474/509] Formatting changes for StringHashMap PR #5417. --- .../AggregateFunctionGroupUniqArray.h | 2 +- dbms/src/Columns/ColumnLowCardinality.cpp | 2 +- dbms/src/Common/HashTable/FixedHashMap.h | 2 +- dbms/src/Common/HashTable/HashMap.h | 18 ++++++------------ dbms/src/Common/HashTable/HashTable.h | 1 - dbms/src/Interpreters/Aggregator.h | 8 -------- 6 files changed, 9 insertions(+), 24 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index 4b52f1e6fd9..38b67efd6dc 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -90,7 +90,7 @@ public: auto & set = this->data(place).value; size_t size = set.size(); writeVarUInt(size, buf); - for (auto & elem : set) + for (const auto & elem : set) writeIntBinary(elem, buf); } diff --git a/dbms/src/Columns/ColumnLowCardinality.cpp b/dbms/src/Columns/ColumnLowCardinality.cpp index fa713f76e5e..32ba2378100 100644 --- a/dbms/src/Columns/ColumnLowCardinality.cpp +++ b/dbms/src/Columns/ColumnLowCardinality.cpp @@ -34,7 +34,7 @@ namespace auto & data = res_col->getData(); data.resize(hash_map.size()); - for (auto val : hash_map) + for (const auto & val : hash_map) data[val.getSecond()] = val.getFirst(); for (auto & ind : index) diff --git a/dbms/src/Common/HashTable/FixedHashMap.h b/dbms/src/Common/HashTable/FixedHashMap.h index ae076ddb877..d50c87a6583 100644 --- a/dbms/src/Common/HashTable/FixedHashMap.h +++ b/dbms/src/Common/HashTable/FixedHashMap.h @@ -11,8 +11,8 @@ struct FixedHashMapCell using State = TState; using value_type = PairNoInit; - bool full; Mapped mapped; + bool full; FixedHashMapCell() {} FixedHashMapCell(const Key &, const State &) : full(true) {} diff --git a/dbms/src/Common/HashTable/HashMap.h b/dbms/src/Common/HashTable/HashMap.h index f82563c4449..98669619d3d 100644 --- a/dbms/src/Common/HashTable/HashMap.h +++ b/dbms/src/Common/HashTable/HashMap.h @@ -128,14 +128,12 @@ struct HashMapCellWithSavedHash : public HashMapCell }; -template -< +template < typename Key, typename Cell, typename Hash = DefaultHash, typename Grower = HashTableGrower<>, - typename Allocator = HashTableAllocator -> + typename Allocator = HashTableAllocator> class HashMapTable : public HashTable { public: @@ -173,23 +171,19 @@ public: }; -template -< +template < typename Key, typename Mapped, typename Hash = DefaultHash, typename Grower = HashTableGrower<>, - typename Allocator = HashTableAllocator -> + typename Allocator = HashTableAllocator> using HashMap = HashMapTable, Hash, Grower, Allocator>; -template -< +template < typename Key, typename Mapped, typename Hash = DefaultHash, typename Grower = HashTableGrower<>, - typename Allocator = HashTableAllocator -> + typename Allocator = HashTableAllocator> using HashMapWithSavedHash = HashMapTable, Hash, Grower, Allocator>; diff --git a/dbms/src/Common/HashTable/HashTable.h b/dbms/src/Common/HashTable/HashTable.h index c5a0c812ee2..d29459a90d5 100644 --- a/dbms/src/Common/HashTable/HashTable.h +++ b/dbms/src/Common/HashTable/HashTable.h @@ -95,7 +95,6 @@ struct HashTableCell /// Create a cell with the given key / key and value. HashTableCell(const Key & key_, const State &) : key(key_) {} -/// HashTableCell(const value_type & value_, const State & state) : key(value_) {} /// Get what the value_type of the container will be. value_type & getValueMutable() { return key; } diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index c3d1d5df8fd..29e24ab282a 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -196,8 +196,6 @@ struct AggregationMethodString using Data = TData; using Key = typename Data::key_type; using Mapped = typename Data::mapped_type; - using iterator = typename Data::iterator; - using const_iterator = typename Data::const_iterator; Data data; @@ -224,8 +222,6 @@ struct AggregationMethodFixedString using Data = TData; using Key = typename Data::key_type; using Mapped = typename Data::mapped_type; - using iterator = typename Data::iterator; - using const_iterator = typename Data::const_iterator; Data data; @@ -254,8 +250,6 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod using Data = typename Base::Data; using Key = typename Base::Key; using Mapped = typename Base::Mapped; - using iterator = typename Base::iterator; - using const_iterator = typename Base::const_iterator; using Base::data; @@ -365,8 +359,6 @@ struct AggregationMethodSerialized using Data = TData; using Key = typename Data::key_type; using Mapped = typename Data::mapped_type; - using iterator = typename Data::iterator; - using const_iterator = typename Data::const_iterator; Data data; From 0b5a7f5f4a0aa0b42b9890160d49444eaed8f13d Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 28 Aug 2019 14:03:29 +0300 Subject: [PATCH 475/509] Disable consecutive key optimization for UInt8/16 LowCardinality columns. A follow-up to #6298, this is as well a part of StringHashMap PR #5417 by Amos Bird. --- dbms/src/Interpreters/Aggregator.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index c3d1d5df8fd..6a27c426283 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -460,8 +460,8 @@ struct AggregatedDataVariants : private boost::noncopyable std::unique_ptr> nullable_keys256_two_level; /// Support for low cardinality. - std::unique_ptr>> low_cardinality_key8; - std::unique_ptr>> low_cardinality_key16; + std::unique_ptr>> low_cardinality_key8; + std::unique_ptr>> low_cardinality_key16; std::unique_ptr>> low_cardinality_key32; std::unique_ptr>> low_cardinality_key64; std::unique_ptr>> low_cardinality_key_string; From 99d62efb4d543f265b91258cc53f3c1ab228e966 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 15:26:13 +0300 Subject: [PATCH 476/509] Added a test --- .../queries/0_stateless/00999_settings_no_extra_quotes.reference | 1 + .../tests/queries/0_stateless/00999_settings_no_extra_quotes.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00999_settings_no_extra_quotes.reference create mode 100644 dbms/tests/queries/0_stateless/00999_settings_no_extra_quotes.sql diff --git a/dbms/tests/queries/0_stateless/00999_settings_no_extra_quotes.reference b/dbms/tests/queries/0_stateless/00999_settings_no_extra_quotes.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00999_settings_no_extra_quotes.reference @@ -0,0 +1 @@ +0 diff --git a/dbms/tests/queries/0_stateless/00999_settings_no_extra_quotes.sql b/dbms/tests/queries/0_stateless/00999_settings_no_extra_quotes.sql new file mode 100644 index 00000000000..55d9ff2780d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00999_settings_no_extra_quotes.sql @@ -0,0 +1 @@ +SELECT DISTINCT description LIKE '"%"' FROM system.settings; From a1101e5278b24781e97fded4251ffe5a04e7b9c5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 16:21:19 +0300 Subject: [PATCH 477/509] Fixed build --- dbms/src/Databases/DatabasesCommon.h | 1 + dbms/src/Storages/Kafka/KafkaBlockInputStream.h | 2 ++ dbms/src/Storages/Kafka/StorageKafka.h | 2 ++ 3 files changed, 5 insertions(+) diff --git a/dbms/src/Databases/DatabasesCommon.h b/dbms/src/Databases/DatabasesCommon.h index 00190d89b1e..734708e4c95 100644 --- a/dbms/src/Databases/DatabasesCommon.h +++ b/dbms/src/Databases/DatabasesCommon.h @@ -4,6 +4,7 @@ #include #include #include +#include /// General functionality for several different database engines. diff --git a/dbms/src/Storages/Kafka/KafkaBlockInputStream.h b/dbms/src/Storages/Kafka/KafkaBlockInputStream.h index fef7f8d0469..011ed5fe046 100644 --- a/dbms/src/Storages/Kafka/KafkaBlockInputStream.h +++ b/dbms/src/Storages/Kafka/KafkaBlockInputStream.h @@ -4,6 +4,8 @@ #include #include +#include + namespace DB { diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index 2353d1abe44..51a06a890db 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include From 9d0e5b925bb2376d5e0a548452eb49a6fce14bd6 Mon Sep 17 00:00:00 2001 From: akonyaev Date: Wed, 28 Aug 2019 16:26:38 +0300 Subject: [PATCH 478/509] ADQM-40 test for orc input format --- .../0_stateless/00900_orc_load.reference | 2 ++ .../queries/0_stateless/00900_orc_load.sh | 17 +++++++++++++++++ .../queries/0_stateless/data_orc/test.orc | Bin 0 -> 771 bytes 3 files changed, 19 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00900_orc_load.reference create mode 100755 dbms/tests/queries/0_stateless/00900_orc_load.sh create mode 100644 dbms/tests/queries/0_stateless/data_orc/test.orc diff --git a/dbms/tests/queries/0_stateless/00900_orc_load.reference b/dbms/tests/queries/0_stateless/00900_orc_load.reference new file mode 100644 index 00000000000..fe79e37ee18 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00900_orc_load.reference @@ -0,0 +1,2 @@ +0 0 0 0 0 2019-01-01 test1 +2147483647 -1 9223372036854775806 123.345345 345345.3453451212 2019-01-01 test2 diff --git a/dbms/tests/queries/0_stateless/00900_orc_load.sh b/dbms/tests/queries/0_stateless/00900_orc_load.sh new file mode 100755 index 00000000000..cd553f6d234 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00900_orc_load.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CUR_DIR/../shell_config.sh + +CB_DIR=$(dirname "$CLICKHOUSE_CLIENT_BINARY") +[ "$CB_DIR" == "." ] && ROOT_DIR=$CUR_DIR/../../../.. +[ "$CB_DIR" != "." ] && BUILD_DIR=$CB_DIR/../.. +[ -z "$ROOT_DIR" ] && ROOT_DIR=$CB_DIR/../../.. + +DATA_FILE=$CUR_DIR/data_orc/test.orc + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (int Int32, smallint Int8, bigint Int64, float Float32, double Float64, date Date, y String) ENGINE = Memory" +cat $DATA_FILE | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" +${CLICKHOUSE_CLIENT} --query="select * from orc_load" + diff --git a/dbms/tests/queries/0_stateless/data_orc/test.orc b/dbms/tests/queries/0_stateless/data_orc/test.orc new file mode 100644 index 0000000000000000000000000000000000000000..1b2c9aa492203b672ef40a48424ed1128e615544 GIT binary patch literal 771 zcmeYdau#G@;9?VE;b012uwY<#Jn30dT0(+=(8C0wM~@Q(etiG`zrK#sK=J?sgE3I0 zI2SVm1A~wN2a^yN2ZO-B|I89V9Kc`()Ee{@taZ}k1ffX}fLcGxb4jQQoMB*NumQ*kLL-n4Hxu6zS?BtD4vYG(sO}xhTYP zJ`N@;4vtIHnFN41fWZpn>gP$R42+6~d}^wUdeA%1Sb_QS=fWXBK-<+fv7?Kkj8eo<)f;Dts>b?Xt zrX;nv#1KRnu`@8ZF^0}#VDLTT&DCJQ!*W4wZRT2zqgNFkz5my4!WUa&XfsK1W!M^4 zov+~qfkz#c{bqf*z^QNg#^iC?(f47iW@Nr8eiZ+4x7*t9GxzS=mdPn6D|GBG`wyUB zZXG^x^w2?(0|!(O9aK4T{Gh^z_du8b5lRr!&|(mI%G7Pdd~`;mQjvoo=YkXS{|6k< zXiQ*wDrz}n$r4M+1r7ooqD$5twe&b7F~h5GrLI?ss9@=o7Q+Uw9~Lv#SXk)!rVBA7 zDa7|QdwMo>Z&lg4<46F9kECY@iva_}21bU*31Tx2=o~$JQ0K#hKqW6lk7m!4UCr(t zb0>DrY@OO0+hu9I-lc1?g}I?R`;jE?Q$Fk$*JN7$>~zTUs>=-WTP5P{mE^ffBzdWj zFuSpUF{2^3z-A^xX@SqohPxRji*mCEGc-@-V4o}?&)$4sKhR^Jy&F6itY8S3!mV?P zMja~#N`q4UKWE0pEKQtK!VxJZY{D+F=L($330juRwrNZZb-5BKbc{jk;p2o& zj~?tWnd`~W(O6@+;5F-x2=Ab7Z=TizQzo`}Ff<=#v1j18%_zVq(a<2E#Kgd(QE$N3 M^o5z(Kgd}E06(l5t^fc4 literal 0 HcmV?d00001 From 90d5410aaa6e7e641efe7b67e0496ab53f2a64ab Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 28 Aug 2019 17:14:09 +0300 Subject: [PATCH 479/509] join_use_nulls with not nullable types --- dbms/src/Interpreters/ExpressionActions.cpp | 8 ++--- dbms/src/Interpreters/Join.cpp | 4 +-- .../00999_join_not_nullable_types.reference | 8 +++++ .../00999_join_not_nullable_types.sql | 34 +++++++++++++++++++ 4 files changed, 48 insertions(+), 6 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00999_join_not_nullable_types.reference create mode 100644 dbms/tests/queries/0_stateless/00999_join_not_nullable_types.sql diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 5ef05569f91..0083820d6e9 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -291,8 +291,8 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings, bool make_nullable = is_null_used_as_default && right_or_full_join; - if (make_nullable && !col.type->isNullable()) - col.type = std::make_shared(col.type); + if (make_nullable && col.type->canBeInsideNullable()) + col.type = makeNullable(col.type); } for (const auto & col : columns_added_by_join) @@ -316,8 +316,8 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings, } } - if (make_nullable && !res_type->isNullable()) - res_type = std::make_shared(res_type); + if (make_nullable && res_type->canBeInsideNullable()) + res_type = makeNullable(res_type); sample_block.insert(ColumnWithTypeAndName(nullptr, res_type, col.name)); } diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 8e91424bc21..63bf88a8437 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -50,7 +50,7 @@ static std::unordered_map requiredRightKeys(const Names & k static void convertColumnToNullable(ColumnWithTypeAndName & column) { - if (column.type->isNullable()) + if (column.type->isNullable() || !column.type->canBeInsideNullable()) return; column.type = makeNullable(column.type); @@ -71,7 +71,7 @@ static ColumnWithTypeAndName correctNullability(ColumnWithTypeAndName && column, if (nullable) { convertColumnToNullable(column); - if (negative_null_map.size()) + if (column.type->isNullable() && negative_null_map.size()) { MutableColumnPtr mutable_column = (*std::move(column.column)).mutate(); assert_cast(*mutable_column).applyNegatedNullMap(negative_null_map); diff --git a/dbms/tests/queries/0_stateless/00999_join_not_nullable_types.reference b/dbms/tests/queries/0_stateless/00999_join_not_nullable_types.reference new file mode 100644 index 00000000000..7b6947fa9a2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00999_join_not_nullable_types.reference @@ -0,0 +1,8 @@ +0 ['left'] 0 ['left'] \N +1 ['left'] 1 ['left'] 1 +2 [] \N [] 2 +['left'] 0 ['left'] \N +['left'] 1 ['left'] 1 +[] \N [] 2 +['left'] 42 \N +['right'] \N 42 diff --git a/dbms/tests/queries/0_stateless/00999_join_not_nullable_types.sql b/dbms/tests/queries/0_stateless/00999_join_not_nullable_types.sql new file mode 100644 index 00000000000..2a24c6dd296 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00999_join_not_nullable_types.sql @@ -0,0 +1,34 @@ +SET join_use_nulls = 1; + +SELECT * FROM +( + SELECT number, ['left'] as ar, number AS left_number FROM system.numbers LIMIT 2 +) +FULL JOIN +( + SELECT number, ['right'] as ar, number AS right_number FROM system.numbers LIMIT 1, 2 +) +USING (number) +ORDER BY number; + +SELECT * FROM +( + SELECT ['left'] as ar, number AS left_number FROM system.numbers LIMIT 2 +) +FULL JOIN +( + SELECT ['right'] as ar, number AS right_number FROM system.numbers LIMIT 1, 2 +) +ON left_number = right_number +ORDER BY left_number; + +SELECT * FROM +( + SELECT ['left'] as ar, 42 AS left_number +) +FULL JOIN +( + SELECT ['right'] as ar, 42 AS right_number +) +USING(ar) +ORDER BY left_number; From 10b878b30b9ef7237b269487d843f6b662036878 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 18:05:38 +0300 Subject: [PATCH 480/509] Disable Poco::AbstractConfiguration substitutions in query in clickhouse-client --- dbms/programs/client/Client.cpp | 2 +- dbms/programs/local/LocalServer.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 6caf57a28bd..0bb6cf62f90 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -672,7 +672,7 @@ private: String text; if (config().has("query")) - text = config().getString("query"); + text = config().getRawString("query"); /// Poco configuration should not process substitutions in form of ${...} inside query. else { /// If 'query' parameter is not set, read a query from stdin. diff --git a/dbms/programs/local/LocalServer.cpp b/dbms/programs/local/LocalServer.cpp index 54383050b6c..1844c037784 100644 --- a/dbms/programs/local/LocalServer.cpp +++ b/dbms/programs/local/LocalServer.cpp @@ -268,7 +268,7 @@ void LocalServer::attachSystemTables() void LocalServer::processQueries() { String initial_create_query = getInitialCreateTableQuery(); - String queries_str = initial_create_query + config().getString("query"); + String queries_str = initial_create_query + config().getRawString("query"); std::vector queries; auto parse_res = splitMultipartQuery(queries_str, queries); From f519234908a585895a6103f8b1307ab3479c6b05 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 18:13:19 +0300 Subject: [PATCH 481/509] Added a test --- .../01000_unneeded_substitutions_client.reference | 1 + .../0_stateless/01000_unneeded_substitutions_client.sh | 6 ++++++ 2 files changed, 7 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01000_unneeded_substitutions_client.reference create mode 100755 dbms/tests/queries/0_stateless/01000_unneeded_substitutions_client.sh diff --git a/dbms/tests/queries/0_stateless/01000_unneeded_substitutions_client.reference b/dbms/tests/queries/0_stateless/01000_unneeded_substitutions_client.reference new file mode 100644 index 00000000000..13a393df666 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01000_unneeded_substitutions_client.reference @@ -0,0 +1 @@ +${} diff --git a/dbms/tests/queries/0_stateless/01000_unneeded_substitutions_client.sh b/dbms/tests/queries/0_stateless/01000_unneeded_substitutions_client.sh new file mode 100755 index 00000000000..f6517fc2a42 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01000_unneeded_substitutions_client.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "SELECT '\${}'" From d06e3503123f616f0a95b36aff4d7406db476625 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 18:20:22 +0300 Subject: [PATCH 482/509] Added setting "replace_running_query_max_wait_ms" --- dbms/src/Core/Settings.h | 1 + dbms/src/Interpreters/ProcessList.cpp | 10 +++++++--- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 92f882e4fc5..f6b65ed19fe 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -62,6 +62,7 @@ struct Settings : public SettingsCollection M(SettingSeconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "") \ M(SettingSeconds, tcp_keep_alive_timeout, 0, "") \ M(SettingMilliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.") \ + M(SettingMilliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.") \ M(SettingUInt64, poll_interval, DBMS_DEFAULT_POLL_INTERVAL, "Block at the query wait loop on the server for the specified number of seconds.") \ M(SettingUInt64, idle_connection_timeout, 3600, "Close idle TCP connections after specified number of seconds.") \ M(SettingUInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.") \ diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index 5a13477147c..71376c6d129 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -87,10 +87,10 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as { std::unique_lock lock(mutex); - const auto max_wait_ms = settings.queue_max_wait_ms.totalMilliseconds(); + const auto queue_max_wait_ms = settings.queue_max_wait_ms.totalMilliseconds(); if (!is_unlimited_query && max_size && processes.size() >= max_size) { - if (!max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(max_wait_ms), [&]{ return processes.size() < max_size; })) + if (!queue_max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(queue_max_wait_ms), [&]{ return processes.size() < max_size; })) throw Exception("Too many simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES); } @@ -127,7 +127,9 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as /// Ask queries to cancel. They will check this flag. running_query->second->is_killed.store(true, std::memory_order_relaxed); - if (!max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(max_wait_ms), [&] + const auto replace_running_query_max_wait_ms = settings.replace_running_query_max_wait_ms.totalMilliseconds(); + if (!replace_running_query_max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(replace_running_query_max_wait_ms), + [&] { running_query = user_process_list->second.queries.find(client_info.current_query_id); if (running_query == user_process_list->second.queries.end()) @@ -135,8 +137,10 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as running_query->second->is_killed.store(true, std::memory_order_relaxed); return false; })) + { throw Exception("Query with id = " + client_info.current_query_id + " is already running and can't be stopped", ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING); + } } } } From 06c0e35e46b4cebcb545112573801fb61dc2a2c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 18:21:57 +0300 Subject: [PATCH 483/509] Added setting "replace_running_query_max_wait_ms" --- dbms/tests/queries/0_stateless/00600_replace_running_query.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00600_replace_running_query.sh b/dbms/tests/queries/0_stateless/00600_replace_running_query.sh index dbbf41dd772..9fc25291548 100755 --- a/dbms/tests/queries/0_stateless/00600_replace_running_query.sh +++ b/dbms/tests/queries/0_stateless/00600_replace_running_query.sh @@ -35,6 +35,6 @@ wait ${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 3, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & wait_for_query_to_start '42' -${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --queue_max_wait_ms=500 --query='SELECT 43' 2>&1 | grep -F "can't be stopped" > /dev/null +${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --replace_running_query_max_wait_ms=500 --query='SELECT 43' 2>&1 | grep -F "can't be stopped" > /dev/null ${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --query='SELECT 44' wait From d1c268c2ed030c3346b54dd9cdf680032e6ac612 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 18:24:23 +0300 Subject: [PATCH 484/509] Added setting "kafka_max_wait_ms" --- dbms/src/Core/Settings.h | 1 + dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index f6b65ed19fe..a1e5ad73c46 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -63,6 +63,7 @@ struct Settings : public SettingsCollection M(SettingSeconds, tcp_keep_alive_timeout, 0, "") \ M(SettingMilliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.") \ M(SettingMilliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.") \ + M(SettingMilliseconds, kafka_max_wait_ms, 5000, "The wait time for reading from Kafka before retry.") \ M(SettingUInt64, poll_interval, DBMS_DEFAULT_POLL_INTERVAL, "Block at the query wait loop on the server for the specified number of seconds.") \ M(SettingUInt64, idle_connection_timeout, 3600, "Close idle TCP connections after specified number of seconds.") \ M(SettingUInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.") \ diff --git a/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp b/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp index 1962e4fbc63..09f97f8c836 100644 --- a/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp +++ b/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp @@ -43,7 +43,7 @@ Block KafkaBlockInputStream::getHeader() const void KafkaBlockInputStream::readPrefixImpl() { - auto timeout = std::chrono::milliseconds(context.getSettingsRef().queue_max_wait_ms.totalMilliseconds()); + auto timeout = std::chrono::milliseconds(context.getSettingsRef().kafka_max_wait_ms.totalMilliseconds()); buffer = storage.popReadBuffer(timeout); claimed = !!buffer; From cced091881743354abbd0a05626a6bff3552484e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 18:24:35 +0300 Subject: [PATCH 485/509] Addition to prev. revision --- dbms/src/Interpreters/ClusterProxy/executeQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp b/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp index 25d5a6eb0d4..d2c11fd0feb 100644 --- a/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -18,6 +18,7 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin { Settings new_settings = settings; new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.max_execution_time); + new_settings.replace_running_query_max_wait_ms = Cluster::saturate(new_settings.replace_running_query_max_wait_ms, settings.max_execution_time); /// Does not matter on remote servers, because queries are sent under different user. new_settings.max_concurrent_queries_for_user = 0; From b6a0dba2df192f5f434c76e0bbcb0314c13ad808 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 18:27:26 +0300 Subject: [PATCH 486/509] Added a setting "connection_pool_max_wait_ms" --- dbms/src/Client/ConnectionPool.h | 2 +- dbms/src/Core/Settings.h | 1 + dbms/src/Interpreters/ClusterProxy/executeQuery.cpp | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Client/ConnectionPool.h b/dbms/src/Client/ConnectionPool.h index 322bad04794..1ecb432c827 100644 --- a/dbms/src/Client/ConnectionPool.h +++ b/dbms/src/Client/ConnectionPool.h @@ -74,7 +74,7 @@ public: { Entry entry; if (settings) - entry = Base::get(settings->queue_max_wait_ms.totalMilliseconds()); + entry = Base::get(settings->connection_pool_max_wait_ms.totalMilliseconds()); else entry = Base::get(-1); diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index a1e5ad73c46..0be8279d3e0 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -62,6 +62,7 @@ struct Settings : public SettingsCollection M(SettingSeconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "") \ M(SettingSeconds, tcp_keep_alive_timeout, 0, "") \ M(SettingMilliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.") \ + M(SettingMilliseconds, connection_pool_max_wait_ms, 0, "The wait time when connection pool is full.") \ M(SettingMilliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.") \ M(SettingMilliseconds, kafka_max_wait_ms, 5000, "The wait time for reading from Kafka before retry.") \ M(SettingUInt64, poll_interval, DBMS_DEFAULT_POLL_INTERVAL, "Block at the query wait loop on the server for the specified number of seconds.") \ diff --git a/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp b/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp index d2c11fd0feb..dc0d3ef27b1 100644 --- a/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -18,6 +18,7 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin { Settings new_settings = settings; new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.max_execution_time); + new_settings.connection_pool_max_wait_ms = Cluster::saturate(new_settings.connection_pool_max_wait_ms, settings.max_execution_time); new_settings.replace_running_query_max_wait_ms = Cluster::saturate(new_settings.replace_running_query_max_wait_ms, settings.max_execution_time); /// Does not matter on remote servers, because queries are sent under different user. From 5e27937227b211865cdb3e169495665fc2c02e23 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 28 Aug 2019 18:44:18 +0300 Subject: [PATCH 487/509] fix RIGHT JOIN for Tuples --- dbms/src/Columns/ColumnTuple.cpp | 10 +++++ dbms/src/Columns/ColumnTuple.h | 1 + ...00999_nullable_nested_types_4877.reference | 26 ++++++++++++ .../00999_nullable_nested_types_4877.sql | 41 +++++++++++++++++++ 4 files changed, 78 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00999_nullable_nested_types_4877.reference create mode 100644 dbms/tests/queries/0_stateless/00999_nullable_nested_types_4877.sql diff --git a/dbms/src/Columns/ColumnTuple.cpp b/dbms/src/Columns/ColumnTuple.cpp index 07599b3456f..3e3e311270f 100644 --- a/dbms/src/Columns/ColumnTuple.cpp +++ b/dbms/src/Columns/ColumnTuple.cpp @@ -81,6 +81,16 @@ MutableColumnPtr ColumnTuple::cloneEmpty() const return ColumnTuple::create(std::move(new_columns)); } +MutableColumnPtr ColumnTuple::cloneResized(size_t new_size) const +{ + const size_t tuple_size = columns.size(); + MutableColumns new_columns(tuple_size); + for (size_t i = 0; i < tuple_size; ++i) + new_columns[i] = columns[i]->cloneResized(new_size); + + return ColumnTuple::create(std::move(new_columns)); +} + Field ColumnTuple::operator[](size_t n) const { return Tuple{ext::map(columns, [n] (const auto & column) { return (*column)[n]; })}; diff --git a/dbms/src/Columns/ColumnTuple.h b/dbms/src/Columns/ColumnTuple.h index 65dd19fc6da..e5e47ac74db 100644 --- a/dbms/src/Columns/ColumnTuple.h +++ b/dbms/src/Columns/ColumnTuple.h @@ -42,6 +42,7 @@ public: const char * getFamilyName() const override { return "Tuple"; } MutableColumnPtr cloneEmpty() const override; + MutableColumnPtr cloneResized(size_t size) const override; size_t size() const override { diff --git a/dbms/tests/queries/0_stateless/00999_nullable_nested_types_4877.reference b/dbms/tests/queries/0_stateless/00999_nullable_nested_types_4877.reference new file mode 100644 index 00000000000..1e2036c94c7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00999_nullable_nested_types_4877.reference @@ -0,0 +1,26 @@ +a ('b','c') ('b','c') +d ('e','f') ('','') +a +x +a ('b','c') ('b','c') +x ('','') ('y','z') +a +d +a +x +a ('b','c') ('b','c') +d ('e','f') ('','') +a ('b','c') ('b','c') +x ('','') ('y','z') +a b ['b','c'] +d e [] +a b ['b','c'] +x ['y','z'] +a +d +a +x +a b ['b','c'] +d e [] +a b ['b','c'] +x \N ['y','z'] diff --git a/dbms/tests/queries/0_stateless/00999_nullable_nested_types_4877.sql b/dbms/tests/queries/0_stateless/00999_nullable_nested_types_4877.sql new file mode 100644 index 00000000000..ca523d77235 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00999_nullable_nested_types_4877.sql @@ -0,0 +1,41 @@ +DROP TABLE IF EXISTS l; +DROP TABLE IF EXISTS r; + +CREATE TABLE l (a String, b Tuple(String, String)) ENGINE = Memory(); +CREATE TABLE r (a String, c Tuple(String, String)) ENGINE = Memory(); + +INSERT INTO l (a, b) VALUES ('a', ('b', 'c')), ('d', ('e', 'f')); +INSERT INTO r (a, c) VALUES ('a', ('b', 'c')), ('x', ('y', 'z')); + +SET join_use_nulls = 0; +SELECT * from l LEFT JOIN r USING a ORDER BY a; +SELECT a from l RIGHT JOIN r USING a ORDER BY a; +SELECT * from l RIGHT JOIN r USING a ORDER BY a; + +SET join_use_nulls = 1; +SELECT a from l LEFT JOIN r USING a ORDER BY a; +SELECT a from l RIGHT JOIN r USING a ORDER BY a; +SELECT * from l LEFT JOIN r USING a ORDER BY a; +SELECT * from l RIGHT JOIN r USING a ORDER BY a; + +DROP TABLE l; +DROP TABLE r; + +CREATE TABLE l (a String, b String) ENGINE = Memory(); +CREATE TABLE r (a String, c Array(String)) ENGINE = Memory(); + +INSERT INTO l (a, b) VALUES ('a', 'b'), ('d', 'e'); +INSERT INTO r (a, c) VALUES ('a', ['b', 'c']), ('x', ['y', 'z']); + +SET join_use_nulls = 0; +SELECT * from l LEFT JOIN r USING a ORDER BY a; +SELECT * from l RIGHT JOIN r USING a ORDER BY a; + +SET join_use_nulls = 1; +SELECT a from l LEFT JOIN r USING a ORDER BY a; +SELECT a from l RIGHT JOIN r USING a ORDER BY a; +SELECT * from l LEFT JOIN r USING a ORDER BY a; +SELECT * from l RIGHT JOIN r USING a ORDER BY a; + +DROP TABLE l; +DROP TABLE r; From 6d6c53d42bb251e185594ab971c23645ad35ca3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 19:15:57 +0300 Subject: [PATCH 488/509] Style --- libs/libcommon/include/common/DateLUTImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index 2258620eb26..ef50d6ede3f 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -28,7 +28,7 @@ enum class WeekModeFlag : UInt8 FIRST_WEEKDAY = 4, NEWYEAR_DAY = 8 }; -typedef std::pair YearWeek; +using YearWeek = std::pair; /** Lookup table to conversion of time to date, and to month / year / day of week / day of month and so on. * First time was implemented for OLAPServer, that needed to do billions of such transformations. From 91bc0eca11f35cb6a1dc84bb7eaf3d71ad534cb3 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 28 Aug 2019 20:00:20 +0300 Subject: [PATCH 489/509] add test for #4858 --- .../0_stateless/00881_unknown_identifier_in_in.reference | 1 + .../queries/0_stateless/00881_unknown_identifier_in_in.sql | 4 ++++ 2 files changed, 5 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00881_unknown_identifier_in_in.reference create mode 100644 dbms/tests/queries/0_stateless/00881_unknown_identifier_in_in.sql diff --git a/dbms/tests/queries/0_stateless/00881_unknown_identifier_in_in.reference b/dbms/tests/queries/0_stateless/00881_unknown_identifier_in_in.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00881_unknown_identifier_in_in.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00881_unknown_identifier_in_in.sql b/dbms/tests/queries/0_stateless/00881_unknown_identifier_in_in.sql new file mode 100644 index 00000000000..2ce709c45be --- /dev/null +++ b/dbms/tests/queries/0_stateless/00881_unknown_identifier_in_in.sql @@ -0,0 +1,4 @@ +SELECT toUInt64(1) x FROM (select 1) +GROUP BY 1 +HAVING x +IN ( SELECT countIf(y, z == 1) FROM (SELECT 1 y, 1 z) ); From b4d98f6e13ac83c10ee73e38008a816e01d6c97d Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 28 Aug 2019 21:23:20 +0300 Subject: [PATCH 490/509] Trying to fix vertical merge --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Interpreters/ExpressionActions.cpp | 2 +- .../MergeTree/IMergedBlockOutputStream.cpp | 4 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 38 +++++++-- dbms/src/Storages/MergeTree/MergeTreeData.h | 16 ++-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 17 +++- .../Storages/MergeTree/MergeTreeIndices.cpp | 1 + .../src/Storages/MergeTree/MergeTreeIndices.h | 14 ++++ ...kip_indices_with_alter_and_merge.reference | 2 + ...test_skip_indices_with_alter_and_merge.sql | 79 +++++++++++++++++++ 10 files changed, 154 insertions(+), 20 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00999_test_skip_indices_with_alter_and_merge.reference create mode 100644 dbms/tests/queries/0_stateless/00999_test_skip_indices_with_alter_and_merge.sql diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 87ab252c583..977e1f4425d 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -447,6 +447,7 @@ namespace ErrorCodes extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW = 470; extern const int SETTINGS_ARE_NOT_SUPPORTED = 471; extern const int IMMUTABLE_SETTING = 472; + extern const int UNSUPPORTED_SKIP_INDEX_EXPRESSION = 473; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index c7b510abcf0..88cd69fe015 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -726,7 +726,7 @@ void ExpressionActions::addImpl(ExpressionAction action, Names & new_names) new_names.push_back(action.result_name); new_names.insert(new_names.end(), action.array_joined_columns.begin(), action.array_joined_columns.end()); - /// Compiled functions are custom functions and them don't need building + /// Compiled functions are custom functions and they don't need building if (action.type == ExpressionAction::APPLY_FUNCTION && !action.is_function_compiled) { if (sample_block.has(action.result_name)) diff --git a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 255fb998446..e673fd4759a 100644 --- a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -336,9 +336,9 @@ void IMergedBlockOutputStream::calculateAndSerializeSkipIndices( size_t skip_index_current_mark = 0; /// Filling and writing skip indices like in IMergedBlockOutputStream::writeColumn - for (size_t i = 0; i < storage.skip_indices.size(); ++i) + for (size_t i = 0; i < skip_indices.size(); ++i) { - const auto index = storage.skip_indices[i]; + const auto index = skip_indices[i]; auto & stream = *skip_indices_streams[i]; size_t prev_pos = 0; skip_index_current_mark = skip_index_mark; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index b2d4a4b9d73..bc34a81c6c2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -91,6 +91,7 @@ namespace ErrorCodes extern const int INCORRECT_FILE_NAME; extern const int BAD_DATA_PART_NAME; extern const int UNKNOWN_SETTING; + extern const int UNSUPPORTED_SKIP_INDEX_EXPRESSION; } @@ -349,6 +350,7 @@ void MergeTreeData::setProperties( MergeTreeIndices new_indices; + auto settings_ptr = getSettings(); if (!indices_description.indices.empty()) { std::set indices_names; @@ -357,11 +359,18 @@ void MergeTreeData::setProperties( { const auto & index_decl = std::dynamic_pointer_cast(index_ast); - new_indices.push_back( - MergeTreeIndexFactory::instance().get( - all_columns, - std::dynamic_pointer_cast(index_decl->clone()), - global_context)); + auto index_ptr = MergeTreeIndexFactory::instance().get( + all_columns, + std::dynamic_pointer_cast(index_decl->clone()), + global_context); + + if (index_ptr->getColumnsRequiredForIndexCalc().size() > 1 && settings_ptr->enable_vertical_merge_algorithm) + throw Exception("Index '" + index_ptr->name + "' contains expression with multiple columns and " + + "'enable_vertical_merge_algorithm' is set to true in storage settings. " + + "Disable vertical merge or use only one column in index expression.", + ErrorCodes::UNSUPPORTED_SKIP_INDEX_EXPRESSION); + + new_indices.push_back(std::move(index_ptr)); if (indices_names.find(new_indices.back()->name) != indices_names.end()) throw Exception( @@ -1293,7 +1302,7 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c } if (columns_alter_forbidden.count(command.column_name)) - throw Exception("trying to ALTER key column " + command.column_name, ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Trying to ALTER key column " + command.column_name, ErrorCodes::ILLEGAL_COLUMN); if (columns_alter_metadata_only.count(command.column_name)) { @@ -1600,7 +1609,7 @@ void MergeTreeData::alterDataPart( true /* sync */, compression_codec, true /* skip_offsets */, - {}, + {}, /// currently restricted unused_written_offsets, part->index_granularity, &part->index_granularity_info); @@ -3085,4 +3094,19 @@ bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const return true; } + +std::vector MergeTreeData::getIndicesForColumn(const String & column_name) const +{ + std::vector result; + + for (size_t i = 0; i < skip_indices.size(); ++i) + { + const auto & index_columns = skip_indices[i]->getColumnsRequiredForIndexCalc(); + if (std::find(index_columns.begin(), index_columns.end(), column_name) != index_columns.end()) + result.emplace_back(skip_indices[i]); + } + + return result; +} + } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 0440a3181c8..6cf1f50dabb 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -621,6 +621,15 @@ public: (settings->enable_mixed_granularity_parts || !has_non_adaptive_index_granularity_parts); } + /// Get constant pointer to storage settings. + /// Copy this pointer into your scope and you will + /// get consistent settings. + MergeTreeSettingsPtr getSettings() const + { + return storage_settings.get(); + } + + std::vector getIndicesForColumn(const String & column_name) const; MergeTreeDataFormatVersion format_version; @@ -679,13 +688,6 @@ public: bool has_non_adaptive_index_granularity_parts = false; - /// Get constant pointer to storage settings. - /// Copy this pointer into your scope and you will - /// get consistent settings. - MergeTreeSettingsPtr getSettings() const - { - return storage_settings.get(); - } protected: diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index ad489a91603..1742a4463b2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -825,6 +825,17 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor rows_sources_read_buf.seek(0, 0); ColumnGathererStream column_gathered_stream(column_name, column_part_streams, rows_sources_read_buf); + + std::vector skip_idx_to_recalc = data.getIndicesForColumn(column_name); + for (const auto & idx : skip_idx_to_recalc) + if (idx->getColumnsRequiredForIndexCalc().size() > 1) + throw Exception("Skip index '" + idx->name + "' has expression on multiple columns. " + + "Vertical merge is not supported for tables with skip indices with expressions on multiple columns. " + + "It's better to avoid indices with multiple columns in expression. " + + "Also you can disable vertical merges with setting enable_vertical_merge_algorithm=0, " + + "but it will lead to additional memory consuption for big merges.", + ErrorCodes::LOGICAL_ERROR); + MergedColumnOnlyOutputStream column_to( data, column_gathered_stream.getHeader(), @@ -832,7 +843,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor false, compression_codec, false, - {}, + skip_idx_to_recalc, written_offset_columns, to.getIndexGranularity() ); @@ -1017,8 +1028,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor { const auto & index = data.skip_indices[i]; const auto & index_cols = index->expr->getRequiredColumns(); - auto it = find(cbegin(index_cols), cend(index_cols), col); - if (it != cend(index_cols) && indices_to_recalc.insert(index).second) + auto it = std::find(std::cbegin(index_cols), std::cend(index_cols), col); + if (it != std::cend(index_cols) && indices_to_recalc.insert(index).second) { ASTPtr expr_list = MergeTreeData::extractKeyExpressionList( storage_from_source_part->getIndices().indices[i]->expr->clone()); diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndices.cpp b/dbms/src/Storages/MergeTree/MergeTreeIndices.cpp index e19aafbd25d..a799cc5cffb 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndices.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeIndices.cpp @@ -51,6 +51,7 @@ std::unique_ptr MergeTreeIndexFactory::get( return lft + ", " + rht.first; }), ErrorCodes::INCORRECT_QUERY); + return it->second(columns, node, context); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndices.h b/dbms/src/Storages/MergeTree/MergeTreeIndices.h index c430d1e8135..0b626d5e10a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndices.h +++ b/dbms/src/Storages/MergeTree/MergeTreeIndices.h @@ -104,11 +104,25 @@ public: virtual MergeTreeIndexConditionPtr createIndexCondition( const SelectQueryInfo & query_info, const Context & context) const = 0; + Names getColumnsRequiredForIndexCalc() const { return expr->getRequiredColumns(); } + + /// Index name String name; + + /// Index expression with columns arguments ExpressionActionsPtr expr; + + /// Names of columns which are used + /// to calculate expression for index Names columns; + + /// Data types of columns DataTypes data_types; + + /// Block with columns and data_types Block header; + + /// Skip index granularity size_t granularity; }; diff --git a/dbms/tests/queries/0_stateless/00999_test_skip_indices_with_alter_and_merge.reference b/dbms/tests/queries/0_stateless/00999_test_skip_indices_with_alter_and_merge.reference new file mode 100644 index 00000000000..5aad65487ae --- /dev/null +++ b/dbms/tests/queries/0_stateless/00999_test_skip_indices_with_alter_and_merge.reference @@ -0,0 +1,2 @@ +201 +201 diff --git a/dbms/tests/queries/0_stateless/00999_test_skip_indices_with_alter_and_merge.sql b/dbms/tests/queries/0_stateless/00999_test_skip_indices_with_alter_and_merge.sql new file mode 100644 index 00000000000..c67f6c594ff --- /dev/null +++ b/dbms/tests/queries/0_stateless/00999_test_skip_indices_with_alter_and_merge.sql @@ -0,0 +1,79 @@ +SET allow_experimental_data_skipping_indices=1; +DROP TABLE IF EXISTS table_test_creation; + +CREATE TABLE table_test_creation ( + k UInt64, + val1 UInt64, + val2 UInt64, + INDEX idx1 val1 * val2 TYPE minmax GRANULARITY 1 +) ENGINE MergeTree() +ORDER BY k; -- { serverError 473 } + +CREATE TABLE table_test_creation ( + k UInt64, + val1 UInt64, + val2 UInt64 +) ENGINE MergeTree() +ORDER BY k; + +ALTER TABLE table_test_creation ADD INDEX idx1 val1 * val2 TYPE minmax GRANULARITY 1; -- { serverError 473 } + +ALTER TABLE table_test_creation ADD INDEX idx1 val1 TYPE minmax GRANULARITY 1; + +ALTER TABLE table_test_creation MODIFY SETTING enable_vertical_merge_algorithm=0; + +ALTER TABLE table_test_creation ADD INDEX idx2 val1 * val2 TYPE minmax GRANULARITY 1; + +DROP TABLE IF EXISTS table_test_creation; + +CREATE TABLE table_test_creation ( + k UInt64, + val1 UInt64, + val2 UInt64, + INDEX idx1 val1 * val2 TYPE minmax GRANULARITY 1 +) ENGINE MergeTree() +ORDER BY k SETTINGS enable_vertical_merge_algorithm=0; + +DROP TABLE IF EXISTS table_test_creation; + +DROP TABLE IF EXISTS test_vertical_merge; + +CREATE TABLE test_vertical_merge ( + k UInt64, + val1 UInt64, + val2 UInt64, + INDEX idx1 val2 TYPE minmax GRANULARITY 1 +) ENGINE MergeTree() +ORDER BY k +SETTINGS vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 1; + +INSERT INTO test_vertical_merge SELECT number, number + 5, number * 12 from numbers(1000); + +SELECT COUNT() from test_vertical_merge WHERE val2 <= 2400; + +OPTIMIZE TABLE test_vertical_merge FINAL; + +SELECT COUNT() from test_vertical_merge WHERE val2 <= 2400; + +DROP TABLE IF EXISTS test_vertical_merge; + +--DROP TABLE IF EXISTS test_alter_multiple_columns; +-- +--CREATE TABLE test_alter_multiple_columns ( +-- k UInt64, +-- val1 UInt64, +-- val2 UInt64, +-- INDEX idx1 val2 * val1 TYPE minmax GRANULARITY 1 +--) ENGINE MergeTree() +--ORDER BY k +--SETTINGS enable_vertical_merge_algorithm=0; +-- +--INSERT INTO test_alter_multiple_columns SELECT number, number + 5, number * 12 from numbers(1000); +-- +--SELECT COUNT() from test_alter_multiple_columns WHERE val2 <= 2400; +-- +--ALTER TABLE test_alter_multiple_columns MODIFY COLUMN val2 UInt16; +-- +--SELECT COUNT() from test_alter_multiple_columns WHERE val2 <= 2400; +-- +--DROP TABLE IF EXISTS test_alter_multiple_columns; From 48dce81e61699f956d1650a99056b53f2534027c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 21:54:30 +0300 Subject: [PATCH 491/509] Minor modifications after #6413 --- dbms/src/Databases/DatabaseOrdinary.cpp | 13 ++++++++----- dbms/src/Interpreters/InterpreterDropQuery.cpp | 10 +++++++--- 2 files changed, 15 insertions(+), 8 deletions(-) diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 9039cb83cdc..b988329127e 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -136,14 +136,14 @@ void DatabaseOrdinary::loadTables( continue; // There are files that we tried to delete previously - const std::string tmp_drop_ext = ".sql.tmp_drop"; - if (endsWith(dir_it.name(), ".sql.tmp_drop")) + static const char * tmp_drop_ext = ".sql.tmp_drop"; + if (endsWith(dir_it.name(), tmp_drop_ext)) { - const std::string table_name = dir_it.name().substr(0, dir_it.name().size() - tmp_drop_ext.size()); + const std::string table_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext)); if (Poco::File(data_path + '/' + table_name).exists()) { Poco::File(dir_it->path()).renameTo(table_name + ".sql"); - LOG_WARNING(log, "Table was not dropped previously"); + LOG_WARNING(log, "Table " << backQuote(table_name) << " was not dropped previously"); } else { @@ -325,7 +325,10 @@ void DatabaseOrdinary::removeTable( Poco::File(table_metadata_path + ".tmp_drop").remove(); return; } - catch (...) {} + catch (...) + { + LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__)); + } attachTable(table_name, res); throw; } diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index caabbc20e3b..226a93aff88 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -91,13 +91,17 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t auto table_lock = database_and_table.second->lockExclusively(context.getCurrentQueryId()); - const auto prev_metadata_name = database_and_table.first->getMetadataPath() + escapeForFileName(database_and_table.second->getTableName()) + ".sql"; - const auto drop_metadata_name = database_and_table.first->getMetadataPath() + escapeForFileName(database_and_table.second->getTableName()) + ".sql.tmp_drop"; + const std::string metadata_file_without_extension = + database_and_table.first->getMetadataPath() + + escapeForFileName(database_and_table.second->getTableName()); + + const auto prev_metadata_name = metadata_file_without_extension + ".sql"; + const auto drop_metadata_name = metadata_file_without_extension + ".sql.tmp_drop"; /// Try to rename metadata file and delete the data try { - //There some kind of tables that have no metadata - ignore renaming + /// There some kind of tables that have no metadata - ignore renaming if (Poco::File(prev_metadata_name).exists()) Poco::File(prev_metadata_name).renameTo(drop_metadata_name); /// Delete table data From 39d50b5144b7d3f11eed2fb9b883e0f4c38d8253 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 22:01:52 +0300 Subject: [PATCH 492/509] Remove mimalloc --- .gitmodules | 3 - CMakeLists.txt | 1 - cmake/find_mimalloc.cmake | 17 --- dbms/CMakeLists.txt | 5 - dbms/src/Common/MiAllocator.cpp | 70 ------------ dbms/src/Common/MiAllocator.h | 27 ----- dbms/src/Common/config.h.in | 1 - dbms/src/Common/tests/CMakeLists.txt | 3 - dbms/src/Common/tests/mi_malloc_test.cpp | 118 -------------------- dbms/src/DataStreams/MarkInCompressedFile.h | 9 +- dbms/src/IO/UncompressedCache.h | 9 -- 11 files changed, 1 insertion(+), 262 deletions(-) delete mode 100644 cmake/find_mimalloc.cmake delete mode 100644 dbms/src/Common/MiAllocator.cpp delete mode 100644 dbms/src/Common/MiAllocator.h delete mode 100644 dbms/src/Common/tests/mi_malloc_test.cpp diff --git a/.gitmodules b/.gitmodules index f6990fed41f..e5be5438cc7 100644 --- a/.gitmodules +++ b/.gitmodules @@ -97,9 +97,6 @@ [submodule "contrib/rapidjson"] path = contrib/rapidjson url = https://github.com/Tencent/rapidjson -[submodule "contrib/mimalloc"] - path = contrib/mimalloc - url = https://github.com/ClickHouse-Extras/mimalloc [submodule "contrib/fastops"] path = contrib/fastops url = https://github.com/ClickHouse-Extras/fastops diff --git a/CMakeLists.txt b/CMakeLists.txt index f84a181a39c..e181bdbc2af 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -476,7 +476,6 @@ include (cmake/find_consistent-hashing.cmake) include (cmake/find_base64.cmake) include (cmake/find_parquet.cmake) include (cmake/find_hyperscan.cmake) -include (cmake/find_mimalloc.cmake) include (cmake/find_simdjson.cmake) include (cmake/find_rapidjson.cmake) include (cmake/find_fastops.cmake) diff --git a/cmake/find_mimalloc.cmake b/cmake/find_mimalloc.cmake deleted file mode 100644 index 1820421379f..00000000000 --- a/cmake/find_mimalloc.cmake +++ /dev/null @@ -1,17 +0,0 @@ -if (OS_LINUX AND NOT SANITIZE AND NOT ARCH_ARM AND NOT ARCH_32 AND NOT ARCH_PPC64LE) - option (ENABLE_MIMALLOC "Set to FALSE to disable usage of mimalloc for internal ClickHouse caches" FALSE) -endif () - -if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mimalloc/include/mimalloc.h") - message (WARNING "submodule contrib/mimalloc is missing. to fix try run: \n git submodule update --init --recursive") - return() -endif () - -if (ENABLE_MIMALLOC) - message (FATAL_ERROR "Mimalloc is not production ready. (Disable with cmake -D ENABLE_MIMALLOC=0). If you want to use mimalloc, you must manually remove this message.") - - set (MIMALLOC_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/mimalloc/include) - set (USE_MIMALLOC 1) - set (MIMALLOC_LIBRARY mimalloc-static) - message (STATUS "Using mimalloc: ${MIMALLOC_INCLUDE_DIR} : ${MIMALLOC_LIBRARY}") -endif () diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 355c66902a8..af59af7642b 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -266,11 +266,6 @@ if(RE2_INCLUDE_DIR) target_include_directories(clickhouse_common_io SYSTEM BEFORE PUBLIC ${RE2_INCLUDE_DIR}) endif() -if (USE_MIMALLOC) - target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${MIMALLOC_INCLUDE_DIR}) - target_link_libraries (clickhouse_common_io PRIVATE ${MIMALLOC_LIBRARY}) -endif () - if(CPUID_LIBRARY) target_link_libraries(clickhouse_common_io PRIVATE ${CPUID_LIBRARY}) endif() diff --git a/dbms/src/Common/MiAllocator.cpp b/dbms/src/Common/MiAllocator.cpp deleted file mode 100644 index 04e61a5de16..00000000000 --- a/dbms/src/Common/MiAllocator.cpp +++ /dev/null @@ -1,70 +0,0 @@ -#include "MiAllocator.h" - -#if USE_MIMALLOC -#include - -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int CANNOT_ALLOCATE_MEMORY; -} - -void * MiAllocator::alloc(size_t size, size_t alignment) -{ - void * ptr; - if (alignment == 0) - { - ptr = mi_malloc(size); - if (!ptr) - DB::throwFromErrno("MiAllocator: Cannot allocate in mimalloc " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); - } - else - { - ptr = mi_malloc_aligned(size, alignment); - if (!ptr) - DB::throwFromErrno("MiAllocator: Cannot allocate in mimalloc (mi_malloc_aligned) " + formatReadableSizeWithBinarySuffix(size) + " with alignment " + toString(alignment) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); - } - return ptr; -} - -void MiAllocator::free(void * buf, size_t) -{ - mi_free(buf); -} - -void * MiAllocator::realloc(void * old_ptr, size_t, size_t new_size, size_t alignment) -{ - if (old_ptr == nullptr) - return alloc(new_size, alignment); - - if (new_size == 0) - { - mi_free(old_ptr); - return nullptr; - } - - void * ptr; - - if (alignment == 0) - { - ptr = mi_realloc(old_ptr, alignment); - if (!ptr) - DB::throwFromErrno("MiAllocator: Cannot reallocate in mimalloc " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); - } - else - { - ptr = mi_realloc_aligned(old_ptr, new_size, alignment); - if (!ptr) - DB::throwFromErrno("MiAllocator: Cannot reallocate in mimalloc (mi_realloc_aligned) " + formatReadableSizeWithBinarySuffix(size) + " with alignment " + toString(alignment) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); - } - return ptr; -} - -} - -#endif diff --git a/dbms/src/Common/MiAllocator.h b/dbms/src/Common/MiAllocator.h deleted file mode 100644 index 127be82434b..00000000000 --- a/dbms/src/Common/MiAllocator.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once - -#include - -#if USE_MIMALLOC -#include - -namespace DB -{ - -/* - * This is a different allocator that is based on mimalloc (Microsoft malloc). - * It can be used separately from main allocator to catch heap corruptions and vulnerabilities (for example, for caches). - * We use MI_SECURE mode in mimalloc to achieve such behaviour. - */ -struct MiAllocator -{ - static void * alloc(size_t size, size_t alignment = 0); - - static void free(void * buf, size_t); - - static void * realloc(void * old_ptr, size_t, size_t new_size, size_t alignment = 0); -}; - -} - -#endif diff --git a/dbms/src/Common/config.h.in b/dbms/src/Common/config.h.in index b6a5b6de2b8..7804068e5c4 100644 --- a/dbms/src/Common/config.h.in +++ b/dbms/src/Common/config.h.in @@ -8,6 +8,5 @@ #cmakedefine01 USE_CPUID #cmakedefine01 USE_CPUINFO #cmakedefine01 USE_BROTLI -#cmakedefine01 USE_MIMALLOC #cmakedefine01 USE_UNWIND #cmakedefine01 CLICKHOUSE_SPLIT_BINARY diff --git a/dbms/src/Common/tests/CMakeLists.txt b/dbms/src/Common/tests/CMakeLists.txt index 2c99c85baec..67c0e376f74 100644 --- a/dbms/src/Common/tests/CMakeLists.txt +++ b/dbms/src/Common/tests/CMakeLists.txt @@ -76,8 +76,5 @@ target_link_libraries (cow_compositions PRIVATE clickhouse_common_io) add_executable (stopwatch stopwatch.cpp) target_link_libraries (stopwatch PRIVATE clickhouse_common_io) -add_executable (mi_malloc_test mi_malloc_test.cpp) -target_link_libraries (mi_malloc_test PRIVATE clickhouse_common_io) - add_executable (symbol_index symbol_index.cpp) target_link_libraries (symbol_index PRIVATE clickhouse_common_io) diff --git a/dbms/src/Common/tests/mi_malloc_test.cpp b/dbms/src/Common/tests/mi_malloc_test.cpp deleted file mode 100644 index ce1e4a3a770..00000000000 --- a/dbms/src/Common/tests/mi_malloc_test.cpp +++ /dev/null @@ -1,118 +0,0 @@ -/** In addition to ClickHouse (Apache 2) license, this file can be also used under MIT license: - -MIT License - -Copyright (c) 2019 Yandex LLC, Alexey Milovidov - -Permission is hereby granted, free of charge, to any person obtaining a copy -of this software and associated documentation files (the "Software"), to deal -in the Software without restriction, including without limitation the rights -to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -copies of the Software, and to permit persons to whom the Software is -furnished to do so, subject to the following conditions: - -The above copyright notice and this permission notice shall be included in all -copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE -SOFTWARE. - -*/ - -#include -#include -#include -#include -#include -#include - -#include - -//#undef USE_MIMALLOC -//#define USE_MIMALLOC 0 - -#if USE_MIMALLOC - -#include -#define malloc mi_malloc -#define free mi_free - -#else - -#include - -#endif - - -size_t total_size{0}; - -struct Allocation -{ - void * ptr = nullptr; - size_t size = 0; - - Allocation() {} - - Allocation(size_t size_) - : size(size_) - { - ptr = malloc(size); - if (!ptr) - throw std::runtime_error("Cannot allocate memory"); - total_size += size; - } - - ~Allocation() - { - if (ptr) - { - free(ptr); - total_size -= size; - } - ptr = nullptr; - } - - Allocation(const Allocation &) = delete; - - Allocation(Allocation && rhs) - { - ptr = rhs.ptr; - size = rhs.size; - rhs.ptr = nullptr; - rhs.size = 0; - } -}; - - -int main(int, char **) -{ - std::vector allocations; - - constexpr size_t limit = 100000000; - constexpr size_t min_alloc_size = 65536; - constexpr size_t max_alloc_size = 10000000; - - std::mt19937 rng; - auto distribution = std::uniform_int_distribution(min_alloc_size, max_alloc_size); - - size_t total_allocations = 0; - - while (true) - { - size_t size = distribution(rng); - - while (total_size + size > limit) - allocations.pop_back(); - - allocations.emplace_back(size); - - ++total_allocations; - if (total_allocations % (1ULL << 20) == 0) - std::cerr << "Total allocations: " << total_allocations << "\n"; - } -} diff --git a/dbms/src/DataStreams/MarkInCompressedFile.h b/dbms/src/DataStreams/MarkInCompressedFile.h index a5970a89738..62fe8eedf76 100644 --- a/dbms/src/DataStreams/MarkInCompressedFile.h +++ b/dbms/src/DataStreams/MarkInCompressedFile.h @@ -6,10 +6,6 @@ #include #include -#include -#if USE_MIMALLOC -#include -#endif namespace DB { @@ -43,9 +39,6 @@ struct MarkInCompressedFile } }; -#if USE_MIMALLOC -using MarksInCompressedFile = PODArray; -#else + using MarksInCompressedFile = PODArray; -#endif } diff --git a/dbms/src/IO/UncompressedCache.h b/dbms/src/IO/UncompressedCache.h index 1f17c5e61b6..86f1530e5b3 100644 --- a/dbms/src/IO/UncompressedCache.h +++ b/dbms/src/IO/UncompressedCache.h @@ -6,11 +6,6 @@ #include #include -#include -#if USE_MIMALLOC -#include -#endif - namespace ProfileEvents { @@ -25,11 +20,7 @@ namespace DB struct UncompressedCacheCell { -#if USE_MIMALLOC - Memory data; -#else Memory<> data; -#endif size_t compressed_size; UInt32 additional_bytes; }; From e9875950a488eaae51488f69e8f2274ed4343b61 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2019 22:36:04 +0300 Subject: [PATCH 493/509] Make test timeout to be more significant --- .../00816_long_concurrent_alter_column.sh | 52 +++++++++++++++++-- 1 file changed, 48 insertions(+), 4 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh b/dbms/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh index 17779b73add..965408065cf 100755 --- a/dbms/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh +++ b/dbms/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh @@ -8,14 +8,58 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "DROP TABLE IF EXISTS concurrent_alter_column" | ${CLICKHOUSE_CLIENT} echo "CREATE TABLE concurrent_alter_column (ts DATETIME) ENGINE = MergeTree PARTITION BY toStartOfDay(ts) ORDER BY tuple()" | ${CLICKHOUSE_CLIENT} -for i in {1..500}; do echo "ALTER TABLE concurrent_alter_column ADD COLUMN c$i DOUBLE;"; done | ${CLICKHOUSE_CLIENT} -n +function thread1() +{ + while true; do + for i in {1..500}; do echo "ALTER TABLE concurrent_alter_column ADD COLUMN c$i DOUBLE;"; done | ${CLICKHOUSE_CLIENT} -n --query_id=alter1 + done +} -for i in {1..100}; do echo "ALTER TABLE concurrent_alter_column ADD COLUMN d DOUBLE" | ${CLICKHOUSE_CLIENT}; sleep `echo 0.0$RANDOM`; echo "ALTER TABLE concurrent_alter_column DROP COLUMN d" | ${CLICKHOUSE_CLIENT} -n; done & -for i in {1..100}; do echo "ALTER TABLE concurrent_alter_column ADD COLUMN e DOUBLE" | ${CLICKHOUSE_CLIENT}; sleep `echo 0.0$RANDOM`; echo "ALTER TABLE concurrent_alter_column DROP COLUMN e" | ${CLICKHOUSE_CLIENT} -n; done & -for i in {1..100}; do echo "ALTER TABLE concurrent_alter_column ADD COLUMN f DOUBLE" | ${CLICKHOUSE_CLIENT}; sleep `echo 0.0$RANDOM`; echo "ALTER TABLE concurrent_alter_column DROP COLUMN f" | ${CLICKHOUSE_CLIENT} -n; done & +function thread2() +{ + while true; do + echo "ALTER TABLE concurrent_alter_column ADD COLUMN d DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter2; + sleep `echo 0.0$RANDOM`; + echo "ALTER TABLE concurrent_alter_column DROP COLUMN d" | ${CLICKHOUSE_CLIENT} --query_id=alter2; + done +} + +function thread3() +{ + while true; do + echo "ALTER TABLE concurrent_alter_column ADD COLUMN e DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter3; + sleep `echo 0.0$RANDOM`; + echo "ALTER TABLE concurrent_alter_column DROP COLUMN e" | ${CLICKHOUSE_CLIENT} --query_id=alter3; + done +} + +function thread4() +{ + while true; do + echo "ALTER TABLE concurrent_alter_column ADD COLUMN f DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter4; + sleep `echo 0.0$RANDOM`; + echo "ALTER TABLE concurrent_alter_column DROP COLUMN f" | ${CLICKHOUSE_CLIENT} --query_id=alter4; + done +} + +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; + +TIMEOUT=30 + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & wait echo "DROP TABLE concurrent_alter_column" | ${CLICKHOUSE_CLIENT} +# Check for deadlocks +echo "SELECT * FROM system.processes WHERE query_id LIKE 'alter%'" | ${CLICKHOUSE_CLIENT} + echo 'did not crash' From b4339f266df17c0cd873ef10249647409e66f229 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Wed, 28 Aug 2019 23:49:37 +0300 Subject: [PATCH 494/509] Make a better build scheme (#6500) * Fix shared build * Major default libs refactor * Fix build with gcc_eh * Link all libraries as a big group. * Use global interface library as a group * Build capnproto using our cmake * Use only internal libunwind --- CMakeLists.txt | 258 ++---------------- cmake/default_libs.cmake | 48 ++++ cmake/find_capnp.cmake | 64 ++--- cmake/find_cxx.cmake | 47 +++- cmake/find_unwind.cmake | 64 +---- cmake/test_compiler.cmake | 47 ---- contrib/CMakeLists.txt | 19 -- contrib/arrow-cmake/CMakeLists.txt | 3 +- contrib/capnproto-cmake/CMakeLists.txt | 68 +++++ contrib/jemalloc-cmake/CMakeLists.txt | 3 +- contrib/libcxx-cmake/CMakeLists.txt | 20 +- contrib/libcxxabi-cmake/CMakeLists.txt | 25 +- contrib/librdkafka-cmake/CMakeLists.txt | 2 +- contrib/libunwind-cmake/CMakeLists.txt | 16 +- .../mariadb-connector-c-cmake/CMakeLists.txt | 5 - dbms/CMakeLists.txt | 19 +- dbms/programs/server/Server.cpp | 2 +- dbms/src/Common/Config/CMakeLists.txt | 2 +- dbms/src/Common/QueryProfiler.cpp | 4 +- dbms/src/Common/QueryProfiler.h | 3 +- dbms/src/Common/StackTrace.cpp | 27 +- dbms/src/Common/ZooKeeper/CMakeLists.txt | 2 +- dbms/src/Dictionaries/CMakeLists.txt | 2 +- dbms/src/Interpreters/tests/CMakeLists.txt | 2 +- dbms/tests/tsan_suppressions.txt | 3 + docker/packager/packager | 4 +- libs/CMakeLists.txt | 4 - libs/libcommon/CMakeLists.txt | 2 - .../include/common/config_common.h.in | 1 - libs/libcommon/src/tests/CMakeLists.txt | 2 +- libs/libglibc-compatibility/CMakeLists.txt | 56 ++-- utils/compressor/CMakeLists.txt | 2 +- 32 files changed, 318 insertions(+), 508 deletions(-) create mode 100644 cmake/default_libs.cmake delete mode 100644 cmake/test_compiler.cmake create mode 100644 contrib/capnproto-cmake/CMakeLists.txt diff --git a/CMakeLists.txt b/CMakeLists.txt index f84a181a39c..fb4ca18126d 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -1,14 +1,22 @@ +foreach(policy + CMP0023 + CMP0048 # CMake 3.0 + CMP0074 # CMake 3.12 + CMP0077 + CMP0079 + ) + if(POLICY ${policy}) + cmake_policy(SET ${policy} NEW) + endif() +endforeach() + project(ClickHouse) cmake_minimum_required(VERSION 3.3) -foreach(policy - CMP0023 - CMP0074 # CMake 3.12 - ) - if(POLICY ${policy}) - cmake_policy(SET ${policy} NEW) - endif() -endforeach() +# Ignore export() since we don't use it, +# but it gets broken with a global targets via link_libraries() +macro (export) +endmacro () set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${CMAKE_CURRENT_SOURCE_DIR}/cmake/Modules/") set(CMAKE_EXPORT_COMPILE_COMMANDS 1) # Write compile_commands.json @@ -128,12 +136,6 @@ if (CMAKE_SYSTEM_PROCESSOR MATCHES "amd64|x86_64") endif () endif () -if (GLIBC_COMPATIBILITY) - set (USE_INTERNAL_MEMCPY ON) -else () - message (WARNING "Option GLIBC_COMPATIBILITY must be turned on for production builds.") -endif () - string(REGEX MATCH "-?[0-9]+(.[0-9]+)?$" COMPILER_POSTFIX ${CMAKE_CXX_COMPILER}) find_program (LLD_PATH NAMES "lld${COMPILER_POSTFIX}" "lld") @@ -172,20 +174,15 @@ if (ARCH_NATIVE) set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native") endif () -# Special options for better optimized code with clang -#if (COMPILER_CLANG) -# set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -Wno-unused-command-line-argument -mllvm -inline-threshold=10000") -#endif () - if (CMAKE_VERSION VERSION_LESS "3.8.0") if (NOT MSVC) - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++1z") + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++17") endif () else () set (CMAKE_CXX_STANDARD 17) set (CMAKE_CXX_EXTENSIONS 0) # https://cmake.org/cmake/help/latest/prop_tgt/CXX_EXTENSIONS.html#prop_tgt:CXX_EXTENSIONS set (CMAKE_CXX_STANDARD_REQUIRED ON) - set (CXX_FLAGS_INTERNAL_COMPILER "-std=c++1z") + set (CXX_FLAGS_INTERNAL_COMPILER "-std=c++17") endif () if (COMPILER_GCC OR COMPILER_CLANG) @@ -207,17 +204,13 @@ endif() set (CMAKE_BUILD_COLOR_MAKEFILE ON) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${COMPILER_FLAGS} ${PLATFORM_EXTRA_CXX_FLAG} -fno-omit-frame-pointer ${COMMON_WARNING_FLAGS} ${CXX_WARNING_FLAGS}") -#set (CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} ${CMAKE_CXX_FLAGS_ADD}") set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_CXX_FLAGS_ADD}") set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline ${CMAKE_CXX_FLAGS_ADD}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${COMPILER_FLAGS} -fno-omit-frame-pointer ${COMMON_WARNING_FLAGS} ${CMAKE_C_FLAGS_ADD}") -#set (CMAKE_C_FLAGS_RELEASE "${CMAKE_C_FLAGS_RELEASE} ${CMAKE_C_FLAGS_ADD}") set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_C_FLAGS_ADD}") set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline ${CMAKE_C_FLAGS_ADD}") -# Uses MAKE_STATIC_LIBRARIES - option (UNBUNDLED "Try find all libraries in system. We recommend to avoid this mode for production builds, because we cannot guarantee exact versions and variants of libraries your system has installed. This mode exists for enthusiastic developers who search for trouble. Also it is useful for maintainers of OS packages." OFF) if (UNBUNDLED) @@ -225,149 +218,28 @@ if (UNBUNDLED) else () set(NOT_UNBUNDLED 1) endif () + # Using system libs can cause lot of warnings in includes. if (UNBUNDLED OR NOT (OS_LINUX OR APPLE) OR ARCH_32) option (NO_WERROR "Disable -Werror compiler option" ON) endif () - -set(THREADS_PREFER_PTHREAD_FLAG ON) -find_package (Threads) - -include (cmake/find_cxx.cmake) - -include (cmake/test_compiler.cmake) - -if (OS_LINUX AND COMPILER_CLANG AND USE_STATIC_LIBRARIES) - option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++ (only make sense on Linux)" ${HAVE_LIBCXX}) - - if (USE_LIBCXX) - set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=0") # More checks in debug build. - endif () -endif () - -if (USE_LIBCXX) - set (STATIC_STDLIB_FLAGS "") -else () - set (STATIC_STDLIB_FLAGS "-static-libgcc -static-libstdc++") -endif () - -if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD)) - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}") - - # Along with executables, we also build example of shared library for "library dictionary source"; and it also should be self-contained. - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}") -endif () - -if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE) - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${FLAG_NO_PIE}") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}") -endif () - # Make this extra-checks for correct library dependencies. if (NOT SANITIZE) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-undefined") set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined") endif () -include (cmake/find_unwind.cmake) +include(cmake/dbms_glob_sources.cmake) +include(cmake/default_libs.cmake) -if (USE_INTERNAL_UNWIND_LIBRARY) - option (USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING "Use internal unwind library for exception handling" ${USE_STATIC_LIBRARIES}) -endif () - - -# Set standard, system and compiler libraries explicitly. -# This is intended for more control of what we are linking. +###################################### +### Add targets below this comment ### +###################################### string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC) set (CMAKE_POSTFIX_VARIABLE "CMAKE_${CMAKE_BUILD_TYPE_UC}_POSTFIX") -set (DEFAULT_LIBS "") -if (OS_LINUX AND NOT UNBUNDLED AND (GLIBC_COMPATIBILITY OR USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING OR USE_LIBCXX)) - # Note: this probably has no effect, but I'm not an expert in CMake. - set (CMAKE_C_IMPLICIT_LINK_LIBRARIES "") - set (CMAKE_CXX_IMPLICIT_LINK_LIBRARIES "") - - # Disable default linked libraries. - set (DEFAULT_LIBS "-nodefaultlibs") - - # We need builtins from Clang's RT even without libcxx - for ubsan+int128. See https://bugs.llvm.org/show_bug.cgi?id=16404 - set (BUILTINS_LIB_PATH "") - if (COMPILER_CLANG) - execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-${CMAKE_SYSTEM_PROCESSOR}.a OUTPUT_VARIABLE BUILTINS_LIB_PATH OUTPUT_STRIP_TRAILING_WHITESPACE) - else () - set (BUILTINS_LIB_PATH "-lgcc") - endif () - - string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC) - set (CMAKE_POSTFIX_VARIABLE "CMAKE_${CMAKE_BUILD_TYPE_UC}_POSTFIX") - - # Add C++ libraries. - # - # This consist of: - # - C++ standard library (like implementation of std::string); - # - C++ ABI implementation (functions for exceptions like __cxa_throw, RTTI, etc); - # - functions for internal implementation of exception handling (stack unwinding based on DWARF info; TODO replace with bundled libunwind); - # - compiler builtins (example: functions for implementation of __int128 operations); - # - # There are two variants of C++ library: libc++ (from LLVM compiler infrastructure) and libstdc++ (from GCC). - - if (USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING) - if (USE_STATIC_LIBRARIES) - set (EXCEPTION_HANDLING_LIBRARY "${ClickHouse_BINARY_DIR}/contrib/libunwind-cmake/libunwind_static${${CMAKE_POSTFIX_VARIABLE}}.a") - else () - set (EXCEPTION_HANDLING_LIBRARY "${ClickHouse_BINARY_DIR}/contrib/libunwind-cmake/libunwind_shared${${CMAKE_POSTFIX_VARIABLE}}.so") - endif () - else () - set (EXCEPTION_HANDLING_LIBRARY "-lgcc_eh") - endif () - - message (STATUS "Using exception handling library: ${EXCEPTION_HANDLING_LIBRARY}") - - if (USE_LIBCXX) - if (USE_INTERNAL_LIBCXX_LIBRARY) - set (LIBCXX_LIBS "${ClickHouse_BINARY_DIR}/contrib/libcxx-cmake/libcxx_static${${CMAKE_POSTFIX_VARIABLE}}.a ${ClickHouse_BINARY_DIR}/contrib/libcxxabi-cmake/libcxxabi_static${${CMAKE_POSTFIX_VARIABLE}}.a") - else () - set (LIBCXX_LIBS "-lc++ -lc++abi -lc++fs") - endif () - - set (DEFAULT_LIBS "${DEFAULT_LIBS} -Wl,-Bstatic ${LIBCXX_LIBS} ${EXCEPTION_HANDLING_LIBRARY} ${BUILTINS_LIB_PATH} -Wl,-Bdynamic") - else () - set (DEFAULT_LIBS "${DEFAULT_LIBS} -Wl,-Bstatic -lstdc++ -lstdc++fs ${EXCEPTION_HANDLING_LIBRARY} ${COVERAGE_OPTION} ${BUILTINS_LIB_PATH} -Wl,-Bdynamic") - endif () - - # Linking with GLIBC prevents portability of binaries to older systems. - # We overcome this behaviour by statically linking with our own implementation of all new symbols (that don't exist in older Libc or have infamous "symbol versioning"). - # The order of linking is important: 'glibc-compatibility' must be before libc but after all other libraries. - if (GLIBC_COMPATIBILITY) - message (STATUS "Some symbols from glibc will be replaced for compatibility") - - string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC) - set (CMAKE_POSTFIX_VARIABLE "CMAKE_${CMAKE_BUILD_TYPE_UC}_POSTFIX") - - # FIXME: glibc-compatibility may be non-static in some builds! - set (DEFAULT_LIBS "${DEFAULT_LIBS} ${ClickHouse_BINARY_DIR}/libs/libglibc-compatibility/libglibc-compatibility${${CMAKE_POSTFIX_VARIABLE}}.a") - endif () - - # Add Libc. GLIBC is actually a collection of interdependent libraries. - set (DEFAULT_LIBS "${DEFAULT_LIBS} -lrt -ldl -lpthread -lm -lc") - - # Note: we'd rather use Musl libc library, but it's little bit more difficult to use. - - message(STATUS "Default libraries: ${DEFAULT_LIBS}") -endif () - -if (NOT GLIBC_COMPATIBILITY) - set (M_LIBRARY m) -endif () - -if (DEFAULT_LIBS) - # Add default libs to all targets as the last dependency. - set(CMAKE_CXX_STANDARD_LIBRARIES ${DEFAULT_LIBS}) - set(CMAKE_C_STANDARD_LIBRARIES ${DEFAULT_LIBS}) -endif () - if (NOT MAKE_STATIC_LIBRARIES) set(CMAKE_POSITION_INDEPENDENT_CODE ON) endif () @@ -420,20 +292,12 @@ if (UNBUNDLED) else () set(NOT_UNBUNDLED 1) endif () + # Using system libs can cause lot of warnings in includes. if (UNBUNDLED OR NOT (OS_LINUX OR APPLE) OR ARCH_32) option (NO_WERROR "Disable -Werror compiler option" ON) endif () -if (USE_LIBCXX) - set (HAVE_LIBCXX 1) - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") -endif() - -if (USE_LIBCXX AND USE_INTERNAL_LIBCXX_LIBRARY) - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -nostdinc++ -isystem ${LIBCXX_INCLUDE_DIR} -isystem ${LIBCXXABI_INCLUDE_DIR}") -endif () - message (STATUS "Building for: ${CMAKE_SYSTEM} ${CMAKE_SYSTEM_PROCESSOR} ${CMAKE_LIBRARY_ARCHITECTURE} ; USE_STATIC_LIBRARIES=${USE_STATIC_LIBRARIES} MAKE_STATIC_LIBRARIES=${MAKE_STATIC_LIBRARIES} SPLIT_SHARED=${SPLIT_SHARED_LIBRARIES} UNBUNDLED=${UNBUNDLED} CCACHE=${CCACHE_FOUND} ${CCACHE_VERSION}") include(GNUInstallDirs) @@ -499,79 +363,11 @@ include (libs/libmysqlxx/cmake/find_mysqlclient.cmake) include (cmake/print_flags.cmake) +install (EXPORT global DESTINATION cmake) + add_subdirectory (contrib EXCLUDE_FROM_ALL) add_subdirectory (libs) add_subdirectory (utils) add_subdirectory (dbms) include (cmake/print_include_directories.cmake) - -if (GLIBC_COMPATIBILITY OR USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING) - # FIXME: actually glibc-compatibility should always be built first, - # because it's unconditionally linked via $DEFAULT_LIBS, - # and these looks like the first places that get linked. - function (add_default_dependencies target_name) - if (TARGET ${target_name}) - if (GLIBC_COMPATIBILITY) - add_dependencies(${target_name} glibc-compatibility) - endif () - - if (USE_LIBCXX AND USE_INTERNAL_LIBCXX_LIBRARY) - add_dependencies(${target_name} cxx_static cxxabi_static) - endif () - - if (USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING) - add_dependencies(${target_name} unwind_static) - endif () - endif () - endfunction () - - add_default_dependencies(ltdl) - add_default_dependencies(zlibstatic) - add_default_dependencies(jemalloc) - add_default_dependencies(memcpy) - add_default_dependencies(Foundation) - add_default_dependencies(common) - add_default_dependencies(gtest) - add_default_dependencies(lz4) - add_default_dependencies(zstd) - add_default_dependencies(snappy) - add_default_dependencies(arrow) - add_default_dependencies(protoc) - add_default_dependencies(thrift_static) - add_default_dependencies(cityhash) - add_default_dependencies(farmhash) - add_default_dependencies(murmurhash) - add_default_dependencies(metrohash) - add_default_dependencies(metrohash128) - add_default_dependencies(consistent-hashing) - add_default_dependencies(double-conversion) - add_default_dependencies(cctz) - add_default_dependencies(kj) - add_default_dependencies(simdjson) - add_default_dependencies(apple_rt) - add_default_dependencies(h3) - add_default_dependencies(re2) - add_default_dependencies(re2_st) - add_default_dependencies(hs_compile_shared) - add_default_dependencies(hs_exec_shared) - add_default_dependencies(hs_shared) - add_default_dependencies(widechar_width) - add_default_dependencies(string_utils) - add_default_dependencies(consistent-hashing-sumbur) - add_default_dependencies(boost_program_options_internal) - add_default_dependencies(boost_system_internal) - add_default_dependencies(boost_regex_internal) - add_default_dependencies(roaring) - add_default_dependencies(btrie) - add_default_dependencies(cpuid) - add_default_dependencies(mysqlclient) - add_default_dependencies(zlib) - add_default_dependencies(thrift) - add_default_dependencies(brotli) - add_default_dependencies(libprotobuf) - add_default_dependencies(base64) - add_default_dependencies(readpassphrase) - add_default_dependencies(unwind_static) - add_default_dependencies(fastops) -endif () diff --git a/cmake/default_libs.cmake b/cmake/default_libs.cmake new file mode 100644 index 00000000000..54a01042558 --- /dev/null +++ b/cmake/default_libs.cmake @@ -0,0 +1,48 @@ +# Set standard, system and compiler libraries explicitly. +# This is intended for more control of what we are linking. + +set (DEFAULT_LIBS "-nodefaultlibs") + +if (OS_LINUX) + # We need builtins from Clang's RT even without libcxx - for ubsan+int128. + # See https://bugs.llvm.org/show_bug.cgi?id=16404 + if (COMPILER_CLANG) + execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-${CMAKE_SYSTEM_PROCESSOR}.a OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) + else () + set (BUILTINS_LIBRARY "-lgcc") + endif () + + set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -lrt -lpthread -ldl") + + message(STATUS "Default libraries: ${DEFAULT_LIBS}") +endif () + +set(CMAKE_CXX_STANDARD_LIBRARIES ${DEFAULT_LIBS}) +set(CMAKE_C_STANDARD_LIBRARIES ${DEFAULT_LIBS}) + +# Global libraries + +add_library(global-libs INTERFACE) + +# Unfortunately '-pthread' doesn't work with '-nodefaultlibs'. +# Just make sure we have pthreads at all. +set(THREADS_PREFER_PTHREAD_FLAG ON) +find_package(Threads REQUIRED) + +add_subdirectory(libs/libglibc-compatibility) +include (cmake/find_unwind.cmake) +include (cmake/find_cxx.cmake) + +add_library(global-group INTERFACE) +target_link_libraries(global-group INTERFACE + -Wl,--start-group + $ + -Wl,--end-group +) + +link_libraries(global-group) + +install( + TARGETS global-group global-libs + EXPORT global +) diff --git a/cmake/find_capnp.cmake b/cmake/find_capnp.cmake index ec591afdc38..572fc1b3341 100644 --- a/cmake/find_capnp.cmake +++ b/cmake/find_capnp.cmake @@ -1,50 +1,20 @@ -option (ENABLE_CAPNP "Enable Cap'n Proto" ON) - -if (ENABLE_CAPNP) - # cmake 3.5.1 bug: - # capnproto uses this cmake feature: - # target_compile_features(kj PUBLIC cxx_constexpr) - # old cmake adds -std=gnu++11 to end of all compile commands (even if -std=gnu++17 already present in compile string) - # cmake 3.9.1 (ubuntu artful) have no this bug (c++17 support added to cmake 3.8.2) - if (CMAKE_VERSION VERSION_LESS "3.8.0") - set (USE_INTERNAL_CAPNP_LIBRARY_DEFAULT 0) - set (MISSING_INTERNAL_CAPNP_LIBRARY 1) - else () - set (USE_INTERNAL_CAPNP_LIBRARY_DEFAULT ${NOT_UNBUNDLED}) - endif () - - option (USE_INTERNAL_CAPNP_LIBRARY "Set to FALSE to use system capnproto library instead of bundled" ${USE_INTERNAL_CAPNP_LIBRARY_DEFAULT}) - - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/capnproto/c++/CMakeLists.txt") - if (USE_INTERNAL_CAPNP_LIBRARY) - message (WARNING "submodule contrib/capnproto is missing. to fix try run: \n git submodule update --init --recursive") - endif () - set (USE_INTERNAL_CAPNP_LIBRARY 0) - set (MISSING_INTERNAL_CAPNP_LIBRARY 1) - endif () - - if (NOT USE_INTERNAL_CAPNP_LIBRARY) - set (CAPNP_PATHS "/usr/local/opt/capnp/lib") - set (CAPNP_INCLUDE_PATHS "/usr/local/opt/capnp/include") - find_library (CAPNP capnp PATHS ${CAPNP_PATHS}) - find_library (CAPNPC capnpc PATHS ${CAPNP_PATHS}) - find_library (KJ kj PATHS ${CAPNP_PATHS}) - set (CAPNP_LIBRARY ${CAPNPC} ${CAPNP} ${KJ}) - find_path (CAPNP_INCLUDE_DIR NAMES capnp/schema-parser.h PATHS ${CAPNP_INCLUDE_PATHS}) - endif () - - if (CAPNP_INCLUDE_DIR AND CAPNP_LIBRARY) - set(USE_CAPNP 1) - elseif (NOT MISSING_INTERNAL_CAPNP_LIBRARY) - set (USE_INTERNAL_CAPNP_LIBRARY 1) - set (CAPNP_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/capnproto/c++/src") - set (CAPNP_LIBRARY capnpc) - set (USE_CAPNP 1) - endif () -endif () +option (USE_CAPNP "Enable Cap'n Proto" ON) if (USE_CAPNP) - message (STATUS "Using capnp=${USE_CAPNP}: ${CAPNP_INCLUDE_DIR} : ${CAPNP_LIBRARY}") -else () - message (STATUS "Build without capnp (support for Cap'n Proto format will be disabled)") + option (USE_INTERNAL_CAPNP_LIBRARY "Set to FALSE to use system capnproto library instead of bundled" ${NOT_UNBUNDLED}) + + # FIXME: refactor to use `add_library(… IMPORTED)` if possible. + if (NOT USE_INTERNAL_CAPNP_LIBRARY) + find_library (KJ kj) + find_library (CAPNP capnp) + find_library (CAPNPC capnpc) + + set (CAPNP_LIBRARIES ${CAPNPC} ${CAPNP} ${KJ}) + else () + add_subdirectory(contrib/capnproto-cmake) + + set (CAPNP_LIBRARIES capnpc) + endif () + + message (STATUS "Using capnp: ${CAPNP_LIBRARIES}") endif () diff --git a/cmake/find_cxx.cmake b/cmake/find_cxx.cmake index 2b2952f6efd..35a0b9d0927 100644 --- a/cmake/find_cxx.cmake +++ b/cmake/find_cxx.cmake @@ -1,26 +1,49 @@ -if (NOT APPLE) +if (OS_LINUX AND COMPILER_CLANG) + option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++" ${HAVE_LIBCXX}) option (USE_INTERNAL_LIBCXX_LIBRARY "Set to FALSE to use system libcxx and libcxxabi libraries instead of bundled" ${NOT_UNBUNDLED}) +endif() + +if (USE_LIBCXX) + set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=0") # More checks in debug build. endif () +# FIXME: make better check for submodule presence if (USE_INTERNAL_LIBCXX_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libcxx/include/vector") message (WARNING "submodule contrib/libcxx is missing. to fix try run: \n git submodule update --init --recursive") set (USE_INTERNAL_LIBCXX_LIBRARY 0) endif () +# FIXME: make better check for submodule presence if (USE_INTERNAL_LIBCXX_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libcxxabi/src") message (WARNING "submodule contrib/libcxxabi is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_LIBCXXABI_LIBRARY 0) + set (USE_INTERNAL_LIBCXX_LIBRARY 0) endif () -if (NOT USE_INTERNAL_LIBCXX_LIBRARY) - find_library (LIBCXX_LIBRARY c++) - find_library (LIBCXXABI_LIBRARY c++abi) +if (USE_LIBCXX) + if (NOT USE_INTERNAL_LIBCXX_LIBRARY) + find_library (LIBCXX_LIBRARY c++) + find_library (LIBCXXFS_LIBRARY c++fs) + find_library (LIBCXXABI_LIBRARY c++abi) + + target_link_libraries(global-libs INTERFACE ${EXCEPTION_HANDLING_LIBRARY}) + else () + set (LIBCXX_LIBRARY cxx) + set (LIBCXXABI_LIBRARY cxxabi) + add_subdirectory(contrib/libcxxabi-cmake) + add_subdirectory(contrib/libcxx-cmake) + + # Exception handling library is embedded into libcxxabi. + endif () + + target_link_libraries(global-libs INTERFACE ${LIBCXX_LIBRARY} ${LIBCXXABI_LIBRARY} ${LIBCXXFS_LIBRARY}) + + set (HAVE_LIBCXX 1) + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") + + message (STATUS "Using libcxx: ${LIBCXX_LIBRARY}") + message (STATUS "Using libcxxfs: ${LIBCXXFS_LIBRARY}") + message (STATUS "Using libcxxabi: ${LIBCXXABI_LIBRARY}") else () - set (LIBCXX_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libcxx/include) - set (LIBCXXABI_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libcxxabi/include) - set (LIBCXX_LIBRARY cxx_static) - set (LIBCXXABI_LIBRARY cxxabi_static) + target_link_libraries(global-libs INTERFACE -l:libstdc++.a -l:libstdc++fs.a) # Always link these libraries as static + target_link_libraries(global-libs INTERFACE ${EXCEPTION_HANDLING_LIBRARY}) endif () - -message (STATUS "Using libcxx: ${LIBCXX_LIBRARY}") -message (STATUS "Using libcxxabi: ${LIBCXXABI_LIBRARY}") diff --git a/cmake/find_unwind.cmake b/cmake/find_unwind.cmake index 25e088e8deb..ea6e1d4bacb 100644 --- a/cmake/find_unwind.cmake +++ b/cmake/find_unwind.cmake @@ -1,59 +1,17 @@ -include (CMakePushCheckState) -cmake_push_check_state () +option (USE_UNWIND "Enable libunwind (better stacktraces)" ON) -option (ENABLE_UNWIND "Enable libunwind (better stacktraces)" ON) +if (NOT CMAKE_SYSTEM MATCHES "Linux" OR ARCH_ARM OR ARCH_32) + set (USE_UNWIND OFF) +endif () -if (ENABLE_UNWIND) +if (USE_UNWIND) + add_subdirectory(contrib/libunwind-cmake) + set (UNWIND_LIBRARIES unwind) + set (EXCEPTION_HANDLING_LIBRARY ${UNWIND_LIBRARIES}) -if (CMAKE_SYSTEM MATCHES "Linux" AND NOT ARCH_ARM AND NOT ARCH_32) - option (USE_INTERNAL_UNWIND_LIBRARY "Set to FALSE to use system unwind library instead of bundled" ${NOT_UNBUNDLED}) + message (STATUS "Using libunwind: ${UNWIND_LIBRARIES}") else () - option (USE_INTERNAL_UNWIND_LIBRARY "Set to FALSE to use system unwind library instead of bundled" OFF) + set (EXCEPTION_HANDLING_LIBRARY gcc_eh) endif () -if (NOT USE_INTERNAL_UNWIND_LIBRARY) - find_library (UNWIND_LIBRARY unwind) - find_path (UNWIND_INCLUDE_DIR NAMES unwind.h PATHS ${UNWIND_INCLUDE_PATHS}) - - include (CheckCXXSourceCompiles) - set(CMAKE_REQUIRED_INCLUDES ${UNWIND_INCLUDE_DIR}) - set(CMAKE_REQUIRED_LIBRARIES ${UNWIND_LIBRARY}) - check_cxx_source_compiles(" - #include - #define UNW_LOCAL_ONLY - #include - int main () { - ucontext_t context; - unw_cursor_t cursor; - unw_init_local(&cursor, &context); - return 0; - } - " HAVE_UNW_INIT_LOCAL) - if (NOT HAVE_UNW_INIT_LOCAL) - set(UNWIND_LIBRARY "") - set(UNWIND_INCLUDE_DIR "") - endif () - -endif () - -if (UNWIND_LIBRARY AND UNWIND_INCLUDE_DIR) - set (USE_UNWIND 1) -elseif (CMAKE_SYSTEM MATCHES "Linux" AND NOT ARCH_ARM AND NOT ARCH_32 AND NOT UNBUNDLED) - set (USE_INTERNAL_UNWIND_LIBRARY 1) - - set (PACKAGE_VERSION "9.0.0svn" CACHE STRING "") - - set (UNWIND_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libunwind/include") - - set (LIBUNWIND_ENABLE_SHARED OFF CACHE BOOL "") - set (LIBUNWIND_ENABLE_STATIC ON CACHE BOOL "") - set (UNWIND_LIBRARY unwind_static) - - set (USE_UNWIND 1) -endif () - -endif () - -message (STATUS "Using unwind=${USE_UNWIND}: ${UNWIND_INCLUDE_DIR} : ${UNWIND_LIBRARY}") - -cmake_pop_check_state () +message (STATUS "Using exception handler: ${EXCEPTION_HANDLING_LIBRARY}") diff --git a/cmake/test_compiler.cmake b/cmake/test_compiler.cmake deleted file mode 100644 index 570c058b9f7..00000000000 --- a/cmake/test_compiler.cmake +++ /dev/null @@ -1,47 +0,0 @@ -include (CheckCXXSourceCompiles) -include (CMakePushCheckState) - -set(THREADS_PREFER_PTHREAD_FLAG ON) -find_package(Threads) - -cmake_push_check_state () - -if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang") -# clang4 : -no-pie cause error -# clang6 : -no-pie cause warning - - if (MAKE_STATIC_LIBRARIES) - set (TEST_FLAG "-Wl,-Bstatic -stdlib=libc++ -lc++ -lc++abi -Wl,-Bdynamic") - else () - set (TEST_FLAG "-stdlib=libc++ -lc++ -lc++abi") - endif () - - set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}") - set (CMAKE_REQUIRED_LIBRARIES ${CMAKE_REQUIRED_LIBRARIES} Threads::Threads) - - check_cxx_source_compiles(" - #include - int main() { - std::cerr << std::endl; - return 0; - } - " HAVE_LIBCXX) - -else () - - set (TEST_FLAG "-no-pie") - set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}") - - check_cxx_source_compiles(" - int main() { - return 0; - } - " HAVE_NO_PIE) - - if (HAVE_NO_PIE) - set (FLAG_NO_PIE ${TEST_FLAG}) - endif () - -endif () - -cmake_pop_check_state () diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index e652c393141..96462de0190 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -23,16 +23,6 @@ if (USE_INTERNAL_ORC_LIBRARY) add_subdirectory(orc) endif() -if (USE_INTERNAL_UNWIND_LIBRARY) - add_subdirectory (libunwind-cmake) -endif () - -if (USE_LIBCXX AND USE_INTERNAL_LIBCXX_LIBRARY) - add_subdirectory(libcxx-cmake) - add_subdirectory(libcxxabi-cmake) -endif() - - if (USE_INTERNAL_BOOST_LIBRARY) add_subdirectory (boost-cmake) endif () @@ -172,15 +162,6 @@ if (ENABLE_ODBC AND USE_INTERNAL_ODBC_LIBRARY) add_library(ODBC::ODBC ALIAS ${ODBC_LIBRARIES}) endif () -if (ENABLE_CAPNP AND USE_INTERNAL_CAPNP_LIBRARY) - set (BUILD_TESTING 0 CACHE INTERNAL "") - set (_save ${CMAKE_CXX_EXTENSIONS}) - set (CMAKE_CXX_EXTENSIONS) - add_subdirectory (capnproto/c++) - set (CMAKE_CXX_EXTENSIONS ${_save}) - target_include_directories(${CAPNP_LIBRARY} PUBLIC $) -endif () - if (USE_INTERNAL_PARQUET_LIBRARY) if (USE_INTERNAL_PARQUET_LIBRARY_NATIVE_CMAKE) # We dont use arrow's cmakefiles because they uses too many depends and download some libs in compile time diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 843ff9cd8af..ba1ddc2414a 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -44,7 +44,6 @@ set( thriftcpp_threads_SOURCES add_library(${THRIFT_LIBRARY} ${thriftcpp_SOURCES} ${thriftcpp_threads_SOURCES}) set_target_properties(${THRIFT_LIBRARY} PROPERTIES CXX_STANDARD 14) # REMOVE after https://github.com/apache/thrift/pull/1641 target_include_directories(${THRIFT_LIBRARY} SYSTEM PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src PRIVATE ${Boost_INCLUDE_DIRS}) -target_link_libraries(${THRIFT_LIBRARY} PRIVATE Threads::Threads) # === orc @@ -219,7 +218,7 @@ endif() add_library(${ARROW_LIBRARY} ${ARROW_SRCS}) add_dependencies(${ARROW_LIBRARY} protoc) target_include_directories(${ARROW_LIBRARY} SYSTEM PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}/cpp/src ${Boost_INCLUDE_DIRS}) -target_link_libraries(${ARROW_LIBRARY} PRIVATE ${DOUBLE_CONVERSION_LIBRARIES} ${Protobuf_LIBRARY} Threads::Threads) +target_link_libraries(${ARROW_LIBRARY} PRIVATE ${DOUBLE_CONVERSION_LIBRARIES} ${Protobuf_LIBRARY}) if (ARROW_WITH_LZ4) target_link_libraries(${ARROW_LIBRARY} PRIVATE ${LZ4_LIBRARY}) endif() diff --git a/contrib/capnproto-cmake/CMakeLists.txt b/contrib/capnproto-cmake/CMakeLists.txt new file mode 100644 index 00000000000..275007c145f --- /dev/null +++ b/contrib/capnproto-cmake/CMakeLists.txt @@ -0,0 +1,68 @@ +set (CAPNPROTO_SOURCE_DIR ${ClickHouse_SOURCE_DIR}/contrib/capnproto/c++/src) + +set (KJ_SRCS + ${CAPNPROTO_SOURCE_DIR}/kj/array.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/common.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/debug.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/exception.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/io.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/memory.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/mutex.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/string.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/hash.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/table.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/thread.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/main.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/arena.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/test-helpers.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/units.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/encoding.c++ + + ${CAPNPROTO_SOURCE_DIR}/kj/refcount.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/string-tree.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/time.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/filesystem.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/filesystem-disk-unix.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/filesystem-disk-win32.c++ + ${CAPNPROTO_SOURCE_DIR}/kj/parse/char.c++ +) + +add_library(kj ${KJ_SRCS}) +target_include_directories(kj INTERFACE ${CAPNPROTO_SOURCE_DIR}) + +set (CAPNP_SRCS + ${CAPNPROTO_SOURCE_DIR}/capnp/c++.capnp.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/blob.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/arena.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/layout.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/list.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/any.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/message.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/schema.capnp.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/serialize.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/serialize-packed.c++ + + ${CAPNPROTO_SOURCE_DIR}/capnp/schema.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/schema-loader.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/dynamic.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/stringify.c++ +) + +add_library(capnp ${CAPNP_SRCS}) +target_link_libraries(capnp PUBLIC kj) + +set (CAPNPC_SRCS + ${CAPNPROTO_SOURCE_DIR}/capnp/compiler/type-id.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/compiler/error-reporter.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/compiler/lexer.capnp.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/compiler/lexer.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/compiler/grammar.capnp.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/compiler/parser.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/compiler/node-translator.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/compiler/compiler.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/schema-parser.c++ + ${CAPNPROTO_SOURCE_DIR}/capnp/serialize-text.c++ +) + +add_library(capnpc ${CAPNPC_SRCS}) +target_link_libraries(capnpc PUBLIC capnp) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 47f057c0559..e44c54d2b37 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -59,7 +59,6 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") if (USE_UNWIND) target_compile_definitions (jemalloc PRIVATE -DJEMALLOC_PROF_LIBUNWIND=1) - target_include_directories (jemalloc BEFORE PRIVATE ${UNWIND_INCLUDE_DIR}) - target_link_libraries (jemalloc PRIVATE ${UNWIND_LIBRARY}) + target_link_libraries (jemalloc PRIVATE ${UNWIND_LIBRARIES}) endif () endif () diff --git a/contrib/libcxx-cmake/CMakeLists.txt b/contrib/libcxx-cmake/CMakeLists.txt index e9ca5e1e7cd..07fa70b9869 100644 --- a/contrib/libcxx-cmake/CMakeLists.txt +++ b/contrib/libcxx-cmake/CMakeLists.txt @@ -1,5 +1,4 @@ set(LIBCXX_SOURCE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libcxx) -#set(LIBCXX_BINARY_DIR ${ClickHouse_BINARY_DIR}/contrib/libcxx) set(SRCS ${LIBCXX_SOURCE_DIR}/src/optional.cpp @@ -16,10 +15,6 @@ ${LIBCXX_SOURCE_DIR}/src/condition_variable.cpp ${LIBCXX_SOURCE_DIR}/src/hash.cpp ${LIBCXX_SOURCE_DIR}/src/string.cpp ${LIBCXX_SOURCE_DIR}/src/debug.cpp -#${LIBCXX_SOURCE_DIR}/src/support/win32/support.cpp -#${LIBCXX_SOURCE_DIR}/src/support/win32/locale_win32.cpp -#${LIBCXX_SOURCE_DIR}/src/support/win32/thread_win32.cpp -#${LIBCXX_SOURCE_DIR}/src/support/solaris/xlocale.cpp ${LIBCXX_SOURCE_DIR}/src/stdexcept.cpp ${LIBCXX_SOURCE_DIR}/src/utility.cpp ${LIBCXX_SOURCE_DIR}/src/any.cpp @@ -43,9 +38,16 @@ ${LIBCXX_SOURCE_DIR}/src/system_error.cpp ${LIBCXX_SOURCE_DIR}/src/random.cpp ) -add_library(cxx_static ${SRCS}) +add_library(cxx ${SRCS}) -target_include_directories(cxx_static PUBLIC ${LIBCXX_SOURCE_DIR}/include) -target_compile_definitions(cxx_static PRIVATE -D_LIBCPP_BUILDING_LIBRARY -DLIBCXX_BUILDING_LIBCXXABI) -target_compile_options(cxx_static PRIVATE -nostdinc++) +target_include_directories(cxx SYSTEM BEFORE PUBLIC $) +target_compile_definitions(cxx PRIVATE -D_LIBCPP_BUILDING_LIBRARY -DLIBCXX_BUILDING_LIBCXXABI) +target_compile_options(cxx PRIVATE -nostdinc++) +target_link_libraries(cxx PUBLIC cxxabi) +install( + TARGETS cxx + EXPORT global + ARCHIVE DESTINATION lib + RUNTIME DESTINATION lib +) diff --git a/contrib/libcxxabi-cmake/CMakeLists.txt b/contrib/libcxxabi-cmake/CMakeLists.txt index 2abece86691..546d39933af 100644 --- a/contrib/libcxxabi-cmake/CMakeLists.txt +++ b/contrib/libcxxabi-cmake/CMakeLists.txt @@ -1,13 +1,10 @@ set(LIBCXXABI_SOURCE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libcxxabi) -set(LIBCXX_SOURCE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libcxx) -#set(LIBCXXABI_BINARY_DIR ${ClickHouse_BINARY_DIR}/contrib/libcxxabi) set(SRCS ${LIBCXXABI_SOURCE_DIR}/src/stdlib_stdexcept.cpp ${LIBCXXABI_SOURCE_DIR}/src/cxa_virtual.cpp ${LIBCXXABI_SOURCE_DIR}/src/cxa_thread_atexit.cpp ${LIBCXXABI_SOURCE_DIR}/src/fallback_malloc.cpp -#${LIBCXXABI_SOURCE_DIR}/src/cxa_noexception.cpp ${LIBCXXABI_SOURCE_DIR}/src/cxa_guard.cpp ${LIBCXXABI_SOURCE_DIR}/src/cxa_default_handlers.cpp ${LIBCXXABI_SOURCE_DIR}/src/cxa_personality.cpp @@ -25,10 +22,24 @@ ${LIBCXXABI_SOURCE_DIR}/src/cxa_vector.cpp ${LIBCXXABI_SOURCE_DIR}/src/stdlib_new_delete.cpp ) -add_library(cxxabi_static ${SRCS}) +add_library(cxxabi ${SRCS}) -target_include_directories(cxxabi_static PUBLIC ${LIBCXXABI_SOURCE_DIR}/include ${LIBCXX_SOURCE_DIR}/include) -target_compile_definitions(cxxabi_static PRIVATE -D_LIBCPP_BUILDING_LIBRARY) -target_compile_options(cxxabi_static PRIVATE -nostdinc++ -fno-sanitize=undefined) # If we don't disable UBSan, infinite recursion happens in dynamic_cast. +target_include_directories(cxxabi SYSTEM BEFORE + PUBLIC $ + PRIVATE $ +) +target_compile_definitions(cxxabi PRIVATE -D_LIBCPP_BUILDING_LIBRARY) +target_compile_options(cxxabi PRIVATE -nostdinc++ -fno-sanitize=undefined) # If we don't disable UBSan, infinite recursion happens in dynamic_cast. +if (USE_UNWIND) + target_link_libraries(cxxabi PRIVATE ${UNWIND_LIBRARIES}) +else () + target_link_libraries(cxxabi PRIVATE gcc_eh) +endif () +install( + TARGETS cxxabi + EXPORT global + ARCHIVE DESTINATION lib + RUNTIME DESTINATION lib +) diff --git a/contrib/librdkafka-cmake/CMakeLists.txt b/contrib/librdkafka-cmake/CMakeLists.txt index 75cd3968204..64dc83fa8b6 100644 --- a/contrib/librdkafka-cmake/CMakeLists.txt +++ b/contrib/librdkafka-cmake/CMakeLists.txt @@ -65,7 +65,7 @@ add_library(rdkafka ${SRCS}) target_include_directories(rdkafka SYSTEM PUBLIC include) target_include_directories(rdkafka SYSTEM PUBLIC ${RDKAFKA_SOURCE_DIR}) # Because weird logic with "include_next" is used. target_include_directories(rdkafka SYSTEM PRIVATE ${ZSTD_INCLUDE_DIR}/common) # Because wrong path to "zstd_errors.h" is used. -target_link_libraries(rdkafka PRIVATE ${ZLIB_LIBRARIES} ${ZSTD_LIBRARY} ${LZ4_LIBRARY} ${LIBGSASL_LIBRARY} Threads::Threads) +target_link_libraries(rdkafka PRIVATE ${ZLIB_LIBRARIES} ${ZSTD_LIBRARY} ${LZ4_LIBRARY} ${LIBGSASL_LIBRARY}) if(OPENSSL_SSL_LIBRARY AND OPENSSL_CRYPTO_LIBRARY) target_link_libraries(rdkafka PRIVATE ${OPENSSL_SSL_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY}) endif() diff --git a/contrib/libunwind-cmake/CMakeLists.txt b/contrib/libunwind-cmake/CMakeLists.txt index 4f24fe249f5..f09d0979692 100644 --- a/contrib/libunwind-cmake/CMakeLists.txt +++ b/contrib/libunwind-cmake/CMakeLists.txt @@ -24,9 +24,15 @@ set(LIBUNWIND_SOURCES ${LIBUNWIND_C_SOURCES} ${LIBUNWIND_ASM_SOURCES}) -add_library(unwind_static ${LIBUNWIND_SOURCES}) +add_library(unwind ${LIBUNWIND_SOURCES}) -target_include_directories(unwind_static SYSTEM BEFORE PUBLIC ${LIBUNWIND_SOURCE_DIR}/include) -target_compile_definitions(unwind_static PRIVATE -D_LIBUNWIND_NO_HEAP=1 -D_DEBUG -D_LIBUNWIND_IS_NATIVE_ONLY) -target_compile_options(unwind_static PRIVATE -fno-exceptions -funwind-tables -fno-sanitize=all -nostdinc++ -fno-rtti) -target_link_libraries(unwind_static PRIVATE Threads::Threads ${CMAKE_DL_LIBS}) +target_include_directories(unwind SYSTEM BEFORE PUBLIC $) +target_compile_definitions(unwind PRIVATE -D_LIBUNWIND_NO_HEAP=1 -D_DEBUG -D_LIBUNWIND_IS_NATIVE_ONLY) +target_compile_options(unwind PRIVATE -fno-exceptions -funwind-tables -fno-sanitize=all -nostdinc++ -fno-rtti) + +install( + TARGETS unwind + EXPORT global + LIBRARY DESTINATION lib + ARCHIVE DESTINATION lib +) diff --git a/contrib/mariadb-connector-c-cmake/CMakeLists.txt b/contrib/mariadb-connector-c-cmake/CMakeLists.txt index 1f453a7f6d1..2e80b0c325f 100644 --- a/contrib/mariadb-connector-c-cmake/CMakeLists.txt +++ b/contrib/mariadb-connector-c-cmake/CMakeLists.txt @@ -62,11 +62,6 @@ endif() add_library(mysqlclient ${SRCS}) -target_link_libraries(mysqlclient PRIVATE ${CMAKE_DL_LIBS} Threads::Threads) -if(M_LIBRARY) - target_link_libraries(mysqlclient PRIVATE ${M_LIBRARY}) -endif() - if(OPENSSL_LIBRARIES) target_link_libraries(mysqlclient PRIVATE ${OPENSSL_LIBRARIES}) target_compile_definitions(mysqlclient PRIVATE -D HAVE_OPENSSL -D HAVE_TLS) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 355c66902a8..98eb23809da 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -97,8 +97,6 @@ add_subdirectory (src) set(dbms_headers) set(dbms_sources) -include(../cmake/dbms_glob_sources.cmake) - add_headers_and_sources(clickhouse_common_io src/Common) add_headers_and_sources(clickhouse_common_io src/Common/HashTable) add_headers_and_sources(clickhouse_common_io src/IO) @@ -163,9 +161,7 @@ if (OS_FREEBSD) endif () if (USE_UNWIND) - if (NOT USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING) - target_link_libraries (clickhouse_common_io PRIVATE ${UNWIND_LIBRARY}) - endif () + target_link_libraries (clickhouse_common_io PRIVATE ${UNWIND_LIBRARIES}) endif () add_subdirectory(src/Common/ZooKeeper) @@ -245,12 +241,6 @@ target_link_libraries(clickhouse_common_io PRIVATE apple_rt PUBLIC - Threads::Threads - PRIVATE - ${CMAKE_DL_LIBS} - PRIVATE - rt - PUBLIC roaring ) @@ -298,7 +288,6 @@ target_link_libraries (dbms ${Boost_FILESYSTEM_LIBRARY} PUBLIC ${Boost_SYSTEM_LIBRARY} - Threads::Threads ) target_include_directories(dbms PUBLIC ${CMAKE_CURRENT_BINARY_DIR}/src/Core/include) @@ -365,10 +354,7 @@ if (USE_ICU) endif () if (USE_CAPNP) - target_link_libraries (dbms PRIVATE ${CAPNP_LIBRARY}) - if (NOT USE_INTERNAL_CAPNP_LIBRARY) - target_include_directories (dbms SYSTEM BEFORE PRIVATE ${CAPNP_INCLUDE_DIR}) - endif () + target_link_libraries (dbms PRIVATE ${CAPNP_LIBRARIES}) endif () if (USE_PARQUET) @@ -381,7 +367,6 @@ endif () if(OPENSSL_CRYPTO_LIBRARY) target_link_libraries(dbms PRIVATE ${OPENSSL_CRYPTO_LIBRARY}) endif () -target_link_libraries(dbms PRIVATE Threads::Threads) target_include_directories (dbms SYSTEM BEFORE PRIVATE ${DIVIDE_INCLUDE_DIR}) target_include_directories (dbms SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index ef61537e38d..5f5e464eb01 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -520,7 +520,7 @@ int Server::main(const std::vector & /*args*/) /// Init trace collector only after trace_log system table was created /// Disable it if we collect test coverage information, because it will work extremely slow. -#if USE_INTERNAL_UNWIND_LIBRARY && !WITH_COVERAGE +#if USE_UNWIND && !WITH_COVERAGE /// QueryProfiler cannot work reliably with any other libunwind or without PHDR cache. if (hasPHDRCache()) global_context->initializeTraceCollector(); diff --git a/dbms/src/Common/Config/CMakeLists.txt b/dbms/src/Common/Config/CMakeLists.txt index e1f78e2c213..c2869763f1b 100644 --- a/dbms/src/Common/Config/CMakeLists.txt +++ b/dbms/src/Common/Config/CMakeLists.txt @@ -4,5 +4,5 @@ add_headers_and_sources(clickhouse_common_config .) add_library(clickhouse_common_config ${clickhouse_common_config_headers} ${clickhouse_common_config_sources}) -target_link_libraries(clickhouse_common_config PUBLIC common PRIVATE clickhouse_common_zookeeper string_utils PUBLIC ${Poco_XML_LIBRARY} ${Poco_Util_LIBRARY} Threads::Threads) +target_link_libraries(clickhouse_common_config PUBLIC common PRIVATE clickhouse_common_zookeeper string_utils PUBLIC ${Poco_XML_LIBRARY} ${Poco_Util_LIBRARY}) target_include_directories(clickhouse_common_config PUBLIC ${DBMS_INCLUDE_DIR}) diff --git a/dbms/src/Common/QueryProfiler.cpp b/dbms/src/Common/QueryProfiler.cpp index b39cdfb4ef5..a0b75c567a9 100644 --- a/dbms/src/Common/QueryProfiler.cpp +++ b/dbms/src/Common/QueryProfiler.cpp @@ -100,7 +100,7 @@ QueryProfilerBase::QueryProfilerBase(const Int32 thread_id, const : log(&Logger::get("QueryProfiler")) , pause_signal(pause_signal_) { -#if USE_INTERNAL_UNWIND_LIBRARY +#if USE_UNWIND /// Sanity check. if (!hasPHDRCache()) throw Exception("QueryProfiler cannot be used without PHDR cache, that is not available for TSan build", ErrorCodes::NOT_IMPLEMENTED); @@ -173,7 +173,7 @@ QueryProfilerBase::~QueryProfilerBase() template void QueryProfilerBase::tryCleanup() { -#if USE_INTERNAL_UNWIND_LIBRARY +#if USE_UNWIND if (timer_id != nullptr && timer_delete(timer_id)) LOG_ERROR(log, "Failed to delete query profiler timer " + errnoToString(ErrorCodes::CANNOT_DELETE_TIMER)); diff --git a/dbms/src/Common/QueryProfiler.h b/dbms/src/Common/QueryProfiler.h index b6420ccc703..abbff6f6c0e 100644 --- a/dbms/src/Common/QueryProfiler.h +++ b/dbms/src/Common/QueryProfiler.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -43,7 +44,7 @@ private: Poco::Logger * log; -#if USE_INTERNAL_UNWIND_LIBRARY +#if USE_UNWIND /// Timer id from timer_create(2) timer_t timer_id = nullptr; #endif diff --git a/dbms/src/Common/StackTrace.cpp b/dbms/src/Common/StackTrace.cpp index a642ec2b73a..9981d0941aa 100644 --- a/dbms/src/Common/StackTrace.cpp +++ b/dbms/src/Common/StackTrace.cpp @@ -1,15 +1,20 @@ -#include -#include -#include #include -#include + #include #include -#include -#include -#include -#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if USE_UNWIND +# include +#endif std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext_t & context) { @@ -215,12 +220,6 @@ StackTrace::StackTrace(NoCapture) { } - -#if USE_UNWIND -extern "C" int unw_backtrace(void **, int); -#endif - - void StackTrace::tryCapture() { size = 0; diff --git a/dbms/src/Common/ZooKeeper/CMakeLists.txt b/dbms/src/Common/ZooKeeper/CMakeLists.txt index 6a611886e65..aa6efcd3ca1 100644 --- a/dbms/src/Common/ZooKeeper/CMakeLists.txt +++ b/dbms/src/Common/ZooKeeper/CMakeLists.txt @@ -4,7 +4,7 @@ add_headers_and_sources(clickhouse_common_zookeeper .) add_library(clickhouse_common_zookeeper ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) -target_link_libraries (clickhouse_common_zookeeper PUBLIC clickhouse_common_io common PRIVATE string_utils PUBLIC ${Poco_Util_LIBRARY} Threads::Threads) +target_link_libraries (clickhouse_common_zookeeper PUBLIC clickhouse_common_io common PRIVATE string_utils PUBLIC ${Poco_Util_LIBRARY}) target_include_directories(clickhouse_common_zookeeper PUBLIC ${DBMS_INCLUDE_DIR}) if (ENABLE_TESTS) diff --git a/dbms/src/Dictionaries/CMakeLists.txt b/dbms/src/Dictionaries/CMakeLists.txt index af858e6b26d..4d066d1f59b 100644 --- a/dbms/src/Dictionaries/CMakeLists.txt +++ b/dbms/src/Dictionaries/CMakeLists.txt @@ -15,7 +15,7 @@ list(REMOVE_ITEM clickhouse_dictionaries_sources DictionaryFactory.cpp Dictionar list(REMOVE_ITEM clickhouse_dictionaries_headers DictionaryFactory.h DictionarySourceFactory.h DictionaryStructure.h) add_library(clickhouse_dictionaries ${clickhouse_dictionaries_sources}) -target_link_libraries(clickhouse_dictionaries PRIVATE dbms clickhouse_common_io ${BTRIE_LIBRARIES} PUBLIC Threads::Threads) +target_link_libraries(clickhouse_dictionaries PRIVATE dbms clickhouse_common_io ${BTRIE_LIBRARIES}) if(Poco_SQL_FOUND AND NOT USE_INTERNAL_POCO_LIBRARY) target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${Poco_SQL_INCLUDE_DIR}) diff --git a/dbms/src/Interpreters/tests/CMakeLists.txt b/dbms/src/Interpreters/tests/CMakeLists.txt index 3fac5424c00..03c06eb7257 100644 --- a/dbms/src/Interpreters/tests/CMakeLists.txt +++ b/dbms/src/Interpreters/tests/CMakeLists.txt @@ -57,5 +57,5 @@ target_link_libraries (users PRIVATE dbms clickhouse_common_config stdc++fs) if (OS_LINUX) add_executable (internal_iotop internal_iotop.cpp) - target_link_libraries (internal_iotop PRIVATE dbms Threads::Threads) + target_link_libraries (internal_iotop PRIVATE dbms) endif () diff --git a/dbms/tests/tsan_suppressions.txt b/dbms/tests/tsan_suppressions.txt index 476e135de14..3dc306ee133 100644 --- a/dbms/tests/tsan_suppressions.txt +++ b/dbms/tests/tsan_suppressions.txt @@ -1,2 +1,5 @@ # libc++ race:locale + +# Too many mutexes: https://github.com/google/sanitizers/issues/950 +deadlock:DB::MergeTreeReadPool::fillPerPartInfo diff --git a/docker/packager/packager b/docker/packager/packager index 0e8bf6ea98d..c132f514569 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -143,10 +143,10 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, cache, di result.append("ALIEN_PKGS='" + ' '.join(['--' + pkg for pkg in alien_pkgs]) + "'") if unbundled: - cmake_flags.append('-DUNBUNDLED=1 -DENABLE_MYSQL=0 -DENABLE_POCO_ODBC=0 -DENABLE_ODBC=0') + cmake_flags.append('-DUNBUNDLED=1 -DENABLE_MYSQL=0 -DENABLE_POCO_ODBC=0 -DENABLE_ODBC=0 -DUSE_CAPNP=0') if split_binary: - cmake_flags.append('-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 -DGLIBC_COMPATIBILITY=ON') + cmake_flags.append('-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1') if with_coverage: cmake_flags.append('-DWITH_COVERAGE=1') diff --git a/libs/CMakeLists.txt b/libs/CMakeLists.txt index d526a662dc0..b5bcbd804be 100644 --- a/libs/CMakeLists.txt +++ b/libs/CMakeLists.txt @@ -15,10 +15,6 @@ if (USE_INTERNAL_MEMCPY) add_subdirectory (libmemcpy) endif() -if (GLIBC_COMPATIBILITY) - add_subdirectory (libglibc-compatibility) -endif () - if (USE_MYSQL) add_subdirectory (libmysqlxx) endif () diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index 2744714a9c4..c78473890dc 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -123,9 +123,7 @@ target_link_libraries (common PUBLIC ${Boost_SYSTEM_LIBRARY} PRIVATE - ${CMAKE_DL_LIBS} ${MALLOC_LIBRARIES} - Threads::Threads ${MEMCPY_LIBRARIES}) if (RT_LIBRARY) diff --git a/libs/libcommon/include/common/config_common.h.in b/libs/libcommon/include/common/config_common.h.in index 1301049b24b..810cf0b87f9 100644 --- a/libs/libcommon/include/common/config_common.h.in +++ b/libs/libcommon/include/common/config_common.h.in @@ -8,5 +8,4 @@ #cmakedefine01 USE_LIBEDIT #cmakedefine01 HAVE_READLINE_HISTORY #cmakedefine01 UNBUNDLED -#cmakedefine01 USE_INTERNAL_UNWIND_LIBRARY #cmakedefine01 WITH_COVERAGE diff --git a/libs/libcommon/src/tests/CMakeLists.txt b/libs/libcommon/src/tests/CMakeLists.txt index 2bb8afe6fa1..15d872ac49d 100644 --- a/libs/libcommon/src/tests/CMakeLists.txt +++ b/libs/libcommon/src/tests/CMakeLists.txt @@ -16,7 +16,7 @@ target_link_libraries (date_lut3 common ${PLATFORM_LIBS}) target_link_libraries (date_lut4 common ${PLATFORM_LIBS}) target_link_libraries (date_lut_default_timezone common ${PLATFORM_LIBS}) target_link_libraries (local_date_time_comparison common) -target_link_libraries (realloc-perf common Threads::Threads) +target_link_libraries (realloc-perf common) add_check(local_date_time_comparison) if(USE_GTEST) diff --git a/libs/libglibc-compatibility/CMakeLists.txt b/libs/libglibc-compatibility/CMakeLists.txt index fe98ae9bf0d..a62f5e75e17 100644 --- a/libs/libglibc-compatibility/CMakeLists.txt +++ b/libs/libglibc-compatibility/CMakeLists.txt @@ -1,25 +1,45 @@ -enable_language(ASM) -include(CheckIncludeFile) +if (GLIBC_COMPATIBILITY) + set (USE_INTERNAL_MEMCPY ON) -check_include_file("sys/random.h" HAVE_SYS_RANDOM_H) + enable_language(ASM) + include(CheckIncludeFile) -if(COMPILER_CLANG) - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-builtin-requires-header") -endif() + check_include_file("sys/random.h" HAVE_SYS_RANDOM_H) -add_headers_and_sources(glibc_compatibility .) -add_headers_and_sources(glibc_compatibility musl) -list(APPEND glibc_compatibility_sources musl/syscall.s musl/longjmp.s) + if(COMPILER_CLANG) + set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-builtin-requires-header") + endif() -list(REMOVE_ITEM glibc_compatibility_sources musl/getentropy.c) -if(HAVE_SYS_RANDOM_H) - list(APPEND glibc_compatibility_sources musl/getentropy.c) -endif() + add_headers_and_sources(glibc_compatibility .) + add_headers_and_sources(glibc_compatibility musl) + list(APPEND glibc_compatibility_sources musl/syscall.s musl/longjmp.s) -if(MAKE_STATIC_LIBRARIES) - list(APPEND glibc_compatibility_sources libcxxabi/cxa_thread_atexit.cpp) -endif() + list(REMOVE_ITEM glibc_compatibility_sources musl/getentropy.c) + if(HAVE_SYS_RANDOM_H) + list(APPEND glibc_compatibility_sources musl/getentropy.c) + endif() -add_library(glibc-compatibility STATIC ${glibc_compatibility_sources}) + if(MAKE_STATIC_LIBRARIES) + list(APPEND glibc_compatibility_sources libcxxabi/cxa_thread_atexit.cpp) + endif() -target_include_directories(glibc-compatibility PRIVATE libcxxabi) + add_library(glibc-compatibility STATIC ${glibc_compatibility_sources}) + + target_include_directories(glibc-compatibility PRIVATE libcxxabi) + + if (USE_STATIC_LIBRARIES=0 AND MAKE_STATIC_LIBRARIES=OFF) + target_compile_options(PRIVATE -fPIC) + endif () + + target_link_libraries(global-libs INTERFACE glibc-compatibility) + + install( + TARGETS glibc-compatibility + EXPORT global + ARCHIVE DESTINATION lib + ) + + message (STATUS "Some symbols from glibc will be replaced for compatibility") +elseif (YANDEX_OFFICIAL_BUILD) + message (WARNING "Option GLIBC_COMPATIBILITY must be turned on for production builds.") +endif () diff --git a/utils/compressor/CMakeLists.txt b/utils/compressor/CMakeLists.txt index 3fdf8aa5eaf..c032054187b 100644 --- a/utils/compressor/CMakeLists.txt +++ b/utils/compressor/CMakeLists.txt @@ -4,7 +4,7 @@ add_executable (zstd_test zstd_test.cpp) if(ZSTD_LIBRARY) target_link_libraries(zstd_test PRIVATE ${ZSTD_LIBRARY}) endif() -target_link_libraries (zstd_test PRIVATE common Threads::Threads) +target_link_libraries (zstd_test PRIVATE common) add_executable (mutator mutator.cpp) target_link_libraries(mutator PRIVATE clickhouse_common_io) From d1a980fcddc9609d35270b6b18b03d7f15bb794f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 29 Aug 2019 01:18:26 +0300 Subject: [PATCH 495/509] Fix for data race in StorageMerge --- dbms/src/Storages/StorageMerge.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 6215550e413..56ab949f30c 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -366,8 +366,8 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr if (storage.get() != this) { - virtual_column->insert(storage->getTableName()); selected_tables.emplace_back(storage, get_lock ? storage->lockStructureForShare(false, query_id) : TableStructureReadLockHolder{}); + virtual_column->insert(storage->getTableName()); } iterator->next(); From 6cbb52128bec45ddcb9a3d47719406689d6c9e7d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 29 Aug 2019 01:51:38 +0300 Subject: [PATCH 496/509] Fix extracting tuple from JSON. --- dbms/src/Functions/FunctionsJSON.h | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/dbms/src/Functions/FunctionsJSON.h b/dbms/src/Functions/FunctionsJSON.h index cae88a40b59..b9fddf57d39 100644 --- a/dbms/src/Functions/FunctionsJSON.h +++ b/dbms/src/Functions/FunctionsJSON.h @@ -733,16 +733,15 @@ struct JSONExtractTree if (!JSONParser::firstArrayElement(it2)) return false; - size_t index = 0; - do + for (size_t index = 0; index != nested.size(); ++index) { if (nested[index]->addValueToColumn(tuple.getColumn(index), it2)) were_valid_elements = true; else tuple.getColumn(index).insertDefault(); - ++index; + if (!JSONParser::nextArrayElement(it2)) + break; } - while (JSONParser::nextArrayElement(it2)); set_size(old_size + static_cast(were_valid_elements)); return were_valid_elements; @@ -756,16 +755,15 @@ struct JSONExtractTree if (!JSONParser::firstObjectMember(it2)) return false; - size_t index = 0; - do + for (size_t index = 0; index != nested.size(); ++index) { if (nested[index]->addValueToColumn(tuple.getColumn(index), it2)) were_valid_elements = true; else tuple.getColumn(index).insertDefault(); - ++index; + if (!JSONParser::nextObjectMember(it2)) + break; } - while (JSONParser::nextObjectMember(it2)); } else { From 4ec12c3d24fb7254255392bc84bafd93ccf7076b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 29 Aug 2019 01:51:02 +0300 Subject: [PATCH 497/509] Add more tests for JSON functions. --- .../0_stateless/00918_json_functions.reference | 14 ++++++++++++++ .../queries/0_stateless/00918_json_functions.sql | 14 ++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00918_json_functions.reference b/dbms/tests/queries/0_stateless/00918_json_functions.reference index 44d4dc1c9bf..a23b177d468 100644 --- a/dbms/tests/queries/0_stateless/00918_json_functions.reference +++ b/dbms/tests/queries/0_stateless/00918_json_functions.reference @@ -46,6 +46,13 @@ hello 1 Thursday Friday +(3,5) +(7,3) +(5,0) +(3,5) +(3,0) +(3,5) +(3,0) --JSONExtractKeysAndValues-- [('a','hello')] [('b',[-100,200,300])] @@ -121,6 +128,13 @@ hello 1 Thursday Friday +(3,5) +(7,3) +(5,0) +(3,5) +(3,0) +(3,5) +(3,0) --JSONExtractKeysAndValues-- [('a','hello')] [('b',[-100,200,300])] diff --git a/dbms/tests/queries/0_stateless/00918_json_functions.sql b/dbms/tests/queries/0_stateless/00918_json_functions.sql index 83f6d1578f9..38bf0a7ffec 100644 --- a/dbms/tests/queries/0_stateless/00918_json_functions.sql +++ b/dbms/tests/queries/0_stateless/00918_json_functions.sql @@ -54,6 +54,13 @@ SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 4, 'Nullable( SELECT JSONExtract('{"passed": true}', 'passed', 'UInt8'); SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)'); SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)'); +SELECT JSONExtract('{"a":3,"b":5,"c":7}', 'Tuple(a Int, b Int)'); +SELECT JSONExtract('{"a":3,"b":5,"c":7}', 'Tuple(c Int, a Int)'); +SELECT JSONExtract('{"a":3,"b":5,"c":7}', 'Tuple(b Int, d Int)'); +SELECT JSONExtract('{"a":3,"b":5,"c":7}', 'Tuple(Int, Int)'); +SELECT JSONExtract('{"a":3}', 'Tuple(Int, Int)'); +SELECT JSONExtract('[3,5,7]', 'Tuple(Int, Int)'); +SELECT JSONExtract('[3]', 'Tuple(Int, Int)'); SELECT '--JSONExtractKeysAndValues--'; SELECT JSONExtractKeysAndValues('{"a": "hello", "b": [-100, 200.0, 300]}', 'String'); @@ -138,6 +145,13 @@ SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 4, 'Nullable( SELECT JSONExtract('{"passed": true}', 'passed', 'UInt8'); SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)'); SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)'); +SELECT JSONExtract('{"a":3,"b":5,"c":7}', 'Tuple(a Int, b Int)'); +SELECT JSONExtract('{"a":3,"b":5,"c":7}', 'Tuple(c Int, a Int)'); +SELECT JSONExtract('{"a":3,"b":5,"c":7}', 'Tuple(b Int, d Int)'); +SELECT JSONExtract('{"a":3,"b":5,"c":7}', 'Tuple(Int, Int)'); +SELECT JSONExtract('{"a":3}', 'Tuple(Int, Int)'); +SELECT JSONExtract('[3,5,7]', 'Tuple(Int, Int)'); +SELECT JSONExtract('[3]', 'Tuple(Int, Int)'); SELECT '--JSONExtractKeysAndValues--'; SELECT JSONExtractKeysAndValues('{"a": "hello", "b": [-100, 200.0, 300]}', 'String'); From d3b378ea10bb5df25e7f83dce9311c9c4c30e961 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 29 Aug 2019 11:02:50 +0300 Subject: [PATCH 498/509] Fix flappy test (descrease number of iterations) --- dbms/tests/queries/0_stateless/00980_alter_settings_race.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00980_alter_settings_race.sh b/dbms/tests/queries/0_stateless/00980_alter_settings_race.sh index 4a948841ed7..3a9e854210d 100755 --- a/dbms/tests/queries/0_stateless/00980_alter_settings_race.sh +++ b/dbms/tests/queries/0_stateless/00980_alter_settings_race.sh @@ -8,7 +8,7 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS table_for_concurrent_alter" $CLICKHOUSE_CLIENT --query="CREATE TABLE table_for_concurrent_alter (id UInt64, Data String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096;"; n=0 -while [ "$n" -lt 100 ]; +while [ "$n" -lt 50 ]; do n=$(( n + 1 )) $CLICKHOUSE_CLIENT --query="INSERT INTO table_for_concurrent_alter VALUES(1, 'Hello')" > /dev/null 2> /dev/null & @@ -17,7 +17,7 @@ done & q=0 -while [ "$q" -lt 100 ]; +while [ "$q" -lt 50 ]; do q=$(( q + 1 )) counter=$(( 100 + q )) From 85afd61c019e7e98fecc31c90cff67ca6b549f60 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 29 Aug 2019 13:44:22 +0300 Subject: [PATCH 499/509] Fix build with internal libcxx --- cmake/find_cxx.cmake | 2 +- contrib/capnproto-cmake/CMakeLists.txt | 3 ++- contrib/libcxx-cmake/CMakeLists.txt | 4 ++-- contrib/libcxxabi-cmake/CMakeLists.txt | 2 +- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/cmake/find_cxx.cmake b/cmake/find_cxx.cmake index 35a0b9d0927..da6e3d48cab 100644 --- a/cmake/find_cxx.cmake +++ b/cmake/find_cxx.cmake @@ -1,5 +1,5 @@ if (OS_LINUX AND COMPILER_CLANG) - option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++" ${HAVE_LIBCXX}) + option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++" ON) option (USE_INTERNAL_LIBCXX_LIBRARY "Set to FALSE to use system libcxx and libcxxabi libraries instead of bundled" ${NOT_UNBUNDLED}) endif() diff --git a/contrib/capnproto-cmake/CMakeLists.txt b/contrib/capnproto-cmake/CMakeLists.txt index 275007c145f..d92a5a282ce 100644 --- a/contrib/capnproto-cmake/CMakeLists.txt +++ b/contrib/capnproto-cmake/CMakeLists.txt @@ -28,7 +28,8 @@ set (KJ_SRCS ) add_library(kj ${KJ_SRCS}) -target_include_directories(kj INTERFACE ${CAPNPROTO_SOURCE_DIR}) +target_include_directories(kj PUBLIC ${CAPNPROTO_SOURCE_DIR}) +target_compile_options(kj PUBLIC -Wno-non-virtual-dtor) set (CAPNP_SRCS ${CAPNPROTO_SOURCE_DIR}/capnp/c++.capnp.c++ diff --git a/contrib/libcxx-cmake/CMakeLists.txt b/contrib/libcxx-cmake/CMakeLists.txt index 07fa70b9869..e74b4a8364f 100644 --- a/contrib/libcxx-cmake/CMakeLists.txt +++ b/contrib/libcxx-cmake/CMakeLists.txt @@ -40,9 +40,9 @@ ${LIBCXX_SOURCE_DIR}/src/random.cpp add_library(cxx ${SRCS}) -target_include_directories(cxx SYSTEM BEFORE PUBLIC $) +target_include_directories(cxx BEFORE PUBLIC $) target_compile_definitions(cxx PRIVATE -D_LIBCPP_BUILDING_LIBRARY -DLIBCXX_BUILDING_LIBCXXABI) -target_compile_options(cxx PRIVATE -nostdinc++) +target_compile_options(cxx PUBLIC -nostdinc++) target_link_libraries(cxx PUBLIC cxxabi) install( diff --git a/contrib/libcxxabi-cmake/CMakeLists.txt b/contrib/libcxxabi-cmake/CMakeLists.txt index 546d39933af..4996c4f2d7b 100644 --- a/contrib/libcxxabi-cmake/CMakeLists.txt +++ b/contrib/libcxxabi-cmake/CMakeLists.txt @@ -29,7 +29,7 @@ target_include_directories(cxxabi SYSTEM BEFORE PRIVATE $ ) target_compile_definitions(cxxabi PRIVATE -D_LIBCPP_BUILDING_LIBRARY) -target_compile_options(cxxabi PRIVATE -nostdinc++ -fno-sanitize=undefined) # If we don't disable UBSan, infinite recursion happens in dynamic_cast. +target_compile_options(cxxabi PRIVATE -nostdinc++ -fno-sanitize=undefined -Wno-macro-redefined) # If we don't disable UBSan, infinite recursion happens in dynamic_cast. if (USE_UNWIND) target_link_libraries(cxxabi PRIVATE ${UNWIND_LIBRARIES}) From 0899b3818ddf5310e63aa303f3b7913c9e890a42 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 29 Aug 2019 13:49:26 +0300 Subject: [PATCH 500/509] Remove redundant changes --- dbms/src/Common/ErrorCodes.cpp | 1 - dbms/src/Parsers/ASTIndexDeclaration.h | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 12 ++-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 23 +++---- ...test_skip_indices_with_alter_and_merge.sql | 61 +------------------ 5 files changed, 15 insertions(+), 84 deletions(-) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 977e1f4425d..87ab252c583 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -447,7 +447,6 @@ namespace ErrorCodes extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW = 470; extern const int SETTINGS_ARE_NOT_SUPPORTED = 471; extern const int IMMUTABLE_SETTING = 472; - extern const int UNSUPPORTED_SKIP_INDEX_EXPRESSION = 473; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Parsers/ASTIndexDeclaration.h b/dbms/src/Parsers/ASTIndexDeclaration.h index feb3f59b9c6..48deffc2c3a 100644 --- a/dbms/src/Parsers/ASTIndexDeclaration.h +++ b/dbms/src/Parsers/ASTIndexDeclaration.h @@ -41,7 +41,7 @@ public: return res; } - void formatImpl(const FormatSettings & s, FormatState &state, FormatStateStacked frame) const override + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override { frame.need_parens = false; std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' '); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index af1b8dee450..1ad84274c29 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -364,12 +364,6 @@ void MergeTreeData::setProperties( std::dynamic_pointer_cast(index_decl->clone()), global_context); - if (index_ptr->getColumnsRequiredForIndexCalc().size() > 1 && settings_ptr->enable_vertical_merge_algorithm) - throw Exception("Index '" + index_ptr->name + "' contains expression with multiple columns and " - + "'enable_vertical_merge_algorithm' is set to true in storage settings. " - + "Disable vertical merge or use only one column in index expression.", - ErrorCodes::UNSUPPORTED_SKIP_INDEX_EXPRESSION); - new_indices.push_back(std::move(index_ptr)); if (indices_names.find(new_indices.back()->name) != indices_names.end()) @@ -1587,7 +1581,8 @@ void MergeTreeData::alterDataPart( if (expression) { BlockInputStreamPtr part_in = std::make_shared( - *this, part, expression->getRequiredColumns(), false, /* take_column_types_from_storage = */ false); + *this, part, expression->getRequiredColumns(), false, /* take_column_types_from_storage = */ false); + auto compression_codec = global_context.chooseCompressionCodec( part->bytes_on_disk, @@ -1609,7 +1604,8 @@ void MergeTreeData::alterDataPart( true /* sync */, compression_codec, true /* skip_offsets */, - {}, /// currently restricted + /// Don't recalc indices because indices alter is restricted + std::vector{}, unused_written_offsets, part->index_granularity, &part->index_granularity_info); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 940bd04377c..b6ecd39e713 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -379,7 +379,7 @@ static void extractMergingAndGatheringColumns( std::set key_columns(sort_key_columns_vec.cbegin(), sort_key_columns_vec.cend()); for (const auto & index : indexes) { - Names index_columns_vec = index->expr->getRequiredColumns(); + Names index_columns_vec = index->getColumnsRequiredForIndexCalc(); std::copy(index_columns_vec.cbegin(), index_columns_vec.cend(), std::inserter(key_columns, key_columns.end())); } @@ -829,16 +829,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor rows_sources_read_buf.seek(0, 0); ColumnGathererStream column_gathered_stream(column_name, column_part_streams, rows_sources_read_buf); - std::vector skip_idx_to_recalc = data.getIndicesForColumn(column_name); - for (const auto & idx : skip_idx_to_recalc) - if (idx->getColumnsRequiredForIndexCalc().size() > 1) - throw Exception("Skip index '" + idx->name + "' has expression on multiple columns. " - + "Vertical merge is not supported for tables with skip indices with expressions on multiple columns. " - + "It's better to avoid indices with multiple columns in expression. " - + "Also you can disable vertical merges with setting enable_vertical_merge_algorithm=0, " - + "but it will lead to additional memory consuption for big merges.", - ErrorCodes::LOGICAL_ERROR); - MergedColumnOnlyOutputStream column_to( data, column_gathered_stream.getHeader(), @@ -846,10 +836,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor false, compression_codec, false, - skip_idx_to_recalc, + /// we don't need to recalc indices here + /// because all of them were already recalculated and written + /// as key part of vertical merge + std::vector{}, written_offset_columns, - to.getIndexGranularity() - ); + to.getIndexGranularity()); + size_t column_elems_written = 0; column_to.writePrefix(); @@ -1030,7 +1023,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor for (size_t i = 0; i < data.skip_indices.size(); ++i) { const auto & index = data.skip_indices[i]; - const auto & index_cols = index->expr->getRequiredColumns(); + const auto & index_cols = index->getColumnsRequiredForIndexCalc(); auto it = std::find(std::cbegin(index_cols), std::cend(index_cols), col); if (it != std::cend(index_cols) && indices_to_recalc.insert(index).second) { diff --git a/dbms/tests/queries/0_stateless/00999_test_skip_indices_with_alter_and_merge.sql b/dbms/tests/queries/0_stateless/00999_test_skip_indices_with_alter_and_merge.sql index c67f6c594ff..55b2f21dc32 100644 --- a/dbms/tests/queries/0_stateless/00999_test_skip_indices_with_alter_and_merge.sql +++ b/dbms/tests/queries/0_stateless/00999_test_skip_indices_with_alter_and_merge.sql @@ -1,48 +1,12 @@ SET allow_experimental_data_skipping_indices=1; -DROP TABLE IF EXISTS table_test_creation; - -CREATE TABLE table_test_creation ( - k UInt64, - val1 UInt64, - val2 UInt64, - INDEX idx1 val1 * val2 TYPE minmax GRANULARITY 1 -) ENGINE MergeTree() -ORDER BY k; -- { serverError 473 } - -CREATE TABLE table_test_creation ( - k UInt64, - val1 UInt64, - val2 UInt64 -) ENGINE MergeTree() -ORDER BY k; - -ALTER TABLE table_test_creation ADD INDEX idx1 val1 * val2 TYPE minmax GRANULARITY 1; -- { serverError 473 } - -ALTER TABLE table_test_creation ADD INDEX idx1 val1 TYPE minmax GRANULARITY 1; - -ALTER TABLE table_test_creation MODIFY SETTING enable_vertical_merge_algorithm=0; - -ALTER TABLE table_test_creation ADD INDEX idx2 val1 * val2 TYPE minmax GRANULARITY 1; - -DROP TABLE IF EXISTS table_test_creation; - -CREATE TABLE table_test_creation ( - k UInt64, - val1 UInt64, - val2 UInt64, - INDEX idx1 val1 * val2 TYPE minmax GRANULARITY 1 -) ENGINE MergeTree() -ORDER BY k SETTINGS enable_vertical_merge_algorithm=0; - -DROP TABLE IF EXISTS table_test_creation; - DROP TABLE IF EXISTS test_vertical_merge; CREATE TABLE test_vertical_merge ( k UInt64, val1 UInt64, val2 UInt64, - INDEX idx1 val2 TYPE minmax GRANULARITY 1 + INDEX idx1 val1 * val2 TYPE minmax GRANULARITY 1, + INDEX idx2 val1 * k TYPE minmax GRANULARITY 1 ) ENGINE MergeTree() ORDER BY k SETTINGS vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 1; @@ -56,24 +20,3 @@ OPTIMIZE TABLE test_vertical_merge FINAL; SELECT COUNT() from test_vertical_merge WHERE val2 <= 2400; DROP TABLE IF EXISTS test_vertical_merge; - ---DROP TABLE IF EXISTS test_alter_multiple_columns; --- ---CREATE TABLE test_alter_multiple_columns ( --- k UInt64, --- val1 UInt64, --- val2 UInt64, --- INDEX idx1 val2 * val1 TYPE minmax GRANULARITY 1 ---) ENGINE MergeTree() ---ORDER BY k ---SETTINGS enable_vertical_merge_algorithm=0; --- ---INSERT INTO test_alter_multiple_columns SELECT number, number + 5, number * 12 from numbers(1000); --- ---SELECT COUNT() from test_alter_multiple_columns WHERE val2 <= 2400; --- ---ALTER TABLE test_alter_multiple_columns MODIFY COLUMN val2 UInt16; --- ---SELECT COUNT() from test_alter_multiple_columns WHERE val2 <= 2400; --- ---DROP TABLE IF EXISTS test_alter_multiple_columns; From 702b47ae5b801c03e43a123162e53e436ed759bc Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 29 Aug 2019 13:56:43 +0300 Subject: [PATCH 501/509] Remove redundant changes --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 28 ++++--------------- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 -- .../src/Storages/MergeTree/MergeTreeIndices.h | 6 ++-- 3 files changed, 8 insertions(+), 28 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 1ad84274c29..10c9465f9ae 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -91,7 +91,6 @@ namespace ErrorCodes extern const int INCORRECT_FILE_NAME; extern const int BAD_DATA_PART_NAME; extern const int UNKNOWN_SETTING; - extern const int UNSUPPORTED_SKIP_INDEX_EXPRESSION; } @@ -350,7 +349,6 @@ void MergeTreeData::setProperties( MergeTreeIndices new_indices; - auto settings_ptr = getSettings(); if (!indices_description.indices.empty()) { std::set indices_names; @@ -359,12 +357,11 @@ void MergeTreeData::setProperties( { const auto & index_decl = std::dynamic_pointer_cast(index_ast); - auto index_ptr = MergeTreeIndexFactory::instance().get( - all_columns, - std::dynamic_pointer_cast(index_decl->clone()), - global_context); - - new_indices.push_back(std::move(index_ptr)); + new_indices.push_back( + MergeTreeIndexFactory::instance().get( + all_columns, + std::dynamic_pointer_cast(index_decl->clone()), + global_context)); if (indices_names.find(new_indices.back()->name) != indices_names.end()) throw Exception( @@ -3090,19 +3087,4 @@ bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const return true; } - -std::vector MergeTreeData::getIndicesForColumn(const String & column_name) const -{ - std::vector result; - - for (size_t i = 0; i < skip_indices.size(); ++i) - { - const auto & index_columns = skip_indices[i]->getColumnsRequiredForIndexCalc(); - if (std::find(index_columns.begin(), index_columns.end(), column_name) != index_columns.end()) - result.emplace_back(skip_indices[i]); - } - - return result; -} - } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index d4c6cbf7c0a..c135c91ddf7 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -629,8 +629,6 @@ public: return storage_settings.get(); } - std::vector getIndicesForColumn(const String & column_name) const; - MergeTreeDataFormatVersion format_version; Context global_context; diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndices.h b/dbms/src/Storages/MergeTree/MergeTreeIndices.h index 0b626d5e10a..22a392ca9c5 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndices.h +++ b/dbms/src/Storages/MergeTree/MergeTreeIndices.h @@ -109,11 +109,11 @@ public: /// Index name String name; - /// Index expression with columns arguments + /// Index expression (x * y) + /// with columns arguments ExpressionActionsPtr expr; - /// Names of columns which are used - /// to calculate expression for index + /// Names of columns for index Names columns; /// Data types of columns From 36abd1e6fca9a6f128bb77a3160ea7d64ba8c3bb Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 29 Aug 2019 14:09:01 +0300 Subject: [PATCH 502/509] Fix build --- contrib/libcxx-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libcxx-cmake/CMakeLists.txt b/contrib/libcxx-cmake/CMakeLists.txt index e74b4a8364f..16aafa7e862 100644 --- a/contrib/libcxx-cmake/CMakeLists.txt +++ b/contrib/libcxx-cmake/CMakeLists.txt @@ -42,7 +42,7 @@ add_library(cxx ${SRCS}) target_include_directories(cxx BEFORE PUBLIC $) target_compile_definitions(cxx PRIVATE -D_LIBCPP_BUILDING_LIBRARY -DLIBCXX_BUILDING_LIBCXXABI) -target_compile_options(cxx PUBLIC -nostdinc++) +target_compile_options(cxx PUBLIC -nostdinc++ -Wno-reserved-id-macro) target_link_libraries(cxx PUBLIC cxxabi) install( From dac55278adcab1c7509624616c3374ef6f1dc300 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 29 Aug 2019 14:15:56 +0300 Subject: [PATCH 503/509] Better comments --- dbms/src/Interpreters/MutationsInterpreter.cpp | 4 +++- dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 6 ++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 27fb48c7b5b..2069d2372e2 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -289,13 +289,15 @@ void MutationsInterpreter::prepare(bool dry_run) throw Exception("Unknown mutation command type: " + DB::toString(command.type), ErrorCodes::UNKNOWN_MUTATION_COMMAND); } + /// We cares about affected indices because we also need to rewrite them + /// when one of index columns updated or filtered with delete if (!affected_indices_columns.empty()) { if (!stages.empty()) { std::vector stages_copy; /// Copy all filled stages except index calculation stage. - for (const auto &stage : stages) + for (const auto & stage : stages) { stages_copy.emplace_back(context); stages_copy.back().column_to_updated = stage.column_to_updated; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index b6ecd39e713..7079996af80 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1042,6 +1042,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor auto indices_recalc_expr = ExpressionAnalyzer( indices_recalc_expr_list, indices_recalc_syntax, context).getActions(false); + + /// We can update only one column, but some skip idx expression may depend on several + /// columns (c1 + c2 * c3). It works because in stream was created with help of + /// MutationsInterpreter which knows about skip indices and stream 'in' already has + /// all required columns. + /// TODO move this logic to single place. in = std::make_shared( std::make_shared(in, indices_recalc_expr)); } From 6d6851e747058c9fb2f0180900d6d0b735e2c1c3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 29 Aug 2019 15:36:41 +0300 Subject: [PATCH 504/509] Add capnp-library do build image --- docker/packager/deb/Dockerfile | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index f08c2dc3eab..7410920a462 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -71,7 +71,9 @@ RUN apt-get --allow-unauthenticated update -y \ odbcinst \ tzdata \ gperf \ - alien + alien \ + libcapnp-dev + COPY build.sh / CMD ["/bin/bash", "/build.sh"] From 549c61cad9c9df3566ce70d5a02b209c6af51856 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 29 Aug 2019 15:38:20 +0300 Subject: [PATCH 505/509] Update MarkInCompressedFile.h --- dbms/src/DataStreams/MarkInCompressedFile.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/DataStreams/MarkInCompressedFile.h b/dbms/src/DataStreams/MarkInCompressedFile.h index 62fe8eedf76..46d078f2b76 100644 --- a/dbms/src/DataStreams/MarkInCompressedFile.h +++ b/dbms/src/DataStreams/MarkInCompressedFile.h @@ -41,4 +41,5 @@ struct MarkInCompressedFile }; using MarksInCompressedFile = PODArray; + } From 0e606c790ed2af32b12bc3379f0bd9a31f17a89f Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 29 Aug 2019 16:38:18 +0300 Subject: [PATCH 506/509] Fix install of libcxx and libcxxabi --- contrib/libcxx-cmake/CMakeLists.txt | 1 + contrib/libcxxabi-cmake/CMakeLists.txt | 1 + 2 files changed, 2 insertions(+) diff --git a/contrib/libcxx-cmake/CMakeLists.txt b/contrib/libcxx-cmake/CMakeLists.txt index 16aafa7e862..8759786c3e8 100644 --- a/contrib/libcxx-cmake/CMakeLists.txt +++ b/contrib/libcxx-cmake/CMakeLists.txt @@ -50,4 +50,5 @@ install( EXPORT global ARCHIVE DESTINATION lib RUNTIME DESTINATION lib + LIBRARY DESTINATION lib ) diff --git a/contrib/libcxxabi-cmake/CMakeLists.txt b/contrib/libcxxabi-cmake/CMakeLists.txt index 4996c4f2d7b..2abe5702132 100644 --- a/contrib/libcxxabi-cmake/CMakeLists.txt +++ b/contrib/libcxxabi-cmake/CMakeLists.txt @@ -42,4 +42,5 @@ install( EXPORT global ARCHIVE DESTINATION lib RUNTIME DESTINATION lib + LIBRARY DESTINATION lib ) From 260b8c7fa7baa21f4283206cf59dd5670b233759 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 29 Aug 2019 18:32:25 +0300 Subject: [PATCH 507/509] Removed obsolete directory --- contrib/mimalloc | 1 - 1 file changed, 1 deletion(-) delete mode 160000 contrib/mimalloc diff --git a/contrib/mimalloc b/contrib/mimalloc deleted file mode 160000 index a787bdebce9..00000000000 --- a/contrib/mimalloc +++ /dev/null @@ -1 +0,0 @@ -Subproject commit a787bdebce94bf3776dc0d1ad597917f479ab8d5 From 5851316742794665b6c559d0f886cbe302782eb5 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Thu, 29 Aug 2019 18:36:07 +0300 Subject: [PATCH 508/509] Don't interrupt copyData() used inside Kafka materialized views (#6597) * Print better offsets on commit * Stop consumption on the read buffer level * Don't use cancellation in the middle of copyData() * Add test * Disable squashing stream for Kafka materialized views * Commit last read message, since rdkafka auto-modifies offset --- .../Interpreters/InterpreterInsertQuery.cpp | 6 +- .../src/Interpreters/InterpreterInsertQuery.h | 5 +- .../Kafka/ReadBufferFromKafkaConsumer.cpp | 57 +++++++++++--- .../Kafka/ReadBufferFromKafkaConsumer.h | 9 ++- dbms/src/Storages/Kafka/StorageKafka.cpp | 9 ++- .../integration/test_storage_kafka/test.py | 74 +++++++++++++++++++ 6 files changed, 140 insertions(+), 20 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 45d528b0e74..f7edca14089 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -38,8 +38,8 @@ namespace ErrorCodes InterpreterInsertQuery::InterpreterInsertQuery( - const ASTPtr & query_ptr_, const Context & context_, bool allow_materialized_) - : query_ptr(query_ptr_), context(context_), allow_materialized(allow_materialized_) + const ASTPtr & query_ptr_, const Context & context_, bool allow_materialized_, bool no_squash_) + : query_ptr(query_ptr_), context(context_), allow_materialized(allow_materialized_), no_squash(no_squash_) { checkStackSize(); } @@ -109,7 +109,7 @@ BlockIO InterpreterInsertQuery::execute() /// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side. /// Client-side bufferization might cause excessive timeouts (especially in case of big blocks). - if (!(context.getSettingsRef().insert_distributed_sync && table->isRemote())) + if (!(context.getSettingsRef().insert_distributed_sync && table->isRemote()) && !no_squash) { out = std::make_shared( out, out->getHeader(), context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.h b/dbms/src/Interpreters/InterpreterInsertQuery.h index 9cde2c274fe..37cb3110984 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.h +++ b/dbms/src/Interpreters/InterpreterInsertQuery.h @@ -15,7 +15,7 @@ namespace DB class InterpreterInsertQuery : public IInterpreter { public: - InterpreterInsertQuery(const ASTPtr & query_ptr_, const Context & context_, bool allow_materialized_ = false); + InterpreterInsertQuery(const ASTPtr & query_ptr_, const Context & context_, bool allow_materialized_ = false, bool no_squash_ = false); /** Prepare a request for execution. Return block streams * - the stream into which you can write data to execute the query, if INSERT; @@ -33,7 +33,8 @@ private: ASTPtr query_ptr; const Context & context; - bool allow_materialized; + const bool allow_materialized; + const bool no_squash; }; diff --git a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp index 01fd09db7e3..960a998932d 100644 --- a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp +++ b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp @@ -4,14 +4,21 @@ namespace DB { using namespace std::chrono_literals; + ReadBufferFromKafkaConsumer::ReadBufferFromKafkaConsumer( - ConsumerPtr consumer_, Poco::Logger * log_, size_t max_batch_size, size_t poll_timeout_, bool intermediate_commit_) + ConsumerPtr consumer_, + Poco::Logger * log_, + size_t max_batch_size, + size_t poll_timeout_, + bool intermediate_commit_, + const std::atomic & stopped_) : ReadBuffer(nullptr, 0) , consumer(consumer_) , log(log_) , batch_size(max_batch_size) , poll_timeout(poll_timeout_) , intermediate_commit(intermediate_commit_) + , stopped(stopped_) , current(messages.begin()) { } @@ -26,11 +33,46 @@ ReadBufferFromKafkaConsumer::~ReadBufferFromKafkaConsumer() void ReadBufferFromKafkaConsumer::commit() { + auto PrintOffsets = [this] (const char * prefix, const cppkafka::TopicPartitionList & offsets) + { + for (const auto & topic_part : offsets) + { + auto print_special_offset = [&topic_part] + { + switch (topic_part.get_offset()) + { + case cppkafka::TopicPartition::OFFSET_BEGINNING: return "BEGINNING"; + case cppkafka::TopicPartition::OFFSET_END: return "END"; + case cppkafka::TopicPartition::OFFSET_STORED: return "STORED"; + case cppkafka::TopicPartition::OFFSET_INVALID: return "INVALID"; + default: return ""; + } + }; + + if (topic_part.get_offset() < 0) + { + LOG_TRACE( + log, + prefix << " " << print_special_offset() << " (topic: " << topic_part.get_topic() + << ", partition: " << topic_part.get_partition() << ")"); + } + else + { + LOG_TRACE( + log, + prefix << " " << topic_part.get_offset() << " (topic: " << topic_part.get_topic() + << ", partition: " << topic_part.get_partition() << ")"); + } + } + }; + + PrintOffsets("Polled offset", consumer->get_offsets_position(consumer->get_assignment())); + if (current != messages.end()) { /// Since we can poll more messages than we already processed, /// commit only processed messages. - consumer->async_commit(*current); + consumer->async_commit(*std::prev(current)); } else { @@ -41,14 +83,7 @@ void ReadBufferFromKafkaConsumer::commit() consumer->async_commit(); } - const auto & offsets = consumer->get_offsets_committed(consumer->get_assignment()); - for (const auto & topic_part : offsets) - { - LOG_TRACE( - log, - "Committed offset " << topic_part.get_offset() << " (topic: " << topic_part.get_topic() - << ", partition: " << topic_part.get_partition() << ")"); - } + PrintOffsets("Committed offset", consumer->get_offsets_committed(consumer->get_assignment())); stalled = false; } @@ -114,7 +149,7 @@ bool ReadBufferFromKafkaConsumer::nextImpl() /// NOTE: ReadBuffer was implemented with an immutable underlying contents in mind. /// If we failed to poll any message once - don't try again. /// Otherwise, the |poll_timeout| expectations get flawn. - if (stalled) + if (stalled || stopped) return false; if (current == messages.end()) diff --git a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h index 8f78eae23e1..c489ce6b5f0 100644 --- a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h +++ b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h @@ -17,7 +17,12 @@ class ReadBufferFromKafkaConsumer : public ReadBuffer { public: ReadBufferFromKafkaConsumer( - ConsumerPtr consumer_, Poco::Logger * log_, size_t max_batch_size, size_t poll_timeout_, bool intermediate_commit_); + ConsumerPtr consumer_, + Poco::Logger * log_, + size_t max_batch_size, + size_t poll_timeout_, + bool intermediate_commit_, + const std::atomic & stopped_); ~ReadBufferFromKafkaConsumer() override; void commit(); // Commit all processed messages. @@ -43,6 +48,8 @@ private: bool stalled = false; bool intermediate_commit = true; + const std::atomic & stopped; + Messages messages; Messages::const_iterator current; diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 2d55eb42f1e..b71ee5c8a18 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -274,8 +274,10 @@ ConsumerBufferPtr StorageKafka::createReadBuffer() batch_size = settings.max_block_size.value; size_t poll_timeout = settings.stream_poll_timeout_ms.totalMilliseconds(); + /// NOTE: we pass |stream_cancelled| by reference here, so the buffers should not outlive the storage. return std::make_shared( - std::make_unique(consumer, log, batch_size, poll_timeout, intermediate_commit), row_delimiter); + std::make_unique(consumer, log, batch_size, poll_timeout, intermediate_commit, stream_cancelled), + row_delimiter); } @@ -371,7 +373,7 @@ bool StorageKafka::streamToViews() block_size = settings.max_block_size; // Create a stream for each consumer and join them in a union stream - InterpreterInsertQuery interpreter{insert, global_context}; + InterpreterInsertQuery interpreter(insert, global_context, false, true); auto block_io = interpreter.execute(); // Create a stream for each consumer and join them in a union stream @@ -396,7 +398,8 @@ bool StorageKafka::streamToViews() else in = streams[0]; - copyData(*in, *block_io.out, &stream_cancelled); + std::atomic stub; + copyData(*in, *block_io.out, &stub); // Check whether the limits were applied during query execution bool limits_applied = false; diff --git a/dbms/tests/integration/test_storage_kafka/test.py b/dbms/tests/integration/test_storage_kafka/test.py index 0f0cf6996ef..47ba957cdef 100644 --- a/dbms/tests/integration/test_storage_kafka/test.py +++ b/dbms/tests/integration/test_storage_kafka/test.py @@ -504,6 +504,7 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster): kafka_check_result(result, True, 'test_kafka_virtual2.reference') +@pytest.mark.timeout(60) def test_kafka_insert(kafka_cluster): instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) @@ -540,6 +541,7 @@ def test_kafka_insert(kafka_cluster): kafka_check_result(result, True) +@pytest.mark.timeout(60) def test_kafka_produce_consume(kafka_cluster): instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) @@ -603,6 +605,78 @@ def test_kafka_produce_consume(kafka_cluster): assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) +@pytest.mark.timeout(120) +def test_kafka_commit_on_block_write(kafka_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'block', + kafka_group_name = 'block', + kafka_format = 'JSONEachRow', + kafka_max_block_size = 100, + kafka_row_delimiter = '\\n'; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; + ''') + + cancel = threading.Event() + + i = [0] + def produce(): + while not cancel.is_set(): + messages = [] + for _ in range(101): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + kafka_produce('block', messages) + + kafka_thread = threading.Thread(target=produce) + kafka_thread.start() + + while int(instance.query('SELECT count() FROM test.view')) == 0: + time.sleep(1) + + cancel.set() + + instance.query(''' + DROP TABLE test.kafka; + ''') + + while int(instance.query("SELECT count() FROM system.tables WHERE database='test' AND name='kafka'")) == 1: + time.sleep(1) + + instance.query(''' + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'block', + kafka_group_name = 'block', + kafka_format = 'JSONEachRow', + kafka_max_block_size = 100, + kafka_row_delimiter = '\\n'; + ''') + + while int(instance.query('SELECT uniqExact(key) FROM test.view')) < i[0]: + time.sleep(1) + + result = int(instance.query('SELECT count() == uniqExact(key) FROM test.view')) + + instance.query(''' + DROP TABLE test.consumer; + DROP TABLE test.view; + ''') + + kafka_thread.join() + + assert result == 1, 'Messages from kafka get duplicated!' + + if __name__ == '__main__': cluster.start() raw_input("Cluster created, press any key to destroy...") From 0bca68e50b540608832b17202ceec6a2d7a5cba4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 29 Aug 2019 21:55:20 +0300 Subject: [PATCH 509/509] Style --- dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 3 ++- dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 7079996af80..fdeb8c3ea96 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -561,7 +561,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor NamesAndTypesList all_columns = data.getColumns().getAllPhysical(); const auto data_settings = data.getSettings(); - NamesAndTypesList gathering_columns, merging_columns; + NamesAndTypesList gathering_columns; + NamesAndTypesList merging_columns; Names gathering_column_names, merging_column_names; extractMergingAndGatheringColumns( all_columns, data.sorting_key_expr, data.skip_indices, diff --git a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 3c15bd54df2..0a852e68b74 100644 --- a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -103,7 +103,6 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG serialize_settings.getter = createStreamGetter(column.name, already_written_offset_columns, skip_offsets); column.type->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[i]); - if (with_final_mark) writeFinalMark(column.name, column.type, offset_columns, skip_offsets, serialize_settings.path); }