mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge branch 'master' into export-logs-in-ci
This commit is contained in:
commit
b6c47ee157
@ -7,8 +7,6 @@
|
||||
#include <base/find_symbols.h>
|
||||
#include <base/preciseExp10.h>
|
||||
|
||||
#include <iostream>
|
||||
|
||||
#define JSON_MAX_DEPTH 100
|
||||
|
||||
|
||||
|
@ -12,7 +12,6 @@
|
||||
#include <tuple>
|
||||
#include <limits>
|
||||
|
||||
#include <boost/multiprecision/cpp_bin_float.hpp>
|
||||
#include <boost/math/special_functions/fpclassify.hpp>
|
||||
|
||||
// NOLINTBEGIN(*)
|
||||
@ -22,6 +21,7 @@
|
||||
#define CONSTEXPR_FROM_DOUBLE constexpr
|
||||
using FromDoubleIntermediateType = long double;
|
||||
#else
|
||||
#include <boost/multiprecision/cpp_bin_float.hpp>
|
||||
/// `wide_integer_from_builtin` can't be constexpr with non-literal `cpp_bin_float_double_extended`
|
||||
#define CONSTEXPR_FROM_DOUBLE
|
||||
using FromDoubleIntermediateType = boost::multiprecision::cpp_bin_float_double_extended;
|
||||
|
@ -19,7 +19,6 @@
|
||||
#include "Poco/UTF16Encoding.h"
|
||||
#include "Poco/Buffer.h"
|
||||
#include "Poco/Exception.h"
|
||||
#include <iostream>
|
||||
|
||||
|
||||
using Poco::Buffer;
|
||||
|
@ -16,7 +16,6 @@
|
||||
#include "Poco/TaskManager.h"
|
||||
#include "Poco/Exception.h"
|
||||
|
||||
#include <iostream>
|
||||
#include <array>
|
||||
|
||||
|
||||
|
@ -14,7 +14,6 @@
|
||||
|
||||
#include "Poco/JSON/Object.h"
|
||||
#include <iostream>
|
||||
#include <sstream>
|
||||
|
||||
|
||||
using Poco::Dynamic::Var;
|
||||
|
@ -26,7 +26,6 @@
|
||||
#include "Poco/CountingStream.h"
|
||||
#include "Poco/RegularExpression.h"
|
||||
#include <sstream>
|
||||
#include <iostream>
|
||||
|
||||
|
||||
using Poco::NumberFormatter;
|
||||
|
@ -1,32 +0,0 @@
|
||||
---
|
||||
slug: /ru/getting-started/example-datasets/wikistat
|
||||
sidebar_position: 17
|
||||
sidebar_label: WikiStat
|
||||
---
|
||||
|
||||
# WikiStat {#wikistat}
|
||||
|
||||
См: http://dumps.wikimedia.org/other/pagecounts-raw/
|
||||
|
||||
Создание таблицы:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE wikistat
|
||||
(
|
||||
date Date,
|
||||
time DateTime,
|
||||
project String,
|
||||
subproject String,
|
||||
path String,
|
||||
hits UInt64,
|
||||
size UInt64
|
||||
) ENGINE = MergeTree(date, (path, time), 8192);
|
||||
```
|
||||
|
||||
Загрузка данных:
|
||||
|
||||
``` bash
|
||||
$ for i in {2007..2016}; do for j in {01..12}; do echo $i-$j >&2; curl -sSL "http://dumps.wikimedia.org/other/pagecounts-raw/$i/$i-$j/" | grep -oE 'pagecounts-[0-9]+-[0-9]+\.gz'; done; done | sort | uniq | tee links.txt
|
||||
$ cat links.txt | while read link; do wget http://dumps.wikimedia.org/other/pagecounts-raw/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1/')/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1-\2/')/$link; done
|
||||
$ ls -1 /opt/wikistat/ | grep gz | while read i; do echo $i; gzip -cd /opt/wikistat/$i | ./wikistat-loader --time="$(echo -n $i | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})([0-9]{2})-([0-9]{2})([0-9]{2})([0-9]{2})\.gz/\1-\2-\3 \4-00-00/')" | clickhouse-client --query="INSERT INTO wikistat FORMAT TabSeparated"; done
|
||||
```
|
1
docs/ru/getting-started/example-datasets/wikistat.md
Symbolic link
1
docs/ru/getting-started/example-datasets/wikistat.md
Symbolic link
@ -0,0 +1 @@
|
||||
../../../en/getting-started/example-datasets/wikistat.md
|
@ -1,32 +0,0 @@
|
||||
---
|
||||
slug: /zh/getting-started/example-datasets/wikistat
|
||||
sidebar_position: 17
|
||||
sidebar_label: WikiStat
|
||||
---
|
||||
|
||||
# WikiStat {#wikistat}
|
||||
|
||||
参考: http://dumps.wikimedia.org/other/pagecounts-raw/
|
||||
|
||||
创建表结构:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE wikistat
|
||||
(
|
||||
date Date,
|
||||
time DateTime,
|
||||
project String,
|
||||
subproject String,
|
||||
path String,
|
||||
hits UInt64,
|
||||
size UInt64
|
||||
) ENGINE = MergeTree(date, (path, time), 8192);
|
||||
```
|
||||
|
||||
加载数据:
|
||||
|
||||
``` bash
|
||||
$ for i in {2007..2016}; do for j in {01..12}; do echo $i-$j >&2; curl -sSL "http://dumps.wikimedia.org/other/pagecounts-raw/$i/$i-$j/" | grep -oE 'pagecounts-[0-9]+-[0-9]+\.gz'; done; done | sort | uniq | tee links.txt
|
||||
$ cat links.txt | while read link; do wget http://dumps.wikimedia.org/other/pagecounts-raw/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1/')/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1-\2/')/$link; done
|
||||
$ ls -1 /opt/wikistat/ | grep gz | while read i; do echo $i; gzip -cd /opt/wikistat/$i | ./wikistat-loader --time="$(echo -n $i | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})([0-9]{2})-([0-9]{2})([0-9]{2})([0-9]{2})\.gz/\1-\2-\3 \4-00-00/')" | clickhouse-client --query="INSERT INTO wikistat FORMAT TabSeparated"; done
|
||||
```
|
1
docs/zh/getting-started/example-datasets/wikistat.md
Symbolic link
1
docs/zh/getting-started/example-datasets/wikistat.md
Symbolic link
@ -0,0 +1 @@
|
||||
../../../en/getting-started/example-datasets/wikistat.md
|
@ -1,4 +1,6 @@
|
||||
#include "ICommand.h"
|
||||
#include <iostream>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
#include <boost/container/flat_set.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <AggregateFunctions/StatCommon.h>
|
||||
#include <iostream>
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
|
@ -14,6 +14,9 @@
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/HashUtils.h>
|
||||
|
||||
#include <numeric>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <iostream>
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include "ConnectionParameters.h"
|
||||
#include <fstream>
|
||||
#include <iostream>
|
||||
#include <Core/Defines.h>
|
||||
#include <Core/Protocol.h>
|
||||
#include <Core/Types.h>
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Poco/URI.h>
|
||||
|
||||
#include <array>
|
||||
#include <iostream>
|
||||
#include <string>
|
||||
#include <unordered_map>
|
||||
|
||||
|
@ -2,17 +2,17 @@
|
||||
#include <Columns/ColumnObject.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnSparse.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
#include <Processors/Transforms/ColumnGathererTransform.h>
|
||||
#include <numeric>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -10,6 +10,7 @@
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
|
||||
using namespace DB;
|
||||
static pcg64 rng(randomSeed());
|
||||
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Poco/Util/LayeredConfiguration.h>
|
||||
#include "ConfigProcessor.h"
|
||||
#include <filesystem>
|
||||
#include <iostream>
|
||||
#include <base/types.h>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <sys/file.h>
|
||||
|
||||
#include <string>
|
||||
#include <iostream>
|
||||
#include <mutex>
|
||||
#include <filesystem>
|
||||
|
||||
|
@ -10,7 +10,6 @@
|
||||
#include <chrono>
|
||||
#include <cstring>
|
||||
#include <memory>
|
||||
#include <iostream>
|
||||
|
||||
|
||||
/// Embedded timezones.
|
||||
|
@ -7,11 +7,11 @@
|
||||
#include <map>
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
#include <iostream>
|
||||
|
||||
#include <base/types.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include "FST.h"
|
||||
#include <algorithm>
|
||||
#include <cassert>
|
||||
#include <iostream>
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
#include <Common/Exception.h>
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <iostream>
|
||||
#include <vector>
|
||||
|
||||
#include <boost/range/adaptor/reversed.hpp>
|
||||
@ -19,6 +18,7 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/VarInt.h>
|
||||
|
||||
|
||||
/*
|
||||
* Implementation of the Filtered Space-Saving for TopK streaming analysis.
|
||||
* http://www.l2f.inesc-id.pt/~fmmb/wiki/uploads/Work/misnis.ref0a.pdf
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include "StudentTTest.h"
|
||||
|
||||
#include <cmath>
|
||||
#include <iostream>
|
||||
#include <iomanip>
|
||||
#include <sstream>
|
||||
#include <stdexcept>
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Common/noexcept_scope.h>
|
||||
|
||||
#include <cassert>
|
||||
#include <iostream>
|
||||
#include <type_traits>
|
||||
|
||||
#include <Poco/Util/Application.h>
|
||||
|
@ -7,7 +7,6 @@
|
||||
#include <Common/UnicodeBar.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
|
||||
#include <iostream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -2,9 +2,9 @@
|
||||
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include <iostream>
|
||||
#include <base/types.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -27,11 +27,6 @@ struct VersionNumber
|
||||
|
||||
std::string toString() const;
|
||||
|
||||
friend std::ostream & operator<<(std::ostream & os, const VersionNumber & v)
|
||||
{
|
||||
return os << v.toString();
|
||||
}
|
||||
|
||||
private:
|
||||
using Components = std::vector<Int64>;
|
||||
Components components;
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <benchmark/benchmark.h>
|
||||
|
||||
#include <iomanip>
|
||||
#include <iostream>
|
||||
#include <random>
|
||||
#include <vector>
|
||||
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <iostream>
|
||||
#include <string>
|
||||
#include <bit>
|
||||
|
||||
|
@ -6,7 +6,6 @@
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ThreadFuzzer.h>
|
||||
|
||||
|
||||
|
@ -4,11 +4,11 @@
|
||||
#include <filesystem>
|
||||
#include <string>
|
||||
#include <iostream>
|
||||
#include <fstream>
|
||||
#include <stdexcept>
|
||||
#include <cstdlib>
|
||||
#include <unistd.h>
|
||||
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
static std::string createTmpPath(const std::string & filename)
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <iomanip>
|
||||
#include <iostream>
|
||||
#include <numeric>
|
||||
|
||||
#include <Interpreters/AggregationCommon.h>
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <iomanip>
|
||||
#include <iostream>
|
||||
#include <gtest/gtest.h>
|
||||
#include <Common/CacheBase.h>
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <iomanip>
|
||||
#include <iostream>
|
||||
|
||||
#include <Common/HashTable/LRUHashMap.h>
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <iomanip>
|
||||
#include <iostream>
|
||||
#include <gtest/gtest.h>
|
||||
#include <Common/LRUResourceCache.h>
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <iomanip>
|
||||
#include <iostream>
|
||||
#include <gtest/gtest.h>
|
||||
#include <Common/CacheBase.h>
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <atomic>
|
||||
#include <iostream>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <atomic>
|
||||
#include <iostream>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <iostream>
|
||||
#include <stdexcept>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
|
@ -1,15 +1,17 @@
|
||||
#ifdef ENABLE_QPL_COMPRESSION
|
||||
|
||||
#include <cstdio>
|
||||
#include <thread>
|
||||
#include <Compression/CompressionCodecDeflateQpl.h>
|
||||
#include <Compression/CompressionFactory.h>
|
||||
#include <Compression/CompressionInfo.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include "libaccel_config.h"
|
||||
#include <Common/MemorySanitizer.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <immintrin.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <iostream>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Common/Exception.h>
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <iostream>
|
||||
#include <string>
|
||||
|
||||
#include <Compression/ICompressionCodec.h>
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <iostream>
|
||||
#include <string>
|
||||
|
||||
#include <Compression/ICompressionCodec.h>
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <iostream>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <iostream>
|
||||
#include <string>
|
||||
|
||||
#include <Compression/ICompressionCodec.h>
|
||||
|
@ -577,7 +577,6 @@ String FeatureFlagsCommand::run()
|
||||
}
|
||||
|
||||
return ret.str();
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <unordered_map>
|
||||
#include <list>
|
||||
#include <atomic>
|
||||
#include <iostream>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <Coordination/SummingStateMachine.h>
|
||||
#include <iostream>
|
||||
#include <cstring>
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <Coordination/pathUtils.h>
|
||||
#include <iostream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include "MySQLCharset.h"
|
||||
#include "config.h"
|
||||
#include <iostream>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#if USE_ICU
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <iostream>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <sys/types.h>
|
||||
#include <unistd.h>
|
||||
#include <iostream>
|
||||
#include <memory>
|
||||
#include <functional>
|
||||
#include <optional>
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <Poco/Util/LayeredConfiguration.h>
|
||||
|
||||
#include <base/defines.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <base/getMemoryAmount.h>
|
||||
#include <Common/logger_useful.h>
|
||||
@ -13,7 +12,6 @@
|
||||
#include <Common/StackTrace.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
#include <base/hex.h>
|
||||
|
||||
#include "config.h"
|
||||
#include "config_version.h"
|
||||
|
@ -25,6 +25,7 @@
|
||||
#include <Dictionaries/HierarchyDictionariesUtils.h>
|
||||
#include <Dictionaries/HashedDictionaryCollectionTraits.h>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric HashedDictionaryThreads;
|
||||
|
@ -13,6 +13,7 @@
|
||||
|
||||
#include <numeric>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
|
||||
#include <iostream>
|
||||
#include <Disks/IO/CachedOnDiskReadBufferFromFile.h>
|
||||
#include <Disks/ObjectStorages/Cached/CachedObjectStorage.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
|
@ -1,13 +1,11 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <IO/WithFileName.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -53,6 +51,7 @@ ColumnsDescription readSchemaFromFormat(
|
||||
bool retry,
|
||||
ContextPtr & context,
|
||||
std::unique_ptr<ReadBuffer> & buf)
|
||||
try
|
||||
{
|
||||
NamesAndTypesList names_and_types;
|
||||
if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name))
|
||||
@ -209,12 +208,23 @@ ColumnsDescription readSchemaFromFormat(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"{} file format doesn't support schema inference. You must specify the structure manually",
|
||||
format_name);
|
||||
|
||||
/// Some formats like CSVWithNames can contain empty column names. We don't support empty column names and further processing can fail with an exception. Let's just remove columns with empty names from the structure.
|
||||
names_and_types.erase(
|
||||
std::remove_if(names_and_types.begin(), names_and_types.end(), [](const NameAndTypePair & pair) { return pair.name.empty(); }),
|
||||
names_and_types.end());
|
||||
return ColumnsDescription(names_and_types);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
if (!buf)
|
||||
throw;
|
||||
auto file_name = getFileNameFromReadBuffer(*buf);
|
||||
if (!file_name.empty())
|
||||
e.addMessage(fmt::format("(in file/uri {})", file_name));
|
||||
throw;
|
||||
}
|
||||
|
||||
|
||||
ColumnsDescription readSchemaFromFormat(
|
||||
const String & format_name,
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <algorithm>
|
||||
#include <numeric>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <iostream>
|
||||
|
||||
#include <Common/PODArray.h>
|
||||
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <string_view>
|
||||
|
||||
#include <iostream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -6,6 +6,8 @@
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <numeric>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -4,7 +4,6 @@
|
||||
|
||||
#include <IO/Archives/IArchiveReader.h>
|
||||
|
||||
#include <iostream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <iostream>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <iostream>
|
||||
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <iostream>
|
||||
#include <iosfwd>
|
||||
|
||||
|
||||
namespace DB::S3
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/StdStreamBufFromReadBuffer.h>
|
||||
#include <iostream>
|
||||
#include <memory>
|
||||
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <iostream>
|
||||
#include <base/types.h>
|
||||
#include <base/defines.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <algorithm>
|
||||
#include <memory>
|
||||
#include <iostream>
|
||||
#include <cassert>
|
||||
#include <cstring>
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <iostream>
|
||||
#include <iosfwd>
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <iostream>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
|
@ -2,14 +2,10 @@
|
||||
#include <set>
|
||||
#include <optional>
|
||||
#include <memory>
|
||||
#include <Poco/Mutex.h>
|
||||
#include <Poco/UUID.h>
|
||||
#include <Poco/Net/IPAddress.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <Common/Macros.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/EventNotifier.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/Throttler.h>
|
||||
@ -17,12 +13,10 @@
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/getMultipleKeysFromConfig.h>
|
||||
#include <Coordination/KeeperDispatcher.h>
|
||||
#include <Compression/ICompressionCodec.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Server/ServerType.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/MarkCache.h>
|
||||
#include <Storages/MergeTree/MergeList.h>
|
||||
#include <Storages/MergeTree/MovesList.h>
|
||||
@ -34,8 +28,6 @@
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorage.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
|
||||
#include <Disks/IO/ThreadPoolReader.h>
|
||||
#include <Disks/StoragePolicy.h>
|
||||
#include <IO/SynchronousReader.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
@ -44,7 +36,6 @@
|
||||
#include <Interpreters/TemporaryDataOnDisk.h>
|
||||
#include <Interpreters/Cache/QueryCache.h>
|
||||
#include <Interpreters/Cache/FileCacheFactory.h>
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
#include <Interpreters/SessionTracker.h>
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <Interpreters/PreparedSets.h>
|
||||
@ -56,7 +47,6 @@
|
||||
#include <Access/EnabledRowPolicies.h>
|
||||
#include <Access/QuotaUsage.h>
|
||||
#include <Access/User.h>
|
||||
#include <Access/Credentials.h>
|
||||
#include <Access/SettingsProfile.h>
|
||||
#include <Access/SettingsProfilesInfo.h>
|
||||
#include <Access/SettingsConstraintsAndProfileIDs.h>
|
||||
@ -70,7 +60,6 @@
|
||||
#include <Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.h>
|
||||
#include <Functions/UserDefined/IUserDefinedSQLObjectsLoader.h>
|
||||
#include <Functions/UserDefined/createUserDefinedSQLObjectsLoader.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Interpreters/InterserverCredentials.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
@ -87,8 +76,6 @@
|
||||
#include <IO/MMappedFileCache.h>
|
||||
#include <IO/WriteSettings.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Common/StackTrace.h>
|
||||
@ -98,14 +85,12 @@
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Common/ShellCommand.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <base/EnumReflection.h>
|
||||
#include <Common/RemoteHostFilter.h>
|
||||
#include <Common/HTTPHeaderFilter.h>
|
||||
#include <Interpreters/AsynchronousInsertQueue.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
||||
#include <Storages/MergeTree/BackgroundJobsAssignee.h>
|
||||
#include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
|
||||
#include <Storages/MergeTree/MergeTreeMetadataCache.h>
|
||||
#include <Interpreters/SynonymsExtensions.h>
|
||||
@ -118,12 +103,8 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/FunctionParameterValuesVisitor.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <base/find_symbols.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
|
||||
#if USE_ROCKSDB
|
||||
#include <rocksdb/table.h>
|
||||
#endif
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
|
@ -1,15 +1,11 @@
|
||||
#include <Interpreters/GraceHashJoin.h>
|
||||
#include <Interpreters/HashJoin.h>
|
||||
#include <Interpreters/TableJoin.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Formats/NativeWriter.h>
|
||||
#include <Interpreters/TemporaryDataOnDisk.h>
|
||||
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <Core/ProtocolDefines.h>
|
||||
#include <Disks/IVolume.h>
|
||||
#include <Disks/TemporaryFileOnDisk.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
|
||||
@ -18,6 +14,9 @@
|
||||
|
||||
#include <Formats/formatBlock.h>
|
||||
|
||||
#include <numeric>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric TemporaryFilesForJoin;
|
||||
|
@ -24,7 +24,6 @@
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <thread>
|
||||
#include <iostream>
|
||||
#include <cstddef>
|
||||
|
||||
|
||||
|
@ -114,7 +114,7 @@ public:
|
||||
* and new table get created - as if previous table was not exist.
|
||||
*/
|
||||
SystemLog(ContextPtr context_,
|
||||
const SystemLogSettings& settings_,
|
||||
const SystemLogSettings & settings_,
|
||||
std::shared_ptr<SystemLogQueue<LogElement>> queue_ = nullptr);
|
||||
|
||||
/** Append a record into log.
|
||||
@ -134,8 +134,6 @@ protected:
|
||||
using Base::queue;
|
||||
|
||||
private:
|
||||
|
||||
|
||||
/* Saving thread data */
|
||||
const StorageID table_id;
|
||||
const String storage_def;
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include "OwnSplitChannel.h"
|
||||
#include "OwnFormattingChannel.h"
|
||||
|
||||
#include <iostream>
|
||||
#include <Core/Block.h>
|
||||
#include <Interpreters/InternalTextLogsQueue.h>
|
||||
#include <Interpreters/TextLog.h>
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <IO/Operators.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
||||
#include <iostream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <Processors/Executors/StreamingFormatExecutor.h>
|
||||
#include <Processors/Transforms/AddingDefaultsTransform.h>
|
||||
#include <iostream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -86,7 +86,21 @@ void IRowInputFormat::logError()
|
||||
Chunk IRowInputFormat::generate()
|
||||
{
|
||||
if (total_rows == 0)
|
||||
readPrefix();
|
||||
{
|
||||
try
|
||||
{
|
||||
readPrefix();
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
auto file_name = getFileNameFromReadBuffer(getReadBuffer());
|
||||
if (!file_name.empty())
|
||||
e.addMessage(fmt::format("(in file/uri {})", file_name));
|
||||
|
||||
e.addMessage("(while reading header)");
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
const Block & header = getPort().getHeader();
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <Processors/IAccumulatingTransform.h>
|
||||
#include <iostream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <Processors/ResizeProcessor.h>
|
||||
#include <iostream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <iostream>
|
||||
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Processors/Transforms/CountingTransform.h>
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <Common/ThreadStatus.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <iostream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <Processors/Transforms/SquashingChunksTransform.h>
|
||||
#include <iostream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -5,9 +5,9 @@
|
||||
#include <Poco/Net/HTTPServerSession.h>
|
||||
#include <Poco/Net/HTTPResponse.h>
|
||||
|
||||
#include <iostream>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -11,7 +11,6 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <vector>
|
||||
#include <unordered_map>
|
||||
#include <iostream>
|
||||
#include <numeric>
|
||||
#include <algorithm>
|
||||
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <iostream>
|
||||
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
|
@ -114,7 +114,6 @@ test_quota/test.py::test_tracking_quota
|
||||
test_quota/test.py::test_users_xml_is_readonly
|
||||
test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database
|
||||
test_profile_events_s3/test.py::test_profile_events
|
||||
test_system_flush_logs/test.py::test_system_logs[system.text_log-0]
|
||||
test_user_defined_object_persistence/test.py::test_persistence
|
||||
test_settings_profile/test.py::test_show_profiles
|
||||
test_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functions_on_cluster
|
||||
|
@ -202,13 +202,14 @@ def prepare_for_hung_check(drop_databases):
|
||||
call(
|
||||
make_query_command(
|
||||
"""
|
||||
select sleepEachRow((
|
||||
select maxOrDefault(300 - elapsed) + 1
|
||||
from system.processes
|
||||
where query not like '%from system.processes%' and elapsed < 300
|
||||
SELECT sleepEachRow((
|
||||
SELECT maxOrDefault(300 - elapsed) + 1
|
||||
FROM system.processes
|
||||
WHERE query NOT LIKE '%FROM system.processes%' AND elapsed < 300
|
||||
) / 300)
|
||||
from numbers(300)
|
||||
format Null
|
||||
FROM numbers(300)
|
||||
FORMAT Null
|
||||
SETTINGS function_sleep_max_microseconds_per_block = 0
|
||||
"""
|
||||
),
|
||||
shell=True,
|
||||
|
@ -45,7 +45,11 @@ def test_system_logs(flush_logs, table, exists):
|
||||
if exists:
|
||||
node.query(q)
|
||||
else:
|
||||
assert "Table {} doesn't exist".format(table) in node.query_and_get_error(q)
|
||||
response = node.query_and_get_error(q)
|
||||
assert (
|
||||
"Table {} doesn't exist".format(table) in response
|
||||
or "Unknown table expression identifier '{}'".format(table) in response
|
||||
)
|
||||
|
||||
|
||||
# Logic is tricky, let's check that there is no hang in case of message queue
|
||||
|
@ -57,11 +57,10 @@ def test_chroot_with_same_root(started_cluster):
|
||||
for j in range(2): # Second insert to test deduplication
|
||||
node.query("INSERT INTO simple VALUES ({0}, {0})".format(i))
|
||||
|
||||
# Replication might take time
|
||||
|
||||
assert_eq_with_retry(node1, "select count() from simple", "2\n")
|
||||
|
||||
assert_eq_with_retry(node2, "select count() from simple", "2\n")
|
||||
node1.query("SYSTEM SYNC REPLICA simple")
|
||||
assert_eq_with_retry(node1, "select count() from simple", "2")
|
||||
node2.query("SYSTEM SYNC REPLICA simple")
|
||||
assert_eq_with_retry(node2, "select count() from simple", "2")
|
||||
|
||||
|
||||
def test_chroot_with_different_root(started_cluster):
|
||||
@ -78,5 +77,7 @@ def test_chroot_with_different_root(started_cluster):
|
||||
for j in range(2): # Second insert to test deduplication
|
||||
node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i))
|
||||
|
||||
assert node1.query("select count() from simple_different").strip() == "1"
|
||||
assert node3.query("select count() from simple_different").strip() == "1"
|
||||
node1.query("SYSTEM SYNC REPLICA simple_different")
|
||||
assert_eq_with_retry(node1, "select count() from simple_different", "1")
|
||||
node3.query("SYSTEM SYNC REPLICA simple_different")
|
||||
assert_eq_with_retry(node3, "select count() from simple_different", "1")
|
||||
|
@ -23,10 +23,12 @@ function run_format_both()
|
||||
|
||||
# NOTE: that those queries may work slow, due to stack trace obtaining
|
||||
run_format 'insert into foo settings max_threads=1' 2> >(grep -m1 -o "Syntax error (query): failed at position .* (end of query):")
|
||||
|
||||
# compatibility
|
||||
run_format 'insert into foo format tsv settings max_threads=1' 2> >(grep -m1 -F -o "Can't format ASTInsertQuery with data, since data will be lost.")
|
||||
run_format_both 'insert into foo format tsv settings max_threads=1' --allow_settings_after_format_in_insert
|
||||
run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT")
|
||||
|
||||
# and via server (since this is a separate code path)
|
||||
$CLICKHOUSE_CLIENT -q 'drop table if exists data_02263'
|
||||
$CLICKHOUSE_CLIENT -q 'create table data_02263 (key Int) engine=Memory()'
|
||||
|
@ -0,0 +1,2 @@
|
||||
in file/uri
|
||||
test.csv
|
11
tests/queries/0_stateless/02836_file_diagnostics_while_reading_header.sh
Executable file
11
tests/queries/0_stateless/02836_file_diagnostics_while_reading_header.sh
Executable file
@ -0,0 +1,11 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
FILENAME="${CLICKHOUSE_TMP}/test.csv"
|
||||
|
||||
printf 'Bad\rHeader\n123\n' > "${FILENAME}"
|
||||
${CLICKHOUSE_LOCAL} --query "SELECT * FROM file('${CLICKHOUSE_TMP}/t*e*s*t.csv')" 2>&1 | grep -o -P 'in file/uri|test\.csv'
|
||||
rm "${FILENAME}"
|
@ -20,7 +20,6 @@ if (ENABLE_UTILS)
|
||||
add_subdirectory (zookeeper-cli)
|
||||
add_subdirectory (zookeeper-dump-tree)
|
||||
add_subdirectory (zookeeper-remove-by-list)
|
||||
add_subdirectory (wikistat-loader)
|
||||
add_subdirectory (check-marks)
|
||||
add_subdirectory (checksum-for-compressed-block)
|
||||
add_subdirectory (check-mysql-binlog)
|
||||
|
@ -1,2 +0,0 @@
|
||||
clickhouse_add_executable (wikistat-loader main.cpp ${SRCS})
|
||||
target_link_libraries (wikistat-loader PRIVATE clickhouse_common_io boost::program_options)
|
@ -1,225 +0,0 @@
|
||||
#include <boost/program_options.hpp>
|
||||
|
||||
#include <base/hex.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadBufferFromFileDescriptor.h>
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
|
||||
|
||||
/** Reads uncompressed wikistat data from stdin,
|
||||
* and writes transformed data in tsv format,
|
||||
* ready to be loaded into ClickHouse.
|
||||
*
|
||||
* Input data has format:
|
||||
*
|
||||
* aa Wikipedia 1 17224
|
||||
* aa.b Main_Page 2 21163
|
||||
*
|
||||
* project, optional subproject, path, hits, total size in bytes.
|
||||
*/
|
||||
|
||||
|
||||
template <bool break_at_dot>
|
||||
static void readString(std::string & s, DB::ReadBuffer & buf)
|
||||
{
|
||||
s.clear();
|
||||
|
||||
while (!buf.eof())
|
||||
{
|
||||
const char * next_pos;
|
||||
|
||||
if (break_at_dot)
|
||||
next_pos = find_first_symbols<' ', '\n', '.'>(buf.position(), buf.buffer().end());
|
||||
else
|
||||
next_pos = find_first_symbols<' ', '\n'>(buf.position(), buf.buffer().end());
|
||||
|
||||
s.append(buf.position(), next_pos - buf.position());
|
||||
buf.position() += next_pos - buf.position();
|
||||
|
||||
if (!buf.hasPendingData())
|
||||
continue;
|
||||
|
||||
if (*buf.position() == ' ' || *buf.position() == '\n' || (break_at_dot && *buf.position() == '.'))
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/** Reads path before whitespace and decodes %xx sequences (to more compact and handy representation),
|
||||
* except %2F '/', %26 '&', %3D '=', %3F '?', %23 '#' (to not break structure of URL).
|
||||
*/
|
||||
static void readPath(std::string & s, DB::ReadBuffer & buf)
|
||||
{
|
||||
s.clear();
|
||||
|
||||
while (!buf.eof())
|
||||
{
|
||||
const char * next_pos = find_first_symbols<' ', '\n', '%'>(buf.position(), buf.buffer().end());
|
||||
|
||||
s.append(buf.position(), next_pos - buf.position());
|
||||
buf.position() += next_pos - buf.position();
|
||||
|
||||
if (!buf.hasPendingData())
|
||||
continue;
|
||||
|
||||
if (*buf.position() == ' ' || *buf.position() == '\n')
|
||||
return;
|
||||
|
||||
if (*buf.position() == '%')
|
||||
{
|
||||
++buf.position();
|
||||
|
||||
char c1;
|
||||
char c2;
|
||||
|
||||
if (buf.eof() || *buf.position() == ' ')
|
||||
break;
|
||||
|
||||
DB::readChar(c1, buf);
|
||||
|
||||
if (buf.eof() || *buf.position() == ' ')
|
||||
break;
|
||||
|
||||
DB::readChar(c2, buf);
|
||||
|
||||
if ((c1 == '2' && (c2 == 'f' || c2 == '6' || c2 == '3' || c2 == 'F'))
|
||||
|| (c1 == '3' && (c2 == 'd' || c2 == 'f' || c2 == 'D' || c2 == 'F')))
|
||||
{
|
||||
s += '%';
|
||||
s += c1;
|
||||
s += c2;
|
||||
}
|
||||
else
|
||||
s += static_cast<char>(static_cast<UInt8>(unhex(c1)) * 16 + static_cast<UInt8>(unhex(c2)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
static void skipUntilNewline(DB::ReadBuffer & buf)
|
||||
{
|
||||
while (!buf.eof())
|
||||
{
|
||||
const char * next_pos = find_first_symbols<'\n'>(buf.position(), buf.buffer().end());
|
||||
|
||||
buf.position() += next_pos - buf.position();
|
||||
|
||||
if (!buf.hasPendingData())
|
||||
continue;
|
||||
|
||||
if (*buf.position() == '\n')
|
||||
{
|
||||
++buf.position();
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
try
|
||||
{
|
||||
boost::program_options::options_description desc("Allowed options");
|
||||
desc.add_options()
|
||||
("help,h", "produce help message")
|
||||
("time", boost::program_options::value<std::string>()->required(),
|
||||
"time of data in YYYY-MM-DD hh:mm:ss form")
|
||||
;
|
||||
|
||||
boost::program_options::variables_map options;
|
||||
boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options);
|
||||
|
||||
if (options.count("help"))
|
||||
{
|
||||
std::cout << "Reads uncompressed wikistat data from stdin and writes transformed data in tsv format." << std::endl;
|
||||
std::cout << "Usage: " << argv[0] << " --time='YYYY-MM-DD hh:00:00' < in > out" << std::endl;
|
||||
std::cout << desc << std::endl;
|
||||
return 1;
|
||||
}
|
||||
|
||||
std::string time_str = options.at("time").as<std::string>();
|
||||
LocalDateTime time(time_str);
|
||||
LocalDate date(time_str);
|
||||
|
||||
DB::ReadBufferFromFileDescriptor in(STDIN_FILENO);
|
||||
DB::WriteBufferFromFileDescriptor out(STDOUT_FILENO);
|
||||
|
||||
std::string project;
|
||||
std::string subproject;
|
||||
std::string path;
|
||||
UInt64 hits = 0;
|
||||
UInt64 size = 0;
|
||||
|
||||
size_t row_num = 0;
|
||||
while (!in.eof())
|
||||
{
|
||||
try
|
||||
{
|
||||
++row_num;
|
||||
readString<true>(project, in);
|
||||
|
||||
if (in.eof())
|
||||
break;
|
||||
|
||||
if (*in.position() == '.')
|
||||
readString<false>(subproject, in);
|
||||
else
|
||||
subproject.clear();
|
||||
|
||||
DB::assertChar(' ', in);
|
||||
readPath(path, in);
|
||||
DB::assertChar(' ', in);
|
||||
DB::readIntText(hits, in);
|
||||
DB::assertChar(' ', in);
|
||||
DB::readIntText(size, in);
|
||||
DB::assertChar('\n', in);
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
/// Sometimes, input data has errors. For example, look at first lines in pagecounts-20130210-130000.gz
|
||||
/// To save rest of data, just skip lines with errors.
|
||||
if (e.code() == DB::ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED)
|
||||
{
|
||||
std::cerr << "At row " << row_num << ": " << DB::getCurrentExceptionMessage(false) << '\n';
|
||||
skipUntilNewline(in);
|
||||
continue;
|
||||
}
|
||||
else
|
||||
throw;
|
||||
}
|
||||
|
||||
DB::writeText(date, out);
|
||||
DB::writeChar('\t', out);
|
||||
DB::writeText(time, out);
|
||||
DB::writeChar('\t', out);
|
||||
DB::writeText(project, out);
|
||||
DB::writeChar('\t', out);
|
||||
DB::writeText(subproject, out);
|
||||
DB::writeChar('\t', out);
|
||||
DB::writeText(path, out);
|
||||
DB::writeChar('\t', out);
|
||||
DB::writeText(hits, out);
|
||||
DB::writeChar('\t', out);
|
||||
DB::writeText(size, out);
|
||||
DB::writeChar('\n', out);
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
std::cerr << DB::getCurrentExceptionMessage(true) << '\n';
|
||||
throw;
|
||||
}
|
Loading…
Reference in New Issue
Block a user