Merge branch 'master' into fix_block_number

This commit is contained in:
alesapin 2024-01-29 10:19:43 +01:00
commit b864b122fa
95 changed files with 243 additions and 2926 deletions

View File

@ -28,7 +28,7 @@ elseif(CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-aarch64")
set(Rust_CARGO_TARGET "aarch64-unknown-linux-gnu")
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "x86_64"))
set(Rust_CARGO_TARGET "x86_64-apple-darwin")
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "darwin"))
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "aarch64"))
set(Rust_CARGO_TARGET "aarch64-apple-darwin")
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "freebsd") AND (CMAKE_TOOLCHAIN_FILE MATCHES "x86_64"))
set(Rust_CARGO_TARGET "x86_64-unknown-freebsd")

View File

@ -1,4 +1,4 @@
option (ENABLE_SSH "Enable support for SSH keys and protocol" ON)
option (ENABLE_SSH "Enable support for SSH keys and protocol" ${ENABLE_LIBRARIES})
if (NOT ENABLE_SSH)
message(STATUS "Not using SSH")

View File

@ -23,13 +23,15 @@ if [ -f /sys/fs/cgroup/cgroup.controllers ]; then
> /sys/fs/cgroup/cgroup.subtree_control
fi
# In case of test hung it is convenient to use pytest --pdb to debug it,
# and on hung you can simply press Ctrl-C and it will spawn a python pdb,
# but on SIGINT dockerd will exit, so ignore it to preserve the daemon.
trap '' INT
# Binding to an IP address without --tlsverify is deprecated. Startup is intentionally being slowed
# unless --tls=false or --tlsverify=false is set
dockerd --host=unix:///var/run/docker.sock --tls=false --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log &
#
# In case of test hung it is convenient to use pytest --pdb to debug it,
# and on hung you can simply press Ctrl-C and it will spawn a python pdb,
# but on SIGINT dockerd will exit, so we spawn new session to ignore SIGINT by
# docker.
# Note, that if you will run it via runner, it will send SIGINT to docker anyway.
setsid dockerd --host=unix:///var/run/docker.sock --tls=false --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log &
set +e
reties=0

View File

@ -8,8 +8,6 @@ sidebar_label: VIEW
You can modify `SELECT` query that was specified when a [materialized view](../create/view.md#materialized) was created with the `ALTER TABLE … MODIFY QUERY` statement without interrupting ingestion process.
The `allow_experimental_alter_materialized_view_structure` setting must be enabled.
This command is created to change materialized view created with `TO [db.]name` clause. It does not change the structure of the underling storage table and it does not change the columns' definition of the materialized view, because of this the application of this command is very limited for materialized views are created without `TO [db.]name` clause.
**Example with TO table**

View File

@ -97,7 +97,7 @@ This feature is deprecated and will be removed in the future.
For your convenience, the old documentation is located [here](https://pastila.nl/?00f32652/fdf07272a7b54bda7e13b919264e449f.md)
## Refreshable Materialized View {#refreshable-materialized-view}
## Refreshable Materialized View [Experimental] {#refreshable-materialized-view}
```sql
CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name
@ -120,7 +120,8 @@ Differences from regular non-refreshable materialized views:
:::note
Refreshable materialized views are a work in progress. Setting `allow_experimental_refreshable_materialized_view = 1` is required for creating one. Current limitations:
* not compatible with Replicated database or table engines,
* not compatible with Replicated database or table engines
* It is not supported in ClickHouse Cloud
* require [Atomic database engine](../../../engines/database-engines/atomic.md),
* no retries for failed refresh - we just skip to the next scheduled refresh time,
* no limit on number of concurrent refreshes.

View File

@ -60,10 +60,16 @@
/// If it is hosted on server, assume that it is the address of ClickHouse.
if (location.protocol != 'file:') {
host = location.origin;
user = 'default';
add_http_cors_header = false;
}
if (window.location.search) {
const params = new URLSearchParams(window.location.search);
if (params.has('host')) { host = params.get('host'); }
if (params.has('user')) { user = params.get('user'); }
if (params.has('password')) { password = params.get('password'); }
}
let map = L.map('space', {
crs: L.CRS.Simple,
center: [-512, 512],

View File

@ -1562,6 +1562,10 @@
<max_entry_size_in_rows>30000000</max_entry_size_in_rows>
</query_cache>
<backups>
<allowed_path>backups</allowed_path>
</backups>
<!-- This allows to disable exposing addresses in stack traces for security reasons.
Please be aware that it does not improve security much, but makes debugging much harder.
The addresses that are small offsets from zero will be displayed nevertheless to show nullptr dereferences.

View File

@ -993,7 +993,16 @@
function renderError(response)
{
clear();
document.getElementById('error').innerText = response ? response : "No response.";
let message = response;
try {
let json = JSON.parse(response);
if (json.exception) {
message = json.exception;
}
} catch (e) {}
document.getElementById('error').innerText = message ? message : "No response.";
document.getElementById('error').style.display = 'block';
document.getElementById('logo-container').style.display = 'none';
}

View File

@ -13,7 +13,7 @@ serde_json = "1.0"
crate-type = ["staticlib"]
[profile.release]
debug = true
debug = false
[profile.release-thinlto]
inherits = "release"

View File

@ -17,7 +17,7 @@ cxx-build = "1.0.83"
crate-type = ["staticlib"]
[profile.release]
debug = true
debug = false
[profile.release-thinlto]
inherits = "release"

View File

@ -73,7 +73,7 @@ namespace
return checkPasswordDoubleSHA1MySQL(scramble, scrambled_password, Util::encodeDoubleSHA1(password_plaintext));
}
#if USE_SSL
#if USE_SSH
bool checkSshSignature(const std::vector<ssh::SSHKey> & keys, std::string_view signature, std::string_view original)
{
for (const auto & key: keys)
@ -243,7 +243,7 @@ bool Authentication::areCredentialsValid(
throw Authentication::Require<SSLCertificateCredentials>("ClickHouse X.509 Authentication");
case AuthenticationType::SSH_KEY:
#if USE_SSL
#if USE_SSH
return checkSshSignature(auth_data.getSSHKeys(), ssh_credentials->getSignature(), ssh_credentials->getOriginal());
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL");

View File

@ -320,7 +320,7 @@ std::shared_ptr<ASTAuthenticationData> AuthenticationData::toAST() const
}
case AuthenticationType::SSH_KEY:
{
#if USE_SSL
#if USE_SSH
for (const auto & key : getSSHKeys())
node->children.push_back(std::make_shared<ASTPublicSSHKey>(key.getBase64(), key.getKeyType()));
@ -353,7 +353,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que
/// For this type of authentication we have ASTPublicSSHKey as children for ASTAuthenticationData
if (query.type && query.type == AuthenticationType::SSH_KEY)
{
#if USE_SSL
#if USE_SSH
AuthenticationData auth_data(*query.type);
std::vector<ssh::SSHKey> keys;

View File

@ -209,7 +209,7 @@ namespace
}
else if (has_ssh_keys)
{
#if USE_SSL
#if USE_SSH
user->auth_data = AuthenticationData{AuthenticationType::SSH_KEY};
Poco::Util::AbstractConfiguration::Keys entries;

View File

@ -316,7 +316,7 @@ void Connection::sendHello()
"Inter-server secret support is disabled, because ClickHouse was built without SSL library");
#endif
}
#if USE_SSL
#if USE_SSH
/// Just inform server that we will authenticate using SSH keys.
else if (!ssh_private_key.isEmpty())
{
@ -346,7 +346,7 @@ void Connection::sendAddendum()
void Connection::performHandshakeForSSHAuth()
{
#if USE_SSL
#if USE_SSH
String challenge;
{
writeVarUInt(Protocol::Client::SSHChallengeRequest, *out);

View File

@ -79,7 +79,7 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati
}
else
{
#if USE_SSL
#if USE_SSH
std::string filename = config.getString("ssh-key-file");
std::string passphrase;
if (config.has("ssh-key-passphrase"))

View File

@ -337,7 +337,7 @@ ColumnPtr ColumnAggregateFunction::indexImpl(const PaddedPODArray<Type> & indexe
assert(limit <= indexes.size());
auto res = createView();
res->data.resize(limit);
res->data.resize_exact(limit);
for (size_t i = 0; i < limit; ++i)
res->data[i] = data[indexes[i]];
@ -626,7 +626,7 @@ void ColumnAggregateFunction::getPermutation(PermutationSortDirection /*directio
size_t /*limit*/, int /*nan_direction_hint*/, IColumn::Permutation & res) const
{
size_t s = data.size();
res.resize(s);
res.resize_exact(s);
iota(res.data(), s, IColumn::Permutation::value_type(0));
}

View File

@ -109,7 +109,7 @@ MutableColumnPtr ColumnArray::cloneResized(size_t to_size) const
offset = getOffsets().back();
}
res->getOffsets().resize(to_size);
res->getOffsets().resize_exact(to_size);
for (size_t i = from_size; i < to_size; ++i)
res->getOffsets()[i] = offset;
}
@ -427,6 +427,12 @@ void ColumnArray::reserve(size_t n)
getData().reserve(n); /// The average size of arrays is not taken into account here. Or it is considered to be no more than 1.
}
void ColumnArray::shrinkToFit()
{
getOffsets().shrink_to_fit();
getData().shrinkToFit();
}
void ColumnArray::ensureOwnership()
{
getData().ensureOwnership();
@ -603,7 +609,7 @@ void ColumnArray::expand(const IColumn::Filter & mask, bool inverted)
ssize_t index = mask.size() - 1;
ssize_t from = offsets_data.size() - 1;
offsets_data.resize(mask.size());
offsets_data.resize_exact(mask.size());
UInt64 last_offset = offsets_data[from];
while (index >= 0)
{
@ -831,7 +837,7 @@ ColumnPtr ColumnArray::indexImpl(const PaddedPODArray<T> & indexes, size_t limit
auto res = ColumnArray::create(data->cloneEmpty());
Offsets & res_offsets = res->getOffsets();
res_offsets.resize(limit);
res_offsets.resize_exact(limit);
size_t current_offset = 0;
for (size_t i = 0; i < limit; ++i)

View File

@ -108,6 +108,7 @@ public:
void updatePermutationWithCollation(const Collator & collator, PermutationSortDirection direction, PermutationSortStability stability,
size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override;
void reserve(size_t n) override;
void shrinkToFit() override;
void ensureOwnership() override;
size_t byteSize() const override;
size_t byteSizeAt(size_t n) const override;

View File

@ -128,7 +128,7 @@ MutableColumns ColumnConst::scatter(ColumnIndex num_columns, const Selector & se
void ColumnConst::getPermutation(PermutationSortDirection /*direction*/, PermutationSortStability /*stability*/,
size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const
{
res.resize(s);
res.resize_exact(s);
iota(res.data(), s, IColumn::Permutation::value_type(0));
}

View File

@ -159,7 +159,7 @@ void ColumnDecimal<T>::getPermutation(IColumn::PermutationSortDirection directio
};
size_t data_size = data.size();
res.resize(data_size);
res.resize_exact(data_size);
if (limit >= data_size)
limit = 0;
@ -318,7 +318,7 @@ MutableColumnPtr ColumnDecimal<T>::cloneResized(size_t size) const
if (size > 0)
{
auto & new_col = static_cast<Self &>(*res);
new_col.data.resize(size);
new_col.data.resize_exact(size);
size_t count = std::min(this->size(), size);

View File

@ -55,6 +55,7 @@ public:
size_t allocatedBytes() const override { return data.allocated_bytes(); }
void protect() override { data.protect(); }
void reserve(size_t n) override { data.reserve(n); }
void shrinkToFit() override { data.shrink_to_fit(); }
void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast<const Self &>(src).getData()[n]); }
void insertData(const char * src, size_t /*length*/) override;

View File

@ -39,7 +39,7 @@ MutableColumnPtr ColumnFixedString::cloneResized(size_t size) const
if (size > 0)
{
auto & new_col = assert_cast<ColumnFixedString &>(*new_col_holder);
new_col.chars.resize(size * n);
new_col.chars.resize_exact(size * n);
size_t count = std::min(this->size(), size);
memcpy(new_col.chars.data(), chars.data(), count * n * sizeof(chars[0]));

View File

@ -181,6 +181,11 @@ public:
chars.reserve(n * size);
}
void shrinkToFit() override
{
chars.shrink_to_fit();
}
void resize(size_t size)
{
chars.resize(n * size);

View File

@ -159,6 +159,7 @@ public:
}
void reserve(size_t n) override { idx.reserve(n); }
void shrinkToFit() override { idx.shrinkToFit(); }
/// Don't count the dictionary size as it can be shared between different blocks.
size_t byteSize() const override { return idx.getPositions()->byteSize(); }
@ -295,6 +296,7 @@ public:
void popBack(size_t n) { positions->popBack(n); }
void reserve(size_t n) { positions->reserve(n); }
void shrinkToFit() { positions->shrinkToFit(); }
UInt64 getMaxPositionForCurrentType() const;

View File

@ -229,6 +229,11 @@ void ColumnMap::reserve(size_t n)
nested->reserve(n);
}
void ColumnMap::shrinkToFit()
{
nested->shrinkToFit();
}
void ColumnMap::ensureOwnership()
{
nested->ensureOwnership();

View File

@ -83,6 +83,7 @@ public:
void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override;
void reserve(size_t n) override;
void shrinkToFit() override;
void ensureOwnership() override;
size_t byteSize() const override;
size_t byteSizeAt(size_t n) const override;

View File

@ -97,7 +97,7 @@ MutableColumnPtr ColumnNullable::cloneResized(size_t new_size) const
if (new_size > 0)
{
new_null_map->getData().resize(new_size);
new_null_map->getData().resize_exact(new_size);
size_t count = std::min(size(), new_size);
memcpy(new_null_map->getData().data(), getNullMapData().data(), count * sizeof(getNullMapData()[0]));
@ -678,6 +678,12 @@ void ColumnNullable::reserve(size_t n)
getNullMapData().reserve(n);
}
void ColumnNullable::shrinkToFit()
{
getNestedColumn().shrinkToFit();
getNullMapData().shrink_to_fit();
}
void ColumnNullable::ensureOwnership()
{
getNestedColumn().ensureOwnership();

View File

@ -109,6 +109,7 @@ public:
void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override;
void reserve(size_t n) override;
void shrinkToFit() override;
void ensureOwnership() override;
size_t byteSize() const override;
size_t byteSizeAt(size_t n) const override;

View File

@ -71,8 +71,8 @@ MutableColumnPtr ColumnString::cloneResized(size_t to_size) const
/// Empty strings are just zero terminating bytes.
res->chars.resize_fill(res->chars.size() + to_size - from_size);
res->offsets.resize_exact(to_size);
res->offsets.resize(to_size);
for (size_t i = from_size; i < to_size; ++i)
{
++offset;
@ -494,6 +494,11 @@ void ColumnString::reserve(size_t n)
offsets.reserve(n);
}
void ColumnString::shrinkToFit()
{
chars.shrink_to_fit();
offsets.shrink_to_fit();
}
void ColumnString::getExtremes(Field & min, Field & max) const
{

View File

@ -259,6 +259,7 @@ public:
ColumnPtr compress() const override;
void reserve(size_t n) override;
void shrinkToFit() override;
void getExtremes(Field & min, Field & max) const override;

View File

@ -444,6 +444,13 @@ void ColumnTuple::reserve(size_t n)
getColumn(i).reserve(n);
}
void ColumnTuple::shrinkToFit()
{
const size_t tuple_size = columns.size();
for (size_t i = 0; i < tuple_size; ++i)
getColumn(i).shrinkToFit();
}
void ColumnTuple::ensureOwnership()
{
const size_t tuple_size = columns.size();

View File

@ -91,6 +91,7 @@ public:
void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges& equal_ranges) const override;
void reserve(size_t n) override;
void shrinkToFit() override;
void ensureOwnership() override;
size_t byteSize() const override;
size_t byteSizeAt(size_t n) const override;

View File

@ -237,7 +237,7 @@ void ColumnVector<T>::getPermutation(IColumn::PermutationSortDirection direction
size_t limit, int nan_direction_hint, IColumn::Permutation & res) const
{
size_t data_size = data.size();
res.resize(data_size);
res.resize_exact(data_size);
if (data_size == 0)
return;
@ -424,7 +424,7 @@ MutableColumnPtr ColumnVector<T>::cloneResized(size_t size) const
if (size > 0)
{
auto & new_col = static_cast<Self &>(*res);
new_col.data.resize(size);
new_col.data.resize_exact(size);
size_t count = std::min(this->size(), size);
memcpy(new_col.data.data(), data.data(), count * sizeof(data[0]));
@ -628,8 +628,8 @@ inline void doFilterAligned(const UInt8 *& filt_pos, const UInt8 *& filt_end_ali
filt_pos += SIMD_ELEMENTS;
data_pos += SIMD_ELEMENTS;
}
/// resize to the real size.
res_data.resize(current_offset);
/// Resize to the real size.
res_data.resize_exact(current_offset);
}
)

View File

@ -182,6 +182,11 @@ public:
data.reserve(n);
}
void shrinkToFit() override
{
data.shrink_to_fit();
}
const char * getFamilyName() const override { return TypeName<T>.data(); }
TypeIndex getDataType() const override { return TypeToTypeIndex<T>; }

View File

@ -399,10 +399,7 @@ public:
/// Requests the removal of unused capacity.
/// It is a non-binding request to reduce the capacity of the underlying container to its size.
virtual MutablePtr shrinkToFit() const
{
return cloneResized(size());
}
virtual void shrinkToFit() {}
/// If we have another column as a source (owner of data), copy all data to ourself and reset source.
virtual void ensureOwnership() {}

View File

@ -2,6 +2,7 @@
#include <limits>
#include <optional>
#include <fmt/format.h>
#include <base/defines.h>
#include <base/scope_guard.h>
#include <Common/ErrorCodes.h>
@ -195,13 +196,6 @@ void LoadTask::remove()
}
}
void LoadTask::detach()
{
jobs.clear();
goal_jobs.clear();
}
AsyncLoader::AsyncLoader(std::vector<PoolInitializer> pool_initializers, bool log_failures_, bool log_progress_)
: log_failures(log_failures_)
, log_progress(log_progress_)
@ -214,7 +208,22 @@ AsyncLoader::AsyncLoader(std::vector<PoolInitializer> pool_initializers, bool lo
AsyncLoader::~AsyncLoader()
{
stop();
// All `LoadTask` objects should be destructed before AsyncLoader destruction because they hold a reference.
// To make sure we check for all pending jobs to be finished.
std::unique_lock lock{mutex};
if (scheduled_jobs.empty() && finished_jobs.empty())
return;
std::vector<String> scheduled;
std::vector<String> finished;
scheduled.reserve(scheduled_jobs.size());
finished.reserve(finished_jobs.size());
for (const auto & [job, _] : scheduled_jobs)
scheduled.push_back(job->name);
for (const auto & job : finished_jobs)
finished.push_back(job->name);
LOG_ERROR(log, "Bug. Destruction with pending ({}) and finished ({}) load jobs.", fmt::join(scheduled, ", "), fmt::join(finished, ", "));
abort();
}
void AsyncLoader::start()
@ -236,6 +245,17 @@ void AsyncLoader::wait()
for (auto & p : pools)
p.thread_pool->wait();
lock.lock();
// If there is no way for all jobs to finish, throw LOGICAL_ERROR instead of deadlock
if (!scheduled_jobs.empty() && !hasWorker(lock))
{
std::vector<String> names;
names.reserve(scheduled_jobs.size());
for (const auto & [job, _] : scheduled_jobs)
names.push_back(job->name);
LOG_ERROR(log, "Waiting for load jobs to finish while being stopped: {}.", fmt::join(names, ", "));
abort();
}
}
}
@ -243,10 +263,12 @@ void AsyncLoader::stop()
{
{
std::unique_lock lock{mutex};
is_running = false;
// NOTE: there is no need to notify because workers never wait
is_running = false; // NOTE: there is no need to notify because workers never wait
}
wait();
// Wait for all currently running jobs to finish (and do NOT wait all pending jobs)
for (auto & p : pools)
p.thread_pool->wait();
}
void AsyncLoader::schedule(LoadTask & task)

View File

@ -198,10 +198,6 @@ public:
// Remove all jobs of this task from AsyncLoader.
void remove();
// Do not track jobs in this task.
// WARNING: Jobs will never be removed() and are going to be stored as finished jobs until ~AsyncLoader().
void detach();
// Return the final jobs in this tasks. This job subset should be used as `dependencies` for dependent jobs or tasks:
// auto load_task = loadSomethingAsync(async_loader, load_after_task.goals(), something);
const LoadJobSet & goals() const { return goal_jobs.empty() ? jobs : goal_jobs; }
@ -333,7 +329,6 @@ private:
public:
AsyncLoader(std::vector<PoolInitializer> pool_initializers, bool log_failures_, bool log_progress_);
// Stops AsyncLoader before destruction
// WARNING: all tasks instances should be destructed before associated AsyncLoader.
~AsyncLoader();

View File

@ -6,6 +6,7 @@
#include <Common/PODArray_fwd.h>
#include <base/getPageSize.h>
#include <boost/noncopyable.hpp>
#include <cstdlib>
#include <cstring>
#include <cstddef>
#include <cassert>
@ -238,6 +239,12 @@ public:
resize_assume_reserved(n);
}
template <typename ... TAllocatorParams>
void shrink_to_fit(TAllocatorParams &&... allocator_params)
{
realloc(PODArrayDetails::minimum_memory_for_elements(size(), ELEMENT_SIZE, pad_left, pad_right), std::forward<TAllocatorParams>(allocator_params)...);
}
void resize_assume_reserved(const size_t n) /// NOLINT
{
c_end = c_start + PODArrayDetails::byte_size(n, ELEMENT_SIZE);

View File

@ -1,5 +1,5 @@
#include <Common/SSH/Wrappers.h>
# if USE_SSL
# if USE_SSH
# include <stdexcept>
# pragma GCC diagnostic push

View File

@ -1,7 +1,7 @@
#pragma once
#include <Common/Exception.h>
#include "config.h"
#if USE_SSL
#if USE_SSH
# include <string_view>
# include <base/types.h>

View File

@ -5,6 +5,7 @@
#cmakedefine01 USE_CPUID
#cmakedefine01 USE_BASE64
#cmakedefine01 USE_SSL
#cmakedefine01 USE_SSH
#cmakedefine01 USE_HDFS
#cmakedefine01 USE_AWS_S3
#cmakedefine01 USE_AZURE_BLOB_STORAGE

View File

@ -622,7 +622,13 @@ TEST(AsyncLoader, CustomDependencyFailure)
auto dependent_job1 = makeLoadJob({ collect_job }, "dependent_job1", dependent_job_func);
auto dependent_job2 = makeLoadJob({ collect_job }, "dependent_job2", dependent_job_func);
auto dependent_job3 = makeLoadJob({ collect_job }, "dependent_job3", dependent_job_func);
auto task = t.schedule({ dependent_job1, dependent_job2, dependent_job3 }); // Other jobs should be discovery automatically
auto task = t.schedule({
dependent_job1, dependent_job2, dependent_job3,
collect_job,
late_dep1, late_dep2, late_dep3,
good_dep1, good_dep2, good_dep3,
evil_dep1, evil_dep2, evil_dep3,
});
t.loader.wait(collect_job, true);
canceled_sync.arrive_and_wait(); // (A)
@ -1022,8 +1028,10 @@ TEST(AsyncLoader, SetMaxThreads)
};
// Generate enough independent jobs
std::vector<LoadTaskPtr> tasks;
tasks.reserve(1000);
for (int i = 0; i < 1000; i++)
t.schedule({makeLoadJob({}, "job", job_func)})->detach();
tasks.push_back(t.schedule({makeLoadJob({}, "job", job_func)}));
t.loader.start();
while (sync_index < syncs.size())

View File

@ -601,7 +601,7 @@ Block Block::shrinkToFit() const
{
Columns new_columns(data.size(), nullptr);
for (size_t i = 0; i < data.size(); ++i)
new_columns[i] = data[i].column->shrinkToFit();
new_columns[i] = data[i].column->cloneResized(data[i].column->size());
return cloneWithColumns(new_columns);
}

View File

@ -208,7 +208,7 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt
data[offset - 1] = 0;
}
data.resize(offset);
data.resize_exact(offset);
}

View File

@ -49,8 +49,9 @@ static void writeData(const ISerialization & serialization, const ColumnPtr & co
{
/** If there are columns-constants - then we materialize them.
* (Since the data type does not know how to serialize / deserialize constants.)
* The same for compressed columns in-memory.
*/
ColumnPtr full_column = column->convertToFullColumnIfConst();
ColumnPtr full_column = column->convertToFullColumnIfConst()->decompress();
ISerialization::SerializeBinaryBulkSettings settings;
settings.getter = [&ostr](ISerialization::SubstreamPath) -> WriteBuffer * { return &ostr; };

View File

@ -716,7 +716,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
setEngine(create);
/// We have to check access rights again (in case engine was changed).
if (create.storage)
if (create.storage && create.storage->engine)
{
auto source_access_type = StorageFactory::instance().getSourceAccessType(create.storage->engine->name);
if (source_access_type != AccessType::NONE)

View File

@ -102,6 +102,7 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
extern const int QUERY_WAS_CANCELLED;
extern const int INCORRECT_DATA;
extern const int SUPPORT_IS_DISABLED;
}
@ -709,10 +710,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{
if (settings.dialect == Dialect::kusto && !internal)
{
ParserKQLStatement parser(end, settings.allow_settings_after_format_in_insert);
/// TODO: parser should fail early when max_query_size limit is reached.
ast = parseKQLQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth);
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Kusto dialect is disabled until these two bugs will be fixed: https://github.com/ClickHouse/ClickHouse/issues/59037 and https://github.com/ClickHouse/ClickHouse/issues/59036");
}
else if (settings.dialect == Dialect::prql && !internal)
{

View File

@ -184,7 +184,11 @@ MergeTreeReadTask::BlockAndProgress MergeTreeReadTask::read(const BlockSizeParam
Block block;
if (read_result.num_rows != 0)
{
for (const auto & column : read_result.columns)
column->assumeMutableRef().shrinkToFit();
block = sample_block.cloneWithColumns(read_result.columns);
}
BlockAndProgress res = {
.block = std::move(block),

View File

@ -140,6 +140,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource(
if (storage.supportsSubcolumns())
options.withSubcolumns();
columns_for_reader = storage_snapshot->getColumnsByNames(options, columns_to_read);
}
else
@ -156,6 +157,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource(
read_settings.local_fs_method = LocalFSReadMethod::pread;
if (read_with_direct_io)
read_settings.direct_io_threshold = 1;
/// Configure throttling
switch (type)
{
@ -224,7 +226,10 @@ try
for (size_t i = 0; i < num_columns; ++i)
{
if (header.has(it->name))
{
columns[i]->assumeMutableRef().shrinkToFit();
res_columns.emplace_back(std::move(columns[i]));
}
++it;
}

View File

@ -79,11 +79,11 @@ public:
for (const auto & elem : block)
compressed_block.insert({ elem.column->compress(), elem.type, elem.name });
new_blocks.emplace_back(compressed_block);
new_blocks.push_back(std::move(compressed_block));
}
else
{
new_blocks.emplace_back(block);
new_blocks.push_back(std::move(block));
}
}
@ -472,9 +472,21 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat
while (auto block = block_in.read())
{
new_bytes += block.bytes();
new_rows += block.rows();
new_blocks.push_back(std::move(block));
if (compress)
{
Block compressed_block;
for (const auto & elem : block)
compressed_block.insert({ elem.column->compress(), elem.type, elem.name });
new_blocks.push_back(std::move(compressed_block));
}
else
{
new_blocks.push_back(std::move(block));
}
new_bytes += new_blocks.back().bytes();
new_rows += new_blocks.back().rows();
}
}

View File

@ -1096,6 +1096,8 @@ void StorageReplicatedMergeTree::drop()
/// Table can be shut down, restarting thread is not active
/// and calling StorageReplicatedMergeTree::getZooKeeper()/getAuxiliaryZooKeeper() won't suffice.
zookeeper = getZooKeeperIfTableShutDown();
/// Update zookeeper client, since existing may be expired, while ZooKeeper is required inside dropAllData().
current_zookeeper = zookeeper;
/// If probably there is metadata in ZooKeeper, we don't allow to drop the table.
if (!zookeeper)

View File

@ -164,6 +164,9 @@ endif ()
if (ENABLE_OPENSSL)
set(USE_OPENSSL_INTREE 1)
endif ()
if (TARGET ch_contrib::ssh)
set(USE_SSH 1)
endif()
if (TARGET ch_contrib::fiu)
set(FIU_ENABLE 1)
endif()

View File

@ -1,7 +1,6 @@
test_access_for_functions/test.py::test_access_rights_for_function
test_build_sets_from_multiple_threads/test.py::test_set
test_concurrent_backups_s3/test.py::test_concurrent_backups
test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary
test_distributed_backward_compatability/test.py::test_distributed_in_tuple
test_distributed_type_object/test.py::test_distributed_type_object
test_executable_table_function/test.py::test_executable_function_input_python

View File

@ -213,6 +213,7 @@ class BuildConfig:
"./programs",
"./packages",
"./docker/packager/packager",
"./rust",
],
exclude_files=[".md"],
docker=["clickhouse/binary-builder"],

View File

@ -281,7 +281,7 @@ def test_reload_after_fail_in_cache_dictionary(started_cluster):
query_and_get_error = instance.query_and_get_error
# Can't get a value from the cache dictionary because the source (table `test.xypairs`) doesn't respond.
expected_error = "Table test.xypairs does not exist"
expected_error = "UNKNOWN_TABLE"
update_error = "Could not update cache dictionary cache_xypairs now"
assert expected_error in query_and_get_error(
"SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(1))"

View File

@ -444,10 +444,10 @@ create view query_logs as
create table query_run_metric_arrays engine File(TSV, 'analyze/query-run-metric-arrays.tsv')
as
with (
-- sumMapState with the list of all keys with '-0.' values. Negative zero is because
-- sumMap removes keys with positive zeros.
-- sumMapState with the list of all keys with 'nan' values. 'nan' is because
-- sumMap removes keys with positive/negative zeros.
with (select groupUniqArrayArray(mapKeys(ProfileEvents)) from query_logs) as all_names
select arrayReduce('sumMapState', [(all_names, arrayMap(x->-0., all_names))])
select arrayReduce('sumMapState', [(all_names, arrayMap(x->nan, all_names))])
) as all_metrics
select test, query_index, version, query_id,
(finalizeAggregation(
@ -460,13 +460,13 @@ create table query_run_metric_arrays engine File(TSV, 'analyze/query-run-metric-
),
arrayReduce('sumMapState', [(
['client_time', 'server_time', 'memory_usage'],
arrayMap(x->if(x != 0., x, -0.), [
arrayMap(x->if(x != 0., x, nan), [
toFloat64(query_runs.time),
toFloat64(query_duration_ms / 1000.),
toFloat64(memory_usage)]))])
]
)) as metrics_tuple).1 metric_names,
metrics_tuple.2 metric_values
arrayMap(x->if(isNaN(x),0,x), metrics_tuple.2) metric_values
from query_logs
right join query_runs
on query_logs.query_id = query_runs.query_id

View File

@ -1,4 +0,0 @@
-- test create table --
Theodore
Diaz
Theodore Diaz 28

View File

@ -1,29 +0,0 @@
DROP TABLE IF EXISTS Customers;
CREATE TABLE Customers
(
FirstName Nullable(String),
LastName String,
Occupation String,
Education String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38);
Select '-- test create table --' ;
Select * from kql(Customers|project FirstName) limit 1;;
DROP TABLE IF EXISTS kql_table1;
CREATE TABLE kql_table1 ENGINE = Memory AS select *, now() as new_column From kql(Customers | project LastName | filter LastName=='Diaz');
select LastName from kql_table1 limit 1;
DROP TABLE IF EXISTS kql_table2;
CREATE TABLE kql_table2
(
FirstName Nullable(String),
LastName String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO kql_table2 select * from kql(Customers|project FirstName,LastName,Age | filter FirstName=='Theodore');
select * from kql_table2 limit 1;
-- select * from kql(Customers | where FirstName !in ("test", "test2"));
DROP TABLE IF EXISTS Customers;
DROP TABLE IF EXISTS kql_table1;
DROP TABLE IF EXISTS kql_table2;

View File

@ -1,105 +0,0 @@
-- bool
true
\N
-- int
123
\N
-- long
123
255
-1
\N
456
-- real
0.01
\N
nan
inf
-inf
-- datetime
2015-12-31 23:59:59.900000000
2015-12-31 00:00:00.000000000
2014-05-25 08:20:03.123456000
2014-11-08 15:55:55.000000000
2014-11-08 15:55:00.000000000
2014-11-08 00:00:00.000000000
\N
2014-05-25 08:20:03.123456000
2014-11-08 15:55:55.123456000
-- time
1216984.12345
45055.123
86400
-86400
6.000000000000001e-9
6e-7
172800
259200
-- guid
\N
-- timespan (time)
172800
1800
10
0.1
0.00001
1e-7
1120343
-- null
1
\N \N \N \N \N
-- decimal
\N
123.345
100000
-- dynamic
\N
1
86400
[1,2,3]
[[1],[2],[3]]
['a','b','c']
-- cast functions
true
1
-- tobool("false")
false
1
-- tobool(1)
true
1
-- tobool(123)
true
1
-- tobool("abc")
\N
\N
-- todouble()
123.4
\N
-- toreal()
123.4
\N
-- toint()
1
\N
-- tostring()
123
1
-- todatetime()
1
\N
-- make_timespan()
01:12:00 01:12:30 1.12:30:55
-- totimespan()
1e-7
60
\N
1120343
-- tolong()
123
\N
-- todecimal()
123.345
\N
\N

View File

@ -1,117 +0,0 @@
set dialect = 'kusto';
print '-- bool'
print bool(true);
print bool(true);
print bool(null);
print '-- int';
print int(123);
print int(null);
print int('4'); -- { clientError BAD_ARGUMENTS }
print '-- long';
print long(123);
print long(0xff);
print long(-1);
print long(null);
print 456;
print '-- real';
print real(0.01);
print real(null);
print real(nan);
print real(+inf);
print real(-inf);
print double('4.2'); -- { clientError BAD_ARGUMENTS }
print '-- datetime';
print datetime(2015-12-31 23:59:59.9);
print datetime(2015-12-31);
print datetime('2014-05-25T08:20:03.123456');
print datetime('2014-11-08 15:55:55');
print datetime('2014-11-08 15:55');
print datetime('2014-11-08');
print datetime(null);
print datetime('2014-05-25T08:20:03.123456Z');
print datetime('2014-11-08 15:55:55.123456Z');
print '-- time';
print time('14.02:03:04.12345');
print time('12:30:55.123');
print time(1d);
print time(-1d);
print time(6nanoseconds);
print time(6tick);
print time(2);
print time(2) + 1d;
print '-- guid'
print guid(74be27de-1e4e-49d9-b579-fe0b331d3642);
print guid(null);
print '-- timespan (time)';
print timespan(2d); -- 2 days
--print timespan(1.5h); -- 1.5 hour
print timespan(30m); -- 30 minutes
print timespan(10s); -- 10 seconds
--print timespan(0.1s); -- 0.1 second
print timespan(100ms); -- 100 millisecond
print timespan(10microsecond); -- 10 microseconds
print timespan(1tick); -- 100 nanoseconds
--print timespan(1.5h) / timespan(30m);
print timespan('12.23:12:23') / timespan(1s);
print '-- null';
print isnull(null);
print bool(null), int(null), long(null), real(null), double(null);
print '-- decimal';
print decimal(null);
print decimal(123.345);
print decimal(1e5);
print '-- dynamic'; -- no support for mixed types and bags for now
print dynamic(null);
print dynamic(1);
print dynamic(timespan(1d));
print dynamic([1,2,3]);
print dynamic([[1], [2], [3]]);
print dynamic(['a', "b", 'c']);
print '-- cast functions'
print '--tobool("true")'; -- == true
print tobool('true'); -- == true
print tobool('true') == toboolean('true'); -- == true
print '-- tobool("false")'; -- == false
print tobool('false'); -- == false
print tobool('false') == toboolean('false'); -- == false
print '-- tobool(1)'; -- == true
print tobool(1); -- == true
print tobool(1) == toboolean(1); -- == true
print '-- tobool(123)'; -- == true
print tobool(123); -- == true
print tobool(123) == toboolean(123); -- == true
print '-- tobool("abc")'; -- == null
print tobool('abc'); -- == null
print tobool('abc') == toboolean('abc'); -- == null
print '-- todouble()';
print todouble('123.4');
print todouble('abc') == null;
print '-- toreal()';
print toreal("123.4");
print toreal('abc') == null;
print '-- toint()';
print toint("123") == int(123);
print toint('abc');
print '-- tostring()';
print tostring(123);
print tostring(null) == '';
print '-- todatetime()';
print todatetime("2015-12-24") == datetime(2015-12-24);
print todatetime('abc') == null;
print '-- make_timespan()';
print v1=make_timespan(1,12), v2=make_timespan(1,12,30), v3=make_timespan(1,12,30,55.123);
print '-- totimespan()';
print totimespan(1tick);
print totimespan('0.00:01:00');
print totimespan('abc');
print totimespan('12.23:12:23') / totimespan(1s);
-- print totimespan(strcat('12.', '23', ':12:', '23')) / timespan(1s); -> 1120343
print '-- tolong()';
print tolong('123');
print tolong('abc');
print '-- todecimal()';
print todecimal(123.345);
print todecimal(null);
print todecimal('abc');
-- print todecimal(4 * 2 + 3); -> 11

View File

@ -1,27 +0,0 @@
-- distinct * --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Peter Nara Skilled Manual Graduate Degree 26
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
\N why Professional Partial College 38
-- distinct one column --
Skilled Manual
Management abcd defg
Professional
-- distinct two column --
Skilled Manual Bachelors
Management abcd defg Bachelors
Skilled Manual Graduate Degree
Professional Graduate Degree
Professional Partial College
-- distinct with where --
Skilled Manual Bachelors
Management abcd defg Bachelors
Skilled Manual Graduate Degree
Professional Graduate Degree
Professional Partial College
-- distinct with where, order --
Skilled Manual Bachelors
Skilled Manual Graduate Degree
Professional Graduate Degree

View File

@ -1,28 +0,0 @@
DROP TABLE IF EXISTS Customers;
CREATE TABLE Customers
(
FirstName Nullable(String),
LastName String,
Occupation String,
Education String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28), ('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38);
set dialect = 'kusto';
print '-- distinct * --';
Customers | distinct *;
print '-- distinct one column --';
Customers | distinct Occupation;
print '-- distinct two column --';
Customers | distinct Occupation, Education;
print '-- distinct with where --';
Customers where Age <30 | distinct Occupation, Education;
print '-- distinct with where, order --';
Customers |where Age <30 | order by Age| distinct Occupation, Education;

View File

@ -1,32 +0,0 @@
-- extend #1 --
Aldi Apple 4 2016-09-10 400
Costco Apple 2 2016-09-11 200
-- extend #2 --
Apple 200
Apple 400
-- extend #3 --
Apple cost 480 on average based on 5 samples.
Snargaluff cost 28080 on average based on 5 samples.
-- extend #4 --
1
-- extend #5 --
Aldi Apple 4 2016-09-10 Apple was purchased from Aldi for $4 on 2016-09-10 400
Costco Apple 2 2016-09-11 Apple was purchased from Costco for $2 on 2016-09-11 200
-- extend #6 --
Aldi Apple 2016-09-10 400
Costco Apple 2016-09-11 200
Aldi Apple 2016-09-10 600
Costco Snargaluff 2016-09-12 10000
Aldi Apple 2016-09-12 700
Aldi Snargaluff 2016-09-11 40000
Costco Snargaluff 2016-09-12 10400
Aldi Apple 2016-09-12 500
Aldi Snargaluff 2016-09-11 60000
Costco Snargaluff 2016-09-10 20000
-- extend #7 --
5
-- extend #8 --
-- extend #9 --
-- extend #10 --
-- extend #11 --
5 [2,1]

View File

@ -1,61 +0,0 @@
-- datatable(Supplier:string, Fruit:string, Price: real, Purchase:datetime)
-- [
-- 'Aldi','Apple',4,'2016-09-10',
-- 'Costco','Apple',2,'2016-09-11',
-- 'Aldi','Apple',6,'2016-09-10',
-- 'Costco','Snargaluff',100,'2016-09-12',
-- 'Aldi','Apple',7,'2016-09-12',
-- 'Aldi','Snargaluff',400,'2016-09-11',
-- 'Costco','Snargaluff',104,'2016-09-12',
-- 'Aldi','Apple',5,'2016-09-12',
-- 'Aldi','Snargaluff',600,'2016-09-11',
-- 'Costco','Snargaluff',200,'2016-09-10',
-- ]
DROP TABLE IF EXISTS Ledger;
CREATE TABLE Ledger
(
Supplier Nullable(String),
Fruit String ,
Price Float64,
Purchase Date
) ENGINE = Memory;
INSERT INTO Ledger VALUES ('Aldi','Apple',4,'2016-09-10'), ('Costco','Apple',2,'2016-09-11'), ('Aldi','Apple',6,'2016-09-10'), ('Costco','Snargaluff',100,'2016-09-12'), ('Aldi','Apple',7,'2016-09-12'), ('Aldi','Snargaluff',400,'2016-09-11'),('Costco','Snargaluff',104,'2016-09-12'),('Aldi','Apple',5,'2016-09-12'),('Aldi','Snargaluff',600,'2016-09-11'),('Costco','Snargaluff',200,'2016-09-10');
-- This test requies sorting after some of aggregations but I don't know KQL, sorry
set max_bytes_before_external_group_by = 0;
set dialect = 'kusto';
print '-- extend #1 --';
Ledger | extend PriceInCents = 100 * Price | take 2;
print '-- extend #2 --';
Ledger | extend PriceInCents = 100 * Price | sort by PriceInCents asc | project Fruit, PriceInCents | take 2;
print '-- extend #3 --';
Ledger | extend PriceInCents = 100 * Price | sort by PriceInCents asc | project Fruit, PriceInCents | summarize AveragePrice = avg(PriceInCents), Purchases = count() by Fruit | extend Sentence = strcat(Fruit, ' cost ', tostring(AveragePrice), ' on average based on ', tostring(Purchases), ' samples.') | project Sentence;
print '-- extend #4 --';
Ledger | extend a = Price | extend b = a | extend c = a, d = b + 500 | extend Pass = bool(b == a and c == a and d == b + 500) | summarize binary_all_and(Pass);
print '-- extend #5 --';
Ledger | take 2 | extend strcat(Fruit, ' was purchased from ', Supplier, ' for $', tostring(Price), ' on ', tostring(Purchase)) | extend PriceInCents = 100 * Price;
print '-- extend #6 --';
Ledger | extend Price = 100 * Price;
print '-- extend #7 --';
print a = 4 | extend a = 5;
print '-- extend #8 --';
-- print x = 5 | extend array_sort_desc(range(0, x), range(1, x + 1))
print '-- extend #9 --';
print x = 19 | extend = 4 + ; -- { clientError SYNTAX_ERROR }
print '-- extend #10 --';
Ledger | extend PriceInCents = * Price | sort by PriceInCents asc | project Fruit, PriceInCents | summarize AveragePrice = avg(PriceInCents), Purchases = count() by Fruit | extend Sentence = strcat(Fruit, ' cost ', tostring(AveragePrice), ' on average based on ', tostring(Purchases), ' samples.') | project Sentence; -- { clientError SYNTAX_ERROR }
print '-- extend #11 --'; -- should ideally return this in the future: 5 [2,1] because of the alias ex
print x = 5 | extend ex = array_sort_desc(dynamic([1, 2]), dynamic([3, 4]));

View File

@ -1,7 +0,0 @@
-- binary functions
4 7
1
1
1
7 3
1

View File

@ -1,8 +0,0 @@
set dialect='kusto';
print ' -- binary functions';
print binary_and(4,7), binary_or(4,7);
print binary_shift_left(1, 1) == binary_shift_left(1, 65);
print binary_shift_right(2, 1) == binary_shift_right(2, 65);
print binary_shift_right(binary_shift_left(1, 65), 65) == 1;
print binary_xor(2, 5), bitset_count_ones(42);
print bitset_count_ones(binary_shift_left(binary_and(4,7), 1));

View File

@ -1,76 +0,0 @@
-- dayofmonth()
31
-- dayofweek()
4.00:00:00
-- dayofyear()
365
-- getmonth()
10
-- getyear()
2015
-- hoursofday()
23
-- startofday()
2017-01-01 00:00:00.000000000
2016-12-31 00:00:00.000000000
2017-01-02 00:00:00.000000000
-- endofday()
2017-01-01 23:59:59.999999000
2016-12-31 23:59:59.999999000
2017-01-02 23:59:59.999999000
-- endofmonth()
2017-01-31 23:59:59.999999000
2016-12-31 23:59:59.999999000
2017-02-28 23:59:59.999999000
2022-09-30 23:59:59.999999000
-- startofweek()
2017-01-01 00:00:00.000000000
2016-12-25 00:00:00.000000000
2017-01-08 00:00:00.000000000
-- endofweek()
2017-01-07 23:59:59.999999000
2016-12-31 23:59:59.999999000
2017-01-14 23:59:59.999999000
-- startofyear()
2017-01-01 00:00:00.000000000
2016-01-01 00:00:00.000000000
2018-01-01 00:00:00.000000000
-- endofyear()
2017-12-31 23:59:59.999999000
2016-12-31 23:59:59.999999000
2018-12-31 23:59:59.999999000
-- unixtime_seconds_todatetime()
2019-01-01 00:00:00.000000000
1970-01-02 00:00:00.000000000
1969-12-31 00:00:00.000000000
-- unixtime_microseconds_todatetime
2019-01-01 00:00:00.000000
-- unixtime_milliseconds_todatetime()
2019-01-01 00:00:00.000
-- unixtime_nanoseconds_todatetime()
2019-01-01 00:00:00.000000000
-- weekofyear()
52
-- monthofyear()
12
-- weekofyear()
52
-- now()
1
-- make_datetime()
1
2017-10-01 12:10:00.0000000
2017-10-01 12:11:00.0000000
-- format_datetime
15-12-14 02:03:04.1234500
17-01-29 [09:00:05] 2017-01-29 [09:00:05] 17-01-29 [09:00:05 AM]
-- format_timespan()
02:03:04.1234500
29.09:00:05:12
-- ago()
-- datetime_diff()
17 2 13 4 29 2 5 10
-- datetime_part()
2017 4 10 44 30 303 01 02 03
-- datetime_add()
2018-01-01 00:00:00.0000000 2017-04-01 00:00:00.0000000 2017-02-01 00:00:00.0000000 2017-01-08 00:00:00.0000000 2017-01-02 00:00:00.0000000 2017-01-01 01:00:00.0000000 2017-01-01 00:01:00.0000000 2017-01-01 00:00:01.0000000

View File

@ -1,86 +0,0 @@
set dialect = 'kusto';
print '-- dayofmonth()';
print dayofmonth(datetime(2015-12-31));
print '-- dayofweek()';
print dayofweek(datetime(2015-12-31));
print '-- dayofyear()';
print dayofyear(datetime(2015-12-31));
print '-- getmonth()';
print getmonth(datetime(2015-10-12));
print '-- getyear()';
print getyear(datetime(2015-10-12));
print '-- hoursofday()';
print hourofday(datetime(2015-12-31 23:59:59.9));
print '-- startofday()';
print startofday(datetime(2017-01-01 10:10:17));
print startofday(datetime(2017-01-01 10:10:17), -1);
print startofday(datetime(2017-01-01 10:10:17), 1);
print '-- endofday()';
print endofday(datetime(2017-01-01 10:10:17));
print endofday(datetime(2017-01-01 10:10:17), -1);
print endofday(datetime(2017-01-01 10:10:17), 1);
print '-- endofmonth()';
print endofmonth(datetime(2017-01-01 10:10:17));
print endofmonth(datetime(2017-01-01 10:10:17), -1);
print endofmonth(datetime(2017-01-01 10:10:17), 1);
print endofmonth(datetime(2022-09-23));
print '-- startofweek()';
print startofweek(datetime(2017-01-01 10:10:17));
print startofweek(datetime(2017-01-01 10:10:17), -1);
print startofweek(datetime(2017-01-01 10:10:17), 1);
print '-- endofweek()';
print endofweek(datetime(2017-01-01 10:10:17));
print endofweek(datetime(2017-01-01 10:10:17), -1);
print endofweek(datetime(2017-01-01 10:10:17), 1);
print '-- startofyear()';
print startofyear(datetime(2017-01-01 10:10:17));
print startofyear(datetime(2017-01-01 10:10:17), -1);
print startofyear(datetime(2017-01-01 10:10:17), 1);
print '-- endofyear()';
print endofyear(datetime(2017-01-01 10:10:17));
print endofyear(datetime(2017-01-01 10:10:17), -1);
print endofyear(datetime(2017-01-01 10:10:17), 1);
print '-- unixtime_seconds_todatetime()';
print unixtime_seconds_todatetime(1546300800);
print unixtime_seconds_todatetime(1d);
print unixtime_seconds_todatetime(-1d);
print '-- unixtime_microseconds_todatetime';
print unixtime_microseconds_todatetime(1546300800000000);
print '-- unixtime_milliseconds_todatetime()';
print unixtime_milliseconds_todatetime(1546300800000);
print '-- unixtime_nanoseconds_todatetime()';
print unixtime_nanoseconds_todatetime(1546300800000000000);
print '-- weekofyear()';
print week_of_year(datetime(2000-01-01));
print '-- monthofyear()';
print monthofyear(datetime(2015-12-31));
print '-- weekofyear()';
print week_of_year(datetime(2000-01-01));
print '-- now()';
print getyear(now(-2d))>1900;
print '-- make_datetime()';
print make_datetime(2017,10,01,12,10) == datetime(2017-10-01 12:10:00);
print year_month_day_hour_minute = make_datetime(2017,10,01,12,10);
print year_month_day_hour_minute_second = make_datetime(2017,10,01,12,11,0.1234567);
print '-- format_datetime';
print format_datetime(datetime(2015-12-14 02:03:04.12345), 'y-M-d h:m:s.fffffff');
print v1=format_datetime(datetime(2017-01-29 09:00:05),'yy-MM-dd [HH:mm:ss]'), v2=format_datetime(datetime(2017-01-29 09:00:05), 'yyyy-M-dd [H:mm:ss]'), v3=format_datetime(datetime(2017-01-29 09:00:05), 'yy-MM-dd [hh:mm:ss tt]');
print '-- format_timespan()';
print format_timespan(time('14.02:03:04.12345'), 'h:m:s.fffffff');
print v1=format_timespan(time('29.09:00:05.12345'), 'dd.hh:mm:ss:FF');
-- print v2=format_timespan(time('29.09:00:05.12345'), 'ddd.h:mm:ss [fffffff]'); == '029.9:00:05 [1234500]'
print '-- ago()';
-- print ago(1d) - now();
print '-- datetime_diff()';
print year = datetime_diff('year',datetime(2017-01-01),datetime(2000-12-31)), quarter = datetime_diff('quarter',datetime(2017-07-01),datetime(2017-03-30)), month = datetime_diff('month',datetime(2017-01-01),datetime(2015-12-30)), week = datetime_diff('week',datetime(2017-10-29 00:00),datetime(2017-09-30 23:59)), day = datetime_diff('day',datetime(2017-10-29 00:00),datetime(2017-09-30 23:59)), hour = datetime_diff('hour',datetime(2017-10-31 01:00),datetime(2017-10-30 23:59)), minute = datetime_diff('minute',datetime(2017-10-30 23:05:01),datetime(2017-10-30 23:00:59)), second = datetime_diff('second',datetime(2017-10-30 23:00:10.100),datetime(2017-10-30 23:00:00.900));
-- millisecond = datetime_diff('millisecond',datetime(2017-10-30 23:00:00.200100),datetime(2017-10-30 23:00:00.100900)),
-- microsecond = datetime_diff('microsecond',datetime(2017-10-30 23:00:00.1009001),datetime(2017-10-30 23:00:00.1008009)),
-- nanosecond = datetime_diff('nanosecond',datetime(2017-10-30 23:00:00.0000000),datetime(2017-10-30 23:00:00.0000007))
print '-- datetime_part()';
print year = datetime_part("year", datetime(2017-10-30 01:02:03.7654321)),quarter = datetime_part("quarter", datetime(2017-10-30 01:02:03.7654321)),month = datetime_part("month", datetime(2017-10-30 01:02:03.7654321)),weekOfYear = datetime_part("week_of_year", datetime(2017-10-30 01:02:03.7654321)),day = datetime_part("day", datetime(2017-10-30 01:02:03.7654321)),dayOfYear = datetime_part("dayOfYear", datetime(2017-10-30 01:02:03.7654321)),hour = datetime_part("hour", datetime(2017-10-30 01:02:03.7654321)),minute = datetime_part("minute", datetime(2017-10-30 01:02:03.7654321)),second = datetime_part("second", datetime(2017-10-30 01:02:03.7654321));
-- millisecond = datetime_part("millisecond", dt),
-- microsecond = datetime_part("microsecond", dt),
-- nanosecond = datetime_part("nanosecond", dt)
print '-- datetime_add()';
print year = datetime_add('year',1,make_datetime(2017,1,1)),quarter = datetime_add('quarter',1,make_datetime(2017,1,1)),month = datetime_add('month',1,make_datetime(2017,1,1)),week = datetime_add('week',1,make_datetime(2017,1,1)),day = datetime_add('day',1,make_datetime(2017,1,1)),hour = datetime_add('hour',1,make_datetime(2017,1,1)),minute = datetime_add('minute',1,make_datetime(2017,1,1)),second = datetime_add('second',1,make_datetime(2017,1,1));

View File

@ -1,152 +0,0 @@
-- constant index value
1 c ['A',NULL,'C']
-- array_length()
1
1
-- array_sum()
1
1
-- array_index_of()
3
1
-- array_iif()
[1,5,3]
[1,5,3]
[1,5,NULL]
[NULL,NULL,NULL]
-- array_concat()
[1,2,3,4,5,6]
-- array_reverse()
[]
[1]
[4,3,2,1]
['example','an','is','this']
-- array_rotate_left()
[]
[]
[]
[3,4,5,1,2]
[1,2,3,4,5]
[3,4,5,1,2]
[4,5,1,2,3]
[1,2,3,4,5]
[4,5,1,2,3]
-- array_rotate_right()
[]
[]
[]
[4,5,1,2,3]
[1,2,3,4,5]
[4,5,1,2,3]
[3,4,5,1,2]
[1,2,3,4,5]
[3,4,5,1,2]
-- array_shift_left()
[]
[]
[]
[3,4,5,NULL,NULL]
[NULL,NULL,1,2,3]
[3,4,5,-1,-1]
['c','','']
-- array_shift_right()
[]
[]
[]
[3,4,5,NULL,NULL]
[NULL,NULL,1,2,3]
[3,4,5,-1,-1]
['c','','']
-- array_slice()
[3,4]
-- array_split()
[[1],[2,3],[4,5]]
[[1,2],[3,4,5]]
[[1],[2,3],[4,5]]
[[1,2,3,4],[],[4,5]]
-- array_sort_asc()
(['a','c','c','d',NULL])
([1,2,3,4])
['a','b','c']
(['p','q','r'],['hello','clickhouse','world'])
([NULL,'a','c','c','d'])
([NULL,'a','c','c','d'])
([NULL,NULL,NULL])
[1,2,3,NULL,NULL]
['a','e','b','c','d']
(['George','John','Paul','Ringo'])
(['blue','green','yellow',NULL,NULL])
([NULL,NULL,'blue','green','yellow'])
-- array_sort_desc()
(['d','c','c','a',NULL])
([4,3,2,1])
['c','b','a']
(['r','q','p'],['world','clickhouse','hello'])
([NULL,'d','c','c','a'])
([NULL,'d','c','c','a'])
([NULL,NULL,NULL])
[3,2,1,NULL,NULL]
['d','c','b','e','a']
(['Ringo','Paul','John','George'])
(['yellow','green','blue',NULL,NULL])
([NULL,NULL,'yellow','green','blue'])
-- jaccard_index()
0.75
0
0
nan
0
0.75
0.25
-- pack_array()
1 2 4 [1,2,4]
['ab','0.0.0.42','4.2']
-- repeat()
[]
[1,1,1]
['asd','asd','asd']
[86400,86400,86400]
[true,true,true]
[NULL]
[NULL]
-- set_difference()
[]
[]
[]
[]
[4,5,6]
[4]
[1,3]
[1,2,3]
['d','s']
['Chewbacca','Han Solo']
-- set_has_element()
0
1
0
1
0
-- set_intersect()
[]
[1,2,3]
[1,2,3]
[]
[5]
[]
['a']
['Darth Vader']
-- set_union()
[]
[1,2,3]
[1,2,3,4,5,6]
[1,2,3,4]
[1,2,3,4,5]
[1,2,3]
['a','d','f','s']
['Chewbacca','Darth Sidious','Darth Vader','Han Solo']
-- zip()
[]
[[1,2],[3,4],[5,6]]
[['Darth','Vader','has a suit'],['Master','Yoda','doesn\'t have a suit']]
[[1,10],[2,20],[3,NULL]]
[[NULL,1],[NULL,2],[NULL,3]]

View File

@ -1,161 +0,0 @@
DROP TABLE IF EXISTS array_test;
CREATE TABLE array_test (floats Array(Float64),
strings Array(String),
nullable_strings Array(Nullable(String))
) ENGINE=Memory;
INSERT INTO array_test VALUES([1.0, 2.5], ['a', 'c'], ['A', NULL, 'C']);
set dialect = 'kusto';
print '-- constant index value';
array_test | project floats[0], strings[1], nullable_strings;
print '-- array_length()';
print array_length(dynamic(['John', 'Denver', 'Bob', 'Marley'])) == 4;
print array_length(dynamic([1, 2, 3])) == 3;
print '-- array_sum()';
print array_sum(dynamic([2, 5, 3])) == 10;
print array_sum(dynamic([2.5, 5.5, 3])) == 11;
print '-- array_index_of()';
print array_index_of(dynamic(['John', 'Denver', 'Bob', 'Marley']), 'Marley');
print array_index_of(dynamic([1, 2, 3]), 2);
print '-- array_iif()';
print array_iif(dynamic([true,false,true]), dynamic([1,2,3]), dynamic([4,5,6]));
print array_iif(dynamic([1,0,1]), dynamic([1,2,3]), dynamic([4,5,6]));
print array_iif(dynamic([true,false,true]), dynamic([1,2]), dynamic([4,5,6]));
print array_iif(dynamic(['a','b','c']), dynamic([1,2,3]), dynamic([4,5,6]));
print '-- array_concat()';
print array_concat(dynamic([1,2,3]),dynamic([4,5,6]));
print '-- array_reverse()';
print array_reverse(dynamic([]));
print array_reverse(dynamic([1]));
print array_reverse(dynamic([1,2,3,4]));
print array_reverse(dynamic(["this", "is", "an", "example"]));
print '-- array_rotate_left()';
print array_rotate_left(dynamic([]), 0);
print array_rotate_left(dynamic([]), 500);
print array_rotate_left(dynamic([]), -500);
print array_rotate_left(dynamic([1,2,3,4,5]), 2);
print array_rotate_left(dynamic([1,2,3,4,5]), 5);
print array_rotate_left(dynamic([1,2,3,4,5]), 7);
print array_rotate_left(dynamic([1,2,3,4,5]), -2);
print array_rotate_left(dynamic([1,2,3,4,5]), -5);
print array_rotate_left(dynamic([1,2,3,4,5]), -7);
print '-- array_rotate_right()';
print array_rotate_right(dynamic([]), 0);
print array_rotate_right(dynamic([]), 500);
print array_rotate_right(dynamic([]), -500);
print array_rotate_right(dynamic([1,2,3,4,5]), 2);
print array_rotate_right(dynamic([1,2,3,4,5]), 5);
print array_rotate_right(dynamic([1,2,3,4,5]), 7);
print array_rotate_right(dynamic([1,2,3,4,5]), -2);
print array_rotate_right(dynamic([1,2,3,4,5]), -5);
print array_rotate_right(dynamic([1,2,3,4,5]), -7);
print '-- array_shift_left()';
print array_shift_left(dynamic([]), 0);
print array_shift_left(dynamic([]), 555);
print array_shift_left(dynamic([]), -555);
print array_shift_left(dynamic([1,2,3,4,5]), 2);
print array_shift_left(dynamic([1,2,3,4,5]), -2);
print array_shift_left(dynamic([1,2,3,4,5]), 2, -1);
print array_shift_left(dynamic(['a', 'b', 'c']), 2);
print '-- array_shift_right()';
print array_shift_left(dynamic([]), 0);
print array_shift_left(dynamic([]), 555);
print array_shift_left(dynamic([]), -555);
print array_shift_right(dynamic([1,2,3,4,5]), -2);
print array_shift_right(dynamic([1,2,3,4,5]), 2);
print array_shift_right(dynamic([1,2,3,4,5]), -2, -1);
print array_shift_right(dynamic(['a', 'b', 'c']), -2);
print '-- array_slice()';
--print array_slice(dynamic([1,2,3]), 1, 2); -- will enable whe analyzer dixed
print array_slice(dynamic([1,2,3,4,5]), -3, -2);
print '-- array_split()';
print array_split(dynamic([1,2,3,4,5]), dynamic([1,-2]));
print array_split(dynamic([1,2,3,4,5]), 2);
print array_split(dynamic([1,2,3,4,5]), dynamic([1,3]));
print array_split(dynamic([1,2,3,4,5]), dynamic([-1,-2]));
print '-- array_sort_asc()';
print array_sort_asc(dynamic([null, 'd', 'a', 'c', 'c']));
print array_sort_asc(dynamic([4, 1, 3, 2]));
print array_sort_asc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))[0];
print array_sort_asc(dynamic(['q', 'p', 'r']), dynamic(['clickhouse','hello', 'world']));
print array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , false);
print array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2);
print array_sort_asc( dynamic([null, null, null]) , false);
print array_sort_asc(dynamic([2, 1, null,3, null]), dynamic([20, 10, 40, 30, 50]), 1 < 2)[0];
print array_sort_asc(dynamic(['1','3','4','5','2']),dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]))[3];
print array_sort_asc(split("John,Paul,George,Ringo", ","));
print array_sort_asc(dynamic([null,"blue","yellow","green",null]));
print array_sort_asc(dynamic([null,"blue","yellow","green",null]), false);
print '-- array_sort_desc()';
print array_sort_desc(dynamic([null, 'd', 'a', 'c', 'c']));
print array_sort_desc(dynamic([4, 1, 3, 2]));
print array_sort_desc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))[0];
print array_sort_desc(dynamic(['q', 'p', 'r']), dynamic(['clickhouse','hello', 'world']));
print array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , false);
print array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2);
print array_sort_desc( dynamic([null, null, null]) , false);
print array_sort_desc(dynamic([2, 1, null,3, null]), dynamic([20, 10, 40, 30, 50]), 1 < 2)[0];
print array_sort_desc(dynamic(['1','3','4','5','2']),dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]))[3];
print array_sort_desc(split("John,Paul,George,Ringo", ","));
print array_sort_desc(dynamic([null,"blue","yellow","green",null]));
print array_sort_desc(dynamic([null,"blue","yellow","green",null]), false);
print '-- jaccard_index()';
print jaccard_index(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3, 4, 4, 4]));
print jaccard_index(dynamic([1, 2, 3]), dynamic([]));
print jaccard_index(dynamic([]), dynamic([1, 2, 3, 4]));
print jaccard_index(dynamic([]), dynamic([]));
print jaccard_index(dynamic([1, 2, 3]), dynamic([4, 5, 6, 7]));
print jaccard_index(dynamic(['a', 's', 'd']), dynamic(['f', 'd', 's', 'a']));
print jaccard_index(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader']));
print '-- pack_array()';
print pack_array(); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
print x = 1 | extend y = x * 2 | extend z = y * 2 | extend pack_array(x,y,z);
print pack_array(strcat('a', 'b'), format_ipv4(42), tostring(4.2));
print '-- repeat()';
print repeat(1, 0);
print repeat(1, 3);
print repeat("asd", 3);
print repeat(timespan(1d), 3);
print repeat(true, 3);
print repeat(1, -3);
print repeat(6.7,-4);
print '-- set_difference()';
print set_difference(dynamic([]), dynamic([]));
print set_difference(dynamic([]), dynamic([9]));
print set_difference(dynamic([]), dynamic(["asd"]));
print set_difference(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3]));
print array_sort_asc(set_difference(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[0];
print set_difference(dynamic([4]), dynamic([1, 2, 3]));
print array_sort_asc(set_difference(dynamic([1, 2, 3, 4, 5]), dynamic([5]), dynamic([2, 4])))[0];
print array_sort_asc(set_difference(dynamic([1, 2, 3]), dynamic([])))[0];
print array_sort_asc(set_difference(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])))[0];
print array_sort_asc(set_difference(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])))[0];
print '-- set_has_element()';
print set_has_element(dynamic([]), 9);
print set_has_element(dynamic(["this", "is", "an", "example"]), "example");
print set_has_element(dynamic(["this", "is", "an", "example"]), "examplee");
print set_has_element(dynamic([1, 2, 3]), 2);
print set_has_element(dynamic([1, 2, 3, 4.2]), 4);
print '-- set_intersect()';
print set_intersect(dynamic([]), dynamic([]));
print array_sort_asc(set_intersect(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])))[0];
print array_sort_asc(set_intersect(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[0];
print set_intersect(dynamic([4]), dynamic([1, 2, 3]));
print set_intersect(dynamic([1, 2, 3, 4, 5]), dynamic([1, 3, 5]), dynamic([2, 5]));
print set_intersect(dynamic([1, 2, 3]), dynamic([]));
print set_intersect(dynamic(['a', 's', 'd']), dynamic(['a', 'f']));
print set_intersect(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader']));
print '-- set_union()';
print set_union(dynamic([]), dynamic([]));
print array_sort_asc(set_union(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])))[0];
print array_sort_asc(set_union(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[0];
print array_sort_asc(set_union(dynamic([4]), dynamic([1, 2, 3])))[0];
print array_sort_asc(set_union(dynamic([1, 3, 4]), dynamic([5]), dynamic([2, 4])))[0];
print array_sort_asc(set_union(dynamic([1, 2, 3]), dynamic([])))[0];
print array_sort_asc(set_union(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])))[0];
print array_sort_asc(set_union(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])))[0];
print '-- zip()';
print zip(dynamic([]), dynamic([]));
print zip(dynamic([1,3,5]), dynamic([2,4,6]));
print zip(dynamic(['Darth','Master']), dynamic(['Vader','Yoda']), dynamic(['has a suit','doesn\'t have a suit']));
print zip(dynamic([1,2,3]), dynamic([10,20]));
print zip(dynamic([]), dynamic([1,2,3]));

View File

@ -1,123 +0,0 @@
-- ipv4_is_private(\'127.0.0.1\')
0
-- ipv4_is_private(\'10.1.2.3\')
1
-- ipv4_is_private(\'192.168.1.1/24\')
1
ipv4_is_private(strcat(\'192.\',\'168.\',\'1.\',\'1\',\'/24\'))
1
-- ipv4_is_private(\'abc\')
\N
-- ipv4_netmask_suffix(\'192.168.1.1/24\')
24
-- ipv4_netmask_suffix(\'192.168.1.1\')
32
-- ipv4_netmask_suffix(\'127.0.0.1/16\')
16
-- ipv4_netmask_suffix(\'abc\')
\N
ipv4_netmask_suffix(strcat(\'127.\', \'0.\', \'0.1/16\'))
16
-- ipv4_is_in_range(\'127.0.0.1\', \'127.0.0.1\')
1
-- ipv4_is_in_range(\'192.168.1.6\', \'192.168.1.1/24\')
1
-- ipv4_is_in_range(\'192.168.1.1\', \'192.168.2.1/24\')
0
-- ipv4_is_in_range(strcat(\'192.\',\'168.\', \'1.1\'), \'192.168.2.1/24\')
0
-- ipv4_is_in_range(\'abc\', \'127.0.0.1\')
\N
-- parse_ipv6(127.0.0.1)
0000:0000:0000:0000:0000:ffff:7f00:0001
-- parse_ipv6(fe80::85d:e82c:9446:7994)
fe80:0000:0000:0000:085d:e82c:9446:7994
-- parse_ipv4(\'127.0.0.1\')
2130706433
-- parse_ipv4(\'192.1.168.1\') < parse_ipv4(\'192.1.168.2\')
1
-- parse_ipv4(arrayStringConcat([\'127\', \'0\', \'0\', \'1\'], \'.\'))
-- parse_ipv4_mask(\'127.0.0.1\', 24) == 2130706432
2130706432
-- parse_ipv4_mask(\'abc\', 31)
\N
\N
-- parse_ipv4_mask(\'192.1.168.2\', 31) == parse_ipv4_mask(\'192.1.168.3\', 31)
3221334018
3221334018
-- ipv4_is_match(\'127.0.0.1\', \'127.0.0.1\')
1
-- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\')
0
-- ipv4_is_match(\'192.168.1.1/24\', \'192.168.1.255/24\')
1
-- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\', 24)
1
-- ipv4_is_match(\'abc\', \'def\', 24)
\N
-- ipv4_compare()
0
-1
1
0
0
0
0
0
0
0
0
-- format_ipv4()
192.168.1.0
192.168.1.1
192.168.1.0
192.168.1.0
1
1
127.0.0.0
-- format_ipv4_mask()
192.168.1.0/24
192.168.1.0/24
192.168.1.0/24
192.168.1.1/32
192.168.1.0/24
1
1
127.0.0.0/24
-- parse_ipv6_mask()
0000:0000:0000:0000:0000:0000:0000:0000
fe80:0000:0000:0000:085d:e82c:9446:7900
0000:0000:0000:0000:0000:ffff:c0a8:ff00
0000:0000:0000:0000:0000:ffff:c0a8:ff00
0000:0000:0000:0000:0000:ffff:ffff:ffff
fe80:0000:0000:0000:085d:e82c:9446:7994
fe80:0000:0000:0000:085d:e82c:9446:7900
0000:0000:0000:0000:0000:ffff:c0a8:ffff
0000:0000:0000:0000:0000:ffff:c0a8:ff00
-- ipv6_is_match()
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1

View File

@ -1,131 +0,0 @@
set dialect='kusto';
print '-- ipv4_is_private(\'127.0.0.1\')';
print ipv4_is_private('127.0.0.1');
print '-- ipv4_is_private(\'10.1.2.3\')';
print ipv4_is_private('10.1.2.3');
print '-- ipv4_is_private(\'192.168.1.1/24\')';
print ipv4_is_private('192.168.1.1/24');
print 'ipv4_is_private(strcat(\'192.\',\'168.\',\'1.\',\'1\',\'/24\'))';
print ipv4_is_private(strcat('192.','168.','1.','1','/24'));
print '-- ipv4_is_private(\'abc\')';
print ipv4_is_private('abc'); -- == null
print '-- ipv4_netmask_suffix(\'192.168.1.1/24\')';
print ipv4_netmask_suffix('192.168.1.1/24'); -- == 24
print '-- ipv4_netmask_suffix(\'192.168.1.1\')';
print ipv4_netmask_suffix('192.168.1.1'); -- == 32
print '-- ipv4_netmask_suffix(\'127.0.0.1/16\')';
print ipv4_netmask_suffix('127.0.0.1/16'); -- == 16
print '-- ipv4_netmask_suffix(\'abc\')';
print ipv4_netmask_suffix('abc'); -- == null
print 'ipv4_netmask_suffix(strcat(\'127.\', \'0.\', \'0.1/16\'))';
print ipv4_netmask_suffix(strcat('127.', '0.', '0.1/16')); -- == 16
print '-- ipv4_is_in_range(\'127.0.0.1\', \'127.0.0.1\')';
print ipv4_is_in_range('127.0.0.1', '127.0.0.1'); -- == true
print '-- ipv4_is_in_range(\'192.168.1.6\', \'192.168.1.1/24\')';
print ipv4_is_in_range('192.168.1.6', '192.168.1.1/24'); -- == true
print '-- ipv4_is_in_range(\'192.168.1.1\', \'192.168.2.1/24\')';
print ipv4_is_in_range('192.168.1.1', '192.168.2.1/24'); -- == false
print '-- ipv4_is_in_range(strcat(\'192.\',\'168.\', \'1.1\'), \'192.168.2.1/24\')';
print ipv4_is_in_range(strcat('192.','168.', '1.1'), '192.168.2.1/24'); -- == false
print '-- ipv4_is_in_range(\'abc\', \'127.0.0.1\')'; -- == null
print ipv4_is_in_range('abc', '127.0.0.1');
print '-- parse_ipv6(127.0.0.1)';
print parse_ipv6('127.0.0.1');
print '-- parse_ipv6(fe80::85d:e82c:9446:7994)';
print parse_ipv6('fe80::85d:e82c:9446:7994');
print '-- parse_ipv4(\'127.0.0.1\')';
print parse_ipv4('127.0.0.1');
print '-- parse_ipv4(\'192.1.168.1\') < parse_ipv4(\'192.1.168.2\')';
print parse_ipv4('192.1.168.1') < parse_ipv4('192.1.168.2');
print '-- parse_ipv4(arrayStringConcat([\'127\', \'0\', \'0\', \'1\'], \'.\'))';
print parse_ipv4(arrayStringConcat(['127', '0', '0', '1'], '.')); -- { clientError UNKNOWN_FUNCTION }
print '-- parse_ipv4_mask(\'127.0.0.1\', 24) == 2130706432';
print parse_ipv4_mask('127.0.0.1', 24);
print '-- parse_ipv4_mask(\'abc\', 31)';
print parse_ipv4_mask('abc', 31)
print '-- parse_ipv4_mask(\'192.1.168.2\', 1000)';
print parse_ipv4_mask('192.1.168.2', 1000);
print '-- parse_ipv4_mask(\'192.1.168.2\', 31) == parse_ipv4_mask(\'192.1.168.3\', 31)';
--print parse_ipv4_mask('192.1.168.2', 31) == parse_ipv4_mask('192.1.168.3', 31); // this qual failed in analyzer 3221334018
print parse_ipv4_mask('192.1.168.2', 31);
print parse_ipv4_mask('192.1.168.3', 31);
print '-- ipv4_is_match(\'127.0.0.1\', \'127.0.0.1\')';
print ipv4_is_match('127.0.0.1', '127.0.0.1');
print '-- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\')';
print ipv4_is_match('192.168.1.1', '192.168.1.255');
print '-- ipv4_is_match(\'192.168.1.1/24\', \'192.168.1.255/24\')';
print ipv4_is_match('192.168.1.1/24', '192.168.1.255/24');
print '-- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\', 24)';
print ipv4_is_match('192.168.1.1', '192.168.1.255', 24);
print '-- ipv4_is_match(\'abc\', \'def\', 24)';
print ipv4_is_match('abc', 'dev', 24);
print '-- ipv4_compare()';
print ipv4_compare('127.0.0.1', '127.0.0.1');
print ipv4_compare('192.168.1.1', '192.168.1.255');
print ipv4_compare('192.168.1.255', '192.168.1.1');
print ipv4_compare('192.168.1.1/24', '192.168.1.255/24');
print ipv4_compare('192.168.1.1', '192.168.1.255', 24);
print ipv4_compare('192.168.1.1/24', '192.168.1.255');
print ipv4_compare('192.168.1.1', '192.168.1.255/24');
print ipv4_compare('192.168.1.1/30', '192.168.1.255/24');
print ipv4_compare('192.168.1.1', '192.168.1.0', 31);
print ipv4_compare('192.168.1.1/24', '192.168.1.255', 31);
print ipv4_compare('192.168.1.1', '192.168.1.255', 24);
print '-- format_ipv4()';
print format_ipv4('192.168.1.255', 24);
print format_ipv4('192.168.1.1', 32);
print format_ipv4('192.168.1.1/24', 32);
print format_ipv4(3232236031, 24);
print format_ipv4('192.168.1.1/24', -1) == '';
print format_ipv4('abc', 24) == '';
print format_ipv4(strcat('127.0', '.0.', '1', '/32'), 12 + 12);
print '-- format_ipv4_mask()';
print format_ipv4_mask('192.168.1.255', 24);
print format_ipv4_mask(3232236031, 24);
print format_ipv4_mask('192.168.1.1', 24);
print format_ipv4_mask('192.168.1.1', 32);
print format_ipv4_mask('192.168.1.1/24', 32);
print format_ipv4_mask('192.168.1.1/24', -1) == '';
print format_ipv4_mask('abc', 24) == '';
print format_ipv4_mask(strcat('127.0', '.0.', '1', '/32'), 12 + 12);
print '-- parse_ipv6_mask()';
print parse_ipv6_mask("127.0.0.1", 24);
print parse_ipv6_mask("fe80::85d:e82c:9446:7994", 120);
print parse_ipv6_mask("192.168.255.255", 120);
print parse_ipv6_mask("192.168.255.255/24", 124);
print parse_ipv6_mask("255.255.255.255", 128);
print parse_ipv6_mask("fe80::85d:e82c:9446:7994", 128);
print parse_ipv6_mask("fe80::85d:e82c:9446:7994/120", 124);
print parse_ipv6_mask("::192.168.255.255", 128);
print parse_ipv6_mask("::192.168.255.255/24", 128);
print '-- ipv6_is_match()';
print ipv6_is_match('::ffff:7f00:1', '127.0.0.1') == true;
print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995') == false;
print ipv6_is_match('192.168.1.1/24', '192.168.1.255/24') == true;
print ipv6_is_match('fe80::85d:e82c:9446:7994/127', 'fe80::85d:e82c:9446:7995/127') == true;
print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995', 127) == true;
print ipv6_is_match('192.168.1.1', '192.168.1.1'); -- // Equal IPs
print ipv6_is_match('192.168.1.1/24', '192.168.1.255'); -- // 24 bit IP4-prefix is used for comparison
print ipv6_is_match('192.168.1.1', '192.168.1.255/24'); -- // 24 bit IP4-prefix is used for comparison
print ipv6_is_match('192.168.1.1/30', '192.168.1.255/24'); -- // 24 bit IP4-prefix is used for comparison
print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7994'); -- // Equal IPs
print ipv6_is_match('fe80::85d:e82c:9446:7994/120', 'fe80::85d:e82c:9446:7998'); -- // 120 bit IP6-prefix is used for comparison
print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7998/120'); -- // 120 bit IP6-prefix is used for comparison
print ipv6_is_match('fe80::85d:e82c:9446:7994/120', 'fe80::85d:e82c:9446:7998/120'); -- // 120 bit IP6-prefix is used for comparison
print ipv6_is_match('192.168.1.1', '::ffff:c0a8:0101'); -- // Equal IPs
print ipv6_is_match('192.168.1.1/24', '::ffff:c0a8:01ff'); -- // 24 bit IP-prefix is used for comparison
print ipv6_is_match('::ffff:c0a8:0101', '192.168.1.255/24'); -- // 24 bit IP-prefix is used for comparison
print ipv6_is_match('::192.168.1.1/30', '192.168.1.255/24'); -- // 24 bit IP-prefix is used for comparison
print ipv6_is_match('192.168.1.1', '192.168.1.0', 31); -- // 31 bit IP4-prefix is used for comparison
print ipv6_is_match('192.168.1.1/24', '192.168.1.255', 31); -- // 24 bit IP4-prefix is used for comparison
print ipv6_is_match('192.168.1.1', '192.168.1.255', 24); -- // 24 bit IP4-prefix is used for comparison
print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995', 127); -- // 127 bit IP6-prefix is used for comparison
print ipv6_is_match('fe80::85d:e82c:9446:7994/127', 'fe80::85d:e82c:9446:7998', 120); -- // 120 bit IP6-prefix is used for comparison
print ipv6_is_match('fe80::85d:e82c:9446:7994/120', 'fe80::85d:e82c:9446:7998', 127); -- // 120 bit IP6-prefix is used for comparison
print ipv6_is_match('192.168.1.1/24', '::ffff:c0a8:01ff', 127); -- // 127 bit IP6-prefix is used for comparison
print ipv6_is_match('::ffff:c0a8:0101', '192.168.1.255', 120); -- // 120 bit IP6-prefix is used for comparison
print ipv6_is_match('::192.168.1.1/30', '192.168.1.255/24', 127); -- // 120 bit IP6-prefix is used for comparison

View File

@ -1,4 +0,0 @@
-- isnan --
1
0
0

View File

@ -1,7 +0,0 @@
set dialect = 'kusto';
print '-- isnan --';
print isnan(double(nan));
print isnan(4.2);
print isnan(4); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }
print isnan(real(+inf));
print isnan(dynamic(null)); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }

View File

@ -1,16 +0,0 @@
-- bin_at()
4.5
-12:0:0
2017-05-14 12:00:00.000000000
2017-05-14 00:00:00.000000000
2018-02-25 15:14:00.000000000 5
2018-02-24 15:14:00.000000000 3
2018-02-23 15:14:00.000000000 4
-- bin()
4
1970-05-11 00:00:00.000000000
336:0:0
1970-05-11 13:45:07.345000000
1970-05-11 13:45:07.345623000
2022-09-26 10:13:23.987232000
1970-05-11 13:45:07.456336000

View File

@ -1,26 +0,0 @@
DROP TABLE IF EXISTS Bin_at_test;
CREATE TABLE Bin_at_test
(
`Date` DateTime('UTC'),
Num Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Bin_at_test VALUES ('2018-02-24T15:14:01',3), ('2018-02-23T16:14:01',4), ('2018-02-26T15:14:01',5);
set dialect = 'kusto';
print '-- bin_at()';
print bin_at(6.5, 2.5, 7);
print bin_at(1h, 1d, 12h);
print bin_at(datetime(2017-05-15 10:20:00.0), 1d, datetime(1970-01-01 12:00:00.0));
print bin_at(datetime(2017-05-17 10:20:00.0), 7d, datetime(2017-06-04 00:00:00.0));
Bin_at_test | summarize sum(Num) by d = todatetime(bin_at(Date, 1d, datetime('2018-02-24 15:14:00'))) | order by d;
print '-- bin()';
print bin(4.5, 1);
print bin(datetime(1970-05-11 13:45:07), 1d);
print bin(16d, 7d);
print bin(datetime(1970-05-11 13:45:07.345623), 1ms);
-- print bin(datetime(2022-09-26 10:13:23.987234), 6ms); -> 2022-09-26 10:13:23.982000000
print bin(datetime(1970-05-11 13:45:07.345623), 1microsecond);
print bin(datetime(2022-09-26 10:13:23.987234), 6microseconds);
print bin(datetime(1970-05-11 13:45:07.456345672), 16microseconds);
-- print bin(datetime(2022-09-26 10:13:23.987234128), 1tick); -> 2022-09-26 10:13:23.987234100
-- print bin(datetime(2022-09-26 10:13:23.987234128), 99nanosecond); -> null

View File

@ -1,360 +0,0 @@
-- test String Functions --
-- Customers |where Education contains \'degree\'
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
-- Customers |where Education !contains \'degree\'
\N why Professional Partial College 38
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers |where Education contains \'Degree\'
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
-- Customers |where Education !contains \'Degree\'
\N why Professional Partial College 38
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where FirstName endswith \'RE\'
Theodore Diaz Skilled Manual Bachelors 28
-- Customers | where ! FirstName endswith \'RE\'
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
--Customers | where FirstName endswith_cs \'re\'
Theodore Diaz Skilled Manual Bachelors 28
-- Customers | where FirstName !endswith_cs \'re\'
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation == \'Skilled Manual\'
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation != \'Skilled Manual\'
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Stephanie Cox Management abcd defg Bachelors 33
-- Customers | where Occupation has \'skilled\'
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation !has \'skilled\'
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Stephanie Cox Management abcd defg Bachelors 33
-- Customers | where Occupation has \'Skilled\'
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation !has \'Skilled\'
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Stephanie Cox Management abcd defg Bachelors 33
-- Customers | where Occupation hasprefix_cs \'Ab\'
-- Customers | where Occupation !hasprefix_cs \'Ab\'
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation hasprefix_cs \'ab\'
Stephanie Cox Management abcd defg Bachelors 33
-- Customers | where Occupation !hasprefix_cs \'ab\'
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation hassuffix \'Ent\'
Stephanie Cox Management abcd defg Bachelors 33
-- Customers | where Occupation !hassuffix \'Ent\'
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation hassuffix \'ent\'
Stephanie Cox Management abcd defg Bachelors 33
-- Customers | where Occupation hassuffix \'ent\'
Stephanie Cox Management abcd defg Bachelors 33
-- Customers |where Education in (\'Bachelors\',\'High School\')
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where Education !in (\'Bachelors\',\'High School\')
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
-- Customers | where FirstName matches regex \'P.*r\'
Peter Nara Skilled Manual Graduate Degree 26
-- Customers | where FirstName startswith \'pet\'
Peter Nara Skilled Manual Graduate Degree 26
-- Customers | where FirstName !startswith \'pet\'
Latoya Shen Professional Graduate Degree 25
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where FirstName startswith_cs \'pet\'
-- Customers | where FirstName !startswith_cs \'pet\'
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where isempty(LastName)
Apple Skilled Manual Bachelors 28
-- Customers | where isnotempty(LastName)
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Peter Nara Skilled Manual Graduate Degree 26
Latoya Shen Professional Graduate Degree 25
\N why Professional Partial College 38
-- Customers | where isnotnull(FirstName)
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where isnull(FirstName)
\N why Professional Partial College 38
-- Customers | project url_decode(\'https%3A%2F%2Fwww.test.com%2Fhello%20word\') | take 1
https://www.test.com/hello word
-- Customers | project url_encode(\'https://www.test.com/hello word\') | take 1
https%3A%2F%2Fwww.test.com%2Fhello%20word
-- Customers | project name_abbr = strcat(substring(FirstName,0,3), \' \', substring(LastName,2))
\N
Lat en
Pet ra
The az
Ste x
App
-- Customers | project name = strcat(FirstName, \' \', LastName)
\N
Latoya Shen
Peter Nara
Theodore Diaz
Stephanie Cox
Apple
-- Customers | project FirstName, strlen(FirstName)
\N \N
Latoya 6
Peter 5
Theodore 8
Stephanie 9
Apple 5
-- Customers | project strrep(FirstName,2,\'_\')
\N
Latoya_Latoya
Peter_Peter
Theodore_Theodore
Stephanie_Stephanie
Apple_Apple
-- Customers | project toupper(FirstName)
\N
LATOYA
PETER
THEODORE
STEPHANIE
APPLE
-- Customers | project tolower(FirstName)
\N
latoya
peter
theodore
stephanie
apple
-- support subquery for in orerator (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/in-cs-operator) (subquery need to be wraped with bracket inside bracket); TODO: case-insensitive not supported yet
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- has_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-all-operator); TODO: subquery not supported yet
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- has_any (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-anyoperator); TODO: subquery not supported yet
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Peter Nara Skilled Manual Graduate Degree 26
Apple Skilled Manual Bachelors 28
-- countof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/countoffunction)
3
3
1
-- extract ( https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractfunction)
PINEAPPLE ice cream is 20
PINEAPPLE
20
20
\N
\N
\N
\N
\N
45.6
45.6
-- extract_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractallfunction); TODO: captureGroups not supported yet
[['T','h','e'],['p','ric','e'],['P','INEAPPL','E'],['i','c','e'],['c','rea','m']]
-- extract_json (https://learn.microsoft.com/en-us/azure/data-explorer/kusto/query/extractjsonfunction)
John
iPhone
\N
26
26
26
26
\N
-- split (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/splitfunction)
['aa','bb']
['bbb']
['']
['a','','b']
['aa','cc']
['aabbcc']
['aaa','bbb','ccc']
[NULL]
-- strcat_delim (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/strcat-delimfunction); TODO: only support string now.
1-2-Ab
-- indexof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/indexoffunction); TODO: length and occurrence not supported yet
2
2
-1
-- base64_encode_fromguid()
8jMxriJurkmwahbmqbIS6w==
-- base64_decode_toarray()
[]
[75,117,115,116,111]
-- base64_decode_toguid()
10e99626-bc2b-4c75-bb3e-fe606de25700
1
-- base64_encode_tostring
S3VzdG8x
-- base64_decode_tostring
Kusto1
-- parse_url()
{"Scheme":"scheme","Host":"","Port":"0","Path":"/this/is/a/path","Username":"username","Password":"password","Query Parameters":{"k1":"v1","k2":"v2"},"Fragment":"fragment"}
-- parse_urlquery()
{"Query Parameters":{"k1":"v1","k2":"v2","k3":"v3"}}
-- strcmp()
0 1 -1 1
-- substring()
CD
-- translate()
kusto xxx
-- trim()
https://www.ibm.com
Te st1
asd
asd
sd
-- trim_start()
www.ibm.com
Te st1// $
asdw
asd
-- trim_end()
https
- Te st1
wasd
asd
-- trim, trim_start, trim_end all at once
--https://bing.com-- -- https://bing.com-- --https://bing.com https://bing.com
-- replace_regex
Number was: 1
-- has_any_index()
0 1 -1 -1
-- parse_version()
1000000020000000300000040
1000000020000000000000000
1000000020000000000000000
\N
\N
\N
\N
1000000020000000300000004
1000000020000000000000000
1000000020000000300000000
1000000000000000000000000
-- parse_json()
[1,2,3]
[{"a":123.5,"b":"{\\"c\\":456}"}]
-- parse_command_line()
[NULL]
[NULL]
-- reverse()
321
43.321
dsa
][
]3,2,1[
]\'redaV\',\'htraD\'[
000000000.00:00:21 51-01-7102
Peter Nara Skilled Manual Graduate Degree 26
Latoya Shen Professional Graduate Degree 25
-- parse_csv()
['']
['aaa']
['aa','b','cc']
['record1','a','b','c']

View File

@ -1,313 +0,0 @@
-- Tags: no-fasttest
DROP TABLE IF EXISTS Customers;
CREATE TABLE Customers
(
FirstName Nullable(String),
LastName String,
Occupation String,
Education String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28), ('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38);
-- datatable (Version:string) [
-- '1.2.3.4',
-- '1.2',
-- '1.2.3',
-- '1'
-- ]
DROP TABLE IF EXISTS Versions;
CREATE TABLE Versions
(
Version String
) ENGINE = Memory;
INSERT INTO Versions VALUES ('1.2.3.4'),('1.2'),('1.2.3'),('1');
set dialect='kusto';
print '-- test String Functions --';
print '-- Customers |where Education contains \'degree\'';
Customers |where Education contains 'degree' | order by LastName;
print '';
print '-- Customers |where Education !contains \'degree\'';
Customers |where Education !contains 'degree' | order by LastName;
print '';
print '-- Customers |where Education contains \'Degree\'';
Customers |where Education contains 'Degree' | order by LastName;
print '';
print '-- Customers |where Education !contains \'Degree\'';
Customers |where Education !contains 'Degree' | order by LastName;
print '';
print '-- Customers | where FirstName endswith \'RE\'';
Customers | where FirstName endswith 'RE' | order by LastName;
print '';
print '-- Customers | where ! FirstName endswith \'RE\'';
Customers | where FirstName ! endswith 'RE' | order by LastName;
print '';
print '--Customers | where FirstName endswith_cs \'re\'';
Customers | where FirstName endswith_cs 're' | order by LastName;
print '';
print '-- Customers | where FirstName !endswith_cs \'re\'';
Customers | where FirstName !endswith_cs 're' | order by LastName;
print '';
print '-- Customers | where Occupation == \'Skilled Manual\'';
Customers | where Occupation == 'Skilled Manual' | order by LastName;
print '';
print '-- Customers | where Occupation != \'Skilled Manual\'';
Customers | where Occupation != 'Skilled Manual' | order by LastName;
print '';
print '-- Customers | where Occupation has \'skilled\'';
Customers | where Occupation has 'skilled' | order by LastName;
print '';
print '-- Customers | where Occupation !has \'skilled\'';
Customers | where Occupation !has 'skilled' | order by LastName;
print '';
print '-- Customers | where Occupation has \'Skilled\'';
Customers | where Occupation has 'Skilled'| order by LastName;
print '';
print '-- Customers | where Occupation !has \'Skilled\'';
Customers | where Occupation !has 'Skilled'| order by LastName;
print '';
print '-- Customers | where Occupation hasprefix_cs \'Ab\'';
Customers | where Occupation hasprefix_cs 'Ab'| order by LastName;
print '';
print '-- Customers | where Occupation !hasprefix_cs \'Ab\'';
Customers | where Occupation !hasprefix_cs 'Ab'| order by LastName;
print '';
print '-- Customers | where Occupation hasprefix_cs \'ab\'';
Customers | where Occupation hasprefix_cs 'ab'| order by LastName;
print '';
print '-- Customers | where Occupation !hasprefix_cs \'ab\'';
Customers | where Occupation !hasprefix_cs 'ab'| order by LastName;
print '';
print '-- Customers | where Occupation hassuffix \'Ent\'';
Customers | where Occupation hassuffix 'Ent'| order by LastName;
print '';
print '-- Customers | where Occupation !hassuffix \'Ent\'';
Customers | where Occupation !hassuffix 'Ent'| order by LastName;
print '';
print '-- Customers | where Occupation hassuffix \'ent\'';
Customers | where Occupation hassuffix 'ent'| order by LastName;
print '';
print '-- Customers | where Occupation hassuffix \'ent\'';
Customers | where Occupation hassuffix 'ent'| order by LastName;
print '';
print '-- Customers |where Education in (\'Bachelors\',\'High School\')';
Customers |where Education in ('Bachelors','High School')| order by LastName;
print '';
print '-- Customers | where Education !in (\'Bachelors\',\'High School\')';
Customers | where Education !in ('Bachelors','High School')| order by LastName;
print '';
print '-- Customers | where FirstName matches regex \'P.*r\'';
Customers | where FirstName matches regex 'P.*r'| order by LastName;
print '';
print '-- Customers | where FirstName startswith \'pet\'';
Customers | where FirstName startswith 'pet'| order by LastName;
print '';
print '-- Customers | where FirstName !startswith \'pet\'';
Customers | where FirstName !startswith 'pet'| order by LastName;
print '';
print '-- Customers | where FirstName startswith_cs \'pet\'';
Customers | where FirstName startswith_cs 'pet'| order by LastName;
print '';
print '-- Customers | where FirstName !startswith_cs \'pet\'';
Customers | where FirstName !startswith_cs 'pet'| order by LastName;
print '';
print '-- Customers | where isempty(LastName)';
Customers | where isempty(LastName);
print '';
print '-- Customers | where isnotempty(LastName)';
Customers | where isnotempty(LastName);
print '';
print '-- Customers | where isnotnull(FirstName)';
Customers | where isnotnull(FirstName)| order by LastName;
print '';
print '-- Customers | where isnull(FirstName)';
Customers | where isnull(FirstName)| order by LastName;
print '';
print '-- Customers | project url_decode(\'https%3A%2F%2Fwww.test.com%2Fhello%20word\') | take 1';
Customers | project url_decode('https%3A%2F%2Fwww.test.com%2Fhello%20word') | take 1;
print '';
print '-- Customers | project url_encode(\'https://www.test.com/hello word\') | take 1';
Customers | project url_encode('https://www.test.com/hello word') | take 1;
print '';
print '-- Customers | project name_abbr = strcat(substring(FirstName,0,3), \' \', substring(LastName,2))';
Customers | project name_abbr = strcat(substring(FirstName,0,3), ' ', substring(LastName,2))| order by LastName;
print '';
print '-- Customers | project name = strcat(FirstName, \' \', LastName)';
Customers | project name = strcat(FirstName, ' ', LastName)| order by LastName;
print '';
print '-- Customers | project FirstName, strlen(FirstName)';
Customers | project FirstName, strlen(FirstName)| order by LastName;
print '';
print '-- Customers | project strrep(FirstName,2,\'_\')';
Customers | project strrep(FirstName,2,'_')| order by LastName;
print '';
print '-- Customers | project toupper(FirstName)';
Customers | project toupper(FirstName)| order by LastName;
print '';
print '-- Customers | project tolower(FirstName)';
Customers | project tolower(FirstName)| order by LastName;
print '';
print '-- support subquery for in orerator (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/in-cs-operator) (subquery need to be wraped with bracket inside bracket); TODO: case-insensitive not supported yet';
Customers | where Age in ((Customers|project Age|where Age < 30)) | order by LastName;
-- Customer | where LastName in~ ("diaz", "cox")
print '';
print '-- has_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-all-operator); TODO: subquery not supported yet';
Customers | where Occupation has_all ('manual', 'skilled') | order by LastName;
print '';
print '-- has_any (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-anyoperator); TODO: subquery not supported yet';
Customers|where Occupation has_any ('Skilled','abcd');
print '';
print '-- countof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/countoffunction)';
Customers | project countof('The cat sat on the mat', 'at') | take 1;
Customers | project countof('The cat sat on the mat', 'at', 'normal') | take 1;
Customers | project countof('The cat sat on the mat', '\\s.he', 'regex') | take 1;
print '';
print '-- extract ( https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractfunction)';
print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 0, 'The price of PINEAPPLE ice cream is 20');
print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 1, 'The price of PINEAPPLE ice cream is 20');
print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 20');
print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 3, 'The price of PINEAPPLE ice cream is 20');
print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 20', typeof(real));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(bool));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(date));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(guid));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(int));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(long));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(real));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(decimal));
print '';
print '-- extract_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractallfunction); TODO: captureGroups not supported yet';
Customers | project extract_all('(\\w)(\\w+)(\\w)','The price of PINEAPPLE ice cream is 20') | take 1;
print '';
print '-- extract_json (https://learn.microsoft.com/en-us/azure/data-explorer/kusto/query/extractjsonfunction)';
print extract_json('', ''); -- { serverError BAD_ARGUMENTS }
print extract_json('a', ''); -- { serverError BAD_ARGUMENTS }
print extract_json('$.firstName', '');
print extract_json('$.phoneNumbers[0].type', '');
print extractjson('$.firstName', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}');
print extract_json('$.phoneNumbers[0].type', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(string));
print extract_json('$.phoneNumbers[0].type', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(int));
print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}');
print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(int));
print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(long));
-- print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(bool)); -> true
print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(double));
print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(guid));
-- print extract_json('$.phoneNumbers', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(dynamic)); we won't be able to handle this particular case for a while, because it should return a dictionary
print '';
print '-- split (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/splitfunction)';
Customers | project split('aa_bb', '_') | take 1;
Customers | project split('aaa_bbb_ccc', '_', 1) | take 1;
Customers | project split('', '_') | take 1;
Customers | project split('a__b', '_') | take 1;
Customers | project split('aabbcc', 'bb') | take 1;
Customers | project split('aabbcc', '') | take 1;
Customers | project split('aaa_bbb_ccc', '_', -1) | take 1;
Customers | project split('aaa_bbb_ccc', '_', 10) | take 1;
print '';
print '-- strcat_delim (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/strcat-delimfunction); TODO: only support string now.';
Customers | project strcat_delim('-', '1', '2', strcat('A','b')) | take 1;
-- Customers | project strcat_delim('-', '1', '2', 'A' , 1s);
print '';
print '-- indexof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/indexoffunction); TODO: length and occurrence not supported yet';
Customers | project indexof('abcdefg','cde') | take 1;
Customers | project indexof('abcdefg','cde',2) | take 1;
Customers | project indexof('abcdefg','cde',6) | take 1;
print '-- base64_encode_fromguid()';
-- print base64_encode_fromguid(guid(null));
print base64_encode_fromguid(guid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb'));
print base64_encode_fromguid(dynamic(null)); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }
print base64_encode_fromguid("abcd1231"); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }
print '-- base64_decode_toarray()';
print base64_decode_toarray('');
print base64_decode_toarray('S3VzdG8=');
print '-- base64_decode_toguid()';
print base64_decode_toguid("JpbpECu8dUy7Pv5gbeJXAA==");
print base64_decode_toguid(base64_encode_fromguid(guid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb'))) == guid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb');
print '-- base64_encode_tostring';
print base64_encode_tostring('');
print base64_encode_tostring('Kusto1');
print '-- base64_decode_tostring';
print base64_decode_tostring('');
print base64_decode_tostring('S3VzdG8x');
print '-- parse_url()';
print parse_url('scheme://username:password@host:1234/this/is/a/path?k1=v1&k2=v2#fragment');
print '-- parse_urlquery()';
print parse_urlquery('k1=v1&k2=v2&k3=v3');
print '-- strcmp()';
print strcmp('ABC','ABC'), strcmp('abc','ABC'), strcmp('ABC','abc'), strcmp('abcde','abc');
print '-- substring()';
print substring("ABCD", -2, 2);
print '-- translate()';
print translate('krasp', 'otsku', 'spark'), translate('abc', '', 'ab'), translate('abc', 'x', 'abc');
print '-- trim()';
print trim("--", "--https://www.ibm.com--");
print trim("[^\w]+", strcat("- ","Te st", "1", "// $"));
print trim("", " asd ");
print trim("a$", "asd");
print trim("^a", "asd");
print '-- trim_start()';
print trim_start("https://", "https://www.ibm.com");
print trim_start("[^\w]+", strcat("- ","Te st", "1", "// $"));
print trim_start("asd$", "asdw");
print trim_start("asd$", "asd");
print trim_start("d$", "asd");
print '-- trim_end()';
print trim_end("://www.ibm.com", "https://www.ibm.com");
print trim_end("[^\w]+", strcat("- ","Te st", "1", "// $"));
print trim_end("^asd", "wasd");
print trim_end("^asd", "asd");
print trim_end("^a", "asd");
print '-- trim, trim_start, trim_end all at once';
print str = "--https://bing.com--", pattern = '--' | extend start = trim_start(pattern, str), end = trim_end(pattern, str), both = trim(pattern, str);
print '-- replace_regex';
print replace_regex(strcat('Number is ', '1'), 'is (\d+)', 'was: \1');
print '-- has_any_index()';
print has_any_index('this is an example', dynamic(['this', 'example'])), has_any_index("this is an example", dynamic(['not', 'example'])), has_any_index("this is an example", dynamic(['not', 'found'])), has_any_index("this is an example", dynamic([]));
print '-- parse_version()';
print parse_version(42); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- print parse_version(''); -> NULL
print parse_version('1.2.3.40');
print parse_version('1.2');
print parse_version(strcat('1.', '2'));
print parse_version('1.2.4.5.6');
print parse_version('moo');
print parse_version('moo.boo.foo');
print parse_version(strcat_delim('.', 'moo', 'boo', 'foo'));
Versions | project parse_version(Version);
print '-- parse_json()';
print parse_json(dynamic([1, 2, 3]));
print parse_json('{"a":123.5, "b":"{\\"c\\":456}"}');
print '-- parse_command_line()';
print parse_command_line(55, 'windows'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- print parse_command_line((52 + 3) * 4 % 2, 'windows'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
print parse_command_line('', 'windows');
print parse_command_line(strrep(' ', 6), 'windows');
-- print parse_command_line('echo \"hello world!\" print$?', 'windows'); -> ["echo","hello world!","print$?"]
-- print parse_command_line("yolo swag 'asd bcd' \"moo moo \"", 'windows'); -> ["yolo","swag","'asd","bcd'","moo moo "]
-- print parse_command_line(strcat_delim(' ', "yolo", "swag", "\'asd bcd\'", "\"moo moo \""), 'windows'); -> ["yolo","swag","'asd","bcd'","moo moo "]
print '-- reverse()';
print reverse(123);
print reverse(123.34);
print reverse('');
print reverse("asd");
print reverse(dynamic([]));
print reverse(dynamic([1, 2, 3]));
print reverse(dynamic(['Darth', "Vader"]));
print reverse(datetime(2017-10-15 12:00));
-- print reverse(timespan(3h)); -> 00:00:30
Customers | where Education contains 'degree' | order by reverse(FirstName);
print '-- parse_csv()';
print parse_csv('');
print parse_csv(65); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
print parse_csv('aaa');
print result=parse_csv('aa,b,cc');
print result_multi_record=parse_csv('record1,a,b,c\nrecord2,x,y,z');
-- print result=parse_csv('aa,"b,b,b",cc,"Escaping quotes: ""Title""","line1\nline2"'); -> ["aa","b,b,b","cc","Escaping quotes: \"Title\"","line1\nline2"]
-- print parse_csv(strcat(strcat_delim(',', 'aa', '"b,b,b"', 'cc', '"Escaping quotes: ""Title"""', '"line1\nline2"'), '\r\n', strcat_delim(',', 'asd', 'qcf'))); -> ["aa","b,b,b","cc","Escaping quotes: \"Title\"","line1\nline2"]

View File

@ -1,60 +0,0 @@
-- from to
Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102]
Costco Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [0,2,0]
Aldi Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [0,500,0]
Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6]
-- from
Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102]
Costco Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000'] [0,2]
Aldi Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000'] [0,500]
Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6]
-- to
Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102]
Costco Apple ['2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [2,0]
Aldi Snargaluff ['2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [500,0]
Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6]
-- without from/to
Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102]
Costco Apple ['2016-09-11 00:00:00.000000000'] [2]
Aldi Snargaluff ['2016-09-11 00:00:00.000000000'] [500]
Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6]
-- without by
['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [70,334,54]
-- without aggregation alias
Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102]
Aldi Snargaluff ['2016-09-11 00:00:00.000000000'] [500]
Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6]
Costco Apple ['2016-09-11 00:00:00.000000000'] [2]
-- assign group alias
Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102]
Aldi Snargaluff ['2016-09-11 00:00:00.000000000'] [500]
Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6]
Costco Apple ['2016-09-11 00:00:00.000000000'] [2]
-- 3d step
Costco Snargaluff ['2016-09-10 00:00:00.000000000'] [134.66666666666666]
Costco Apple ['2016-09-10 00:00:00.000000000'] [2]
Aldi Snargaluff ['2016-09-10 00:00:00.000000000'] [500]
Aldi Apple ['2016-09-10 00:00:00.000000000'] [5.5]
-- numeric column
Costco Snargaluff [10,11,12,13,14] [200,0,102,0,0]
Aldi Snargaluff [10,11,12,13,14] [0,500,0,0,0]
Aldi Apple [10,11,12,13,14] [5,0,6,0,0]
Costco Apple [10,11,12,13,14] [0,2,0,0,0]
-- from
Costco Snargaluff [10,11,12] [200,0,102]
Aldi Snargaluff [10,11] [0,500]
Aldi Apple [10,11,12] [5,0,6]
Costco Apple [10,11] [0,2]
-- to
Costco Snargaluff [8,12,16] [200,102,0]
Aldi Snargaluff [8,12,16] [500,0,0]
Aldi Apple [8,12,16] [5,6,0]
Costco Apple [8,12,16] [2,0,0]
-- without from/to
Costco Snargaluff [10,12] [200,102]
Aldi Snargaluff [10] [500]
Aldi Apple [10,12] [5,6]
Costco Apple [10] [2]
-- without by
[10,12] [202,54]
['2017-01-01 00:00:00.000000000','2017-01-02 00:00:00.000000000','2017-01-03 00:00:00.000000000','2017-01-04 00:00:00.000000000','2017-01-05 00:00:00.000000000','2017-01-06 00:00:00.000000000','2017-01-07 00:00:00.000000000','2017-01-08 00:00:00.000000000','2017-01-09 00:00:00.000000000'] [4,3,5,0,10.5,4,3,8,6.5]

View File

@ -1,77 +0,0 @@
-- Azure Data Explore Test Data
-- let make_series_test_table = datatable (Supplier:string, Fruit:string, Price: real, Purchase:datetime)
-- [
-- 'Aldi','Apple',4,'2016-09-10',
-- 'Costco','Apple',2,'2016-09-11',
-- 'Aldi','Apple',6,'2016-09-10',
-- 'Costco','Snargaluff',100,'2016-09-12',
-- 'Aldi','Apple',7,'2016-09-12',
-- 'Aldi','Snargaluff',400,'2016-09-11',
-- 'Costco','Snargaluff',104,'2016-09-12',
-- 'Aldi','Apple',5,'2016-09-12',
-- 'Aldi','Snargaluff',600,'2016-09-11',
-- 'Costco','Snargaluff',200,'2016-09-10',
-- ];
DROP TABLE IF EXISTS make_series_test_table;
CREATE TABLE make_series_test_table
(
Supplier Nullable(String),
Fruit String ,
Price Float64,
Purchase Date
) ENGINE = Memory;
INSERT INTO make_series_test_table VALUES ('Aldi','Apple',4,'2016-09-10'), ('Costco','Apple',2,'2016-09-11'), ('Aldi','Apple',6,'2016-09-10'), ('Costco','Snargaluff',100,'2016-09-12'), ('Aldi','Apple',7,'2016-09-12'), ('Aldi','Snargaluff',400,'2016-09-11'),('Costco','Snargaluff',104,'2016-09-12'),('Aldi','Apple',5,'2016-09-12'),('Aldi','Snargaluff',600,'2016-09-11'),('Costco','Snargaluff',200,'2016-09-10');
DROP TABLE IF EXISTS make_series_test_table2;
CREATE TABLE make_series_test_table2
(
Supplier Nullable(String),
Fruit String ,
Price Int32,
Purchase Int32
) ENGINE = Memory;
INSERT INTO make_series_test_table2 VALUES ('Aldi','Apple',4,10),('Costco','Apple',2,11),('Aldi','Apple',6,10),('Costco','Snargaluff',100,12),('Aldi','Apple',7,12),('Aldi','Snargaluff',400,11),('Costco','Snargaluff',104,12),('Aldi','Apple',5,12),('Aldi','Snargaluff',600,11),('Costco','Snargaluff',200,10);
DROP TABLE IF EXISTS make_series_test_table3;
CREATE TABLE make_series_test_table3
(
timestamp datetime,
metric Float64,
) ENGINE = Memory;
INSERT INTO make_series_test_table3 VALUES (parseDateTimeBestEffort('2016-12-31T06:00', 'UTC'), 50), (parseDateTimeBestEffort('2017-01-01', 'UTC'), 4), (parseDateTimeBestEffort('2017-01-02', 'UTC'), 3), (parseDateTimeBestEffort('2017-01-03', 'UTC'), 4), (parseDateTimeBestEffort('2017-01-03T03:00', 'UTC'), 6), (parseDateTimeBestEffort('2017-01-05', 'UTC'), 8), (parseDateTimeBestEffort('2017-01-05T13:40', 'UTC'), 13), (parseDateTimeBestEffort('2017-01-06', 'UTC'), 4), (parseDateTimeBestEffort('2017-01-07', 'UTC'), 3), (parseDateTimeBestEffort('2017-01-08', 'UTC'), 8), (parseDateTimeBestEffort('2017-01-08T21:00', 'UTC'), 8), (parseDateTimeBestEffort('2017-01-09', 'UTC'), 2), (parseDateTimeBestEffort('2017-01-09T12:00', 'UTC'), 11), (parseDateTimeBestEffort('2017-01-10T05:00', 'UTC'), 5);
-- This test requies sorting after some of aggregations but I don't know KQL, sorry
set max_bytes_before_external_group_by = 0;
set dialect = 'kusto';
print '-- from to';
make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) to datetime(2016-09-13) step 1d by Supplier, Fruit | order by Supplier, Fruit;
print '-- from';
make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) step 1d by Supplier, Fruit | order by Supplier, Fruit;
print '-- to';
make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase to datetime(2016-09-13) step 1d by Supplier, Fruit | order by Supplier, Fruit;
print '-- without from/to';
make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase step 1d by Supplier, Fruit | order by Supplier, Fruit;
print '-- without by';
make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase step 1d;
print '-- without aggregation alias';
make_series_test_table | make-series avg(Price) default=0 on Purchase step 1d by Supplier, Fruit;
print '-- assign group alias';
make_series_test_table | make-series avg(Price) default=0 on Purchase step 1d by Supplier_Name = Supplier, Fruit;
print '-- 3d step';
make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) to datetime(2016-09-13) step 3d by Supplier, Fruit | order by Supplier, Fruit;
print '-- numeric column'
print '-- from to';
make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase from 10 to 15 step 1.0 by Supplier, Fruit;
print '-- from';
make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase from 10 step 1.0 by Supplier, Fruit;
print '-- to';
make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase to 18 step 4.0 by Supplier, Fruit;
print '-- without from/to';
make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase step 2.0 by Supplier, Fruit;
print '-- without by';
make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase step 2.0;
make_series_test_table3 | make-series avg(metric) default=0 on timestamp from datetime(2017-01-01) to datetime(2017-01-10) step 1d
-- print '-- summarize --'
-- make_series_test_table | summarize count() by format_datetime(bin(Purchase, 1d), 'yy-MM-dd');

View File

@ -1,65 +0,0 @@
-- mv-expand --
-- mv_expand_test_table | mv-expand c --
1 ['Salmon','Steak','Chicken'] 1 [5,6,7,8]
1 ['Salmon','Steak','Chicken'] 2 [5,6,7,8]
1 ['Salmon','Steak','Chicken'] 3 [5,6,7,8]
1 ['Salmon','Steak','Chicken'] 4 [5,6,7,8]
-- mv_expand_test_table | mv-expand c, d --
1 ['Salmon','Steak','Chicken'] 1 5
1 ['Salmon','Steak','Chicken'] 2 6
1 ['Salmon','Steak','Chicken'] 3 7
1 ['Salmon','Steak','Chicken'] 4 8
-- mv_expand_test_table | mv-expand b | mv-expand c --
1 Salmon 1 [5,6,7,8]
1 Salmon 2 [5,6,7,8]
1 Salmon 3 [5,6,7,8]
1 Salmon 4 [5,6,7,8]
1 Steak 1 [5,6,7,8]
1 Steak 2 [5,6,7,8]
1 Steak 3 [5,6,7,8]
1 Steak 4 [5,6,7,8]
1 Chicken 1 [5,6,7,8]
1 Chicken 2 [5,6,7,8]
1 Chicken 3 [5,6,7,8]
1 Chicken 4 [5,6,7,8]
-- mv_expand_test_table | mv-expand with_itemindex=index b, c, d --
0 1 Salmon 1 5
1 1 Steak 2 6
2 1 Chicken 3 7
3 1 4 8
-- mv_expand_test_table | mv-expand array_concat(c,d) --
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 1
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 2
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 3
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 4
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 5
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 6
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 7
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 8
-- mv_expand_test_table | mv-expand x = c, y = d --
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 1 5
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 2 6
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 3 7
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 4 8
-- mv_expand_test_table | mv-expand xy = array_concat(c, d) --
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 1
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 2
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 3
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 4
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 5
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 6
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 7
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 8
-- mv_expand_test_table | mv-expand xy = array_concat(c, d) limit 2| summarize count() by xy --
1 1
2 1
-- mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bool) --
0 1 ['Salmon','Steak','Chicken'] 1 true
1 1 ['Salmon','Steak','Chicken'] 2 true
2 1 ['Salmon','Steak','Chicken'] 3 true
3 1 ['Salmon','Steak','Chicken'] 4 true
-- mv_expand_test_table | mv-expand c to typeof(bool) --
1 ['Salmon','Steak','Chicken'] [5,6,7,8] true
1 ['Salmon','Steak','Chicken'] [5,6,7,8] true
1 ['Salmon','Steak','Chicken'] [5,6,7,8] true
1 ['Salmon','Steak','Chicken'] [5,6,7,8] true

View File

@ -1,35 +0,0 @@
-- datatable(a: int, b: dynamic, c: dynamic, d: dynamic) [
-- 1, dynamic(['Salmon', 'Steak', 'Chicken']), dynamic([1, 2, 3, 4]), dynamic([5, 6, 7, 8])
-- ]
DROP TABLE IF EXISTS mv_expand_test_table;
CREATE TABLE mv_expand_test_table
(
a UInt8,
b Array(String),
c Array(Int8),
d Array(Int8)
) ENGINE = Memory;
INSERT INTO mv_expand_test_table VALUES (1, ['Salmon', 'Steak','Chicken'],[1,2,3,4],[5,6,7,8]);
set dialect='kusto';
print '-- mv-expand --';
print '-- mv_expand_test_table | mv-expand c --';
mv_expand_test_table | mv-expand c;
print '-- mv_expand_test_table | mv-expand c, d --';
mv_expand_test_table | mv-expand c, d;
print '-- mv_expand_test_table | mv-expand b | mv-expand c --';
mv_expand_test_table | mv-expand b | mv-expand c;
print '-- mv_expand_test_table | mv-expand with_itemindex=index b, c, d --';
mv_expand_test_table | mv-expand with_itemindex=index b, c, d;
print '-- mv_expand_test_table | mv-expand array_concat(c,d) --';
mv_expand_test_table | mv-expand array_concat(c,d);
print '-- mv_expand_test_table | mv-expand x = c, y = d --';
mv_expand_test_table | mv-expand x = c, y = d;
print '-- mv_expand_test_table | mv-expand xy = array_concat(c, d) --';
mv_expand_test_table | mv-expand xy = array_concat(c, d);
print '-- mv_expand_test_table | mv-expand xy = array_concat(c, d) limit 2| summarize count() by xy --';
mv_expand_test_table | mv-expand xy = array_concat(c, d) limit 2| summarize count() by xy;
print '-- mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bool) --';
mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bool);
print '-- mv_expand_test_table | mv-expand c to typeof(bool) --';
mv_expand_test_table | mv-expand c to typeof(bool);

View File

@ -1,23 +0,0 @@
numeric
kusto
00:00:00
00:00:00.0000001
00:00:00.0010000
00:00:42
01:06:00
2.18:00:00
5.00:00:00
7.00:00:00
14.00:00:00
('00:01:12','21.00:00:00','00:00:00.0000002')
numeric
99
100
1
42
66
66
5
1
2
(72,3,200)

View File

@ -1,16 +0,0 @@
select value from system.settings where name = 'interval_output_format';
{% for format in ['kusto', 'numeric'] -%}
select '{{ format }}';
set interval_output_format = '{{ format }}';
select toIntervalNanosecond(99);
select toIntervalNanosecond(100);
select toIntervalMillisecond(1);
select toIntervalSecond(42);
select toIntervalMinute(66);
select toIntervalHour(66);
select toIntervalDay(5);
select toIntervalWeek(1);
select toIntervalWeek(2);
select toIntervalSecond(72) + toIntervalWeek(3) + toIntervalNanosecond(200);
{% endfor -%}

View File

@ -1,60 +0,0 @@
-- #1 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- #2 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Peter Nara Skilled Manual Graduate Degree 26
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #3 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #4 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #5 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #6 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #7 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #8 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #9 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Peter Nara Skilled Manual Graduate Degree 26
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #10 --
-- #11 --
-- #12 --
-- #13 --
-- #14 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #15 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28

View File

@ -1,42 +0,0 @@
DROP TABLE IF EXISTS Customers;
CREATE TABLE Customers
(
FirstName Nullable(String),
LastName String,
Occupation String,
Education String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38);
Select '-- #1 --' ;
select * from kql($$Customers | where FirstName !in ('Peter', 'Latoya')$$);
Select '-- #2 --' ;
select * from kql($$Customers | where FirstName !in ("test", "test2")$$);
Select '-- #3 --' ;
select * from kql($$Customers | where FirstName !contains 'Pet'$$);
Select '-- #4 --' ;
select * from kql($$Customers | where FirstName !contains_cs 'Pet'$$);
Select '-- #5 --' ;
select * from kql($$Customers | where FirstName !endswith 'ter'$$);
Select '-- #6 --' ;
select * from kql($$Customers | where FirstName !endswith_cs 'ter'$$);
Select '-- #7 --' ;
select * from kql($$Customers | where FirstName != 'Peter'$$);
Select '-- #8 --' ;
select * from kql($$Customers | where FirstName !has 'Peter'$$);
Select '-- #9 --' ;
select * from kql($$Customers | where FirstName !has_cs 'peter'$$);
Select '-- #10 --' ;
-- select * from kql($$Customers | where FirstName !hasprefix 'Peter'$$); -- will enable when analyzer fixed `and` issue
Select '-- #11 --' ;
--select * from kql($$Customers | where FirstName !hasprefix_cs 'Peter'$$);
Select '-- #12 --' ;
--select * from kql($$Customers | where FirstName !hassuffix 'Peter'$$);
Select '-- #13 --' ;
--select * from kql($$Customers | where FirstName !hassuffix_cs 'Peter'$$);
Select '-- #14 --' ;
select * from kql($$Customers | where FirstName !startswith 'Peter'$$);
Select '-- #15 --' ;
select * from kql($$Customers | where FirstName !startswith_cs 'Peter'$$);
DROP TABLE IF EXISTS Customers;

View File

@ -1,92 +0,0 @@
-- test summarize --
12 25 46 32.416666666666664 389
Skilled Manual 5 26 36 30.2 151
Professional 6 25 46 34.166666666666664 205
Management abcd defg 1 33 33 33 33
Skilled Manual 0
Professional 2
Management abcd defg 0
Skilled Manual 36
Professional 38
Management abcd defg 33
Skilled Manual 26
Professional 25
Management abcd defg 33
Skilled Manual 30.2
Professional 29.25
Management abcd defg 33
Skilled Manual 151
Professional 117
Management abcd defg 33
4
2
40 2
30 4
20 6
Skilled Manual 5
Professional 6
Management abcd defg 1
-- make_list() --
Skilled Manual ['Bachelors','Graduate Degree','High School','Partial College','Bachelors']
Professional ['Graduate Degree','Partial College','Partial College','Partial College','Partial College','Partial College']
Management abcd defg ['Bachelors']
Skilled Manual ['Bachelors','Graduate Degree']
Professional ['Graduate Degree','Partial College']
Management abcd defg ['Bachelors']
-- make_list_if() --
Skilled Manual ['Edward','Christine']
Professional ['Dalton','Angel']
Management abcd defg ['Stephanie']
Skilled Manual ['Edward']
Professional ['Dalton']
Management abcd defg ['Stephanie']
-- make_set() --
Skilled Manual ['Graduate Degree','High School','Partial College','Bachelors']
Professional ['Graduate Degree','Partial College']
Management abcd defg ['Bachelors']
Skilled Manual ['Graduate Degree','Bachelors']
Professional ['Graduate Degree','Partial College']
Management abcd defg ['Bachelors']
-- make_set_if() --
Skilled Manual ['Partial College','High School']
Professional ['Partial College']
Management abcd defg ['Bachelors']
Skilled Manual ['High School']
Professional ['Partial College']
Management abcd defg ['Bachelors']
-- stdev() --
6.855102059227432
-- stdevif() --
7.557189365836421
-- binary_all_and --
42
-- binary_all_or --
46
-- binary_all_xor --
4
43.8
25.55 30.5 43.8
30.5
35
[25,35,45]
-- Summarize following sort --
Skilled Manual 5
Professional 6
Management abcd defg 1
-- summarize with bin --
0 1
245000 2
0 1
245 2
0 1
245 2
2015-10-12 00:00:00.000000000
2016-10-12 00:00:00.000000000
-- make_list_with_nulls --
['Theodore','Stephanie','Peter','Latoya','Joshua','Edward','Dalton','Christine','Cameron','Angel','Apple',NULL]
Skilled Manual ['Theodore','Peter','Edward','Christine','Apple']
Professional ['Latoya','Joshua','Dalton','Cameron','Angel',NULL]
Management abcd defg ['Stephanie']
Skilled Manual ['Theodore','Peter','Edward','Christine','Apple'] [28,26,36,33,28]
Professional ['Latoya','Joshua','Dalton','Cameron','Angel',NULL] [25,26,42,28,46,38]
Management abcd defg ['Stephanie'] [33]

View File

@ -1,102 +0,0 @@
-- datatable(FirstName:string, LastName:string, Occupation:string, Education:string, Age:int) [
-- 'Theodore', 'Diaz', 'Skilled Manual', 'Bachelors', 28,
-- 'Stephanie', 'Cox', 'Management abcd defg', 'Bachelors', 33,
-- 'Peter', 'Nara', 'Skilled Manual', 'Graduate Degree', 26,
-- 'Latoya', 'Shen', 'Professional', 'Graduate Degree', 25,
-- 'Joshua', 'Lee', 'Professional', 'Partial College', 26,
-- 'Edward', 'Hernandez', 'Skilled Manual', 'High School', 36,
-- 'Dalton', 'Wood', 'Professional', 'Partial College', 42,
-- 'Christine', 'Nara', 'Skilled Manual', 'Partial College', 33,
-- 'Cameron', 'Rodriguez', 'Professional', 'Partial College', 28,
-- 'Angel', 'Stewart', 'Professional', 'Partial College', 46,
-- 'Apple', '', 'Skilled Manual', 'Bachelors', 28,
-- dynamic(null), 'why', 'Professional', 'Partial College', 38
-- ]
DROP TABLE IF EXISTS Customers;
CREATE TABLE Customers
(
FirstName Nullable(String),
LastName String,
Occupation String,
Education String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Joshua','Lee','Professional','Partial College',26),('Edward','Hernandez','Skilled Manual','High School',36),('Dalton','Wood','Professional','Partial College',42),('Christine','Nara','Skilled Manual','Partial College',33),('Cameron','Rodriguez','Professional','Partial College',28),('Angel','Stewart','Professional','Partial College',46),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38);
drop table if exists EventLog;
create table EventLog
(
LogEntry String,
Created Int64
) ENGINE = Memory;
insert into EventLog values ('Darth Vader has entered the room.', 546), ('Rambo is suspciously looking at Darth Vader.', 245234), ('Darth Sidious electrocutes both using Force Lightning.', 245554);
drop table if exists Dates;
create table Dates
(
EventTime DateTime,
) ENGINE = Memory;
Insert into Dates VALUES ('2015-10-12') , ('2016-10-12')
Select '-- test summarize --' ;
set dialect='kusto';
Customers | summarize count(), min(Age), max(Age), avg(Age), sum(Age);
Customers | summarize count(), min(Age), max(Age), avg(Age), sum(Age) by Occupation | order by Occupation;
Customers | summarize countif(Age>40) by Occupation | order by Occupation;
Customers | summarize MyMax = maxif(Age, Age<40) by Occupation | order by Occupation;
Customers | summarize MyMin = minif(Age, Age<40) by Occupation | order by Occupation;
Customers | summarize MyAvg = avgif(Age, Age<40) by Occupation | order by Occupation;
Customers | summarize MySum = sumif(Age, Age<40) by Occupation | order by Occupation;
Customers | summarize dcount(Education);
Customers | summarize dcountif(Education, Occupation=='Professional');
Customers | summarize count_ = count() by bin(Age, 10) | order by count_ asc;
Customers | summarize job_count = count() by Occupation | where job_count > 0 | order by Occupation;
Customers | summarize 'Edu Count'=count() by Education | sort by 'Edu Count' desc; -- { clientError 62 }
print '-- make_list() --';
Customers | summarize f_list = make_list(Education) by Occupation | sort by Occupation;
Customers | summarize f_list = make_list(Education, 2) by Occupation | sort by Occupation;
print '-- make_list_if() --';
Customers | summarize f_list = make_list_if(FirstName, Age>30) by Occupation | sort by Occupation;
Customers | summarize f_list = make_list_if(FirstName, Age>30, 1) by Occupation | sort by Occupation;
print '-- make_set() --';
Customers | summarize f_list = make_set(Education) by Occupation | sort by Occupation;
Customers | summarize f_list = make_set(Education, 2) by Occupation | sort by Occupation;
print '-- make_set_if() --';
Customers | summarize f_list = make_set_if(Education, Age>30) by Occupation | sort by Occupation;
Customers | summarize f_list = make_set_if(Education, Age>30, 1) by Occupation | sort by Occupation;
print '-- stdev() --';
Customers | project Age | summarize stdev(Age);
print '-- stdevif() --';
Customers | project Age | summarize stdevif(Age, Age%2==0);
print '-- binary_all_and --';
Customers | project Age | where Age > 40 | summarize binary_all_and(Age);
print '-- binary_all_or --';
Customers | project Age | where Age > 40 | summarize binary_all_or(Age);
print '-- binary_all_xor --';
Customers | project Age | where Age > 40 | summarize binary_all_xor(Age);
Customers | project Age | summarize percentile(Age, 95);
Customers | project Age | summarize percentiles(Age, 5, 50, 95)|project round(percentiles_Age[0],2),round(percentiles_Age[1],2),round(percentiles_Age[2],2);
Customers | project Age | summarize percentiles(Age, 5, 50, 95)[1];
Customers | summarize w=count() by AgeBucket=bin(Age, 5) | summarize percentilew(AgeBucket, w, 75);
Customers | summarize w=count() by AgeBucket=bin(Age, 5) | summarize percentilesw(AgeBucket, w, 50, 75, 99.9);
print '-- Summarize following sort --';
Customers | sort by FirstName | summarize count() by Occupation | sort by Occupation;
print '-- summarize with bin --';
EventLog | summarize count=count() by bin(Created, 1000) | sort by count asc;
EventLog | summarize count=count() by bin(unixtime_seconds_todatetime(Created/1000), 1s) | sort by count asc;
EventLog | summarize count=count() by time_label=bin(Created/1000, 1s) | sort by count asc;
Dates | project bin(datetime(EventTime), 1m);
print '-- make_list_with_nulls --';
Customers | summarize t = make_list_with_nulls(FirstName);
Customers | summarize f_list = make_list_with_nulls(FirstName) by Occupation | sort by Occupation;
Customers | summarize f_list = make_list_with_nulls(FirstName), a_list = make_list_with_nulls(Age) by Occupation | sort by Occupation;
-- TODO:
-- arg_max()
-- arg_min()

View File

@ -1,139 +0,0 @@
-- test Query only has table name: --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management Bachelors 33
Peter Nara Skilled Manual Graduate Degree 26
Latoya Shen Professional Graduate Degree 25
Joshua Lee Professional Partial College 26
Edward Hernandez Skilled Manual High School 36
Dalton Wood Professional Partial College 42
Christine Nara Skilled Manual Partial College 33
Cameron Rodriguez Professional Partial College 28
Angel Stewart Professional Partial College 46
-- Query has Column Selection --
Theodore Diaz Skilled Manual
Stephanie Cox Management
Peter Nara Skilled Manual
Latoya Shen Professional
Joshua Lee Professional
Edward Hernandez Skilled Manual
Dalton Wood Professional
Christine Nara Skilled Manual
Cameron Rodriguez Professional
Angel Stewart Professional
-- Query has limit --
Theodore Diaz Skilled Manual
Stephanie Cox Management
Peter Nara Skilled Manual
Latoya Shen Professional
Joshua Lee Professional
Theodore Diaz Skilled Manual
Stephanie Cox Management
Peter Nara Skilled Manual
Latoya Shen Professional
Joshua Lee Professional
-- Query has second limit with bigger value --
Theodore Diaz Skilled Manual
Stephanie Cox Management
Peter Nara Skilled Manual
Latoya Shen Professional
Joshua Lee Professional
-- Query has second limit with smaller value --
Theodore Diaz Skilled Manual
Stephanie Cox Management
Peter Nara Skilled Manual
-- Query has second Column selection --
Theodore Diaz
Stephanie Cox
Peter Nara
-- Query has second Column selection with extra column --
-- Query with desc sort --
Theodore
Stephanie
Peter
Latoya
Joshua
Skilled Manual
Skilled Manual
Professional
Professional
Management
-- Query with asc sort --
Management
Professional
Professional
Skilled Manual
Skilled Manual
-- Query with sort (without keyword asc desc) --
Theodore
Stephanie
Peter
Latoya
Joshua
Skilled Manual
Skilled Manual
Professional
Professional
Management
-- Query with sort 2 Columns with different direction --
Stephanie Cox Management
Latoya Shen Professional
Joshua Lee Professional
Peter Nara Skilled Manual
Theodore Diaz Skilled Manual
-- Query with second sort --
Stephanie Cox Management
Latoya Shen Professional
Joshua Lee Professional
Peter Nara Skilled Manual
Theodore Diaz Skilled Manual
-- Test String Equals (==) --
Theodore Diaz Skilled Manual
Peter Nara Skilled Manual
Edward Hernandez Skilled Manual
Christine Nara Skilled Manual
-- Test String Not equals (!=) --
Stephanie Cox Management
Latoya Shen Professional
Joshua Lee Professional
Dalton Wood Professional
Cameron Rodriguez Professional
Angel Stewart Professional
-- Test Filter using a list (in) --
Theodore Diaz Skilled Manual Bachelors
Stephanie Cox Management Bachelors
Edward Hernandez Skilled Manual High School
-- Test Filter using a list (!in) --
Peter Nara Skilled Manual Graduate Degree
Latoya Shen Professional Graduate Degree
Joshua Lee Professional Partial College
Dalton Wood Professional Partial College
Christine Nara Skilled Manual Partial College
Cameron Rodriguez Professional Partial College
Angel Stewart Professional Partial College
-- Test Filter using common string operations (contains_cs) --
Joshua Lee Professional Partial College
Dalton Wood Professional Partial College
Christine Nara Skilled Manual Partial College
Cameron Rodriguez Professional Partial College
Angel Stewart Professional Partial College
-- Test Filter using common string operations (startswith_cs) --
Latoya Shen Professional Graduate Degree
Joshua Lee Professional Partial College
Dalton Wood Professional Partial College
Cameron Rodriguez Professional Partial College
Angel Stewart Professional Partial College
-- Test Filter using common string operations (endswith_cs) --
Latoya Shen Professional Graduate Degree
Joshua Lee Professional Partial College
-- Test Filter using numerical equal (==) --
Peter Nara Skilled Manual Graduate Degree 26
Joshua Lee Professional Partial College 26
-- Test Filter using numerical great and less (> , <) --
Stephanie Cox Management Bachelors 33
Edward Hernandez Skilled Manual High School 36
Christine Nara Skilled Manual Partial College 33
-- Test Filter using multi where --
Dalton Wood Professional Partial College 42
Angel Stewart Professional Partial College 46
-- Complex query with unknown function --
-- Missing column in front of startsWith --

View File

@ -1,88 +0,0 @@
DROP TABLE IF EXISTS Customers;
CREATE TABLE Customers
(
FirstName Nullable(String),
LastName String,
Occupation String,
Education String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28), ('Stephanie','Cox','Management','Bachelors',33), ('Peter','Nara','Skilled Manual','Graduate Degree',26), ('Latoya','Shen','Professional','Graduate Degree',25), ('Joshua','Lee','Professional','Partial College',26), ('Edward','Hernandez','Skilled Manual','High School',36), ('Dalton','Wood','Professional','Partial College',42), ('Christine','Nara','Skilled Manual','Partial College',33), ('Cameron','Rodriguez','Professional','Partial College',28), ('Angel','Stewart','Professional','Partial College',46);
set dialect='kusto';
print '-- test Query only has table name: --';
Customers;
print '-- Query has Column Selection --';
Customers | project FirstName,LastName,Occupation;
print '-- Query has limit --';
Customers | project FirstName,LastName,Occupation | take 5;
Customers | project FirstName,LastName,Occupation | limit 5;
print '-- Query has second limit with bigger value --';
Customers | project FirstName,LastName,Occupation | take 5 | take 7;
print '-- Query has second limit with smaller value --';
Customers | project FirstName,LastName,Occupation | take 5 | take 3;
print '-- Query has second Column selection --';
Customers | project FirstName,LastName,Occupation | take 3 | project FirstName,LastName;
print '-- Query has second Column selection with extra column --';
Customers| project FirstName,LastName,Occupation | take 3 | project FirstName,LastName,Education;-- { serverError 47 }
print '-- Query with desc sort --';
Customers | project FirstName | take 5 | sort by FirstName desc;
Customers | project Occupation | take 5 | order by Occupation desc;
print '-- Query with asc sort --';
Customers | project Occupation | take 5 | sort by Occupation asc;
print '-- Query with sort (without keyword asc desc) --';
Customers | project FirstName | take 5 | sort by FirstName;
Customers | project Occupation | take 5 | order by Occupation;
print '-- Query with sort 2 Columns with different direction --';
Customers | project FirstName,LastName,Occupation | take 5 | sort by Occupation asc, LastName desc;
print '-- Query with second sort --';
Customers | project FirstName,LastName,Occupation | take 5 | sort by Occupation desc |sort by Occupation asc, LastName desc;
print '-- Test String Equals (==) --';
Customers | project FirstName,LastName,Occupation | where Occupation == 'Skilled Manual';
print '-- Test String Not equals (!=) --';
Customers | project FirstName,LastName,Occupation | where Occupation != 'Skilled Manual';
print '-- Test Filter using a list (in) --';
Customers | project FirstName,LastName,Occupation,Education | where Education in ('Bachelors','High School');
print '-- Test Filter using a list (!in) --';
set dialect='kusto';
Customers | project FirstName,LastName,Occupation,Education | where Education !in ('Bachelors','High School');
print '-- Test Filter using common string operations (contains_cs) --';
Customers | project FirstName,LastName,Occupation,Education | where Education contains_cs 'Coll';
print '-- Test Filter using common string operations (startswith_cs) --';
Customers | project FirstName,LastName,Occupation,Education | where Occupation startswith_cs 'Prof';
print '-- Test Filter using common string operations (endswith_cs) --';
Customers | project FirstName,LastName,Occupation,Education | where FirstName endswith_cs 'a';
print '-- Test Filter using numerical equal (==) --';
Customers | project FirstName,LastName,Occupation,Education,Age | where Age == 26;
print '-- Test Filter using numerical great and less (> , <) --';
Customers | project FirstName,LastName,Occupation,Education,Age | where Age > 30 and Age < 40;
print '-- Test Filter using multi where --';
Customers | project FirstName,LastName,Occupation,Education,Age | where Age > 30 | where Occupation == 'Professional';
print '-- Complex query with unknown function --';
hits | where CounterID == 62 and EventDate >= '2013-07-14' and EventDate <= '2013-07-15' and IsRefresh == 0 and DontCountHits == 0 | summarize count() by d=bin(poopoo(EventTime), 1m) | order by d | limit 10; -- { clientError UNKNOWN_FUNCTION }
print '-- Missing column in front of startsWith --';
StormEvents | where startswith "W" | summarize Count=count() by State; -- { clientError SYNTAX_ERROR }

View File

@ -5,6 +5,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# In case of parallel parsing and small block
# (--min_chunk_bytes_for_parallel_parsing) we may have multiple blocks, and
# this will break sorting order, so let's limit number of threads to avoid
# reordering.
CLICKHOUSE_CLIENT+="--allow_repeated_settings --max_threads 1"
echo "Integers"
$CLICKHOUSE_CLIENT -q "insert into function file(02475_data.bsonEachRow) select number::Bool as bool, number::Int8 as int8, number::UInt8 as uint8, number::Int16 as int16, number::UInt16 as uint16, number::Int32 as int32, number::UInt32 as uint32, number::Int64 as int64, number::UInt64 as uint64 from numbers(5) settings engine_file_truncate_on_insert=1"
$CLICKHOUSE_CLIENT -q "select * from file(02475_data.bsonEachRow, auto, 'bool Bool, int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64')"

View File

@ -0,0 +1,2 @@
0
1000000 Hello, world Hello, world

View File

@ -0,0 +1,31 @@
#!/usr/bin/env bash
# Tags: no-parallel, no-fasttest
# Because we are creating a backup with fixed path.
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT --multiquery "
DROP TABLE IF EXISTS test;
CREATE TABLE test (x String) ENGINE = Memory SETTINGS compress = 1;
INSERT INTO test SELECT 'Hello, world' FROM numbers(1000000);
"
$CLICKHOUSE_CLIENT --multiquery "
BACKUP TABLE test TO File('test.zip');
" --format Null
$CLICKHOUSE_CLIENT --multiquery "
TRUNCATE TABLE test;
SELECT count() FROM test;
"
$CLICKHOUSE_CLIENT --multiquery "
RESTORE TABLE test FROM File('test.zip');
" --format Null
$CLICKHOUSE_CLIENT --multiquery "
SELECT count(), min(x), max(x) FROM test;
DROP TABLE test;
"

View File

@ -68,73 +68,3 @@ Simple functions with non compilable function without key
4611686725751467379 9223371678237104442 3626326766789368100 61384643584599682996279588 408650940859.2896 104735.01095549858 8873898 9223372036854775807 4611686018427387904 3818489297630359920
Simple functions if combinator without key
4611687533683519016 9223371678237104442 4124667747700004330 930178817930.5122 321189.2280948817 4434274 9223372036854775806 4611686018427387904 2265422677606390266
Aggregation without JIT compilation
Simple functions
1704509 4611700827100483880 9223360787015464643 10441337359398154812 19954243669348.844 9648741.579254271 523264 9223372036854775807 4611686018427387904 4544239379628300646
732797 4611701940806302259 9223355550934604746 977192643464016658 2054229034942.3723 51998323.94457991 475698 9223372036854775807 4611686018427387904 4091184823334377716
598875 4611701407242345792 9223362250391155632 9312163881623734456 27615161624211.875 12261797.824844675 337212 9223372036854775807 4611686018427387904 3725992504798702670
792887 4611699550286611812 9223290551912005343 6930300520201292824 27479710385933.586 53095331.60360441 252197 9223372036854775807 4611686018427387904 6536441508464694614
3807842 4611710821592843606 9223326163906184987 16710274896338005145 85240848090850.69 22373416.533275086 196036 9223372036854775807 4611686018427387904 1797862753609257231
25703952 4611709443519524003 9223353913449113943 9946868158853570839 67568783303242.086 3154349.826950714 147211 9223372036854775807 4611686018427387904 8737124378202300429
716829 4611852156092872082 9223361623076951140 15381015774917924786 170693446547158.72 201431892.4773785 90109 9223372036854775807 4611686018427387904 8209915323001116338
59183 4611730685242027332 9223354909338698162 8078812522502896568 94622946187035.42 1425270865.0901496 85379 9223372036854775807 4611686018427387904 8909082036598843562
33010362 4611704682869732882 9223268545373999677 2064452191838585926 26532987929602.555 3695122.4062526934 77807 9223372036854775807 4611686018427387904 5411365383789552292
800784 4611752907938305166 9223340418389788041 18082918611792817587 233352070043266.62 36535786.81446395 77492 9223372036854775807 4611686018427387904 2059255810151375435
20810645 4611712185532639162 9223218900001937412 4996531385439292694 68246505203164.63 6316535.831023813 73213 9223372036854775807 4611686018427387904 8852740550386113674
25843850 4611690025407720929 9223346023778617822 12755881190906812868 185015319325648.16 9962165.34831339 68945 9223372036854775807 4611686018427387904 7849665866595760148
23447120 4611796031755620254 9223329309291309758 17231649548755339966 255019232629204.38 7937191.271698021 67570 9223372036854775807 4611686018427387904 3435410911925610424
14739804 4611692230555590277 9223313509005166531 2458378896777063244 38308020331864.36 14590240.469105456 64174 9223372036854775807 4611686018427387904 511910855240035342
32077710 4611884228437061959 9223352444952988904 12965822147651192908 214467085941034.7 7257521.096258734 60456 9223372036854775807 4611686018427387904 2256071920672551964
22446879 4611846229717089436 9223124373140579096 13530160492087688838 231724477077663.4 4737362.521046629 58389 9223372036854775807 4611686018427387904 6236276364886386410
170282 4611833225706935900 9223371583739401906 8076893424988479310 141657635880324.8 1613795518.1065989 57017 9223372036854775807 4611686018427387904 4755775861151848768
11482817 4611708000353743073 9223337838355779113 14841435427430843458 283531099960470.8 9938452.835998287 52345 9223372036854775807 4611686018427387904 5371586112642152558
63469 4611695097019173921 9223353530156141191 6296784708578574520 120762239817777.88 579655378.4603049 52142 9223372036854775807 4611686018427387904 4150567963952988110
29103473 4611744585914335132 9223333530281362537 5908285283932344933 123712996438970.34 867841.595541967 47758 9223372036854775807 4611686018427387904 3238284030821087319
Simple functions with non compilable function
1704509 4611700827100483880 9223360787015464643 10441337359398154812 3620921835565807284859452 19954243669348.844 9648741.579254271 523264 9223372036854775807 4611686018427387904 4544239379628300646
732797 4611701940806302259 9223355550934604746 977192643464016658 3289442827160604417733394 2054229034942.3723 51998323.94457991 475698 9223372036854775807 4611686018427387904 4091184823334377716
598875 4611701407242345792 9223362250391155632 9312163881623734456 2330921446573746856380600 27615161624211.875 12261797.824844675 337212 9223372036854775807 4611686018427387904 3725992504798702670
792887 4611699550286611812 9223290551912005343 6930300520201292824 1745179600137886041476120 27479710385933.586 53095331.60360441 252197 9223372036854775807 4611686018427387904 6536441508464694614
3807842 4611710821592843606 9223326163906184987 16710274896338005145 1356295121550317411019929 85240848090850.69 22373416.533275086 196036 9223372036854775807 4611686018427387904 1797862753609257231
25703952 4611709443519524003 9223353913449113943 9946868158853570839 1018731388338768841564439 67568783303242.086 3154349.826950714 147211 9223372036854775807 4611686018427387904 8737124378202300429
716829 4611852156092872082 9223361623076951140 15381015774917924786 623810478612337115371442 170693446547158.72 201431892.4773785 90109 9223372036854775807 4611686018427387904 8209915323001116338
59183 4611730685242027332 9223354909338698162 8078812522502896568 589916507545680254024632 94622946187035.42 1425270865.0901496 85379 9223372036854775807 4611686018427387904 8909082036598843562
33010362 4611704682869732882 9223268545373999677 2064452191838585926 538517864195994778911814 26532987929602.555 3695122.4062526934 77807 9223372036854775807 4611686018427387904 5411365383789552292
800784 4611752907938305166 9223340418389788041 18082918611792817587 535545510122473785781683 233352070043266.62 36535786.81446395 77492 9223372036854775807 4611686018427387904 2059255810151375435
20810645 4611712185532639162 9223218900001937412 4996531385439292694 506405014842860050255126 68246505203164.63 6316535.831023813 73213 9223372036854775807 4611686018427387904 8852740550386113674
25843850 4611690025407720929 9223346023778617822 12755881190906812868 476547495537329753708996 185015319325648.16 9962165.34831339 68945 9223372036854775807 4611686018427387904 7849665866595760148
23447120 4611796031755620254 9223329309291309758 17231649548755339966 467236365548464278670014 255019232629204.38 7937191.271698021 67570 9223372036854775807 4611686018427387904 3435410911925610424
14739804 4611692230555590277 9223313509005166531 2458378896777063244 444126268697527941770060 38308020331864.36 14590240.469105456 64174 9223372036854775807 4611686018427387904 511910855240035342
32077710 4611884228437061959 9223352444952988904 12965822147651192908 417407443977973675608140 214467085941034.7 7257521.096258734 60456 9223372036854775807 4611686018427387904 2256071920672551964
22446879 4611846229717089436 9223124373140579096 13530160492087688838 403462269796593691082374 231724477077663.4 4737362.521046629 58389 9223372036854775807 4611686018427387904 6236276364886386410
170282 4611833225706935900 9223371583739401906 8076893424988479310 394417911933408911581006 141657635880324.8 1613795518.1065989 57017 9223372036854775807 4611686018427387904 4755775861151848768
11482817 4611708000353743073 9223337838355779113 14841435427430843458 361995300393829962204226 283531099960470.8 9938452.835998287 52345 9223372036854775807 4611686018427387904 5371586112642152558
63469 4611695097019173921 9223353530156141191 6296784708578574520 360843057610541117735096 120762239817777.88 579655378.4603049 52142 9223372036854775807 4611686018427387904 4150567963952988110
29103473 4611744585914335132 9223333530281362537 5908285283932344933 330534668598011678200421 123712996438970.34 867841.595541967 47758 9223372036854775807 4611686018427387904 3238284030821087319
Simple functions if combinator
1704509 4611700827100483880 9223310246721229500 16398241567152875142 62618822667209.71 2224726.7626273884 261874 9223372036854775806 4611686018427387904 4518874482384062894
732797 4611721382223060002 9223355550934604746 16281585268876620522 68472164943295.68 5898616.931652982 237784 9223372036854775806 4611686018427387904 3641900047478154650
598875 4611701407242345792 9223362250391155632 3577699408183553052 21300140553347.42 53771550.26565126 167966 9223372036854775806 4611686018427387904 1688477495230210408
792887 4611699550286611812 9223164887726235740 7088177025760385824 56461952267903.89 92835869.96920013 125539 9223372036854775806 4611686018427387904 4850868151095058072
3807842 4611710821592843606 9223283397553859544 5756765290752687660 58835559208469.4 39794091.419183925 97845 9223372036854775806 4611686018427387904 6845214684357194564
25703952 4611784761593342388 9223241341744449690 4782279928971192568 65182094768443.91 9276773.708181158 73368 9223372036854775806 4611686018427387904 1384302533387727316
716829 4611852156092872082 9223361623076951140 8613712481895484190 191445613359755.62 291083243.75407773 44993 9223372036854775806 4611686018427387904 6344483471397203854
59183 4611730685242027332 9223354909338698162 18369075291092794110 429013599530392 5925109959.715378 42817 9223372036854775806 4611686018427387904 5909305558020042898
33010362 4611704682869732882 9223092117352620518 9991152681891671022 257099731913529.5 12412830.045471078 38861 9223372036854775806 4611686018427387904 4672855013852508626
800784 4611752907938305166 9223309994342931384 5251877538869750510 135472890315726.03 53535427.52018088 38767 9223372036854775806 4611686018427387904 7801864489649220514
20810645 4611712185532639162 9223218900001937412 11803718472901310700 323593455407553 10496765.20741332 36477 9223372036854775806 4611686018427387904 5941995311893397960
25843850 4611744529689964352 9223346023778617822 127137885677350808 3700925266420.715 18966925.191309396 34353 9223372036854775806 4611686018427387904 6700111718676827412
23447120 4611796031755620254 9223329309291309758 1841522159325376278 54534534450526.42 6271211.193812284 33768 9223372036854775806 4611686018427387904 2325654077031843898
14739804 4611762063154116632 9223007205463222212 16302703534054321116 506987919332451.8 6885575.861759452 32156 9223372036854775806 4611686018427387904 2114922310535979832
32077710 4612033458080771112 9223352444952988904 421072759851674408 13955745719596.793 12220152.393889504 30172 9223372036854775806 4611686018427387904 4399934528735249092
22446879 4611846229717089436 9223124373140579096 6577134317587565298 224866980668999.47 2482202.163802278 29249 9223372036854775806 4611686018427387904 8763910740678180498
170282 4611833225706935900 9223371583739401906 15764226366913732386 551447384017691 2515144222.953728 28587 9223372036854775806 4611686018427387904 8217388408377809010
11482817 4611990575414646848 9223302669582414438 9828522700609834800 378121905921203.2 34845264.2080656 25993 9223372036854775806 4611686018427387904 4689180182672571856
63469 4612175339998036670 9222961628400798084 17239621485933250238 663164390134376.5 7825349797.6059 25996 9223372036854775806 4611686018427387904 2067736879306995526
29103473 4611744585914335132 9223035551850347954 12590190375872647672 525927999326314.7 26049107.15514301 23939 9223372036854775806 4611686018427387904 8318055464870862444
Simple functions without key
4611686725751467379 9223371678237104442 3626326766789368100 408650940859.2896 104735.01095549858 8873898 9223372036854775807 4611686018427387904 3818489297630359920
Simple functions with non compilable function without key
4611686725751467379 9223371678237104442 3626326766789368100 61384643584599682996279588 408650940859.2896 104735.01095549858 8873898 9223372036854775807 4611686018427387904 3818489297630359920
Simple functions if combinator without key
4611687533683519016 9223371678237104442 4124667747700004330 930178817930.5122 321189.2280948817 4434274 9223372036854775806 4611686018427387904 2265422677606390266

View File

@ -1,4 +1,3 @@
SET compile_aggregate_expressions = 1;
SET min_count_to_compile_aggregate_expression = 0;
-- The test uses many aggregations. A low max_bytes_before_external_group_by value will lead to high disk usage
-- which in CI leads to timeouts
@ -103,104 +102,3 @@ SELECT
FROM test.hits
ORDER BY min_watch_id
DESC LIMIT 20;
SET compile_aggregate_expressions = 0;
SELECT 'Aggregation without JIT compilation';
SELECT 'Simple functions';
SELECT
CounterID,
min(WatchID),
max(WatchID),
sum(WatchID),
avg(WatchID),
avgWeighted(WatchID, CounterID),
count(WatchID),
groupBitOr(WatchID),
groupBitAnd(WatchID),
groupBitXor(WatchID)
FROM test.hits
GROUP BY CounterID ORDER BY count() DESC LIMIT 20;
SELECT 'Simple functions with non compilable function';
SELECT
CounterID,
min(WatchID),
max(WatchID),
sum(WatchID),
sum(toUInt128(WatchID)),
avg(WatchID),
avgWeighted(WatchID, CounterID),
count(WatchID),
groupBitOr(WatchID),
groupBitAnd(WatchID),
groupBitXor(WatchID)
FROM test.hits
GROUP BY CounterID ORDER BY count() DESC LIMIT 20;
SELECT 'Simple functions if combinator';
WITH (WatchID % 2 == 0) AS predicate
SELECT
CounterID,
minIf(WatchID,predicate),
maxIf(WatchID, predicate),
sumIf(WatchID, predicate),
avgIf(WatchID, predicate),
avgWeightedIf(WatchID, CounterID, predicate),
countIf(WatchID, predicate),
groupBitOrIf(WatchID, predicate),
groupBitAndIf(WatchID, predicate),
groupBitXorIf(WatchID, predicate)
FROM test.hits
GROUP BY CounterID ORDER BY count() DESC LIMIT 20;
SELECT 'Simple functions without key';
SELECT
min(WatchID) AS min_watch_id,
max(WatchID),
sum(WatchID),
avg(WatchID),
avgWeighted(WatchID, CounterID),
count(WatchID),
groupBitOr(WatchID),
groupBitAnd(WatchID),
groupBitXor(WatchID)
FROM test.hits
ORDER BY min_watch_id DESC LIMIT 20;
SELECT 'Simple functions with non compilable function without key';
SELECT
min(WatchID) AS min_watch_id,
max(WatchID),
sum(WatchID),
sum(toUInt128(WatchID)),
avg(WatchID),
avgWeighted(WatchID, CounterID),
count(WatchID),
groupBitOr(WatchID),
groupBitAnd(WatchID),
groupBitXor(WatchID)
FROM test.hits
ORDER BY min_watch_id DESC LIMIT 20;
SELECT 'Simple functions if combinator without key';
WITH (WatchID % 2 == 0) AS predicate
SELECT
minIf(WatchID, predicate) as min_watch_id,
maxIf(WatchID, predicate),
sumIf(WatchID, predicate),
avgIf(WatchID, predicate),
avgWeightedIf(WatchID, CounterID, predicate),
countIf(WatchID, predicate),
groupBitOrIf(WatchID, predicate),
groupBitAndIf(WatchID, predicate),
groupBitXorIf(WatchID, predicate)
FROM test.hits
ORDER BY min_watch_id
DESC LIMIT 20;