未验证 提交 a6975607 编写于 作者: A Alexander Kuzmenkov 提交者: GitHub

Merge pull request #11224 from ClickHouse/aku/merge-arrow-stream

Merge ArrowStream format #11088
......@@ -37,6 +37,8 @@ The supported formats are:
| [Avro](#data-format-avro) | ✔ | ✔ |
| [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ |
| [Parquet](#data-format-parquet) | ✔ | ✔ |
| [Arrow](#data-format-arrow) | ✔ | ✔ |
| [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ |
| [ORC](#data-format-orc) | ✔ | ✗ |
| [RowBinary](#rowbinary) | ✔ | ✔ |
| [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ |
......@@ -985,9 +987,9 @@ See also [how to read/write length-delimited protobuf messages in popular langua
## Avro {#data-format-avro}
[Apache Avro](http://avro.apache.org/) is a row-oriented data serialization framework developed within Apache’s Hadoop project.
[Apache Avro](https://avro.apache.org/) is a row-oriented data serialization framework developed within Apache’s Hadoop project.
ClickHouse Avro format supports reading and writing [Avro data files](http://avro.apache.org/docs/current/spec.html#Object+Container+Files).
ClickHouse Avro format supports reading and writing [Avro data files](https://avro.apache.org/docs/current/spec.html#Object+Container+Files).
### Data Types Matching {#data_types-matching}
......@@ -1009,7 +1011,7 @@ The table below shows supported data types and how they match ClickHouse [data t
| `long (timestamp-millis)` \* | [DateTime64(3)](../sql-reference/data-types/datetime.md) | `long (timestamp-millis)` \* |
| `long (timestamp-micros)` \* | [DateTime64(6)](../sql-reference/data-types/datetime.md) | `long (timestamp-micros)` \* |
\* [Avro logical types](http://avro.apache.org/docs/current/spec.html#Logical+Types)
\* [Avro logical types](https://avro.apache.org/docs/current/spec.html#Logical+Types)
Unsupported Avro data types: `record` (non-root), `map`
......@@ -1095,7 +1097,7 @@ SELECT * FROM topic1_stream;
## Parquet {#data-format-parquet}
[Apache Parquet](http://parquet.apache.org/) is a columnar storage format widespread in the Hadoop ecosystem. ClickHouse supports read and write operations for this format.
[Apache Parquet](https://parquet.apache.org/) is a columnar storage format widespread in the Hadoop ecosystem. ClickHouse supports read and write operations for this format.
### Data Types Matching {#data_types-matching-2}
......@@ -1141,6 +1143,16 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_
To exchange data with Hadoop, you can use [HDFS table engine](../engines/table-engines/integrations/hdfs.md).
## Arrow {#data-format-arrow}
[Apache Arrow](https://arrow.apache.org/) comes with two built-in columnar storage formats. ClickHouse supports read and write operations for these formats.
`Arrow` is Apache Arrow's "file mode" format. It is designed for in-memory random access.
## ArrowStream {#data-format-arrow-stream}
`ArrowStream` is Apache Arrow's "stream mode" format. It is designed for in-memory stream processing.
## ORC {#data-format-orc}
[Apache ORC](https://orc.apache.org/) is a columnar storage format widespread in the Hadoop ecosystem. You can only insert data in this format to ClickHouse.
......
......@@ -16,12 +16,12 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int UNKNOWN_EXCEPTION;
extern const int CANNOT_READ_ALL_DATA;
}
ArrowBlockInputFormat::ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_)
: IInputFormat(header_, in_)
ArrowBlockInputFormat::ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_, bool stream_)
: IInputFormat(header_, in_), stream{stream_}
{
prepareReader();
}
......@@ -30,12 +30,23 @@ Chunk ArrowBlockInputFormat::generate()
{
Chunk res;
const Block & header = getPort().getHeader();
if (record_batch_current >= record_batch_total)
return res;
std::vector<std::shared_ptr<arrow::RecordBatch>> single_batch(1);
arrow::Status read_status = file_reader->ReadRecordBatch(record_batch_current, &single_batch[0]);
arrow::Status read_status;
if (stream)
{
read_status = stream_reader->ReadNext(&single_batch[0]);
if (!single_batch[0])
return res;
}
else
{
if (record_batch_current >= record_batch_total)
return res;
read_status = file_reader->ReadRecordBatch(record_batch_current, &single_batch[0]);
}
if (!read_status.ok())
throw Exception{"Error while reading batch of Arrow data: " + read_status.ToString(),
ErrorCodes::CANNOT_READ_ALL_DATA};
......@@ -57,30 +68,54 @@ void ArrowBlockInputFormat::resetParser()
{
IInputFormat::resetParser();
file_reader.reset();
if (stream)
stream_reader.reset();
else
file_reader.reset();
prepareReader();
}
void ArrowBlockInputFormat::prepareReader()
{
arrow::Status open_status = arrow::ipc::RecordBatchFileReader::Open(asArrowFile(in), &file_reader);
if (!open_status.ok())
throw Exception(open_status.ToString(), ErrorCodes::BAD_ARGUMENTS);
record_batch_total = file_reader->num_record_batches();
arrow::Status status;
if (stream)
status = arrow::ipc::RecordBatchStreamReader::Open(asArrowFile(in), &stream_reader);
else
status = arrow::ipc::RecordBatchFileReader::Open(asArrowFile(in), &file_reader);
if (!status.ok())
throw Exception{"Error while opening a table: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION};
if (stream)
record_batch_total = -1;
else
record_batch_total = file_reader->num_record_batches();
record_batch_current = 0;
}
void registerInputFormatProcessorArrow(FormatFactory &factory)
{
factory.registerInputFormatProcessor(
"Arrow",
[](ReadBuffer & buf,
const Block & sample,
const RowInputFormatParams & /* params */,
const FormatSettings & /* format_settings */)
{
return std::make_shared<ArrowBlockInputFormat>(buf, sample);
});
"Arrow",
[](ReadBuffer & buf,
const Block & sample,
const RowInputFormatParams & /* params */,
const FormatSettings & /* format_settings */)
{
return std::make_shared<ArrowBlockInputFormat>(buf, sample, false);
});
factory.registerInputFormatProcessor(
"ArrowStream",
[](ReadBuffer & buf,
const Block & sample,
const RowInputFormatParams & /* params */,
const FormatSettings & /* format_settings */)
{
return std::make_shared<ArrowBlockInputFormat>(buf, sample, true);
});
}
}
......
......@@ -4,6 +4,7 @@
#include <Processors/Formats/IInputFormat.h>
namespace arrow { class RecordBatchReader; }
namespace arrow::ipc { class RecordBatchFileReader; }
namespace DB
......@@ -14,7 +15,7 @@ class ReadBuffer;
class ArrowBlockInputFormat : public IInputFormat
{
public:
ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_);
ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_, bool stream_);
void resetParser() override;
......@@ -24,12 +25,17 @@ protected:
Chunk generate() override;
private:
void prepareReader();
private:
// Whether to use ArrowStream format
bool stream;
// This field is only used for ArrowStream format
std::shared_ptr<arrow::RecordBatchReader> stream_reader;
// The following fields are used only for Arrow format
std::shared_ptr<arrow::ipc::RecordBatchFileReader> file_reader;
int record_batch_total = 0;
int record_batch_current = 0;
void prepareReader();
};
}
......
......@@ -15,8 +15,8 @@ namespace ErrorCodes
extern const int UNKNOWN_EXCEPTION;
}
ArrowBlockOutputFormat::ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_)
: IOutputFormat(header_, out_), format_settings{format_settings_}, arrow_ostream{std::make_shared<ArrowBufferedOutputStream>(out_)}
ArrowBlockOutputFormat::ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, bool stream_, const FormatSettings & format_settings_)
: IOutputFormat(header_, out_), stream{stream_}, format_settings{format_settings_}, arrow_ostream{std::make_shared<ArrowBufferedOutputStream>(out_)}
{
}
......@@ -29,12 +29,7 @@ void ArrowBlockOutputFormat::consume(Chunk chunk)
CHColumnToArrowColumn::chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Arrow");
if (!writer)
{
// TODO: should we use arrow::ipc::IpcOptions::alignment?
auto status = arrow::ipc::RecordBatchFileWriter::Open(arrow_ostream.get(), arrow_table->schema(), &writer);
if (!status.ok())
throw Exception{"Error while opening a table: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION};
}
prepareWriter(arrow_table->schema());
// TODO: calculate row_group_size depending on a number of rows and table size
auto status = writer->WriteTable(*arrow_table, format_settings.arrow.row_group_size);
......@@ -53,6 +48,20 @@ void ArrowBlockOutputFormat::finalize()
}
}
void ArrowBlockOutputFormat::prepareWriter(const std::shared_ptr<arrow::Schema> & schema)
{
arrow::Status status;
// TODO: should we use arrow::ipc::IpcOptions::alignment?
if (stream)
status = arrow::ipc::RecordBatchStreamWriter::Open(arrow_ostream.get(), schema, &writer);
else
status = arrow::ipc::RecordBatchFileWriter::Open(arrow_ostream.get(), schema, &writer);
if (!status.ok())
throw Exception{"Error while opening a table writer: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION};
}
void registerOutputFormatProcessorArrow(FormatFactory & factory)
{
factory.registerOutputFormatProcessor(
......@@ -62,7 +71,17 @@ void registerOutputFormatProcessorArrow(FormatFactory & factory)
FormatFactory::WriteCallback,
const FormatSettings & format_settings)
{
return std::make_shared<ArrowBlockOutputFormat>(buf, sample, format_settings);
return std::make_shared<ArrowBlockOutputFormat>(buf, sample, false, format_settings);
});
factory.registerOutputFormatProcessor(
"ArrowStream",
[](WriteBuffer & buf,
const Block & sample,
FormatFactory::WriteCallback,
const FormatSettings & format_settings)
{
return std::make_shared<ArrowBlockOutputFormat>(buf, sample, true, format_settings);
});
}
......
......@@ -6,6 +6,7 @@
#include <Processors/Formats/IOutputFormat.h>
#include "ArrowBufferedStreams.h"
namespace arrow { class Schema; }
namespace arrow::ipc { class RecordBatchWriter; }
namespace DB
......@@ -14,7 +15,7 @@ namespace DB
class ArrowBlockOutputFormat : public IOutputFormat
{
public:
ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_);
ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, bool stream_, const FormatSettings & format_settings_);
String getName() const override { return "ArrowBlockOutputFormat"; }
void consume(Chunk) override;
......@@ -23,9 +24,12 @@ public:
String getContentType() const override { return "application/octet-stream"; }
private:
bool stream;
const FormatSettings format_settings;
std::shared_ptr<ArrowBufferedOutputStream> arrow_ostream;
std::shared_ptr<arrow::ipc::RecordBatchWriter> writer;
void prepareWriter(const std::shared_ptr<arrow::Schema> & schema);
};
}
......
......@@ -31,8 +31,6 @@
992
991
990
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
......
......@@ -34,14 +34,6 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_numbers ORDER BY number DESC L
${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_numbers"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_events"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_events (event String, value UInt64, description String) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.events FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_events FORMAT Arrow"
${CLICKHOUSE_CLIENT} --query="SELECT event, description FROM arrow_events WHERE event IN ('ContextLock', 'Query') ORDER BY event"
${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_events"
${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"
......@@ -61,8 +53,6 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( -128,
# 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')"
# 'SELECT -127,-128,-129,126,127,128,255,256,257,-32767,-32768,-32769,32766,32767,32768,65535,65536,65537, -2147483647,-2147483648,-2147483649,2147483646,2147483647,2147483648,4294967295,4294967296,4294967297, -9223372036854775807,-9223372036854775808,9223372036854775806,9223372036854775807,9223372036854775808,18446744073709551615';
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types2 FORMAT Arrow"
echo original:
......@@ -98,7 +88,6 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types5 (int8 Nullable(Int
${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="SELECT * FROM arrow_types5 ORDER BY int8 FORMAT Arrow" > ${CLICKHOUSE_TMP}/arrow_all_types_5.arrow
#${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types5 FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT 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"
echo dest from null:
......
9999
9998
9997
9996
9995
9994
9993
9992
9991
9990
99999
99998
99997
99996
99995
99994
99993
99992
99991
99990
2
1
0
999
998
997
996
995
994
993
992
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
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
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 1970-01-01 06:29:04
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
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
max:
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 1970-01-01 06:22:27 2003-02-03 04:05:06
-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 1970-01-01 06:09:16 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 1970-01-01 06:29:36 2004-02-03 04:05:06
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
#!/usr/bin/env bash
set -e
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CUR_DIR/../shell_config.sh
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS contributors"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE contributors (name String) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.contributors ORDER BY name DESC FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO contributors FORMAT ArrowStream"
# random results
${CLICKHOUSE_CLIENT} --query="SELECT * FROM contributors LIMIT 10" > /dev/null
${CLICKHOUSE_CLIENT} --query="DROP TABLE contributors"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_numbers"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_numbers (number UInt64) ENGINE = Memory"
# less than default block size (65k)
${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 10000 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_numbers FORMAT ArrowStream"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_numbers ORDER BY number DESC LIMIT 10"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE arrow_numbers"
# More than default block size
${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 100000 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_numbers FORMAT ArrowStream"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_numbers ORDER BY number DESC LIMIT 10"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE arrow_numbers"
${CLICKHOUSE_CLIENT} --max_block_size=2 --query="SELECT * FROM system.numbers LIMIT 3 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_numbers FORMAT ArrowStream"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_numbers ORDER BY number DESC LIMIT 10"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE arrow_numbers"
${CLICKHOUSE_CLIENT} --max_block_size=1 --query="SELECT * FROM system.numbers LIMIT 1000 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_numbers FORMAT ArrowStream"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_numbers ORDER BY number DESC LIMIT 10"
${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_numbers"
${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"
# 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"
# 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="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')"
# 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')"
# 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="SELECT * FROM arrow_types1 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types2 FORMAT ArrowStream"
echo original:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8" | tee ${CLICKHOUSE_TMP}/arrow_all_types_1.dump
echo converted:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types2 ORDER BY int8" | tee ${CLICKHOUSE_TMP}/arrow_all_types_2.dump
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8 FORMAT ArrowStream" > ${CLICKHOUSE_TMP}/arrow_all_types_1.arrow
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types2 ORDER BY int8 FORMAT ArrowStream" > ${CLICKHOUSE_TMP}/arrow_all_types_2.arrow
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="SELECT * FROM arrow_types3 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types2 FORMAT ArrowStream"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types3 FORMAT ArrowStream"
${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="SELECT * FROM arrow_types4 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types2 FORMAT ArrowStream"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types4 FORMAT ArrowStream"
echo dest:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types2 ORDER BY int8"
echo min:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types3 ORDER BY int8"
echo max:
${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="SELECT * FROM arrow_types5 ORDER BY int8 FORMAT ArrowStream" > ${CLICKHOUSE_TMP}/arrow_all_types_5.arrow
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types5 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT ArrowStream"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT ArrowStream"
echo dest from null:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types6 ORDER BY int8"
${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_types5"
${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_types6"
${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_types1"
${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_types2"
${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_types3"
${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_types4"
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册