mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-18 20:32:43 +00:00
Deal with includes
This commit is contained in:
parent
c28ae71b3f
commit
bc93760c5a
@ -11,7 +11,6 @@
|
||||
#include <Storages/ObjectStorage/S3/Configuration.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include "Common/DateLUT.h"
|
||||
#include <Common/logger_useful.h>
|
||||
#include "Storages/ColumnsDescription.h"
|
||||
|
||||
@ -47,9 +46,11 @@ public:
|
||||
{
|
||||
BaseStorageConfiguration::update(object_storage, local_context);
|
||||
|
||||
bool existed = current_metadata != nullptr;
|
||||
|
||||
if (updateMetadataObjectIfNeeded(object_storage, local_context))
|
||||
{
|
||||
if (hasExternalDynamicMetadata())
|
||||
if (hasExternalDynamicMetadata() && existed)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::FORMAT_VERSION_TOO_OLD,
|
||||
@ -103,6 +104,7 @@ public:
|
||||
BaseStorageConfiguration::setPaths(current_metadata->getDataFiles());
|
||||
BaseStorageConfiguration::setPartitionColumns(current_metadata->getPartitionColumns());
|
||||
}
|
||||
|
||||
return ColumnsDescription{current_metadata->getTableSchema()};
|
||||
}
|
||||
|
||||
|
@ -1,45 +1,25 @@
|
||||
#include <memory>
|
||||
#include <Poco/Logger.h>
|
||||
#include "Common/Config/ConfigProcessor.h"
|
||||
#include "Common/DateLUT.h"
|
||||
#include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h"
|
||||
#include "config.h"
|
||||
|
||||
#if USE_AVRO
|
||||
|
||||
# include <Columns/ColumnString.h>
|
||||
# include <Columns/ColumnTuple.h>
|
||||
# include <Columns/IColumn.h>
|
||||
# include <Core/Settings.h>
|
||||
# include <DataTypes/DataTypeArray.h>
|
||||
# include <DataTypes/DataTypeDate.h>
|
||||
# include <DataTypes/DataTypeDateTime64.h>
|
||||
# include <DataTypes/DataTypeFactory.h>
|
||||
# include <DataTypes/DataTypeFixedString.h>
|
||||
# include <DataTypes/DataTypeMap.h>
|
||||
# include <DataTypes/DataTypeNullable.h>
|
||||
# include <DataTypes/DataTypeString.h>
|
||||
# include <DataTypes/DataTypeTuple.h>
|
||||
# include <DataTypes/DataTypeUUID.h>
|
||||
# include <DataTypes/DataTypesDecimal.h>
|
||||
# include <DataTypes/DataTypesNumber.h>
|
||||
# include <Formats/FormatFactory.h>
|
||||
# include <IO/ReadBufferFromFileBase.h>
|
||||
# include <IO/ReadBufferFromString.h>
|
||||
# include <IO/ReadHelpers.h>
|
||||
# include <Processors/Formats/Impl/AvroRowInputFormat.h>
|
||||
# include <Storages/ObjectStorage/DataLakes/Common.h>
|
||||
# include <Storages/ObjectStorage/StorageObjectStorageSource.h>
|
||||
# include <Common/logger_useful.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Processors/Formats/Impl/AvroRowInputFormat.h>
|
||||
#include <Storages/ObjectStorage/DataLakes/Common.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorageSource.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
# include <filesystem>
|
||||
# include <sstream>
|
||||
#include "Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h"
|
||||
#include "Storages/ObjectStorage/DataLakes/Iceberg/Utils.h"
|
||||
|
||||
# include "Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h"
|
||||
# include "Storages/ObjectStorage/DataLakes/Iceberg/Utils.h"
|
||||
|
||||
# include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileImpl.h"
|
||||
# include "Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h"
|
||||
#include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileImpl.h"
|
||||
#include "Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -255,12 +235,15 @@ bool IcebergMetadata::update(const ContextPtr & local_context)
|
||||
|
||||
const auto [metadata_version, metadata_file_path] = getMetadataFileAndVersion(object_storage, *configuration_ptr);
|
||||
|
||||
if (metadata_version == current_metadata_version)
|
||||
return false;
|
||||
|
||||
current_metadata_version = metadata_version;
|
||||
|
||||
auto metadata_object = readJson(metadata_file_path, local_context);
|
||||
|
||||
chassert(format_version == metadata_object->getValue<int>("format-version"));
|
||||
|
||||
if (metadata_version == current_metadata_version)
|
||||
return false;
|
||||
|
||||
auto manifest_list_file = getRelevantManifestList(metadata_object);
|
||||
if (manifest_list_file && (!current_snapshot.has_value() || (manifest_list_file.value() != current_snapshot->getName())))
|
||||
|
@ -3,24 +3,21 @@
|
||||
|
||||
#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format.
|
||||
|
||||
# include <Core/Types.h>
|
||||
# include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
# include <Interpreters/Context_fwd.h>
|
||||
# include <Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h>
|
||||
# include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
|
||||
# include <Poco/JSON/Array.h>
|
||||
# include <Poco/JSON/Object.h>
|
||||
# include <Poco/JSON/Parser.h>
|
||||
#include <Poco/JSON/Array.h>
|
||||
#include <Poco/JSON/Object.h>
|
||||
#include <Poco/JSON/Parser.h>
|
||||
|
||||
# include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h"
|
||||
# include "Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h"
|
||||
# include "Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h"
|
||||
#include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h"
|
||||
#include "Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h"
|
||||
#include "Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h"
|
||||
|
||||
# include <memory>
|
||||
# include <mutex>
|
||||
# include <optional>
|
||||
# include <unordered_map>
|
||||
#include <unordered_map>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -3,26 +3,12 @@
|
||||
#if USE_AVRO
|
||||
|
||||
|
||||
# include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileImpl.h"
|
||||
# include "Storages/ObjectStorage/DataLakes/Iceberg/Utils.h"
|
||||
#include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileImpl.h"
|
||||
#include "Storages/ObjectStorage/DataLakes/Iceberg/Utils.h"
|
||||
|
||||
|
||||
# include <Core/Types.h>
|
||||
# include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
# include <Interpreters/Context_fwd.h>
|
||||
# include <Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h>
|
||||
# include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
# include <DataFile.hh>
|
||||
|
||||
# include <Poco/JSON/Array.h>
|
||||
# include <Poco/JSON/Object.h>
|
||||
# include <Poco/JSON/Parser.h>
|
||||
|
||||
# include <Common/Exception.h>
|
||||
# include "DataTypes/DataTypeTuple.h"
|
||||
# include "Formats/FormatSettings.h"
|
||||
|
||||
# include <Processors/Formats/Impl/AvroRowInputFormat.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include "DataTypes/DataTypeTuple.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,11 +1,11 @@
|
||||
#pragma once
|
||||
#include <memory>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if USE_AVRO
|
||||
|
||||
# include <cstdint>
|
||||
# include <Common/Exception.h>
|
||||
#include <cstdint>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -2,28 +2,8 @@
|
||||
|
||||
#if USE_AVRO
|
||||
|
||||
|
||||
# include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h"
|
||||
# include "Storages/ObjectStorage/DataLakes/Iceberg/Utils.h"
|
||||
|
||||
|
||||
# include <Core/Types.h>
|
||||
# include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
# include <Interpreters/Context_fwd.h>
|
||||
# include <Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h>
|
||||
# include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
# include <DataFile.hh>
|
||||
|
||||
# include <Poco/JSON/Array.h>
|
||||
# include <Poco/JSON/Object.h>
|
||||
# include <Poco/JSON/Parser.h>
|
||||
|
||||
# include <Common/Exception.h>
|
||||
# include "DataTypes/DataTypeTuple.h"
|
||||
# include "Formats/FormatSettings.h"
|
||||
|
||||
# include <Processors/Formats/Impl/AvroRowInputFormat.h>
|
||||
|
||||
#include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h"
|
||||
#include <Processors/Formats/Impl/AvroRowInputFormat.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include "Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h"
|
||||
#include <Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h>
|
||||
|
||||
#include <Poco/JSON/Array.h>
|
||||
#include <Poco/JSON/Object.h>
|
||||
@ -21,14 +21,18 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
namespace Iceberg
|
||||
{
|
||||
|
||||
|
@ -2,20 +2,19 @@
|
||||
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
#include "config.h"
|
||||
|
||||
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
|
||||
|
||||
#include <Poco/JSON/Array.h>
|
||||
#include <Poco/JSON/Object.h>
|
||||
#include <Poco/JSON/Parser.h>
|
||||
|
||||
#include <unordered_map>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include "config.h"
|
||||
|
||||
#if USE_AVRO
|
||||
# include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h"
|
||||
#include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,26 +1,10 @@
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format.
|
||||
#if USE_AVRO
|
||||
|
||||
# include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h"
|
||||
|
||||
# include <Core/Types.h>
|
||||
# include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
# include <Interpreters/Context_fwd.h>
|
||||
# include <Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h>
|
||||
# include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
# include <DataFile.hh>
|
||||
|
||||
# include <Poco/JSON/Array.h>
|
||||
# include <Poco/JSON/Object.h>
|
||||
# include <Poco/JSON/Parser.h>
|
||||
|
||||
# include <Common/Exception.h>
|
||||
# include "DataTypes/DataTypeTuple.h"
|
||||
# include "Formats/FormatSettings.h"
|
||||
|
||||
# include <Processors/Formats/Impl/AvroRowInputFormat.h>
|
||||
#include <Processors/Formats/Impl/AvroRowInputFormat.h>
|
||||
#include <Storages/ObjectStorage/DataLakes/Iceberg/Utils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -2,24 +2,9 @@
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format.
|
||||
#if USE_AVRO
|
||||
|
||||
|
||||
# include <Core/Types.h>
|
||||
# include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
# include <Interpreters/Context_fwd.h>
|
||||
# include <Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h>
|
||||
# include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
# include <DataFile.hh>
|
||||
|
||||
# include <Poco/JSON/Array.h>
|
||||
# include <Poco/JSON/Object.h>
|
||||
# include <Poco/JSON/Parser.h>
|
||||
|
||||
# include <Common/Exception.h>
|
||||
# include "Formats/FormatSettings.h"
|
||||
|
||||
# include <Processors/Formats/Impl/AvroRowInputFormat.h>
|
||||
#include <Processors/Formats/Impl/AvroRowInputFormat.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user