mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #34561 from bigo-sg/arrow_type_timestamp
Implement transformation between CH DateTime64 and arrow timestamp column
This commit is contained in:
commit
5e8b2228e0
@ -14,6 +14,7 @@
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
#include <base/types.h>
|
||||
#include <Processors/Chunk.h>
|
||||
@ -206,38 +207,19 @@ static ColumnWithTypeAndName readColumnWithDate64Data(std::shared_ptr<arrow::Chu
|
||||
|
||||
static ColumnWithTypeAndName readColumnWithTimestampData(std::shared_ptr<arrow::ChunkedArray> & arrow_column, const String & column_name)
|
||||
{
|
||||
auto internal_type = std::make_shared<DataTypeUInt32>();
|
||||
const auto & arrow_type = static_cast<const arrow::TimestampType &>(*(arrow_column->type()));
|
||||
const UInt8 scale = arrow_type.unit() * 3;
|
||||
auto internal_type = std::make_shared<DataTypeDateTime64>(scale, arrow_type.timezone());
|
||||
auto internal_column = internal_type->createColumn();
|
||||
auto & column_data = assert_cast<ColumnVector<UInt32> &>(*internal_column).getData();
|
||||
auto & column_data = assert_cast<ColumnDecimal<DateTime64> &>(*internal_column).getData();
|
||||
column_data.reserve(arrow_column->length());
|
||||
|
||||
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i)
|
||||
{
|
||||
auto & chunk = dynamic_cast<arrow::TimestampArray &>(*(arrow_column->chunk(chunk_i)));
|
||||
const auto & type = static_cast<const ::arrow::TimestampType &>(*chunk.type());
|
||||
|
||||
UInt32 divide = 1;
|
||||
const auto unit = type.unit();
|
||||
switch (unit)
|
||||
{
|
||||
case arrow::TimeUnit::SECOND:
|
||||
divide = 1;
|
||||
break;
|
||||
case arrow::TimeUnit::MILLI:
|
||||
divide = 1000;
|
||||
break;
|
||||
case arrow::TimeUnit::MICRO:
|
||||
divide = 1000000;
|
||||
break;
|
||||
case arrow::TimeUnit::NANO:
|
||||
divide = 1000000000;
|
||||
break;
|
||||
}
|
||||
|
||||
const auto & chunk = dynamic_cast<const arrow::TimestampArray &>(*(arrow_column->chunk(chunk_i)));
|
||||
for (size_t value_i = 0, length = static_cast<size_t>(chunk.length()); value_i < length; ++value_i)
|
||||
{
|
||||
auto timestamp = static_cast<UInt32>(chunk.Value(value_i) / divide); // ms! TODO: check other 's' 'ns' ...
|
||||
column_data.emplace_back(timestamp);
|
||||
column_data.emplace_back(chunk.Value(value_i));
|
||||
}
|
||||
}
|
||||
return {std::move(internal_column), std::move(internal_type), column_name};
|
||||
@ -485,7 +467,6 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
|
||||
return readColumnWithNumericData<CPP_NUMERIC_TYPE>(arrow_column, column_name);
|
||||
FOR_ARROW_NUMERIC_TYPES(DISPATCH)
|
||||
# undef DISPATCH
|
||||
// TODO: support TIMESTAMP_MICROS and TIMESTAMP_MILLIS with truncated micro- and milliseconds?
|
||||
// TODO: read JSON as a string?
|
||||
// TODO: read UUID as a string?
|
||||
default:
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#if USE_ARROW || USE_PARQUET
|
||||
|
||||
// #include <base/Decimal.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
@ -17,6 +18,7 @@
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <Processors/Formats/IOutputFormat.h>
|
||||
#include <arrow/api.h>
|
||||
#include <arrow/builder.h>
|
||||
@ -55,6 +57,7 @@ namespace DB
|
||||
extern const int UNKNOWN_EXCEPTION;
|
||||
extern const int UNKNOWN_TYPE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int DECIMAL_OVERFLOW;
|
||||
}
|
||||
|
||||
static const std::initializer_list<std::pair<String, std::shared_ptr<arrow::DataType>>> internal_type_to_arrow_type =
|
||||
@ -120,6 +123,42 @@ namespace DB
|
||||
checkStatus(status, write_column->getName(), format_name);
|
||||
}
|
||||
|
||||
static void fillArrowArrayWithDateTime64ColumnData(
|
||||
const DataTypeDateTime64 * type,
|
||||
ColumnPtr write_column,
|
||||
const PaddedPODArray<UInt8> * null_bytemap,
|
||||
const String & format_name,
|
||||
arrow::ArrayBuilder* array_builder,
|
||||
size_t start,
|
||||
size_t end)
|
||||
{
|
||||
const auto & column = assert_cast<const ColumnDecimal<DateTime64> &>(*write_column);
|
||||
arrow::TimestampBuilder & builder = assert_cast<arrow::TimestampBuilder &>(*array_builder);
|
||||
arrow::Status status;
|
||||
|
||||
auto scale = type->getScale();
|
||||
bool need_rescale = scale % 3;
|
||||
auto rescale_multiplier = DecimalUtils::scaleMultiplier<DateTime64::NativeType>(3 - scale % 3);
|
||||
for (size_t value_i = start; value_i < end; ++value_i)
|
||||
{
|
||||
if (null_bytemap && (*null_bytemap)[value_i])
|
||||
{
|
||||
status = builder.AppendNull();
|
||||
}
|
||||
else
|
||||
{
|
||||
auto value = static_cast<Int64>(column[value_i].get<DecimalField<DateTime64>>().getValue());
|
||||
if (need_rescale)
|
||||
{
|
||||
if (common::mulOverflow(value, rescale_multiplier, value))
|
||||
throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW);
|
||||
}
|
||||
status = builder.Append(value);
|
||||
}
|
||||
checkStatus(status, write_column->getName(), format_name);
|
||||
}
|
||||
}
|
||||
|
||||
static void fillArrowArray(
|
||||
const String & column_name,
|
||||
ColumnPtr & column,
|
||||
@ -480,6 +519,11 @@ namespace DB
|
||||
if (!callOnIndexAndDataType<void>(column_type->getTypeId(), fill_decimal))
|
||||
throw Exception{ErrorCodes::LOGICAL_ERROR, "Cannot fill arrow array with decimal data with type {}", column_type_name};
|
||||
}
|
||||
else if (isDateTime64(column_type))
|
||||
{
|
||||
const auto * datetime64_type = assert_cast<const DataTypeDateTime64 *>(column_type.get());
|
||||
fillArrowArrayWithDateTime64ColumnData(datetime64_type, column, null_bytemap, format_name, array_builder, start, end);
|
||||
}
|
||||
#define DISPATCH(CPP_NUMERIC_TYPE, ARROW_BUILDER_TYPE) \
|
||||
else if (#CPP_NUMERIC_TYPE == column_type_name) \
|
||||
{ \
|
||||
@ -546,6 +590,18 @@ namespace DB
|
||||
}
|
||||
}
|
||||
|
||||
static arrow::TimeUnit::type getArrowTimeUnit(const DataTypeDateTime64 * type)
|
||||
{
|
||||
UInt32 scale = type->getScale();
|
||||
if (scale == 0)
|
||||
return arrow::TimeUnit::SECOND;
|
||||
if (scale > 0 && scale <= 3)
|
||||
return arrow::TimeUnit::MILLI;
|
||||
if (scale > 3 && scale <= 6)
|
||||
return arrow::TimeUnit::MICRO;
|
||||
return arrow::TimeUnit::NANO;
|
||||
}
|
||||
|
||||
static std::shared_ptr<arrow::DataType> getArrowType(
|
||||
DataTypePtr column_type, ColumnPtr column, const std::string & column_name, const std::string & format_name, bool * out_is_column_nullable)
|
||||
{
|
||||
@ -638,6 +694,12 @@ namespace DB
|
||||
return arrow_type_it->second;
|
||||
}
|
||||
|
||||
if (isDateTime64(column_type))
|
||||
{
|
||||
const auto * datetime64_type = assert_cast<const DataTypeDateTime64 *>(column_type.get());
|
||||
return arrow::timestamp(getArrowTimeUnit(datetime64_type), datetime64_type->getTimeZone().getTimeZone());
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::UNKNOWN_TYPE,
|
||||
"The type '{}' of a column '{}' is not supported for conversion into {} data format.",
|
||||
column_type->getName(), column_name, format_name);
|
||||
|
@ -34,32 +34,32 @@
|
||||
ContextLock Number of times the lock of Context was acquired or tried to acquire. This is global lock.
|
||||
Query Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries.
|
||||
original:
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 2003-02-03 04:05:06.789000000
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 2002-02-03 04:05:06.789000000
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06 2004-02-03 04:05:06.789000000
|
||||
converted:
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 2003-02-03 04:05:06.789000000
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 2002-02-03 04:05:06.789000000
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06 2004-02-03 04:05:06.789000000
|
||||
diff:
|
||||
dest:
|
||||
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 00:00:00
|
||||
80 81 82 83 84 85 86 87 88 89 str02 fstr2\0\0\0\0\0\0\0\0\0\0 2005-03-04 2006-08-09 10:11:12
|
||||
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 00:00:00 2004-05-06 07:08:09.012000000
|
||||
80 81 82 83 84 85 86 87 88 89 str02 fstr2\0\0\0\0\0\0\0\0\0\0 2005-03-04 2006-08-09 10:11:12 2006-08-09 10:11:12.345000000
|
||||
min:
|
||||
-128 0 0 0 0 0 0 0 -1 -1 string-1\0\0\0\0\0\0\0 fixedstring-1\0\0 2003-04-05 2003-02-03
|
||||
-108 108 8 92 -8 108 -40 -116 -1 -1 string-0\0\0\0\0\0\0\0 fixedstring\0\0\0\0 2001-02-03 2002-02-03
|
||||
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06
|
||||
127 -1 -1 -1 -1 -1 -1 -1 -1 -1 string-2\0\0\0\0\0\0\0 fixedstring-2\0\0 2004-06-07 2004-02-03
|
||||
-128 0 0 0 0 0 0 0 -1 -1 string-1\0\0\0\0\0\0\0 fixedstring-1\0\0 2003-04-05 2003-02-03 2003-02-03 04:05:06.789000000
|
||||
-108 108 8 92 -8 108 -40 -116 -1 -1 string-0\0\0\0\0\0\0\0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 2002-02-03 04:05:06.789000000
|
||||
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 2004-05-06 07:08:09.012000000
|
||||
127 -1 -1 -1 -1 -1 -1 -1 -1 -1 string-2\0\0\0\0\0\0\0 fixedstring-2\0\0 2004-06-07 2004-02-03 2004-02-03 04:05:06.789000000
|
||||
max:
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 2003-04-05 00:00:00 2003-02-03 04:05:06
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 2001-02-03 00:00:00 2002-02-03 04:05:06
|
||||
80 81 82 83 84 85 86 87 88 89 str02 fstr2 2005-03-04 05:06:07 2006-08-09 10:11:12
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 2004-06-07 00:00:00 2004-02-03 04:05:06
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 2003-04-05 00:00:00 2003-02-03 04:05:06 2003-02-03 04:05:06.789000000
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 2001-02-03 00:00:00 2002-02-03 04:05:06 2002-02-03 04:05:06.789000000
|
||||
80 81 82 83 84 85 86 87 88 89 str02 fstr2 2005-03-04 05:06:07 2006-08-09 10:11:12 2006-08-09 10:11:12.345000000
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 2004-06-07 00:00:00 2004-02-03 04:05:06 2004-02-03 04:05:06.789000000
|
||||
dest from null:
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06
|
||||
\N \N \N \N \N \N \N \N \N \N \N \N \N \N
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 2003-02-03 04:05:06.789000000
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 2002-02-03 04:05:06.789000000
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06 2004-02-03 04:05:06.789000000
|
||||
\N \N \N \N \N \N \N \N \N \N \N \N \N \N \N
|
||||
1 [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.2,10,4] [4,10000.1,10000.1] [1000000000,90,101001.01]
|
||||
1 [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.2,10,4] [4,10000.1,10000.1] [1000000000,90,101001.01]
|
||||
2 [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] []
|
||||
|
@ -46,20 +46,20 @@ ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types1"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types2"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types3"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types4"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory"
|
||||
# convert min type
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types3 (int8 Int8, uint8 Int8, int16 Int8, uint16 Int8, int32 Int8, uint32 Int8, int64 Int8, uint64 Int8, float32 Int8, float64 Int8, string FixedString(15), fixedstring FixedString(15), date Date, datetime Date) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types3 (int8 Int8, uint8 Int8, int16 Int8, uint16 Int8, int32 Int8, uint32 Int8, int64 Int8, uint64 Int8, float32 Int8, float64 Int8, string FixedString(15), fixedstring FixedString(15), date Date, datetime Date, datetime64 DateTime64(9)) ENGINE = Memory"
|
||||
# convert max type
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types4 (int8 Int64, uint8 Int64, int16 Int64, uint16 Int64, int32 Int64, uint32 Int64, int64 Int64, uint64 Int64, float32 Int64, float64 Int64, string String, fixedstring String, date DateTime, datetime DateTime) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types4 (int8 Int64, uint8 Int64, int16 Int64, uint16 Int64, int32 Int64, uint32 Int64, int64 Int64, uint64 Int64, float32 Int64, float64 Int64, string String, fixedstring String, date DateTime, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -108, 108, -1016, 1116, -1032, 1132, -1064, 1164, -1.032, -1.064, 'string-0', 'fixedstring', '2001-02-03', '2002-02-03 04:05:06')"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -108, 108, -1016, 1116, -1032, 1132, -1064, 1164, -1.032, -1.064, 'string-0', 'fixedstring', '2001-02-03', '2002-02-03 04:05:06', toDateTime64('2002-02-03 04:05:06.789', 9))"
|
||||
|
||||
# min
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -128, 0, -32768, 0, -2147483648, 0, -9223372036854775808, 0, -1.032, -1.064, 'string-1', 'fixedstring-1', '2003-04-05', '2003-02-03 04:05:06')"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -128, 0, -32768, 0, -2147483648, 0, -9223372036854775808, 0, -1.032, -1.064, 'string-1', 'fixedstring-1', '2003-04-05', '2003-02-03 04:05:06', toDateTime64('2003-02-03 04:05:06.789', 9))"
|
||||
|
||||
# max
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06')"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06', toDateTime64('2004-02-03 04:05:06.789', 9))"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet"
|
||||
|
||||
@ -73,11 +73,11 @@ echo diff:
|
||||
diff "${CLICKHOUSE_TMP}"/parquet_all_types_1.dump "${CLICKHOUSE_TMP}"/parquet_all_types_2.dump
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_types2"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types3 values ( 79, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str01', 'fstr1', '2003-03-04', '2004-05-06')"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types3 values ( 79, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str01', 'fstr1', '2003-03-04', '2004-05-06', toDateTime64('2004-05-06 07:08:09.012', 9))"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types3 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types3 FORMAT Parquet"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types4 values ( 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str02', 'fstr2', '2005-03-04 05:06:07', '2006-08-09 10:11:12')"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types4 values ( 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str02', 'fstr2', '2005-03-04 05:06:07', '2006-08-09 10:11:12', toDateTime64('2006-08-09 10:11:12.345', 9))"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types4 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types4 FORMAT Parquet"
|
||||
|
||||
@ -92,9 +92,9 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types4 ORDER BY int8"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types5"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types6"
|
||||
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_types2"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types5 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime)) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime)) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types5 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types5 ORDER BY int8 FORMAT Parquet" > "${CLICKHOUSE_TMP}"/parquet_all_types_5.parquet
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types5 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types6 FORMAT Parquet"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types6 FORMAT Parquet"
|
||||
|
@ -1,4 +1,4 @@
|
||||
0 0 0 0 0 2019-01-01 test1
|
||||
2147483647 -1 9223372036854775806 123.345345 345345.3453451212 2019-01-01 test2
|
||||
0 0 0 0 0 2019-01-01 test1
|
||||
2147483647 -1 9223372036854775806 123.345345 345345.3453451212 2019-01-01 test2
|
||||
0 0 0 0 0 2019-01-01 test1 2019-01-01 02:03:04.567
|
||||
2147483647 -1 9223372036854775806 123.345345 345345.3453451212 2019-01-01 test2 2019-01-01 02:03:04.567
|
||||
0 0 0 0 0 2019-01-01 test1 2019-01-01 02:03:04.567
|
||||
2147483647 -1 9223372036854775806 123.345345 345345.3453451212 2019-01-01 test2 2019-01-01 02:03:04.567
|
||||
|
@ -8,9 +8,13 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
DATA_FILE=$CUR_DIR/data_orc/test.orc
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (int Int32, smallint Int8, bigint Int64, float Float32, double Float64, date Date, y String) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (int Int32, smallint Int8, bigint Int64, float Float32, double Float64, date Date, y String, datetime64 DateTime64(3)) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="insert into orc_load values (0, 0, 0, 0, 0, '2019-01-01', 'test1', toDateTime64('2019-01-01 02:03:04.567', 3)), (2147483647, -1, 9223372036854775806, 123.345345, 345345.3453451212, '2019-01-01', 'test2', toDateTime64('2019-01-01 02:03:04.567', 3))"
|
||||
${CLICKHOUSE_CLIENT} --query="select * from orc_load FORMAT ORC" > $DATA_FILE
|
||||
${CLICKHOUSE_CLIENT} --query="truncate table orc_load"
|
||||
|
||||
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC"
|
||||
timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < $DATA_FILE
|
||||
${CLICKHOUSE_CLIENT} --query="select * from orc_load"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="drop table orc_load"
|
||||
rm -rf "$DATA_FILE"
|
||||
|
@ -32,29 +32,29 @@
|
||||
991
|
||||
990
|
||||
original:
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 2003-02-03 04:05:06.789012
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 2002-02-03 04:05:06.789012
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06 2004-02-03 04:05:06.789012
|
||||
converted:
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 2003-02-03 04:05:06.789012
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 2002-02-03 04:05:06.789012
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06 2004-02-03 04:05:06.789012
|
||||
diff:
|
||||
dest:
|
||||
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 00:00:00
|
||||
80 81 82 83 84 85 86 87 88 89 str02 fstr2\0\0\0\0\0\0\0\0\0\0 2005-03-04 2006-08-09 10:11:12
|
||||
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 00:00:00 2005-02-03 04:05:06.789012
|
||||
80 81 82 83 84 85 86 87 88 89 str02 fstr2\0\0\0\0\0\0\0\0\0\0 2005-03-04 2006-08-09 10:11:12 2007-02-03 04:05:06.789012
|
||||
min:
|
||||
-128 0 0 0 0 0 0 0 -1 -1 string-1\0\0\0\0\0\0\0 fixedstring-1\0\0 2003-04-05 2003-02-03
|
||||
-108 108 8 92 -8 108 -40 -116 -1 -1 string-0\0\0\0\0\0\0\0 fixedstring\0\0\0\0 2001-02-03 2002-02-03
|
||||
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06
|
||||
127 -1 -1 -1 -1 -1 -1 -1 -1 -1 string-2\0\0\0\0\0\0\0 fixedstring-2\0\0 2004-06-07 2004-02-03
|
||||
-128 0 0 0 0 0 0 0 -1 -1 string-1\0\0\0\0\0\0\0 fixedstring-1\0\0 2003-04-05 2003-02-03 2003-02-03 04:05:06.789012
|
||||
-108 108 8 92 -8 108 -40 -116 -1 -1 string-0\0\0\0\0\0\0\0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 2002-02-03 04:05:06.789012
|
||||
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 2005-02-03 04:05:06.789012
|
||||
127 -1 -1 -1 -1 -1 -1 -1 -1 -1 string-2\0\0\0\0\0\0\0 fixedstring-2\0\0 2004-06-07 2004-02-03 2004-02-03 04:05:06.789012
|
||||
max:
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 2003-04-05 00:00:00 2003-02-03 04:05:06
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 2001-02-03 00:00:00 2002-02-03 04:05:06
|
||||
80 81 82 83 84 85 86 87 88 89 str02 fstr2 2005-03-04 05:06:07 2006-08-09 10:11:12
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 2004-06-07 00:00:00 2004-02-03 04:05:06
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 2003-04-05 00:00:00 2003-02-03 04:05:06 2003-02-03 04:05:06.789012
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 2001-02-03 00:00:00 2002-02-03 04:05:06 2002-02-03 04:05:06.789012
|
||||
80 81 82 83 84 85 86 87 88 89 str02 fstr2 2005-03-04 05:06:07 2006-08-09 10:11:12 2007-02-03 04:05:06.789012
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 2004-06-07 00:00:00 2004-02-03 04:05:06 2004-02-03 04:05:06.789012
|
||||
dest from null:
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06
|
||||
\N \N \N \N \N \N \N \N \N \N \N \N \N \N
|
||||
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 2003-02-03 04:05:06.789
|
||||
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 2002-02-03 04:05:06.789
|
||||
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06 2004-02-03 04:05:06.789
|
||||
\N \N \N \N \N \N \N \N \N \N \N \N \N \N \N
|
||||
|
@ -40,20 +40,20 @@ ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types1"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types2"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types3"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types4"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(6)) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(6)) ENGINE = Memory"
|
||||
# convert min type
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types3 (int8 Int8, uint8 Int8, int16 Int8, uint16 Int8, int32 Int8, uint32 Int8, int64 Int8, uint64 Int8, float32 Int8, float64 Int8, string FixedString(15), fixedstring FixedString(15), date Date, datetime Date) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types3 (int8 Int8, uint8 Int8, int16 Int8, uint16 Int8, int32 Int8, uint32 Int8, int64 Int8, uint64 Int8, float32 Int8, float64 Int8, string FixedString(15), fixedstring FixedString(15), date Date, datetime Date, datetime64 DateTime64(6)) ENGINE = Memory"
|
||||
# convert max type
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types4 (int8 Int64, uint8 Int64, int16 Int64, uint16 Int64, int32 Int64, uint32 Int64, int64 Int64, uint64 Int64, float32 Int64, float64 Int64, string String, fixedstring String, date DateTime, datetime DateTime) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types4 (int8 Int64, uint8 Int64, int16 Int64, uint16 Int64, int32 Int64, uint32 Int64, int64 Int64, uint64 Int64, float32 Int64, float64 Int64, string String, fixedstring String, date DateTime, datetime DateTime, datetime64 DateTime64(6)) ENGINE = Memory"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( -108, 108, -1016, 1116, -1032, 1132, -1064, 1164, -1.032, -1.064, 'string-0', 'fixedstring', '2001-02-03', '2002-02-03 04:05:06')"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( -108, 108, -1016, 1116, -1032, 1132, -1064, 1164, -1.032, -1.064, 'string-0', 'fixedstring', '2001-02-03', '2002-02-03 04:05:06', toDateTime64('2002-02-03 04:05:06.789012', 6))"
|
||||
|
||||
# min
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( -128, 0, -32768, 0, -2147483648, 0, -9223372036854775808, 0, -1.032, -1.064, 'string-1', 'fixedstring-1', '2003-04-05', '2003-02-03 04:05:06')"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( -128, 0, -32768, 0, -2147483648, 0, -9223372036854775808, 0, -1.032, -1.064, 'string-1', 'fixedstring-1', '2003-04-05', '2003-02-03 04:05:06', toDateTime64('2003-02-03 04:05:06.789012', 6))"
|
||||
|
||||
# max
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06')"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06', toDateTime64('2004-02-03 04:05:06.789012', 6))"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types2 FORMAT Arrow"
|
||||
|
||||
@ -67,11 +67,11 @@ echo diff:
|
||||
diff "${CLICKHOUSE_TMP}"/arrow_all_types_1.dump "${CLICKHOUSE_TMP}"/arrow_all_types_2.dump
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE arrow_types2"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types3 values ( 79, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str01', 'fstr1', '2003-03-04', '2004-05-06')"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types3 values ( 79, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str01', 'fstr1', '2003-03-04', '2004-05-06', toDateTime64('2005-02-03 04:05:06.789012', 6))"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types3 ORDER BY int8 FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types2 FORMAT Arrow"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8 FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types3 FORMAT Arrow"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types4 values ( 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str02', 'fstr2', '2005-03-04 05:06:07', '2006-08-09 10:11:12')"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types4 values ( 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str02', 'fstr2', '2005-03-04 05:06:07', '2006-08-09 10:11:12', toDateTime64('2007-02-03 04:05:06.789012', 6))"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types4 ORDER BY int8 FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types2 FORMAT Arrow"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8 FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types4 FORMAT Arrow"
|
||||
|
||||
@ -86,9 +86,9 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types4 ORDER BY int8"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types5"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types6"
|
||||
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE arrow_types2"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types5 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime)) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime)) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types5 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64)) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64)) ENGINE = Memory"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types5 ORDER BY int8 FORMAT Arrow" > "${CLICKHOUSE_TMP}"/arrow_all_types_5.arrow
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types5 ORDER BY int8 FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT Arrow"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8 FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT Arrow"
|
||||
|
Binary file not shown.
Loading…
Reference in New Issue
Block a user