Merge branch 'master' into fix-mv-link

This commit is contained in:
Dan Roscigno 2022-09-27 10:23:44 -04:00 committed by GitHub
commit b983ed35f2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
111 changed files with 3074 additions and 2994 deletions

View File

@ -338,6 +338,12 @@ echo $previous_release_tag | download_release_packets && echo -e 'Download scrip
|| echo -e 'Download script failed\tFAIL' >> /test_output/test_results.tsv
mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.clean.log
for table in query_log trace_log
do
clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.tsv.gz ||:
done
tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||:
# Check if we cloned previous release repository successfully
if ! [ "$(ls -A previous_release_repository/tests/queries)" ]
@ -454,6 +460,7 @@ else
-e "This engine is deprecated and is not supported in transactions" \
-e "[Queue = DB::MergeMutateRuntimeQueue]: Code: 235. DB::Exception: Part" \
-e "The set of parts restored in place of" \
-e "(ReplicatedMergeTreeAttachThread): Initialization failed. Error" \
/var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "<Error>" > /test_output/bc_check_error_messages.txt \
&& echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
@ -496,6 +503,12 @@ else
# Remove file bc_check_fatal_messages.txt if it's empty
[ -s /test_output/bc_check_fatal_messages.txt ] || rm /test_output/bc_check_fatal_messages.txt
tar -chf /test_output/coordination.backward.tar /var/lib/clickhouse/coordination ||:
for table in query_log trace_log
do
clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.backward.tsv.gz ||:
done
fi
dmesg -T > /test_output/dmesg.log
@ -505,17 +518,8 @@ grep -q -F -e 'Out of memory: Killed process' -e 'oom_reaper: reaped process' -e
&& echo -e 'OOM in dmesg\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'No OOM in dmesg\tOK' >> /test_output/test_results.tsv
tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||:
mv /var/log/clickhouse-server/stderr.log /test_output/
# Replace the engine with Ordinary to avoid extra symlinks stuff in artifacts.
# (so that clickhouse-local --path can read it w/o extra care).
sed -i -e "s/ATTACH DATABASE _ UUID '[^']*'/ATTACH DATABASE system/" -e "s/Atomic/Ordinary/" /var/lib/clickhouse/metadata/system.sql
for table in query_log trace_log; do
sed -i "s/ATTACH TABLE _ UUID '[^']*'/ATTACH TABLE $table/" /var/lib/clickhouse/metadata/system/${table}.sql
tar -chf /test_output/${table}_dump.tar /var/lib/clickhouse/metadata/system.sql /var/lib/clickhouse/metadata/system/${table}.sql /var/lib/clickhouse/data/system/${table} ||:
done
# Write check result into check_status.tsv
clickhouse-local --structure "test String, res String" -q "SELECT 'failure', test FROM table WHERE res != 'OK' order by (lower(test) like '%hung%'), rowNumberInAllBlocks() LIMIT 1" < /test_output/test_results.tsv > /test_output/check_status.tsv
[ -s /test_output/check_status.tsv ] || echo -e "success\tNo errors found" > /test_output/check_status.tsv

View File

@ -12,7 +12,16 @@ then
DIR="amd64"
elif [ "${ARCH}" = "aarch64" -o "${ARCH}" = "arm64" ]
then
DIR="aarch64"
# If the system has >=ARMv8.2 (https://en.wikipedia.org/wiki/AArch64), choose the corresponding build, else fall back to a v8.0
# compat build. Unfortunately, the ARM ISA level cannot be read directly, we need to guess from the "features" in /proc/cpuinfo.
# Also, the flags in /proc/cpuinfo are named differently than the flags passed to the compiler (cmake/cpu_features.cmake).
ARMV82=$(grep -m 1 'Features' /proc/cpuinfo | awk '/asimd/ && /sha1/ && /aes/ && /atomics/')
if [ "${ARMV82}" ]
then
DIR="aarch64"
else
DIR="aarch64v80compat"
fi
elif [ "${ARCH}" = "powerpc64le" -o "${ARCH}" = "ppc64le" ]
then
DIR="powerpc64le"
@ -22,12 +31,6 @@ then
if [ "${ARCH}" = "x86_64" -o "${ARCH}" = "amd64" ]
then
DIR="freebsd"
elif [ "${ARCH}" = "aarch64" -o "${ARCH}" = "arm64" ]
then
DIR="freebsd-aarch64"
elif [ "${ARCH}" = "powerpc64le" -o "${ARCH}" = "ppc64le" ]
then
DIR="freebsd-powerpc64le"
fi
elif [ "${OS}" = "Darwin" ]
then
@ -42,7 +45,7 @@ fi
if [ -z "${DIR}" ]
then
echo "The '${OS}' operating system with the '${ARCH}' architecture is not supported."
echo "Operating system '${OS}' / architecture '${ARCH}' is unsupported."
exit 1
fi

View File

@ -58,7 +58,9 @@ void processFile(const fs::path & file_path, const fs::path & dst_path, bool tes
}
else
{
auto src_buf = createReadBufferFromFileBase(file_path, {}, fs::file_size(file_path));
ReadSettings read_settings{};
read_settings.local_fs_method = LocalFSReadMethod::pread;
auto src_buf = createReadBufferFromFileBase(file_path, read_settings, fs::file_size(file_path));
std::shared_ptr<WriteBuffer> dst_buf;
/// test mode for integration tests.

View File

@ -198,6 +198,7 @@ protected:
SharedContextHolder shared_context;
ContextMutablePtr global_context;
/// thread status should be destructed before shared context because it relies on process list.
std::optional<ThreadStatus> thread_status;
ServerConnectionPtr connection;

View File

@ -7,15 +7,11 @@
#include <base/unaligned.h>
#include <Core/Field.h>
#include <Common/assert_cast.h>
#include <Common/TargetSpecific.h>
#include <Core/TypeId.h>
#include <base/TypeName.h>
#include "config_core.h"
#if USE_MULTITARGET_CODE
# include <immintrin.h>
#endif
namespace DB
{
@ -395,124 +391,6 @@ protected:
Container data;
};
DECLARE_DEFAULT_CODE(
template <typename Container, typename Type>
inline void vectorIndexImpl(const Container & data, const PaddedPODArray<Type> & indexes, size_t limit, Container & res_data)
{
for (size_t i = 0; i < limit; ++i)
res_data[i] = data[indexes[i]];
}
);
DECLARE_AVX512VBMI_SPECIFIC_CODE(
template <typename Container, typename Type>
inline void vectorIndexImpl(const Container & data, const PaddedPODArray<Type> & indexes, size_t limit, Container & res_data)
{
static constexpr UInt64 MASK64 = 0xffffffffffffffff;
const size_t limit64 = limit & ~63;
size_t pos = 0;
size_t data_size = data.size();
auto data_pos = reinterpret_cast<const UInt8 *>(data.data());
auto indexes_pos = reinterpret_cast<const UInt8 *>(indexes.data());
auto res_pos = reinterpret_cast<UInt8 *>(res_data.data());
if (data_size <= 64)
{
/// one single mask load for table size <= 64
__mmask64 last_mask = MASK64 >> (64 - data_size);
__m512i table1 = _mm512_maskz_loadu_epi8(last_mask, data_pos);
/// 64 bytes table lookup using one single permutexvar_epi8
while (pos < limit64)
{
__m512i vidx = _mm512_loadu_epi8(indexes_pos + pos);
__m512i out = _mm512_permutexvar_epi8(vidx, table1);
_mm512_storeu_epi8(res_pos + pos, out);
pos += 64;
}
/// tail handling
if (limit > limit64)
{
__mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit);
__m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos);
__m512i out = _mm512_permutexvar_epi8(vidx, table1);
_mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out);
}
}
else if (data_size <= 128)
{
/// table size (64, 128] requires 2 zmm load
__mmask64 last_mask = MASK64 >> (128 - data_size);
__m512i table1 = _mm512_loadu_epi8(data_pos);
__m512i table2 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 64);
/// 128 bytes table lookup using one single permute2xvar_epi8
while (pos < limit64)
{
__m512i vidx = _mm512_loadu_epi8(indexes_pos + pos);
__m512i out = _mm512_permutex2var_epi8(table1, vidx, table2);
_mm512_storeu_epi8(res_pos + pos, out);
pos += 64;
}
if (limit > limit64)
{
__mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit);
__m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos);
__m512i out = _mm512_permutex2var_epi8(table1, vidx, table2);
_mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out);
}
}
else
{
if (data_size > 256)
{
/// byte index will not exceed 256 boundary.
data_size = 256;
}
__m512i table1 = _mm512_loadu_epi8(data_pos);
__m512i table2 = _mm512_loadu_epi8(data_pos + 64);
__m512i table3, table4;
if (data_size <= 192)
{
/// only 3 tables need to load if size <= 192
__mmask64 last_mask = MASK64 >> (192 - data_size);
table3 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 128);
table4 = _mm512_setzero_si512();
}
else
{
__mmask64 last_mask = MASK64 >> (256 - data_size);
table3 = _mm512_loadu_epi8(data_pos + 128);
table4 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 192);
}
/// 256 bytes table lookup can use: 2 permute2xvar_epi8 plus 1 blender with MSB
while (pos < limit64)
{
__m512i vidx = _mm512_loadu_epi8(indexes_pos + pos);
__m512i tmp1 = _mm512_permutex2var_epi8(table1, vidx, table2);
__m512i tmp2 = _mm512_permutex2var_epi8(table3, vidx, table4);
__mmask64 msb = _mm512_movepi8_mask(vidx);
__m512i out = _mm512_mask_blend_epi8(msb, tmp1, tmp2);
_mm512_storeu_epi8(res_pos + pos, out);
pos += 64;
}
if (limit > limit64)
{
__mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit);
__m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos);
__m512i tmp1 = _mm512_permutex2var_epi8(table1, vidx, table2);
__m512i tmp2 = _mm512_permutex2var_epi8(table3, vidx, table4);
__mmask64 msb = _mm512_movepi8_mask(vidx);
__m512i out = _mm512_mask_blend_epi8(msb, tmp1, tmp2);
_mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out);
}
}
}
);
template <typename T>
template <typename Type>
ColumnPtr ColumnVector<T>::indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const
@ -521,18 +399,8 @@ ColumnPtr ColumnVector<T>::indexImpl(const PaddedPODArray<Type> & indexes, size_
auto res = this->create(limit);
typename Self::Container & res_data = res->getData();
#if USE_MULTITARGET_CODE
if constexpr (sizeof(T) == 1 && sizeof(Type) == 1)
{
/// VBMI optimization only applicable for (U)Int8 types
if (isArchSupported(TargetArch::AVX512VBMI))
{
TargetSpecific::AVX512VBMI::vectorIndexImpl<Container, Type>(data, indexes, limit, res_data);
return res;
}
}
#endif
TargetSpecific::Default::vectorIndexImpl<Container, Type>(data, indexes, limit, res_data);
for (size_t i = 0; i < limit; ++i)
res_data[i] = data[indexes[i]];
return res;
}

View File

@ -5,7 +5,7 @@
#include <Common/Exception.h>
#include <Common/hex.h>
#include <Core/Settings.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
namespace DB
{
@ -226,6 +226,30 @@ String TracingContext::composeTraceparentHeader() const
static_cast<uint8_t>(trace_flags));
}
void TracingContext::deserialize(ReadBuffer & buf)
{
buf >> this->trace_id
>> "\n"
>> this->span_id
>> "\n"
>> this->tracestate
>> "\n"
>> this->trace_flags
>> "\n";
}
void TracingContext::serialize(WriteBuffer & buf) const
{
buf << this->trace_id
<< "\n"
<< this->span_id
<< "\n"
<< this->tracestate
<< "\n"
<< this->trace_flags
<< "\n";
}
const TracingContextOnThread & CurrentContext()
{
return current_thread_trace_context;

View File

@ -7,6 +7,8 @@ namespace DB
struct Settings;
class OpenTelemetrySpanLog;
class WriteBuffer;
class ReadBuffer;
namespace OpenTelemetry
{
@ -63,6 +65,9 @@ struct TracingContext
{
return trace_id != UUID();
}
void deserialize(ReadBuffer & buf);
void serialize(WriteBuffer & buf) const;
};
/// Tracing context kept on each thread
@ -157,5 +162,16 @@ struct SpanHolder : public Span
}
inline WriteBuffer & operator<<(WriteBuffer & buf, const OpenTelemetry::TracingContext & context)
{
context.serialize(buf);
return buf;
}
inline ReadBuffer & operator>> (ReadBuffer & buf, OpenTelemetry::TracingContext & context)
{
context.deserialize(buf);
return buf;
}
}

View File

@ -705,7 +705,7 @@ void KeeperServer::waitInit()
int64_t timeout = coordination_settings->startup_timeout.totalMilliseconds();
if (!initialized_cv.wait_for(lock, std::chrono::milliseconds(timeout), [&] { return initialized_flag.load(); }))
throw Exception(ErrorCodes::RAFT_ERROR, "Failed to wait RAFT initialization");
LOG_WARNING(log, "Failed to wait for RAFT initialization in {}ms, will continue in background", timeout);
}
std::vector<int64_t> KeeperServer::getDeadSessions()

View File

@ -8,6 +8,7 @@
#include <Interpreters/executeQuery.h>
#include <Parsers/queryToString.h>
#include <Common/Exception.h>
#include <Common/OpenTelemetryTraceContext.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/ZooKeeper/Types.h>
#include <Common/ZooKeeper/ZooKeeper.h>
@ -642,6 +643,7 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex
entry.query = queryToString(query);
entry.initiator = ddl_worker->getCommonHostID();
entry.setSettingsIfRequired(query_context);
entry.tracing_context = OpenTelemetry::CurrentContext();
String node_path = ddl_worker->tryEnqueueAndExecuteEntry(entry, query_context);
Strings hosts_to_wait = getZooKeeper()->getChildren(zookeeper_path + "/replicas");

View File

@ -221,6 +221,10 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr
/// NOTE Possibly it would be better to execute initial query on the most up-to-date node,
/// but it requires more complex logic around /try node.
OpenTelemetry::SpanHolder span(__FUNCTION__);
span.addAttribute("clickhouse.cluster", database->getDatabaseName());
entry.tracing_context = OpenTelemetry::CurrentContext();
auto zookeeper = getAndSetZooKeeper();
UInt32 our_log_ptr = getLogPointer();
UInt32 max_log_ptr = parse<UInt32>(zookeeper->get(database->zookeeper_path + "/max_log_ptr"));

View File

@ -37,7 +37,7 @@ namespace ErrorCodes
AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRemoteFS(
AsynchronousReaderPtr reader_,
IAsynchronousReader & reader_,
const ReadSettings & settings_,
std::shared_ptr<ReadBufferFromRemoteFSGather> impl_,
size_t min_bytes_for_seek_)
@ -111,7 +111,7 @@ std::future<IAsynchronousReader::Result> AsynchronousReadIndirectBufferFromRemot
request.ignore = bytes_to_ignore;
bytes_to_ignore = 0;
}
return reader->submit(request);
return reader.submit(request);
}

View File

@ -31,7 +31,7 @@ class AsynchronousReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase
{
public:
explicit AsynchronousReadIndirectBufferFromRemoteFS(
AsynchronousReaderPtr reader_, const ReadSettings & settings_,
IAsynchronousReader & reader_, const ReadSettings & settings_,
std::shared_ptr<ReadBufferFromRemoteFSGather> impl_,
size_t min_bytes_for_seek = DBMS_DEFAULT_BUFFER_SIZE);
@ -64,7 +64,7 @@ private:
std::future<IAsynchronousReader::Result> asyncReadInto(char * data, size_t size);
AsynchronousReaderPtr reader;
IAsynchronousReader & reader;
Int32 priority;

View File

@ -198,31 +198,10 @@ std::future<IAsynchronousReader::Result> ThreadPoolReader::submit(Request reques
ProfileEvents::increment(ProfileEvents::ThreadPoolReaderPageCacheMiss);
ThreadGroupStatusPtr running_group;
if (CurrentThread::isInitialized() && CurrentThread::get().getThreadGroup())
running_group = CurrentThread::get().getThreadGroup();
auto schedule = threadPoolCallbackRunner<Result>(pool, "ThreadPoolRead");
ContextPtr query_context;
if (CurrentThread::isInitialized())
query_context = CurrentThread::get().getQueryContext();
auto task = std::make_shared<std::packaged_task<Result()>>([request, fd, running_group, query_context]
return schedule([request, fd]() -> Result
{
ThreadStatus thread_status;
SCOPE_EXIT({
if (running_group)
thread_status.detachQuery();
});
if (running_group)
thread_status.attachQuery(running_group);
if (query_context)
thread_status.attachQueryContext(query_context);
setThreadName("ThreadPoolRead");
Stopwatch watch(CLOCK_MONOTONIC);
SCOPE_EXIT({
watch.stop();
@ -260,14 +239,7 @@ std::future<IAsynchronousReader::Result> ThreadPoolReader::submit(Request reques
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadBytes, bytes_read);
return Result{ .size = bytes_read, .offset = request.ignore };
});
auto future = task->get_future();
/// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority".
pool.scheduleOrThrow([task]{ (*task)(); }, -request.priority);
return future;
}, request.priority);
}
}

View File

@ -2,6 +2,7 @@
#include <IO/AsynchronousReader.h>
#include <Common/ThreadPool.h>
#include <Interpreters/threadPoolCallbackRunner.h>
namespace DB
@ -31,8 +32,11 @@ private:
public:
ThreadPoolReader(size_t pool_size, size_t queue_size_);
std::future<Result> submit(Request request) override;
void wait() override { pool.wait(); }
/// pool automatically waits for all tasks in destructor.
};

View File

@ -5,7 +5,6 @@
#include <Common/CurrentMetrics.h>
#include <Common/Stopwatch.h>
#include <Common/assert_cast.h>
#include <Common/setThreadName.h>
#include <Common/CurrentThread.h>
#include <Common/config.h>
#include <IO/SeekableReadBuffer.h>
@ -40,33 +39,10 @@ ThreadPoolRemoteFSReader::ThreadPoolRemoteFSReader(size_t pool_size, size_t queu
std::future<IAsynchronousReader::Result> ThreadPoolRemoteFSReader::submit(Request request)
{
ThreadGroupStatusPtr running_group;
if (CurrentThread::isInitialized() && CurrentThread::get().getThreadGroup())
running_group = CurrentThread::get().getThreadGroup();
auto schedule = threadPoolCallbackRunner<Result>(pool, "VFSRead");
ContextPtr query_context;
if (CurrentThread::isInitialized())
query_context = CurrentThread::get().getQueryContext();
auto task = std::make_shared<std::packaged_task<Result()>>([request, running_group, query_context]
return schedule([request]() -> Result
{
ThreadStatus thread_status;
SCOPE_EXIT({
if (running_group)
thread_status.detachQuery();
});
/// To be able to pass ProfileEvents.
if (running_group)
thread_status.attachQuery(running_group);
/// Save query context if any, because cache implementation needs it.
if (query_context)
thread_status.attachQueryContext(query_context);
setThreadName("VFSRead");
CurrentMetrics::Increment metric_increment{CurrentMetrics::Read};
auto * remote_fs_fd = assert_cast<RemoteFSFileDescriptor *>(request.descriptor.get());
@ -80,14 +56,7 @@ std::future<IAsynchronousReader::Result> ThreadPoolRemoteFSReader::submit(Reques
ProfileEvents::increment(ProfileEvents::ThreadpoolReaderReadBytes, result.offset ? result.size - result.offset : result.size);
return Result{ .size = result.size, .offset = result.offset };
});
auto future = task->get_future();
/// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority".
pool.scheduleOrThrow([task]{ (*task)(); }, -request.priority);
return future;
}, request.priority);
}
}

View File

@ -3,6 +3,7 @@
#include <IO/AsynchronousReader.h>
#include <IO/ReadBuffer.h>
#include <Common/ThreadPool.h>
#include <Interpreters/threadPoolCallbackRunner.h>
namespace DB
{
@ -14,6 +15,8 @@ public:
std::future<IAsynchronousReader::Result> submit(Request request) override;
void wait() override { pool.wait(); }
private:
ThreadPool pool;
};

View File

@ -11,19 +11,20 @@
namespace DB
{
static constexpr auto DEFAULT_RETRY_NUM = 3;
WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage(
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
const String & blob_path_,
size_t max_single_part_upload_size_,
size_t buf_size_,
const WriteSettings & write_settings_,
std::optional<std::map<std::string, std::string>> attributes_)
const WriteSettings & write_settings_)
: BufferWithOwnMemory<WriteBuffer>(buf_size_, nullptr, 0)
, blob_container_client(blob_container_client_)
, log(&Poco::Logger::get("WriteBufferFromAzureBlobStorage"))
, max_single_part_upload_size(max_single_part_upload_size_)
, blob_path(blob_path_)
, write_settings(write_settings_)
, attributes(attributes_)
, blob_container_client(blob_container_client_)
{
}
@ -33,63 +34,69 @@ WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage()
finalize();
}
void WriteBufferFromAzureBlobStorage::finalizeImpl()
void WriteBufferFromAzureBlobStorage::execWithRetry(std::function<void()> func, size_t num_tries)
{
if (attributes.has_value())
auto handle_exception = [&](const auto & e, size_t i)
{
auto blob_client = blob_container_client->GetBlobClient(blob_path);
Azure::Storage::Metadata metadata;
for (const auto & [key, value] : *attributes)
metadata[key] = value;
blob_client.SetMetadata(metadata);
}
if (i == num_tries - 1)
throw;
const size_t max_tries = 3;
for (size_t i = 0; i < max_tries; ++i)
LOG_DEBUG(log, "Write at attempt {} for blob `{}` failed: {}", i + 1, blob_path, e.Message);
};
for (size_t i = 0; i < num_tries; ++i)
{
try
{
next();
func();
break;
}
catch (const Azure::Core::Http::TransportException & e)
{
handle_exception(e, i);
}
catch (const Azure::Core::RequestFailedException & e)
{
if (i == max_tries - 1)
throw;
LOG_INFO(&Poco::Logger::get("WriteBufferFromAzureBlobStorage"),
"Exception caught during finalizing azure storage write at attempt {}: {}", i + 1, e.Message);
handle_exception(e, i);
}
}
}
void WriteBufferFromAzureBlobStorage::finalizeImpl()
{
execWithRetry([this](){ next(); }, DEFAULT_RETRY_NUM);
}
void WriteBufferFromAzureBlobStorage::nextImpl()
{
if (!offset())
return;
auto * buffer_begin = working_buffer.begin();
auto len = offset();
char * buffer_begin = working_buffer.begin();
size_t total_size = offset();
auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path);
size_t read = 0;
size_t current_size = 0;
std::vector<std::string> block_ids;
while (read < len)
while (current_size < total_size)
{
auto part_len = std::min(len - read, max_single_part_upload_size);
size_t part_len = std::min(total_size - current_size, max_single_part_upload_size);
const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64));
auto block_id = getRandomASCIIString(64);
block_ids.push_back(block_id);
Azure::Core::IO::MemoryBodyStream tmp_buffer(reinterpret_cast<uint8_t *>(buffer_begin + current_size), part_len);
execWithRetry([&](){ block_blob_client.StageBlock(block_id, tmp_buffer); }, DEFAULT_RETRY_NUM);
Azure::Core::IO::MemoryBodyStream tmp_buffer(reinterpret_cast<uint8_t *>(buffer_begin + read), part_len);
block_blob_client.StageBlock(block_id, tmp_buffer);
read += part_len;
current_size += part_len;
LOG_TRACE(log, "Staged block (id: {}) of size {} (written {}/{}, blob path: {}).", block_id, part_len, current_size, total_size, blob_path);
}
block_blob_client.CommitBlockList(block_ids);
execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, DEFAULT_RETRY_NUM);
LOG_TRACE(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path);
if (write_settings.remote_throttler)
write_settings.remote_throttler->add(read);
write_settings.remote_throttler->add(total_size);
}
}

View File

@ -13,20 +13,25 @@
#include <azure/core/io/body_stream.hpp>
namespace Poco
{
class Logger;
}
namespace DB
{
class WriteBufferFromAzureBlobStorage : public BufferWithOwnMemory<WriteBuffer>
{
public:
using AzureClientPtr = std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient>;
WriteBufferFromAzureBlobStorage(
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
AzureClientPtr blob_container_client_,
const String & blob_path_,
size_t max_single_part_upload_size_,
size_t buf_size_,
const WriteSettings & write_settings_,
std::optional<std::map<std::string, std::string>> attributes_ = {});
const WriteSettings & write_settings_);
~WriteBufferFromAzureBlobStorage() override;
@ -34,12 +39,15 @@ public:
private:
void finalizeImpl() override;
void execWithRetry(std::function<void()> func, size_t num_tries);
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
size_t max_single_part_upload_size;
const String blob_path;
WriteSettings write_settings;
std::optional<std::map<std::string, std::string>> attributes;
Poco::Logger * log;
const size_t max_single_part_upload_size;
const std::string blob_path;
const WriteSettings write_settings;
AzureClientPtr blob_container_client;
};
}

View File

@ -77,13 +77,21 @@ std::unique_ptr<ReadBufferFromFileBase> createReadBufferFromFileBase(
}
else if (settings.local_fs_method == LocalFSReadMethod::pread_fake_async)
{
static AsynchronousReaderPtr reader = std::make_shared<SynchronousReader>();
auto context = Context::getGlobalContextInstance();
if (!context)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context not initialized");
auto & reader = context->getThreadPoolReader(Context::FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER);
res = std::make_unique<AsynchronousReadBufferFromFileWithDescriptorsCache>(
reader, settings.priority, filename, buffer_size, actual_flags, existing_memory, alignment, file_size);
}
else if (settings.local_fs_method == LocalFSReadMethod::pread_threadpool)
{
static AsynchronousReaderPtr reader = std::make_shared<ThreadPoolReader>(16, 1000000);
auto context = Context::getGlobalContextInstance();
if (!context)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context not initialized");
auto & reader = context->getThreadPoolReader(Context::FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER);
res = std::make_unique<AsynchronousReadBufferFromFileWithDescriptorsCache>(
reader, settings.priority, filename, buffer_size, actual_flags, existing_memory, alignment, file_size);
}

View File

@ -107,7 +107,7 @@ std::unique_ptr<ReadBufferFromFileBase> AzureObjectStorage::readObjects( /// NOL
if (disk_read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
{
auto reader = getThreadPoolReader();
auto & reader = getThreadPoolReader();
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, disk_read_settings, std::move(reader_impl));
}
else

View File

@ -11,22 +11,25 @@ namespace DB
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
}
AsynchronousReaderPtr IObjectStorage::getThreadPoolReader()
IAsynchronousReader & IObjectStorage::getThreadPoolReader()
{
constexpr size_t pool_size = 50;
constexpr size_t queue_size = 1000000;
static AsynchronousReaderPtr reader = std::make_shared<ThreadPoolRemoteFSReader>(pool_size, queue_size);
return reader;
auto context = Context::getGlobalContextInstance();
if (!context)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context not initialized");
return context->getThreadPoolReader(Context::FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER);
}
ThreadPool & IObjectStorage::getThreadPoolWriter()
{
constexpr size_t pool_size = 100;
constexpr size_t queue_size = 1000000;
static ThreadPool writer(pool_size, pool_size, queue_size);
return writer;
auto context = Context::getGlobalContextInstance();
if (!context)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context not initialized");
return context->getThreadPoolWriter();
}
void IObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT

View File

@ -130,7 +130,7 @@ public:
/// Path to directory with objects cache
virtual const std::string & getCacheBasePath() const;
static AsynchronousReaderPtr getThreadPoolReader();
static IAsynchronousReader & getThreadPoolReader();
static ThreadPool & getThreadPoolWriter();

View File

@ -190,7 +190,7 @@ std::unique_ptr<ReadBufferFromFileBase> S3ObjectStorage::readObjects( /// NOLINT
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
{
auto reader = getThreadPoolReader();
auto & reader = getThreadPoolReader();
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, disk_read_settings, std::move(s3_impl));
}
else
@ -230,6 +230,8 @@ std::unique_ptr<WriteBufferFromFileBase> S3ObjectStorage::writeObject( /// NOLIN
throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 doesn't support append to files");
auto settings_ptr = s3_settings.get();
auto scheduler = threadPoolCallbackRunner<void>(getThreadPoolWriter(), "VFSWrite");
auto s3_buffer = std::make_unique<WriteBufferFromS3>(
client.get(),
bucket,
@ -237,7 +239,7 @@ std::unique_ptr<WriteBufferFromFileBase> S3ObjectStorage::writeObject( /// NOLIN
settings_ptr->s3_settings,
attributes,
buf_size,
threadPoolCallbackRunner(getThreadPoolWriter()),
std::move(scheduler),
disk_write_settings);
return std::make_unique<WriteIndirectBufferFromRemoteFS>(

View File

@ -168,7 +168,7 @@ std::unique_ptr<ReadBufferFromFileBase> WebObjectStorage::readObject( /// NOLINT
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
{
auto reader = IObjectStorage::getThreadPoolReader();
auto & reader = IObjectStorage::getThreadPoolReader();
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, read_settings, std::move(web_impl), min_bytes_for_seek);
}
else

View File

@ -24,7 +24,7 @@ namespace ErrorCodes
AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile(
AsynchronousReaderPtr reader_,
IAsynchronousReader & reader_,
Int32 priority_,
const std::string & file_name_,
size_t buf_size,
@ -32,7 +32,7 @@ AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile(
char * existing_memory,
size_t alignment,
std::optional<size_t> file_size_)
: AsynchronousReadBufferFromFileDescriptor(std::move(reader_), priority_, -1, buf_size, existing_memory, alignment, file_size_)
: AsynchronousReadBufferFromFileDescriptor(reader_, priority_, -1, buf_size, existing_memory, alignment, file_size_)
, file_name(file_name_)
{
ProfileEvents::increment(ProfileEvents::FileOpen);
@ -58,7 +58,7 @@ AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile(
AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile(
AsynchronousReaderPtr reader_,
IAsynchronousReader & reader_,
Int32 priority_,
int & fd_,
const std::string & original_file_name,
@ -66,7 +66,7 @@ AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile(
char * existing_memory,
size_t alignment,
std::optional<size_t> file_size_)
: AsynchronousReadBufferFromFileDescriptor(std::move(reader_), priority_, fd_, buf_size, existing_memory, alignment, file_size_)
: AsynchronousReadBufferFromFileDescriptor(reader_, priority_, fd_, buf_size, existing_memory, alignment, file_size_)
, file_name(original_file_name.empty() ? "(fd = " + toString(fd_) + ")" : original_file_name)
{
fd_ = -1;
@ -105,4 +105,3 @@ AsynchronousReadBufferFromFileWithDescriptorsCache::~AsynchronousReadBufferFromF
}

View File

@ -14,7 +14,7 @@ protected:
public:
explicit AsynchronousReadBufferFromFile(
AsynchronousReaderPtr reader_,
IAsynchronousReader & reader_,
Int32 priority_,
const std::string & file_name_,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
@ -25,7 +25,7 @@ public:
/// Use pre-opened file descriptor.
explicit AsynchronousReadBufferFromFile(
AsynchronousReaderPtr reader_,
IAsynchronousReader & reader_,
Int32 priority_,
int & fd, /// Will be set to -1 if constructor didn't throw and ownership of file descriptor is passed to the object.
const std::string & original_file_name = {},
@ -45,7 +45,6 @@ public:
}
};
/** Similar to AsynchronousReadBufferFromFile but also transparently shares open file descriptors.
*/
class AsynchronousReadBufferFromFileWithDescriptorsCache : public AsynchronousReadBufferFromFileDescriptor
@ -56,7 +55,7 @@ private:
public:
AsynchronousReadBufferFromFileWithDescriptorsCache(
AsynchronousReaderPtr reader_,
IAsynchronousReader & reader_,
Int32 priority_,
const std::string & file_name_,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
@ -64,7 +63,7 @@ public:
char * existing_memory = nullptr,
size_t alignment = 0,
std::optional<size_t> file_size_ = std::nullopt)
: AsynchronousReadBufferFromFileDescriptor(std::move(reader_), priority_, -1, buf_size, existing_memory, alignment, file_size_)
: AsynchronousReadBufferFromFileDescriptor(reader_, priority_, -1, buf_size, existing_memory, alignment, file_size_)
, file_name(file_name_)
{
file = OpenedFileCache::instance().get(file_name, flags);
@ -80,4 +79,3 @@ public:
};
}

View File

@ -54,7 +54,7 @@ std::future<IAsynchronousReader::Result> AsynchronousReadBufferFromFileDescripto
return std::async(std::launch::deferred, [] { return IAsynchronousReader::Result{.size = 0, .offset = 0}; });
}
return reader->submit(request);
return reader.submit(request);
}
@ -140,7 +140,7 @@ void AsynchronousReadBufferFromFileDescriptor::finalize()
AsynchronousReadBufferFromFileDescriptor::AsynchronousReadBufferFromFileDescriptor(
AsynchronousReaderPtr reader_,
IAsynchronousReader & reader_,
Int32 priority_,
int fd_,
size_t buf_size,
@ -148,7 +148,7 @@ AsynchronousReadBufferFromFileDescriptor::AsynchronousReadBufferFromFileDescript
size_t alignment,
std::optional<size_t> file_size_)
: ReadBufferFromFileBase(buf_size, existing_memory, alignment, file_size_)
, reader(std::move(reader_))
, reader(reader_)
, priority(priority_)
, required_alignment(alignment)
, fd(fd_)

View File

@ -16,7 +16,7 @@ namespace DB
class AsynchronousReadBufferFromFileDescriptor : public ReadBufferFromFileBase
{
protected:
AsynchronousReaderPtr reader;
IAsynchronousReader & reader;
Int32 priority;
Memory<> prefetch_buffer;
@ -36,7 +36,7 @@ protected:
public:
AsynchronousReadBufferFromFileDescriptor(
AsynchronousReaderPtr reader_,
IAsynchronousReader & reader_,
Int32 priority_,
int fd_,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,

View File

@ -4,6 +4,7 @@
#include <optional>
#include <memory>
#include <future>
#include <boost/noncopyable.hpp>
namespace DB
@ -18,7 +19,7 @@ namespace DB
* For example, this interface may not suffice if you want to serve 10 000 000 of 4 KiB requests per second.
* This interface is fairly limited.
*/
class IAsynchronousReader
class IAsynchronousReader : private boost::noncopyable
{
public:
/// For local filesystems, the file descriptor is simply integer
@ -68,6 +69,8 @@ public:
/// The method can be called concurrently from multiple threads.
virtual std::future<Result> submit(Request request) = 0;
virtual void wait() = 0;
/// Destructor must wait for all not completed request and ignore the results.
/// It may also cancel the requests.
virtual ~IAsynchronousReader() = default;

View File

@ -43,7 +43,7 @@ struct ParallelReadBuffer::ReadWorker
};
ParallelReadBuffer::ParallelReadBuffer(
std::unique_ptr<ReadBufferFactory> reader_factory_, CallbackRunner schedule_, size_t max_working_readers_)
std::unique_ptr<ReadBufferFactory> reader_factory_, ThreadPoolCallbackRunner<void> schedule_, size_t max_working_readers_)
: SeekableReadBuffer(nullptr, 0)
, max_working_readers(max_working_readers_)
, schedule(std::move(schedule_))
@ -71,7 +71,7 @@ bool ParallelReadBuffer::addReaderToPool()
auto worker = read_workers.emplace_back(std::make_shared<ReadWorker>(std::move(reader)));
++active_working_reader;
schedule([this, worker = std::move(worker)]() mutable { readerThreadFunction(std::move(worker)); });
schedule([this, worker = std::move(worker)]() mutable { readerThreadFunction(std::move(worker)); }, 0);
return true;
}

View File

@ -33,12 +33,13 @@ public:
class ReadBufferFactory : public WithFileSize
{
public:
~ReadBufferFactory() override = default;
virtual SeekableReadBufferPtr getReader() = 0;
virtual ~ReadBufferFactory() override = default;
virtual off_t seek(off_t off, int whence) = 0;
};
explicit ParallelReadBuffer(std::unique_ptr<ReadBufferFactory> reader_factory_, CallbackRunner schedule_, size_t max_working_readers);
ParallelReadBuffer(std::unique_ptr<ReadBufferFactory> reader_factory_, ThreadPoolCallbackRunner<void> schedule_, size_t max_working_readers);
~ParallelReadBuffer() override { finishAndWait(); }
@ -75,7 +76,7 @@ private:
size_t max_working_readers;
std::atomic_size_t active_working_reader{0};
CallbackRunner schedule;
ThreadPoolCallbackRunner<void> schedule;
std::unique_ptr<ReadBufferFactory> reader_factory;

View File

@ -13,7 +13,8 @@ class SynchronousReader final : public IAsynchronousReader
{
public:
std::future<Result> submit(Request request) override;
void wait() override {}
};
}

View File

@ -74,7 +74,7 @@ WriteBufferFromS3::WriteBufferFromS3(
const S3Settings::ReadWriteSettings & s3_settings_,
std::optional<std::map<String, String>> object_metadata_,
size_t buffer_size_,
ScheduleFunc schedule_,
ThreadPoolCallbackRunner<void> schedule_,
const WriteSettings & write_settings_)
: BufferWithOwnMemory<WriteBuffer>(buffer_size_, nullptr, 0)
, bucket(bucket_)
@ -292,7 +292,7 @@ void WriteBufferFromS3::writePart()
}
task_finish_notify();
});
}, 0);
}
catch (...)
{
@ -442,7 +442,7 @@ void WriteBufferFromS3::makeSinglepartUpload()
}
task_notify_finish();
});
}, 0);
}
catch (...)
{

View File

@ -15,6 +15,7 @@
#include <IO/WriteBuffer.h>
#include <IO/WriteSettings.h>
#include <Storages/StorageS3Settings.h>
#include <Interpreters/threadPoolCallbackRunner.h>
#include <aws/core/utils/memory/stl/AWSStringStream.h>
@ -33,7 +34,6 @@ namespace Aws::S3::Model
namespace DB
{
using ScheduleFunc = std::function<void(std::function<void()>)>;
class WriteBufferFromFile;
/**
@ -53,7 +53,7 @@ public:
const S3Settings::ReadWriteSettings & s3_settings_,
std::optional<std::map<String, String>> object_metadata_ = std::nullopt,
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
ScheduleFunc schedule_ = {},
ThreadPoolCallbackRunner<void> schedule_ = {},
const WriteSettings & write_settings_ = {});
~WriteBufferFromS3() override;
@ -106,7 +106,7 @@ private:
/// Following fields are for background uploads in thread pool (if specified).
/// We use std::function to avoid dependency of Interpreters
const ScheduleFunc schedule;
const ThreadPoolCallbackRunner<void> schedule;
std::unique_ptr<PutObjectTask> put_object_task; /// Does not need protection by mutex because of the logic around is_finished field.
std::list<UploadPartTask> TSA_GUARDED_BY(bg_tasks_mutex) upload_object_tasks;

View File

@ -31,6 +31,9 @@
#include <Storages/StorageS3Settings.h>
#include <Disks/DiskLocal.h>
#include <Disks/ObjectStorages/IObjectStorage.h>
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
#include <Disks/IO/ThreadPoolReader.h>
#include <IO/SynchronousReader.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Interpreters/ActionLocksManager.h>
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
@ -228,6 +231,12 @@ struct ContextSharedPart : boost::noncopyable
mutable std::unique_ptr<BackgroundSchedulePool> distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends)
mutable std::unique_ptr<BackgroundSchedulePool> message_broker_schedule_pool; /// A thread pool that can run different jobs in background (used for message brokers, like RabbitMQ and Kafka)
mutable std::unique_ptr<IAsynchronousReader> asynchronous_remote_fs_reader;
mutable std::unique_ptr<IAsynchronousReader> asynchronous_local_fs_reader;
mutable std::unique_ptr<IAsynchronousReader> synchronous_local_fs_reader;
mutable std::unique_ptr<ThreadPool> threadpool_writer;
mutable ThrottlerPtr replicated_fetches_throttler; /// A server-wide throttler for replicated fetches
mutable ThrottlerPtr replicated_sends_throttler; /// A server-wide throttler for replicated sends
mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads
@ -311,23 +320,76 @@ struct ContextSharedPart : boost::noncopyable
~ContextSharedPart()
{
try
/// Wait for thread pool for background reads and writes,
/// since it may use per-user MemoryTracker which will be destroyed here.
if (asynchronous_remote_fs_reader)
{
/// Wait for thread pool for background writes,
/// since it may use per-user MemoryTracker which will be destroyed here.
IObjectStorage::getThreadPoolWriter().wait();
/// Make sure that threadpool is destructed before this->process_list
/// because thread_status, which was created for threads inside threadpool,
/// relies on it.
if (load_marks_threadpool)
try
{
LOG_DEBUG(log, "Desctructing remote fs threadpool reader");
asynchronous_remote_fs_reader->wait();
asynchronous_remote_fs_reader.reset();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
if (asynchronous_local_fs_reader)
{
try
{
LOG_DEBUG(log, "Desctructing local fs threadpool reader");
asynchronous_local_fs_reader->wait();
asynchronous_local_fs_reader.reset();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
if (synchronous_local_fs_reader)
{
try
{
LOG_DEBUG(log, "Desctructing local fs threadpool reader");
synchronous_local_fs_reader->wait();
synchronous_local_fs_reader.reset();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
if (threadpool_writer)
{
try
{
LOG_DEBUG(log, "Desctructing threadpool writer");
threadpool_writer->wait();
threadpool_writer.reset();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
if (load_marks_threadpool)
{
try
{
LOG_DEBUG(log, "Desctructing marks loader");
load_marks_threadpool->wait();
load_marks_threadpool.reset();
}
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
try
@ -3378,6 +3440,66 @@ OrdinaryBackgroundExecutorPtr Context::getCommonExecutor() const
return shared->common_executor;
}
IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const
{
const auto & config = getConfigRef();
auto lock = getLock();
switch (type)
{
case FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER:
{
if (!shared->asynchronous_remote_fs_reader)
{
auto pool_size = config.getUInt(".threadpool_remote_fs_reader_pool_size", 100);
auto queue_size = config.getUInt(".threadpool_remote_fs_reader_queue_size", 1000000);
shared->asynchronous_remote_fs_reader = std::make_unique<ThreadPoolRemoteFSReader>(pool_size, queue_size);
}
return *shared->asynchronous_remote_fs_reader;
}
case FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER:
{
if (!shared->asynchronous_local_fs_reader)
{
auto pool_size = config.getUInt(".threadpool_local_fs_reader_pool_size", 100);
auto queue_size = config.getUInt(".threadpool_local_fs_reader_queue_size", 1000000);
shared->asynchronous_local_fs_reader = std::make_unique<ThreadPoolReader>(pool_size, queue_size);
}
return *shared->asynchronous_local_fs_reader;
}
case FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER:
{
if (!shared->synchronous_local_fs_reader)
{
shared->synchronous_local_fs_reader = std::make_unique<SynchronousReader>();
}
return *shared->synchronous_local_fs_reader;
}
}
}
ThreadPool & Context::getThreadPoolWriter() const
{
const auto & config = getConfigRef();
auto lock = getLock();
if (!shared->threadpool_writer)
{
auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100);
auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000);
shared->threadpool_writer = std::make_unique<ThreadPool>(pool_size, pool_size, queue_size);
}
return *shared->threadpool_writer;
}
ReadSettings Context::getReadSettings() const
{

View File

@ -1011,6 +1011,17 @@ public:
OrdinaryBackgroundExecutorPtr getFetchesExecutor() const;
OrdinaryBackgroundExecutorPtr getCommonExecutor() const;
enum class FilesystemReaderType
{
SYNCHRONOUS_LOCAL_FS_READER,
ASYNCHRONOUS_LOCAL_FS_READER,
ASYNCHRONOUS_REMOTE_FS_READER,
};
IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const;
ThreadPool & getThreadPoolWriter() const;
/** Get settings for reading from filesystem. */
ReadSettings getReadSettings() const;

View File

@ -50,21 +50,26 @@ bool HostID::isLocalAddress(UInt16 clickhouse_port) const
void DDLLogEntry::assertVersion() const
{
constexpr UInt64 max_version = 2;
if (version == 0 || max_version < version)
if (version == 0
/// NORMALIZE_CREATE_ON_INITIATOR_VERSION does not change the entry format, it uses versioin 2, so there shouldn't be such version
|| version == NORMALIZE_CREATE_ON_INITIATOR_VERSION
|| version > DDL_ENTRY_FORMAT_MAX_VERSION)
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown DDLLogEntry format version: {}."
"Maximum supported version is {}", version, max_version);
"Maximum supported version is {}", version, DDL_ENTRY_FORMAT_MAX_VERSION);
}
void DDLLogEntry::setSettingsIfRequired(ContextPtr context)
{
version = context->getSettingsRef().distributed_ddl_entry_format_version;
if (version <= 0 || version > DDL_ENTRY_FORMAT_MAX_VERSION)
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown distributed_ddl_entry_format_version: {}."
"Maximum supported version is {}.", version, DDL_ENTRY_FORMAT_MAX_VERSION);
/// NORMALIZE_CREATE_ON_INITIATOR_VERSION does not affect entry format in ZooKeeper
if (version == NORMALIZE_CREATE_ON_INITIATOR_VERSION)
version = SETTINGS_IN_ZK_VERSION;
if (version == SETTINGS_IN_ZK_VERSION)
if (version >= SETTINGS_IN_ZK_VERSION)
settings.emplace(context->getSettingsRef().changes());
}
@ -94,6 +99,9 @@ String DDLLogEntry::toString() const
wb << "settings: " << serializeAST(ast) << "\n";
}
if (version >= OPENTELEMETRY_ENABLED_VERSION)
wb << "tracing: " << this->tracing_context;
return wb.str();
}
@ -106,7 +114,7 @@ void DDLLogEntry::parse(const String & data)
Strings host_id_strings;
rb >> "query: " >> escape >> query >> "\n";
if (version == 1)
if (version == OLDEST_VERSION)
{
rb >> "hosts: " >> host_id_strings >> "\n";
@ -115,9 +123,8 @@ void DDLLogEntry::parse(const String & data)
else
initiator.clear();
}
else if (version == 2)
else if (version >= SETTINGS_IN_ZK_VERSION)
{
if (!rb.eof() && *rb.position() == 'h')
rb >> "hosts: " >> host_id_strings >> "\n";
if (!rb.eof() && *rb.position() == 'i')
@ -134,6 +141,12 @@ void DDLLogEntry::parse(const String & data)
}
}
if (version >= OPENTELEMETRY_ENABLED_VERSION)
{
if (!rb.eof() && *rb.position() == 't')
rb >> "tracing: " >> this->tracing_context;
}
assertEOF(rb);
if (!host_id_strings.empty())

View File

@ -2,6 +2,7 @@
#include <Core/Types.h>
#include <Interpreters/Cluster.h>
#include <Common/OpenTelemetryTraceContext.h>
#include <Common/ZooKeeper/Types.h>
#include <filesystem>
@ -69,12 +70,18 @@ struct DDLLogEntry
static constexpr const UInt64 OLDEST_VERSION = 1;
static constexpr const UInt64 SETTINGS_IN_ZK_VERSION = 2;
static constexpr const UInt64 NORMALIZE_CREATE_ON_INITIATOR_VERSION = 3;
static constexpr const UInt64 OPENTELEMETRY_ENABLED_VERSION = 4;
/// Add new version here
/// Remember to update the value below once new version is added
static constexpr const UInt64 DDL_ENTRY_FORMAT_MAX_VERSION = 4;
UInt64 version = 1;
String query;
std::vector<HostID> hosts;
String initiator; // optional
std::optional<SettingsChanges> settings;
OpenTelemetry::TracingContext tracing_context;
void setSettingsIfRequired(ContextPtr context);
String toString() const;

View File

@ -19,6 +19,7 @@
#include <Interpreters/executeQuery.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/Context.h>
#include <Common/OpenTelemetryTraceContext.h>
#include <Common/setThreadName.h>
#include <Common/randomSeed.h>
#include <Common/ZooKeeper/ZooKeeper.h>
@ -515,6 +516,11 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper)
LOG_DEBUG(log, "Processing task {} ({})", task.entry_name, task.entry.query);
chassert(!task.completely_processed);
/// Setup tracing context on current thread for current DDL
OpenTelemetry::TracingContextHolder tracing_ctx_holder(__PRETTY_FUNCTION__ ,
task.entry.tracing_context,
this->context->getOpenTelemetrySpanLog());
String active_node_path = task.getActiveNodePath();
String finished_node_path = task.getFinishedNodePath();

View File

@ -766,6 +766,16 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat
throw Exception("Column " + backQuoteIfNeed(column.name) + " already exists", ErrorCodes::DUPLICATE_COLUMN);
}
/// Check if _row_exists for lightweight delete column in column_lists for merge tree family.
if (create.storage && create.storage->engine && endsWith(create.storage->engine->name, "MergeTree"))
{
auto search = all_columns.find(LightweightDeleteDescription::FILTER_COLUMN.name);
if (search != all_columns.end())
throw Exception("Cannot create table with column '" + LightweightDeleteDescription::FILTER_COLUMN.name + "' "
"for *MergeTree engines because it is reserved for lightweight delete feature",
ErrorCodes::ILLEGAL_COLUMN);
}
const auto & settings = getContext()->getSettingsRef();
/// Check low cardinality types in creating table if it was not allowed in setting

View File

@ -55,6 +55,8 @@ bool isSupportedAlterType(int type)
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, const DDLQueryOnClusterParams & params)
{
OpenTelemetry::SpanHolder span(__FUNCTION__);
if (context->getCurrentTransaction() && context->getSettingsRef().throw_on_unsupported_query_inside_transaction)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ON CLUSTER queries inside transactions are not supported");
@ -88,6 +90,8 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context,
cluster = context->getCluster(query->cluster);
}
span.addAttribute("clickhouse.cluster", query->cluster);
/// TODO: support per-cluster grant
context->checkAccess(AccessType::CLUSTER);
@ -164,6 +168,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context,
entry.query = queryToString(query_ptr);
entry.initiator = ddl_worker.getCommonHostID();
entry.setSettingsIfRequired(context);
entry.tracing_context = OpenTelemetry::CurrentContext();
String node_path = ddl_worker.enqueueQuery(entry);
return getDistributedDDLStatus(node_path, entry, context);

View File

@ -1,40 +1,44 @@
#include "threadPoolCallbackRunner.h"
#include <Common/scope_guard_safe.h>
#include <Common/CurrentThread.h>
#include <Common/setThreadName.h>
#include <IO/AsynchronousReader.h>
#include <future>
namespace DB
{
CallbackRunner threadPoolCallbackRunner(ThreadPool & pool)
template <typename Result> ThreadPoolCallbackRunner<Result> threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name)
{
return [pool = &pool, thread_group = CurrentThread::getGroup()](auto callback) mutable
return [pool = &pool, thread_group = CurrentThread::getGroup(), thread_name](std::function<Result()> && callback, size_t priority) mutable -> std::future<Result>
{
pool->scheduleOrThrow(
[&, callback = std::move(callback), thread_group]()
{
auto task = std::make_shared<std::packaged_task<Result()>>([thread_group, thread_name, callback = std::move(callback)]() -> Result
{
if (thread_group)
CurrentThread::attachTo(thread_group);
SCOPE_EXIT_SAFE({
if (thread_group)
CurrentThread::attachTo(thread_group);
SCOPE_EXIT_SAFE({
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
/// After we detached from the thread_group, parent for memory_tracker inside ThreadStatus will be reset to it's parent.
/// Typically, it may be changes from Process to User.
/// Usually it could be ok, because thread pool task is executed before user-level memory tracker is destroyed.
/// However, thread could stay alive inside the thread pool, and it's ThreadStatus as well.
/// When, finally, we destroy the thread (and the ThreadStatus),
/// it can use memory tracker in the ~ThreadStatus in order to alloc/free untracked_memory,
/// and by this time user-level memory tracker may be already destroyed.
///
/// As a work-around, reset memory tracker to total, which is always alive.
CurrentThread::get().memory_tracker.setParent(&total_memory_tracker);
});
callback();
CurrentThread::detachQueryIfNotDetached();
});
setThreadName(thread_name.data());
return callback();
});
auto future = task->get_future();
/// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority".
pool->scheduleOrThrow([task]{ (*task)(); }, -priority);
return future;
};
}
template ThreadPoolCallbackRunner<void> threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name);
template ThreadPoolCallbackRunner<IAsynchronousReader::Result> threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name);
}

View File

@ -1,15 +1,18 @@
#pragma once
#include <Common/ThreadPool.h>
#include <future>
namespace DB
{
/// High-order function to run callbacks (functions with 'void()' signature) somewhere asynchronously
using CallbackRunner = std::function<void(std::function<void()>)>;
/// High-order function to run callbacks (functions with 'void()' signature) somewhere asynchronously.
template <typename Result>
using ThreadPoolCallbackRunner = std::function<std::future<Result>(std::function<Result()> &&, size_t priority)>;
/// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()'
CallbackRunner threadPoolCallbackRunner(ThreadPool & pool);
/// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()'.
template <typename Result>
ThreadPoolCallbackRunner<Result> threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name);
}

File diff suppressed because it is too large Load Diff

View File

@ -9,26 +9,6 @@ namespace DB
{
class ParserArray : public IParserBase
{
protected:
const char * getName() const override { return "array"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** If in parenthesis an expression from one element - returns this element in `node`;
* or if there is a SELECT subquery in parenthesis, then this subquery returned in `node`;
* otherwise returns `tuple` function from the contents of brackets.
*/
class ParserParenthesisExpression : public IParserBase
{
protected:
const char * getName() const override { return "parenthesized expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** The SELECT subquery is in parenthesis.
*/
class ParserSubquery : public IParserBase
@ -141,36 +121,6 @@ protected:
ColumnTransformers allowed_transformers;
};
/** A function, for example, f(x, y + 1, g(z)).
* Or an aggregate function: sum(x + f(y)), corr(x, y). The syntax is the same as the usual function.
* Or a parametric aggregate function: quantile(0.9)(x + y).
* Syntax - two pairs of parentheses instead of one. The first is for parameters, the second for arguments.
* For functions, the DISTINCT modifier can be specified, for example, count(DISTINCT x, y).
*/
class ParserFunction : public IParserBase
{
public:
explicit ParserFunction(bool allow_function_parameters_ = true, bool is_table_function_ = false)
: allow_function_parameters(allow_function_parameters_), is_table_function(is_table_function_)
{
}
protected:
const char * getName() const override { return "function"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
bool allow_function_parameters;
bool is_table_function;
};
// A special function parser for view and viewIfPermitted table functions.
// It parses an SELECT query as its argument and doesn't support getColumnName().
class ParserTableFunctionView : public IParserBase
{
protected:
const char * getName() const override { return "function"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
// Allows to make queries like SELECT SUM(<expr>) FILTER(WHERE <cond>) FROM ...
class ParserFilterClause : public IParserBase
{
@ -394,16 +344,6 @@ protected:
};
/** The expression element is one of: an expression in parentheses, an array, a literal, a function, an identifier, an asterisk.
*/
class ParserExpressionElement : public IParserBase
{
protected:
const char * getName() const override { return "element of expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** An expression element, possibly with an alias, if appropriate.
*/
class ParserWithOptionalAlias : public IParserBase

File diff suppressed because it is too large Load Diff

View File

@ -116,6 +116,36 @@ private:
SelectUnionModes union_modes;
};
class ParserArray : public IParserBase
{
protected:
const char * getName() const override { return "array"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** A function, for example, f(x, y + 1, g(z)).
* Or an aggregate function: sum(x + f(y)), corr(x, y). The syntax is the same as the usual function.
* Or a parametric aggregate function: quantile(0.9)(x + y).
* Syntax - two pairs of parentheses instead of one. The first is for parameters, the second for arguments.
* For functions, the DISTINCT modifier can be specified, for example, count(DISTINCT x, y).
*/
class ParserFunction : public IParserBase
{
public:
explicit ParserFunction(bool allow_function_parameters_ = true, bool is_table_function_ = false)
: allow_function_parameters(allow_function_parameters_), is_table_function(is_table_function_)
{
}
protected:
const char * getName() const override { return "function"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
bool allow_function_parameters;
bool is_table_function;
};
/** An expression with an infix binary left-associative operator.
* For example, a + b - c + d.
*/
@ -123,31 +153,13 @@ class ParserLeftAssociativeBinaryOperatorList : public IParserBase
{
private:
Operators_t operators;
Operators_t overlapping_operators_to_skip = { (const char *[]){ nullptr } };
ParserPtr first_elem_parser;
ParserPtr remaining_elem_parser;
/// =, !=, <, > ALL (subquery) / ANY (subquery)
bool comparison_expression = false;
ParserPtr elem_parser;
public:
/** `operators_` - allowed operators and their corresponding functions
*/
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, ParserPtr && first_elem_parser_)
: operators(operators_), first_elem_parser(std::move(first_elem_parser_))
{
}
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_,
Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_, bool comparison_expression_ = false)
: operators(operators_), overlapping_operators_to_skip(overlapping_operators_to_skip_),
first_elem_parser(std::move(first_elem_parser_)), comparison_expression(comparison_expression_)
{
}
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, ParserPtr && first_elem_parser_,
ParserPtr && remaining_elem_parser_)
: operators(operators_), first_elem_parser(std::move(first_elem_parser_)),
remaining_elem_parser(std::move(remaining_elem_parser_))
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, ParserPtr && elem_parser_)
: operators(operators_), elem_parser(std::move(elem_parser_))
{
}
@ -158,295 +170,8 @@ protected:
};
/** Expression with an infix operator of arbitrary arity.
* For example, a AND b AND c AND d.
*/
class ParserVariableArityOperatorList : public IParserBase
class ParserExpression : public IParserBase
{
private:
const char * infix;
const char * function_name;
ParserPtr elem_parser;
public:
ParserVariableArityOperatorList(const char * infix_, const char * function_, ParserPtr && elem_parser_)
: infix(infix_), function_name(function_), elem_parser(std::move(elem_parser_))
{
}
protected:
const char * getName() const override { return "list, delimited by operator of variable arity"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** An expression with a prefix unary operator.
* Example, NOT x.
*/
class ParserPrefixUnaryOperatorExpression : public IParserBase
{
private:
Operators_t operators;
ParserPtr elem_parser;
public:
/** `operators_` - allowed operators and their corresponding functions
*/
ParserPrefixUnaryOperatorExpression(Operators_t operators_, ParserPtr && elem_parser_)
: operators(operators_), elem_parser(std::move(elem_parser_))
{
}
protected:
const char * getName() const override { return "expression with prefix unary operator"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// CAST operator "::". This parser is used if left argument
/// of operator cannot be read as simple literal from text of query.
/// Example: "[1, 1 + 1, 1 + 2]::Array(UInt8)"
class ParserCastExpression : public IParserBase
{
private:
ParserPtr elem_parser;
public:
explicit ParserCastExpression(ParserPtr && elem_parser_)
: elem_parser(std::move(elem_parser_))
{
}
protected:
const char * getName() const override { return "CAST expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserArrayElementExpression : public IParserBase
{
private:
static const char * operators[];
protected:
const char * getName() const override{ return "array element expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserTupleElementExpression : public IParserBase
{
private:
static const char * operators[];
protected:
const char * getName() const override { return "tuple element expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserUnaryExpression : public IParserBase
{
private:
static const char * operators[];
ParserPrefixUnaryOperatorExpression operator_parser {operators, std::make_unique<ParserCastExpression>(std::make_unique<ParserTupleElementExpression>())};
protected:
const char * getName() const override { return "unary expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserMultiplicativeExpression : public IParserBase
{
private:
static const char * operators[];
ParserLeftAssociativeBinaryOperatorList operator_parser {operators, std::make_unique<ParserUnaryExpression>()};
protected:
const char * getName() const override { return "multiplicative expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
/// DATE operator. "DATE '2001-01-01'" would be parsed as "toDate('2001-01-01')".
class ParserDateOperatorExpression : public IParserBase
{
protected:
ParserMultiplicativeExpression next_parser;
const char * getName() const override { return "DATE operator expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// TIMESTAMP operator. "TIMESTAMP '2001-01-01 12:34:56'" would be parsed as "toDateTime('2001-01-01 12:34:56')".
class ParserTimestampOperatorExpression : public IParserBase
{
protected:
ParserDateOperatorExpression next_parser;
const char * getName() const override { return "TIMESTAMP operator expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// Optional conversion to INTERVAL data type. Example: "INTERVAL x SECOND" parsed as "toIntervalSecond(x)".
class ParserIntervalOperatorExpression : public IParserBase
{
protected:
ParserTimestampOperatorExpression next_parser;
const char * getName() const override { return "INTERVAL operator expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
private:
static bool parseArgumentAndIntervalKind(Pos & pos, ASTPtr & expr, IntervalKind & interval_kind, Expected & expected);
};
class ParserAdditiveExpression : public IParserBase
{
private:
static const char * operators[];
ParserLeftAssociativeBinaryOperatorList operator_parser {operators, std::make_unique<ParserIntervalOperatorExpression>()};
protected:
const char * getName() const override { return "additive expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
class ParserConcatExpression : public IParserBase
{
ParserVariableArityOperatorList operator_parser {"||", "concat", std::make_unique<ParserAdditiveExpression>()};
protected:
const char * getName() const override { return "string concatenation expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
class ParserBetweenExpression : public IParserBase
{
private:
ParserConcatExpression elem_parser;
protected:
const char * getName() const override { return "BETWEEN expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserComparisonExpression : public IParserBase
{
private:
static const char * operators[];
static const char * overlapping_operators_to_skip[];
ParserLeftAssociativeBinaryOperatorList operator_parser {operators,
overlapping_operators_to_skip, std::make_unique<ParserBetweenExpression>(), true};
protected:
const char * getName() const override{ return "comparison expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
/** Parser for nullity checking with IS (NOT) NULL.
*/
class ParserNullityChecking : public IParserBase
{
private:
ParserComparisonExpression elem_parser;
protected:
const char * getName() const override { return "nullity checking"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserLogicalNotExpression : public IParserBase
{
private:
static const char * operators[];
ParserPrefixUnaryOperatorExpression operator_parser {operators, std::make_unique<ParserNullityChecking>()};
protected:
const char * getName() const override{ return "logical-NOT expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
class ParserLogicalAndExpression : public IParserBase
{
private:
ParserVariableArityOperatorList operator_parser {"AND", "and", std::make_unique<ParserLogicalNotExpression>()};
protected:
const char * getName() const override { return "logical-AND expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
class ParserLogicalOrExpression : public IParserBase
{
private:
ParserVariableArityOperatorList operator_parser {"OR", "or", std::make_unique<ParserLogicalAndExpression>()};
protected:
const char * getName() const override { return "logical-OR expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
/** An expression with ternary operator.
* For example, a = 1 ? b + 1 : c * 2.
*/
class ParserTernaryOperatorExpression : public IParserBase
{
private:
ParserLogicalOrExpression elem_parser;
protected:
const char * getName() const override { return "expression with ternary operator"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserLambdaExpression : public IParserBase
{
private:
ParserTernaryOperatorExpression elem_parser;
protected:
const char * getName() const override { return "lambda expression"; }
@ -457,9 +182,6 @@ protected:
// It's used to parse expressions in table function.
class ParserTableFunctionExpression : public IParserBase
{
private:
ParserLambdaExpression elem_parser;
protected:
const char * getName() const override { return "table function expression"; }
@ -467,13 +189,10 @@ protected:
};
using ParserExpression = ParserLambdaExpression;
class ParserExpressionWithOptionalAlias : public IParserBase
{
public:
explicit ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword, bool is_table_function = false);
explicit ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword_, bool is_table_function_ = false);
protected:
ParserPtr impl;

View File

@ -138,14 +138,28 @@ void IAST::updateTreeHashImpl(SipHash & hash_state) const
}
size_t IAST::checkDepthImpl(size_t max_depth, size_t level) const
size_t IAST::checkDepthImpl(size_t max_depth) const
{
size_t res = level + 1;
for (const auto & child : children)
std::vector<std::pair<ASTPtr, size_t>> stack;
stack.reserve(children.size());
for (const auto & i: children)
stack.push_back({i, 1});
size_t res = 0;
while (!stack.empty())
{
if (level >= max_depth)
auto top = stack.back();
stack.pop_back();
if (top.second >= max_depth)
throw Exception("AST is too deep. Maximum: " + toString(max_depth), ErrorCodes::TOO_DEEP_AST);
res = std::max(res, child->checkDepthImpl(max_depth, level + 1));
res = std::max(res, top.second);
for (const auto & i: top.first->children)
stack.push_back({i, top.second + 1});
}
return res;

View File

@ -92,7 +92,7 @@ public:
*/
size_t checkDepth(size_t max_depth) const
{
return checkDepthImpl(max_depth, 0);
return checkDepthImpl(max_depth);
}
/** Get total number of tree elements
@ -273,7 +273,7 @@ public:
static const char * hilite_none;
private:
size_t checkDepthImpl(size_t max_depth, size_t level) const;
size_t checkDepthImpl(size_t max_depth) const;
/** Forward linked list of ASTPtr to delete.
* Used in IAST destructor to avoid possible stack overflow.

View File

@ -20,7 +20,7 @@ bool ParserCreateFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp
ParserKeyword s_on("ON");
ParserIdentifier function_name_p;
ParserKeyword s_as("AS");
ParserLambdaExpression lambda_p;
ParserExpression lambda_p;
ASTPtr function_name;
ASTPtr function_core;

View File

@ -152,7 +152,7 @@ bool ParserConstraintDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected &
ParserKeyword s_assume("ASSUME");
ParserIdentifier name_p;
ParserLogicalOrExpression expression_p;
ParserExpression expression_p;
ASTPtr name;
ASTPtr expr;
@ -858,8 +858,8 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
ParserStorage storage_p;
ParserStorage storage_inner;
ParserTablePropertiesDeclarationList table_properties_p;
ParserIntervalOperatorExpression watermark_p;
ParserIntervalOperatorExpression lateness_p;
ParserExpression watermark_p;
ParserExpression lateness_p;
ParserSelectWithUnionQuery select_p;
ASTPtr table;

View File

@ -134,7 +134,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
ParserKeyword s_remove{"REMOVE"};
ParserKeyword s_type{"TYPE"};
ParserKeyword s_collate{"COLLATE"};
ParserTernaryOperatorExpression expr_parser;
ParserExpression expr_parser;
ParserStringLiteral string_literal_parser;
ParserLiteral literal_parser;
ParserCodec codec_parser;

View File

@ -20,7 +20,7 @@ bool ParserDictionaryAttributeDeclaration::parseImpl(Pos & pos, ASTPtr & node, E
ParserKeyword s_is_object_id{"IS_OBJECT_ID"};
ParserLiteral default_parser;
ParserArrayOfLiterals array_literals_parser;
ParserTernaryOperatorExpression expression_parser;
ParserExpression expression_parser;
/// mandatory attribute name
ASTPtr name;

View File

@ -3,7 +3,7 @@
#include <Parsers/ASTExternalDDLQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserDropQuery.h>
#include <Parsers/ParserExternalDDLQuery.h>
#include <Parsers/ParserRenameQuery.h>

View File

@ -226,7 +226,7 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec
else if (ParserKeyword("ON").ignore(pos, expected))
{
/// OR is operator with lowest priority, so start parsing from it.
if (!ParserLogicalOrExpression().parse(pos, table_join->on_expression, expected))
if (!ParserExpression().parse(pos, table_join->on_expression, expected))
return false;
}
else

View File

@ -26,6 +26,7 @@
#include <Storages/AlterCommands.h>
#include <Storages/IStorage.h>
#include <Storages/LightweightDeleteDescription.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Common/typeid_cast.h>
#include <Common/randomSeed.h>
@ -1056,6 +1057,10 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
throw Exception{"Data type have to be specified for column " + backQuote(column_name) + " to add",
ErrorCodes::BAD_ARGUMENTS};
if (column_name == LightweightDeleteDescription::FILTER_COLUMN.name && std::dynamic_pointer_cast<MergeTreeData>(table))
throw Exception{"Cannot add column " + backQuote(column_name) + ": this column name is reserved for lightweight delete feature",
ErrorCodes::ILLEGAL_COLUMN};
if (command.codec)
CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs);
@ -1240,6 +1245,10 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
throw Exception{"Cannot rename to " + backQuote(command.rename_to) + ": column with this name already exists",
ErrorCodes::DUPLICATE_COLUMN};
if (command.rename_to == LightweightDeleteDescription::FILTER_COLUMN.name && std::dynamic_pointer_cast<MergeTreeData>(table))
throw Exception{"Cannot rename to " + backQuote(command.rename_to) + ": this column name is reserved for lightweight delete feature",
ErrorCodes::ILLEGAL_COLUMN};
if (modified_columns.contains(column_name))
throw Exception{"Cannot rename and modify the same column " + backQuote(column_name) + " in a single ALTER query",
ErrorCodes::NOT_IMPLEMENTED};

View File

@ -35,7 +35,7 @@ namespace ErrorCodes
}
AsynchronousReadBufferFromHDFS::AsynchronousReadBufferFromHDFS(
AsynchronousReaderPtr reader_, const ReadSettings & settings_, std::shared_ptr<ReadBufferFromHDFS> impl_)
IAsynchronousReader & reader_, const ReadSettings & settings_, std::shared_ptr<ReadBufferFromHDFS> impl_)
: BufferWithOwnMemory<SeekableReadBuffer>(settings_.remote_fs_buffer_size)
, reader(reader_)
, priority(settings_.priority)
@ -72,7 +72,7 @@ std::future<IAsynchronousReader::Result> AsynchronousReadBufferFromHDFS::asyncRe
request.offset = file_offset_of_buffer_end;
request.priority = priority;
request.ignore = 0;
return reader->submit(request);
return reader.submit(request);
}
void AsynchronousReadBufferFromHDFS::prefetch()

View File

@ -24,7 +24,7 @@ class AsynchronousReadBufferFromHDFS : public BufferWithOwnMemory<SeekableReadBu
{
public:
AsynchronousReadBufferFromHDFS(
AsynchronousReaderPtr reader_,
IAsynchronousReader & reader_,
const ReadSettings & settings_,
std::shared_ptr<ReadBufferFromHDFS> impl_);
@ -51,7 +51,7 @@ private:
std::future<IAsynchronousReader::Result> asyncReadInto(char * data, size_t size);
AsynchronousReaderPtr reader;
IAsynchronousReader & reader;
Int32 priority;
std::shared_ptr<ReadBufferFromHDFS> impl;
std::future<IAsynchronousReader::Result> prefetch_future;

View File

@ -508,7 +508,7 @@ std::unique_ptr<ReadBuffer> StorageS3Source::createS3ReadBuffer(const String & k
LOG_TRACE(
log, "Downloading from S3 in {} threads. Object size: {}, Range size: {}.", download_thread_num, object_size, download_buffer_size);
return std::make_unique<ParallelReadBuffer>(std::move(factory), threadPoolCallbackRunner(IOThreadPool::get()), download_thread_num);
return std::make_unique<ParallelReadBuffer>(std::move(factory), threadPoolCallbackRunner<void>(IOThreadPool::get(), "S3ParallelRead"), download_thread_num);
}
String StorageS3Source::getName() const
@ -617,7 +617,7 @@ public:
s3_configuration_.rw_settings,
std::nullopt,
DBMS_DEFAULT_BUFFER_SIZE,
threadPoolCallbackRunner(IOThreadPool::get()),
threadPoolCallbackRunner<void>(IOThreadPool::get(), "S3ParallelRead"),
context->getWriteSettings()),
compression_method,
3);

View File

@ -351,7 +351,7 @@ namespace
return wrapReadBufferWithCompressionMethod(
std::make_unique<ParallelReadBuffer>(
std::move(read_buffer_factory),
threadPoolCallbackRunner(IOThreadPool::get()),
threadPoolCallbackRunner<void>(IOThreadPool::get(), "URLParallelRead"),
download_threads),
compression_method,
settings.zstd_window_log_max);

View File

@ -25,7 +25,7 @@ int main()
String path = "/path/to/hdfs/file";
ReadSettings settings = {};
auto in = std::make_unique<ReadBufferFromHDFS>(hdfs_namenode_url, path, *config, settings);
auto reader = IObjectStorage::getThreadPoolReader();
auto & reader = IObjectStorage::getThreadPoolReader();
AsynchronousReadBufferFromHDFS buf(reader, {}, std::move(in));
String output;

View File

@ -61,7 +61,7 @@ namespace DB
ColumnsDescription /*cached_columns_*/) const
{
const Settings & settings = context_->getSettings();
ParserLambdaExpression partition_by_parser;
ParserExpression partition_by_parser;
ASTPtr partition_by_ast = parseQuery(
partition_by_parser,
"(" + partition_by_def + ")",

View File

@ -136,6 +136,7 @@ CI_CONFIG = {
"build_type": "",
"sanitizer": "",
"package_type": "binary",
"static_binary_name": "aarch64v80compat",
"libraries": "static",
"tidy": "disable",
"with_coverage": False,

View File

@ -3,7 +3,7 @@ set -xeuo pipefail
echo "Running prepare script"
export DEBIAN_FRONTEND=noninteractive
export RUNNER_VERSION=2.293.0
export RUNNER_VERSION=2.296.2
export RUNNER_HOME=/home/ubuntu/actions-runner
deb_arch() {

View File

@ -865,6 +865,12 @@ class TestCase:
stdout=PIPE,
universal_newlines=True,
).communicate()[0]
if diff.startswith("Binary files "):
diff += "Content of stdout:\n===================\n"
file = open(self.stdout_file, "r")
diff += str(file.read())
file.close()
diff += "==================="
description += f"\n{diff}\n"
if debug_log:
description += "\n"

View File

@ -0,0 +1,41 @@
import socket
import time
def get_keeper_socket(cluster, node, port=9181):
hosts = cluster.get_instance_ip(node.name)
client = socket.socket()
client.settimeout(10)
client.connect((hosts, port))
return client
def send_4lw_cmd(cluster, node, cmd="ruok", port=9181):
client = None
try:
client = get_keeper_socket(cluster, node, port)
client.send(cmd.encode())
data = client.recv(100_000)
data = data.decode()
return data
finally:
if client is not None:
client.close()
NOT_SERVING_REQUESTS_ERROR_MSG = "This instance is not currently serving requests"
def wait_until_connected(cluster, node, port=9181):
while send_4lw_cmd(cluster, node, "mntr", port) == NOT_SERVING_REQUESTS_ERROR_MSG:
time.sleep(0.1)
def wait_until_quorum_lost(cluster, node, port=9181):
while send_4lw_cmd(cluster, node, "mntr", port) != NOT_SERVING_REQUESTS_ERROR_MSG:
time.sleep(0.1)
def wait_nodes(cluster, nodes):
for node in nodes:
wait_until_connected(cluster, node)

View File

@ -15,6 +15,7 @@ node1 = cluster.add_instance(
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()

View File

@ -1,22 +0,0 @@
<clickhouse>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>9234</port>
</server>
</raft_configuration>
</keeper_server>
</clickhouse>

View File

@ -1,8 +0,0 @@
<clickhouse>
<zookeeper>
<node index="1">
<host>node1</host>
<port>9181</port>
</node>
</zookeeper>
</clickhouse>

View File

@ -2,6 +2,7 @@ import os
import pytest
import socket
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import time
@ -62,37 +63,6 @@ def get_fake_zk(nodename, timeout=30.0):
return _fake_zk_instance
def get_keeper_socket(node_name):
hosts = cluster.get_instance_ip(node_name)
client = socket.socket()
client.settimeout(10)
client.connect((hosts, 9181))
return client
def send_4lw_cmd(node_name, cmd="ruok"):
client = None
try:
client = get_keeper_socket(node_name)
client.send(cmd.encode())
data = client.recv(100_000)
data = data.decode()
return data
finally:
if client is not None:
client.close()
def wait_until_connected(node_name):
while send_4lw_cmd(node_name, "mntr") == NOT_SERVING_REQUESTS_ERROR_MSG:
time.sleep(0.1)
def wait_nodes(nodes):
for node in nodes:
wait_until_connected(node.name)
def wait_and_assert_data(zk, path, data):
while zk.retry(zk.exists, path) is None:
time.sleep(0.1)
@ -104,9 +74,6 @@ def close_zk(zk):
zk.close()
NOT_SERVING_REQUESTS_ERROR_MSG = "This instance is not currently serving requests"
def test_cluster_recovery(started_cluster):
node_zks = []
try:
@ -114,7 +81,7 @@ def test_cluster_recovery(started_cluster):
for node in nodes[CLUSTER_SIZE:]:
node.stop_clickhouse()
wait_nodes(nodes[:CLUSTER_SIZE])
keeper_utils.wait_nodes(cluster, nodes[:CLUSTER_SIZE])
node_zks = [get_fake_zk(node.name) for node in nodes[:CLUSTER_SIZE]]
@ -152,7 +119,7 @@ def test_cluster_recovery(started_cluster):
wait_and_assert_data(node_zk, "/test_force_recovery_extra", "somedataextra")
nodes[0].start_clickhouse()
wait_until_connected(nodes[0].name)
keeper_utils.wait_until_connected(cluster, nodes[0])
node_zks[0] = get_fake_zk(nodes[0].name)
wait_and_assert_data(node_zks[0], "/test_force_recovery_extra", "somedataextra")
@ -167,8 +134,7 @@ def test_cluster_recovery(started_cluster):
node.stop_clickhouse()
# wait for node1 to lose quorum
while send_4lw_cmd(nodes[0].name, "mntr") != NOT_SERVING_REQUESTS_ERROR_MSG:
time.sleep(0.2)
keeper_utils.wait_until_quorum_lost(cluster, nodes[0])
nodes[0].copy_file_to_container(
os.path.join(CONFIG_DIR, "recovered_keeper1.xml"),
@ -177,9 +143,15 @@ def test_cluster_recovery(started_cluster):
nodes[0].query("SYSTEM RELOAD CONFIG")
assert send_4lw_cmd(nodes[0].name, "mntr") == NOT_SERVING_REQUESTS_ERROR_MSG
send_4lw_cmd(nodes[0].name, "rcvr")
assert send_4lw_cmd(nodes[0].name, "mntr") == NOT_SERVING_REQUESTS_ERROR_MSG
assert (
keeper_utils.send_4lw_cmd(cluster, nodes[0], "mntr")
== keeper_utils.NOT_SERVING_REQUESTS_ERROR_MSG
)
keeper_utils.send_4lw_cmd(cluster, nodes[0], "rcvr")
assert (
keeper_utils.send_4lw_cmd(cluster, nodes[0], "mntr")
== keeper_utils.NOT_SERVING_REQUESTS_ERROR_MSG
)
# add one node to restore the quorum
nodes[CLUSTER_SIZE].copy_file_to_container(
@ -191,10 +163,10 @@ def test_cluster_recovery(started_cluster):
)
nodes[CLUSTER_SIZE].start_clickhouse()
wait_until_connected(nodes[CLUSTER_SIZE].name)
keeper_utils.wait_until_connected(cluster, nodes[CLUSTER_SIZE])
# node1 should have quorum now and accept requests
wait_until_connected(nodes[0].name)
keeper_utils.wait_until_connected(cluster, nodes[0])
node_zks.append(get_fake_zk(nodes[CLUSTER_SIZE].name))
@ -206,7 +178,7 @@ def test_cluster_recovery(started_cluster):
f"/etc/clickhouse-server/config.d/enable_keeper{i+1}.xml",
)
node.start_clickhouse()
wait_until_connected(node.name)
keeper_utils.wait_until_connected(cluster, node)
node_zks.append(get_fake_zk(node.name))
# refresh old zk sessions
@ -223,7 +195,7 @@ def test_cluster_recovery(started_cluster):
wait_and_assert_data(node_zks[-1], "/test_force_recovery_last", "somedatalast")
nodes[0].start_clickhouse()
wait_until_connected(nodes[0].name)
keeper_utils.wait_until_connected(cluster, nodes[0])
node_zks[0] = get_fake_zk(nodes[0].name)
for zk in node_zks[:nodes_left]:
assert_all_data(zk)

View File

@ -2,10 +2,11 @@ import os
import pytest
import socket
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import time
from kazoo.client import KazooClient
from kazoo.client import KazooClient, KazooRetry
CLUSTER_SIZE = 3
@ -45,47 +46,19 @@ def started_cluster():
def get_fake_zk(nodename, timeout=30.0):
_fake_zk_instance = KazooClient(
hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout
hosts=cluster.get_instance_ip(nodename) + ":9181",
timeout=timeout,
connection_retry=KazooRetry(max_tries=10),
command_retry=KazooRetry(max_tries=10),
)
_fake_zk_instance.start()
return _fake_zk_instance
def get_keeper_socket(node_name):
hosts = cluster.get_instance_ip(node_name)
client = socket.socket()
client.settimeout(10)
client.connect((hosts, 9181))
return client
def send_4lw_cmd(node_name, cmd="ruok"):
client = None
try:
client = get_keeper_socket(node_name)
client.send(cmd.encode())
data = client.recv(100_000)
data = data.decode()
return data
finally:
if client is not None:
client.close()
def wait_until_connected(node_name):
while send_4lw_cmd(node_name, "mntr") == NOT_SERVING_REQUESTS_ERROR_MSG:
time.sleep(0.1)
def wait_nodes(nodes):
for node in nodes:
wait_until_connected(node.name)
def wait_and_assert_data(zk, path, data):
while zk.exists(path) is None:
while zk.retry(zk.exists, path) is None:
time.sleep(0.1)
assert zk.get(path)[0] == data.encode()
assert zk.retry(zk.get, path)[0] == data.encode()
def close_zk(zk):
@ -93,20 +66,17 @@ def close_zk(zk):
zk.close()
NOT_SERVING_REQUESTS_ERROR_MSG = "This instance is not currently serving requests"
def test_cluster_recovery(started_cluster):
node_zks = []
try:
wait_nodes(nodes)
keeper_utils.wait_nodes(cluster, nodes)
node_zks = [get_fake_zk(node.name) for node in nodes]
data_in_cluster = []
def add_data(zk, path, data):
zk.create(path, data.encode())
zk.retry(zk.create, path, data.encode())
data_in_cluster.append((path, data))
def assert_all_data(zk):
@ -137,7 +107,7 @@ def test_cluster_recovery(started_cluster):
wait_and_assert_data(node_zk, "/test_force_recovery_extra", "somedataextra")
nodes[0].start_clickhouse()
wait_until_connected(nodes[0].name)
keeper_utils.wait_until_connected(cluster, nodes[0])
node_zks[0] = get_fake_zk(nodes[0].name)
wait_and_assert_data(node_zks[0], "/test_force_recovery_extra", "somedataextra")
@ -156,7 +126,7 @@ def test_cluster_recovery(started_cluster):
)
nodes[0].start_clickhouse()
wait_until_connected(nodes[0].name)
keeper_utils.wait_until_connected(cluster, nodes[0])
assert_all_data(get_fake_zk(nodes[0].name))
finally:

View File

@ -1,6 +1,7 @@
import socket
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import random
import string
import os
@ -25,6 +26,10 @@ node3 = cluster.add_instance(
from kazoo.client import KazooClient, KazooState
def wait_nodes():
keeper_utils.wait_nodes(cluster, [node1, node2, node3])
@pytest.fixture(scope="module")
def started_cluster():
try:
@ -56,28 +61,6 @@ def clear_znodes():
destroy_zk_client(zk)
def wait_node(node):
for _ in range(100):
zk = None
try:
zk = get_fake_zk(node.name, timeout=30.0)
# zk.create("/test", sequence=True)
print("node", node.name, "ready")
break
except Exception as ex:
time.sleep(0.2)
print("Waiting until", node.name, "will be ready, exception", ex)
finally:
destroy_zk_client(zk)
else:
raise Exception("Can't wait node", node.name, "to become ready")
def wait_nodes():
for n in [node1, node2, node3]:
wait_node(n)
def get_fake_zk(nodename, timeout=30.0):
_fake_zk_instance = KazooClient(
hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout
@ -86,23 +69,15 @@ def get_fake_zk(nodename, timeout=30.0):
return _fake_zk_instance
def get_keeper_socket(node_name):
hosts = cluster.get_instance_ip(node_name)
client = socket.socket()
client.settimeout(10)
client.connect((hosts, 9181))
return client
def close_keeper_socket(cli):
if cli is not None:
cli.close()
def reset_node_stats(node_name=node1.name):
def reset_node_stats(node=node1):
client = None
try:
client = get_keeper_socket(node_name)
client = keeper_utils.get_keeper_socket(cluster, node)
client.send(b"srst")
client.recv(10)
finally:
@ -110,23 +85,10 @@ def reset_node_stats(node_name=node1.name):
client.close()
def send_4lw_cmd(node_name=node1.name, cmd="ruok"):
def reset_conn_stats(node=node1):
client = None
try:
client = get_keeper_socket(node_name)
client.send(cmd.encode())
data = client.recv(100_000)
data = data.decode()
return data
finally:
if client is not None:
client.close()
def reset_conn_stats(node_name=node1.name):
client = None
try:
client = get_keeper_socket(node_name)
client = keeper_utils.get_keeper_socket(cluster, node)
client.send(b"crst")
client.recv(10_000)
finally:
@ -138,7 +100,7 @@ def test_cmd_ruok(started_cluster):
client = None
try:
wait_nodes()
data = send_4lw_cmd(cmd="ruok")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="ruok")
assert data == "imok"
finally:
close_keeper_socket(client)
@ -187,7 +149,7 @@ def test_cmd_mntr(started_cluster):
clear_znodes()
# reset stat first
reset_node_stats(node1.name)
reset_node_stats(node1)
zk = get_fake_zk(node1.name, timeout=30.0)
do_some_action(
@ -200,7 +162,7 @@ def test_cmd_mntr(started_cluster):
delete_cnt=2,
)
data = send_4lw_cmd(cmd="mntr")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="mntr")
# print(data.decode())
reader = csv.reader(data.split("\n"), delimiter="\t")
@ -252,10 +214,10 @@ def test_cmd_srst(started_cluster):
wait_nodes()
clear_znodes()
data = send_4lw_cmd(cmd="srst")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="srst")
assert data.strip() == "Server stats reset."
data = send_4lw_cmd(cmd="mntr")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="mntr")
assert len(data) != 0
# print(data)
@ -279,7 +241,7 @@ def test_cmd_conf(started_cluster):
wait_nodes()
clear_znodes()
data = send_4lw_cmd(cmd="conf")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="conf")
reader = csv.reader(data.split("\n"), delimiter="=")
result = {}
@ -335,8 +297,8 @@ def test_cmd_conf(started_cluster):
def test_cmd_isro(started_cluster):
wait_nodes()
assert send_4lw_cmd(node1.name, "isro") == "rw"
assert send_4lw_cmd(node2.name, "isro") == "ro"
assert keeper_utils.send_4lw_cmd(cluster, node1, "isro") == "rw"
assert keeper_utils.send_4lw_cmd(cluster, node2, "isro") == "ro"
def test_cmd_srvr(started_cluster):
@ -345,12 +307,12 @@ def test_cmd_srvr(started_cluster):
wait_nodes()
clear_znodes()
reset_node_stats(node1.name)
reset_node_stats(node1)
zk = get_fake_zk(node1.name, timeout=30.0)
do_some_action(zk, create_cnt=10)
data = send_4lw_cmd(cmd="srvr")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="srvr")
print("srvr output -------------------------------------")
print(data)
@ -380,13 +342,13 @@ def test_cmd_stat(started_cluster):
try:
wait_nodes()
clear_znodes()
reset_node_stats(node1.name)
reset_conn_stats(node1.name)
reset_node_stats(node1)
reset_conn_stats(node1)
zk = get_fake_zk(node1.name, timeout=30.0)
do_some_action(zk, create_cnt=10)
data = send_4lw_cmd(cmd="stat")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="stat")
print("stat output -------------------------------------")
print(data)
@ -440,7 +402,7 @@ def test_cmd_cons(started_cluster):
zk = get_fake_zk(node1.name, timeout=30.0)
do_some_action(zk, create_cnt=10)
data = send_4lw_cmd(cmd="cons")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="cons")
print("cons output -------------------------------------")
print(data)
@ -485,12 +447,12 @@ def test_cmd_crst(started_cluster):
zk = get_fake_zk(node1.name, timeout=30.0)
do_some_action(zk, create_cnt=10)
data = send_4lw_cmd(cmd="crst")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="crst")
print("crst output -------------------------------------")
print(data)
data = send_4lw_cmd(cmd="cons")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="cons")
print("cons output(after crst) -------------------------------------")
print(data)
@ -537,7 +499,7 @@ def test_cmd_dump(started_cluster):
zk = get_fake_zk(node1.name, timeout=30.0)
do_some_action(zk, ephemeral_cnt=2)
data = send_4lw_cmd(cmd="dump")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="dump")
print("dump output -------------------------------------")
print(data)
@ -563,7 +525,7 @@ def test_cmd_wchs(started_cluster):
zk = get_fake_zk(node1.name, timeout=30.0)
do_some_action(zk, create_cnt=2, watch_cnt=2)
data = send_4lw_cmd(cmd="wchs")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="wchs")
print("wchs output -------------------------------------")
print(data)
@ -598,7 +560,7 @@ def test_cmd_wchc(started_cluster):
zk = get_fake_zk(node1.name, timeout=30.0)
do_some_action(zk, create_cnt=2, watch_cnt=2)
data = send_4lw_cmd(cmd="wchc")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="wchc")
print("wchc output -------------------------------------")
print(data)
@ -622,7 +584,7 @@ def test_cmd_wchp(started_cluster):
zk = get_fake_zk(node1.name, timeout=30.0)
do_some_action(zk, create_cnt=2, watch_cnt=2)
data = send_4lw_cmd(cmd="wchp")
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="wchp")
print("wchp output -------------------------------------")
print(data)

View File

@ -204,7 +204,7 @@ JUST_WRONG_CONFIG = """
"""
def test_duplicate_endpoint(started_cluster):
def test_invalid_configs(started_cluster):
node1.stop_clickhouse()
def assert_config_fails(config):

View File

@ -1,6 +1,7 @@
#!/usr/bin/env python3
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import pytest
import random
import string
@ -37,40 +38,22 @@ def started_cluster():
cluster.shutdown()
def get_keeper_socket(node_name):
hosts = cluster.get_instance_ip(node_name)
client = socket.socket()
client.settimeout(10)
client.connect((hosts, 9181))
return client
def close_keeper_socket(cli):
if cli is not None:
cli.close()
def send_4lw_cmd(node_name, cmd="ruok"):
client = None
try:
client = get_keeper_socket(node_name)
client.send(cmd.encode())
data = client.recv(100_000)
data = data.decode()
return data
finally:
if client is not None:
client.close()
def test_aggressive_mntr(started_cluster):
def go_mntr(node_name):
for _ in range(100000):
print(node_name, send_4lw_cmd(node_name, "mntr"))
def go_mntr(node):
for _ in range(10000):
try:
print(node.name, keeper_utils.send_4lw_cmd(cluster, node, "mntr"))
except ConnectionRefusedError:
pass
node1_thread = threading.Thread(target=lambda: go_mntr(node1.name))
node2_thread = threading.Thread(target=lambda: go_mntr(node2.name))
node3_thread = threading.Thread(target=lambda: go_mntr(node3.name))
node1_thread = threading.Thread(target=lambda: go_mntr(node1))
node2_thread = threading.Thread(target=lambda: go_mntr(node2))
node3_thread = threading.Thread(target=lambda: go_mntr(node3))
node1_thread.start()
node2_thread.start()
node3_thread.start()
@ -78,8 +61,7 @@ def test_aggressive_mntr(started_cluster):
node2.stop_clickhouse()
node3.stop_clickhouse()
while send_4lw_cmd(node1.name, "mntr") != NOT_SERVING_REQUESTS_ERROR_MSG:
time.sleep(0.2)
keeper_utils.wait_until_quorum_lost(cluster, node1)
node1.stop_clickhouse()
starters = []

View File

@ -1,5 +1,6 @@
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import random
import string
import os
@ -55,31 +56,6 @@ def smaller_exception(ex):
return "\n".join(str(ex).split("\n")[0:2])
def wait_node(node):
for _ in range(100):
zk = None
try:
node.query("SELECT * FROM system.zookeeper WHERE path = '/'")
zk = get_fake_zk(node.name, timeout=30.0)
zk.create("/test", sequence=True)
print("node", node.name, "ready")
break
except Exception as ex:
time.sleep(0.2)
print("Waiting until", node.name, "will be ready, exception", ex)
finally:
if zk:
zk.stop()
zk.close()
else:
raise Exception("Can't wait node", node.name, "to become ready")
def wait_nodes():
for node in [node1, node2, node3]:
wait_node(node)
def get_fake_zk(nodename, timeout=30.0):
_fake_zk_instance = KazooClient(
hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout
@ -88,6 +64,10 @@ def get_fake_zk(nodename, timeout=30.0):
return _fake_zk_instance
def wait_nodes():
keeper_utils.wait_nodes(cluster, [node1, node2, node3])
# in extremely rare case it can take more than 5 minutes in debug build with sanitizer
@pytest.mark.timeout(600)
def test_blocade_leader(started_cluster):

View File

@ -1,5 +1,6 @@
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import random
import string
import os
@ -43,29 +44,8 @@ def smaller_exception(ex):
return "\n".join(str(ex).split("\n")[0:2])
def wait_node(node):
for _ in range(100):
zk = None
try:
node.query("SELECT * FROM system.zookeeper WHERE path = '/'")
zk = get_fake_zk(node.name, timeout=30.0)
zk.create("/test", sequence=True)
print("node", node.name, "ready")
break
except Exception as ex:
time.sleep(0.2)
print("Waiting until", node.name, "will be ready, exception", ex)
finally:
if zk:
zk.stop()
zk.close()
else:
raise Exception("Can't wait node", node.name, "to become ready")
def wait_nodes():
for node in [node1, node2, node3]:
wait_node(node)
keeper_utils.wait_nodes(cluster, [node1, node2, node3])
def get_fake_zk(nodename, timeout=30.0):

View File

@ -2,6 +2,7 @@
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import random
import string
import os
@ -41,9 +42,11 @@ def started_cluster():
def start(node):
node.start_clickhouse()
keeper_utils.wait_until_connected(cluster, node)
def test_nodes_add(started_cluster):
keeper_utils.wait_until_connected(cluster, node1)
zk_conn = get_fake_zk(node1)
for i in range(100):
@ -62,6 +65,7 @@ def test_nodes_add(started_cluster):
)
node1.query("SYSTEM RELOAD CONFIG")
waiter.wait()
keeper_utils.wait_until_connected(cluster, node2)
zk_conn2 = get_fake_zk(node2)
@ -93,6 +97,7 @@ def test_nodes_add(started_cluster):
node2.query("SYSTEM RELOAD CONFIG")
waiter.wait()
keeper_utils.wait_until_connected(cluster, node3)
zk_conn3 = get_fake_zk(node3)
for i in range(100):

View File

@ -11,6 +11,7 @@ import os
import time
from multiprocessing.dummy import Pool
from helpers.test_tools import assert_eq_with_retry
import helpers.keeper_utils as keeper_utils
from kazoo.client import KazooClient, KazooState
cluster = ClickHouseCluster(__file__)
@ -41,6 +42,7 @@ def started_cluster():
def start(node):
node.start_clickhouse()
keeper_utils.wait_until_connected(cluster, node)
def get_fake_zk(node, timeout=30.0):

View File

@ -2,6 +2,7 @@
import pytest
from helpers.cluster import ClickHouseCluster
import time
import os
from kazoo.client import KazooClient, KazooState
@ -79,9 +80,12 @@ def test_nodes_remove(started_cluster):
assert zk_conn.exists("test_two_" + str(i)) is not None
assert zk_conn.exists("test_two_" + str(100 + i)) is not None
with pytest.raises(Exception):
try:
zk_conn3 = get_fake_zk(node3)
zk_conn3.sync("/test_two_0")
time.sleep(0.1)
except Exception:
pass
node3.stop_clickhouse()
@ -91,6 +95,7 @@ def test_nodes_remove(started_cluster):
)
node1.query("SYSTEM RELOAD CONFIG")
zk_conn = get_fake_zk(node1)
zk_conn.sync("/test_two_0")
@ -98,8 +103,11 @@ def test_nodes_remove(started_cluster):
assert zk_conn.exists("test_two_" + str(i)) is not None
assert zk_conn.exists("test_two_" + str(100 + i)) is not None
with pytest.raises(Exception):
try:
zk_conn2 = get_fake_zk(node2)
zk_conn2.sync("/test_two_0")
time.sleep(0.1)
except Exception:
pass
node2.stop_clickhouse()

View File

@ -46,6 +46,10 @@ def get_connection_zk(nodename, timeout=30.0):
return _fake_zk_instance
def restart_clickhouse():
node.restart_clickhouse(kill=True)
def test_state_after_restart(started_cluster):
try:
node_zk = None
@ -62,7 +66,7 @@ def test_state_after_restart(started_cluster):
if i % 7 == 0:
node_zk.delete("/test_state_after_restart/node" + str(i))
node.restart_clickhouse(kill=True)
restart_clickhouse()
node_zk2 = get_connection_zk("node")
@ -111,7 +115,7 @@ def test_state_duplicate_restart(started_cluster):
if i % 7 == 0:
node_zk.delete("/test_state_duplicated_restart/node" + str(i))
node.restart_clickhouse(kill=True)
restart_clickhouse()
node_zk2 = get_connection_zk("node")
@ -119,7 +123,7 @@ def test_state_duplicate_restart(started_cluster):
node_zk2.create("/test_state_duplicated_restart/just_test2")
node_zk2.create("/test_state_duplicated_restart/just_test3")
node.restart_clickhouse(kill=True)
restart_clickhouse()
node_zk3 = get_connection_zk("node")
@ -159,6 +163,7 @@ def test_state_duplicate_restart(started_cluster):
# http://zookeeper-user.578899.n2.nabble.com/Why-are-ephemeral-nodes-written-to-disk-tp7583403p7583418.html
def test_ephemeral_after_restart(started_cluster):
try:
node_zk = None
node_zk2 = None
@ -176,7 +181,7 @@ def test_ephemeral_after_restart(started_cluster):
if i % 7 == 0:
node_zk.delete("/test_ephemeral_after_restart/node" + str(i))
node.restart_clickhouse(kill=True)
restart_clickhouse()
node_zk2 = get_connection_zk("node")

View File

@ -1,6 +1,7 @@
#!/usr/bin/env python3
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import random
import string
import os
@ -26,10 +27,15 @@ node3 = cluster.add_instance(
from kazoo.client import KazooClient, KazooState
def wait_nodes():
keeper_utils.wait_nodes(cluster, [node1, node2, node3])
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
wait_nodes()
yield cluster
@ -100,6 +106,8 @@ def test_restart_multinode(started_cluster):
node1.restart_clickhouse(kill=True)
node2.restart_clickhouse(kill=True)
node3.restart_clickhouse(kill=True)
wait_nodes()
for i in range(100):
try:
node1_zk = get_fake_zk("node1")

View File

@ -1,34 +0,0 @@
<clickhouse>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>9234</port>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
</raft_configuration>
</keeper_server>
</clickhouse>

View File

@ -1,34 +0,0 @@
<clickhouse>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>2</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>9234</port>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
</raft_configuration>
</keeper_server>
</clickhouse>

View File

@ -1,34 +0,0 @@
<clickhouse>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>3</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>9234</port>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
</raft_configuration>
</keeper_server>
</clickhouse>

View File

@ -1,28 +0,0 @@
<clickhouse>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>9234</port>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
</raft_configuration>
</keeper_server>
</clickhouse>

View File

@ -1,28 +0,0 @@
<clickhouse>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>2</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>9234</port>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
</raft_configuration>
</keeper_server>
</clickhouse>

View File

@ -1,28 +0,0 @@
<clickhouse>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>3</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>9234</port>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
</raft_configuration>
</keeper_server>
</clickhouse>

View File

@ -1,6 +1,7 @@
#!/usr/bin/env python3
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import random
import string
import os
@ -84,6 +85,7 @@ def test_recover_from_snapshot(started_cluster):
# stale node should recover from leader's snapshot
# with some sanitizers can start longer than 5 seconds
node3.start_clickhouse(20)
keeper_utils.wait_until_connected(cluster, node3)
print("Restarted")
try:

View File

@ -40,4 +40,4 @@ def started_cluster():
def test_connection(started_cluster):
# just nothrow
node2.query("SELECT * FROM system.zookeeper WHERE path = '/'")
node2.query_with_retry("SELECT * FROM system.zookeeper WHERE path = '/'")

View File

@ -1,5 +1,6 @@
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import time
import socket
import struct
@ -52,25 +53,8 @@ def destroy_zk_client(zk):
pass
def wait_node(node):
for _ in range(100):
zk = None
try:
zk = get_fake_zk(node.name, timeout=30.0)
print("node", node.name, "ready")
break
except Exception as ex:
time.sleep(0.2)
print("Waiting until", node.name, "will be ready, exception", ex)
finally:
destroy_zk_client(zk)
else:
raise Exception("Can't wait node", node.name, "to become ready")
def wait_nodes():
for n in [node1, node2, node3]:
wait_node(n)
keeper_utils.wait_nodes(cluster, [node1, node2, node3])
def get_fake_zk(nodename, timeout=30.0):

View File

@ -2,6 +2,7 @@
##!/usr/bin/env python3
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
from multiprocessing.dummy import Pool
from kazoo.client import KazooClient, KazooState
import random
@ -22,7 +23,7 @@ node3 = cluster.add_instance(
def start_zookeeper(node):
node1.exec_in_container(["bash", "-c", "/opt/zookeeper/bin/zkServer.sh start"])
node.exec_in_container(["bash", "-c", "/opt/zookeeper/bin/zkServer.sh start"])
def stop_zookeeper(node):
@ -66,6 +67,7 @@ def stop_clickhouse(node):
def start_clickhouse(node):
node.start_clickhouse()
keeper_utils.wait_until_connected(cluster, node)
def copy_zookeeper_data(make_zk_snapshots, node):

View File

@ -3,6 +3,7 @@
#!/usr/bin/env python3
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import random
import string
import os
@ -50,6 +51,11 @@ def get_connection_zk(nodename, timeout=30.0):
return _fake_zk_instance
def restart_clickhouse():
node.restart_clickhouse(kill=True)
keeper_utils.wait_until_connected(cluster, node)
def test_state_after_restart(started_cluster):
try:
node_zk = None
@ -69,7 +75,7 @@ def test_state_after_restart(started_cluster):
else:
existing_children.append("node" + str(i))
node.restart_clickhouse(kill=True)
restart_clickhouse()
node_zk2 = get_connection_zk("node")
@ -123,7 +129,7 @@ def test_ephemeral_after_restart(started_cluster):
else:
existing_children.append("node" + str(i))
node.restart_clickhouse(kill=True)
restart_clickhouse()
node_zk2 = get_connection_zk("node")

View File

@ -1,6 +1,7 @@
#!/usr/bin/env python3
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import random
import string
import os
@ -20,6 +21,10 @@ node3 = cluster.add_instance(
from kazoo.client import KazooClient, KazooState
def wait_nodes():
keeper_utils.wait_nodes(cluster, [node1, node2, node3])
@pytest.fixture(scope="module")
def started_cluster():
try:
@ -94,6 +99,8 @@ def test_restart_multinode(started_cluster):
node1.restart_clickhouse(kill=True)
node2.restart_clickhouse(kill=True)
node3.restart_clickhouse(kill=True)
wait_nodes()
for i in range(100):
try:
node1_zk = get_fake_zk("node1")

View File

@ -1,34 +0,0 @@
<clickhouse>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>9234</port>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
</raft_configuration>
</keeper_server>
</clickhouse>

View File

@ -1,34 +0,0 @@
<clickhouse>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>2</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>9234</port>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
</raft_configuration>
</keeper_server>
</clickhouse>

View File

@ -1,34 +0,0 @@
<clickhouse>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>3</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>9234</port>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
</raft_configuration>
</keeper_server>
</clickhouse>

View File

@ -1,6 +1,7 @@
#!/usr/bin/env python3
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import random
import string
import os
@ -48,6 +49,7 @@ def started_cluster():
def start(node):
node.start_clickhouse()
keeper_utils.wait_until_connected(cluster, node)
def delete_with_retry(node_name, path):
@ -138,6 +140,7 @@ def test_restart_third_node(started_cluster):
node1_zk.create("/test_restart", b"aaaa")
node3.restart_clickhouse()
keeper_utils.wait_until_connected(cluster, node3)
assert node3.contains_in_log(
"Connected to ZooKeeper (or Keeper) before internal Keeper start"

View File

@ -2,6 +2,7 @@
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import random
import string
import os
@ -40,29 +41,8 @@ def smaller_exception(ex):
return "\n".join(str(ex).split("\n")[0:2])
def wait_node(node):
for _ in range(100):
zk = None
try:
node.query("SELECT * FROM system.zookeeper WHERE path = '/'")
zk = get_fake_zk(node.name, timeout=30.0)
zk.create("/test", sequence=True)
print("node", node.name, "ready")
break
except Exception as ex:
time.sleep(0.2)
print("Waiting until", node.name, "will be ready, exception", ex)
finally:
if zk:
zk.stop()
zk.close()
else:
raise Exception("Can't wait node", node.name, "to become ready")
def wait_nodes():
for node in [node1, node2]:
wait_node(node)
keeper_utils.wait_nodes(cluster, [node1, node2])
def get_fake_zk(nodename, timeout=30.0):

View File

@ -1,5 +1,6 @@
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import random
import string
import os
@ -42,29 +43,8 @@ def smaller_exception(ex):
return "\n".join(str(ex).split("\n")[0:2])
def wait_node(node):
for _ in range(100):
zk = None
try:
node.query("SELECT * FROM system.zookeeper WHERE path = '/'")
zk = get_fake_zk(node.name, timeout=30.0)
zk.create("/test", sequence=True)
print("node", node.name, "ready")
break
except Exception as ex:
time.sleep(0.2)
print("Waiting until", node.name, "will be ready, exception", ex)
finally:
if zk:
zk.stop()
zk.close()
else:
raise Exception("Can't wait node", node.name, "to become ready")
def wait_nodes():
for node in [node1, node2, node3]:
wait_node(node)
keeper_utils.wait_nodes(cluster, [node1, node2, node3])
def get_fake_zk(nodename, timeout=30.0):
@ -129,6 +109,7 @@ def test_server_restart(started_cluster):
node1_zk.set("/test_server_restart/" + str(child_node), b"somevalue")
node3.restart_clickhouse(kill=True)
keeper_utils.wait_until_connected(cluster, node3)
node2_zk = get_fake_zk("node2")
node3_zk = get_fake_zk("node3")

View File

@ -1,6 +1,7 @@
#!/usr/bin/env python3
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
from kazoo.client import KazooClient, KazooState
from kazoo.security import ACL, make_digest_acl, make_acl
from kazoo.exceptions import (
@ -60,6 +61,7 @@ def stop_clickhouse():
def start_clickhouse():
node.start_clickhouse()
keeper_utils.wait_until_connected(cluster, node)
def copy_zookeeper_data(make_zk_snapshots):

Some files were not shown because too many files have changed in this diff Show More