ClickHouse/tests/integration/test_storage_delta/test.py

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

404 lines
12 KiB
Python
Raw Normal View History

2022-09-02 07:06:24 +00:00
import helpers.client
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
import pytest
import logging
import os
import json
import time
import glob
import pyspark
import delta
from delta import *
from pyspark.sql.types import (
StructType,
StructField,
StringType,
IntegerType,
DateType,
TimestampType,
BooleanType,
ArrayType,
)
from pyspark.sql.functions import current_timestamp
from datetime import datetime
2023-03-24 21:35:12 +00:00
from pyspark.sql.functions import monotonically_increasing_id, row_number
from pyspark.sql.window import Window
from helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents
2022-09-02 07:06:24 +00:00
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
2022-09-06 18:05:33 +00:00
2022-09-02 07:06:24 +00:00
2023-04-11 15:23:05 +00:00
def get_spark():
builder = (
pyspark.sql.SparkSession.builder.appName("spark_test")
.config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension")
.config(
"spark.sql.catalog.spark_catalog",
"org.apache.spark.sql.delta.catalog.DeltaCatalog",
)
.master("local")
)
2023-04-12 12:38:39 +00:00
return builder.master("local").getOrCreate()
2023-04-11 15:23:05 +00:00
2022-09-02 07:06:24 +00:00
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster = ClickHouseCluster(__file__, with_spark=True)
2023-03-24 21:35:12 +00:00
cluster.add_instance(
"node1",
main_configs=["configs/config.d/named_collections.xml"],
2023-06-14 11:45:53 +00:00
user_configs=["configs/users.d/users.xml"],
2023-03-24 21:35:12 +00:00
with_minio=True,
)
2022-09-02 07:06:24 +00:00
logging.info("Starting cluster...")
cluster.start()
prepare_s3_bucket(cluster)
2023-04-11 15:23:05 +00:00
cluster.spark_session = get_spark()
2022-09-02 07:06:24 +00:00
yield cluster
2022-09-06 18:05:33 +00:00
2022-09-02 07:06:24 +00:00
finally:
cluster.shutdown()
2022-09-06 18:05:33 +00:00
2023-03-24 21:35:12 +00:00
def write_delta_from_file(spark, path, result_path, mode="overwrite"):
spark.read.load(path).write.mode(mode).option("compression", "none").format(
"delta"
).option("delta.columnMapping.mode", "name").save(result_path)
2023-04-03 18:56:10 +00:00
def write_delta_from_df(spark, df, result_path, mode="overwrite", partition_by=None):
if partition_by is None:
df.write.mode(mode).option("compression", "none").format("delta").option(
"delta.columnMapping.mode", "name"
).save(result_path)
else:
df.write.mode(mode).option("compression", "none").format("delta").option(
"delta.columnMapping.mode", "name"
).partitionBy("a").save(result_path)
2023-03-24 21:35:12 +00:00
def generate_data(spark, start, end):
a = spark.range(start, end, 1).toDF("a")
b = spark.range(start + 1, end + 1, 1).toDF("b")
b = b.withColumn("b", b["b"].cast(StringType()))
2023-03-28 18:51:02 +00:00
a = a.withColumn(
"row_index", row_number().over(Window.orderBy(monotonically_increasing_id()))
)
b = b.withColumn(
"row_index", row_number().over(Window.orderBy(monotonically_increasing_id()))
)
2023-03-24 21:35:12 +00:00
df = a.join(b, on=["row_index"]).drop("row_index")
return df
def get_delta_metadata(delta_metadata_file):
jsons = [json.loads(x) for x in delta_metadata_file.splitlines()]
combined_json = {}
for d in jsons:
combined_json.update(d)
return combined_json
2022-09-02 07:06:24 +00:00
2023-03-24 21:35:12 +00:00
def create_delta_table(node, table_name):
2023-03-21 19:19:30 +00:00
node.query(
2023-03-24 21:35:12 +00:00
f"""
DROP TABLE IF EXISTS {table_name};
CREATE TABLE {table_name}
ENGINE=DeltaLake(s3, filename = '{table_name}/')"""
2023-03-21 19:19:30 +00:00
)
2023-04-05 18:32:37 +00:00
def create_initial_data_file(
cluster, node, query, table_name, compression_method="none"
):
2023-03-24 21:35:12 +00:00
node.query(
f"""
INSERT INTO TABLE FUNCTION
file('{table_name}.parquet')
SETTINGS
output_format_parquet_compression_method='{compression_method}',
s3_truncate_on_insert=1 {query}
FORMAT Parquet"""
)
2023-04-05 18:32:37 +00:00
user_files_path = os.path.join(
SCRIPT_DIR, f"{cluster.instances_dir_name}/node1/database/user_files"
)
result_path = f"{user_files_path}/{table_name}.parquet"
2023-03-24 21:35:12 +00:00
return result_path
2023-03-21 19:19:30 +00:00
2023-03-24 21:35:12 +00:00
def test_single_log_file(started_cluster):
instance = started_cluster.instances["node1"]
2023-04-11 15:23:05 +00:00
spark = started_cluster.spark_session
2023-03-24 21:35:12 +00:00
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
2023-03-28 18:51:02 +00:00
TABLE_NAME = "test_single_log_file"
2022-09-02 07:06:24 +00:00
2023-03-24 21:35:12 +00:00
inserted_data = "SELECT number, toString(number + 1) FROM numbers(100)"
2023-04-05 18:32:37 +00:00
parquet_data_path = create_initial_data_file(
started_cluster, instance, inserted_data, TABLE_NAME
)
2023-03-24 21:35:12 +00:00
write_delta_from_file(spark, parquet_data_path, f"/{TABLE_NAME}")
files = upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "")
2023-03-30 16:29:55 +00:00
assert len(files) == 2 # 1 metadata files + 1 data file
2023-03-24 21:35:12 +00:00
create_delta_table(instance, TABLE_NAME)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100
assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query(
inserted_data
)
2023-04-03 18:56:10 +00:00
def test_partition_by(started_cluster):
instance = started_cluster.instances["node1"]
2023-04-11 15:23:05 +00:00
spark = started_cluster.spark_session
2023-04-03 18:56:10 +00:00
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
TABLE_NAME = "test_partition_by"
write_delta_from_df(
spark,
generate_data(spark, 0, 10),
f"/{TABLE_NAME}",
mode="overwrite",
partition_by="a",
)
files = upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "")
2023-04-03 19:27:05 +00:00
assert len(files) == 11 # 10 partitions and 1 metadata file
2023-04-03 18:56:10 +00:00
create_delta_table(instance, TABLE_NAME)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 10
2023-04-04 14:00:59 +00:00
def test_checkpoint(started_cluster):
instance = started_cluster.instances["node1"]
2023-04-11 15:23:05 +00:00
spark = started_cluster.spark_session
2023-04-04 14:00:59 +00:00
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
TABLE_NAME = "test_checkpoint"
write_delta_from_df(
spark,
generate_data(spark, 0, 1),
f"/{TABLE_NAME}",
mode="overwrite",
)
for i in range(1, 25):
write_delta_from_df(
spark,
generate_data(spark, i, i + 1),
f"/{TABLE_NAME}",
mode="append",
)
files = upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "")
# 25 data files
# 25 metadata files
# 1 last_metadata file
# 2 checkpoints
assert len(files) == 25 * 2 + 3
ok = False
for file in files:
if file.endswith("last_checkpoint"):
ok = True
assert ok
create_delta_table(instance, TABLE_NAME)
assert (
int(
instance.query(
f"SELECT count() FROM {TABLE_NAME} SETTINGS input_format_parquet_allow_missing_columns=1"
)
)
== 25
)
table = DeltaTable.forPath(spark, f"/{TABLE_NAME}")
table.delete("a < 10")
files = upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "")
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 15
for i in range(0, 5):
write_delta_from_df(
spark,
generate_data(spark, i, i + 1),
f"/{TABLE_NAME}",
mode="append",
)
# + 1 metadata files (for delete)
# + 5 data files
# + 5 metadata files
# + 1 checkpoint file
# + 1 ?
files = upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "")
assert len(files) == 53 + 1 + 5 * 2 + 1 + 1
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 20
2023-04-04 20:59:16 +00:00
assert (
instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY 1").strip()
== instance.query(
2023-04-18 14:25:21 +00:00
"SELECT * FROM ("
2023-04-04 20:59:16 +00:00
"SELECT number, toString(number + 1) FROM numbers(5) "
2023-04-18 14:25:21 +00:00
"UNION ALL SELECT number, toString(number + 1) FROM numbers(10, 15) "
") ORDER BY 1"
2023-04-04 20:59:16 +00:00
).strip()
)
2023-04-03 19:27:05 +00:00
2023-03-24 21:35:12 +00:00
def test_multiple_log_files(started_cluster):
instance = started_cluster.instances["node1"]
2023-04-11 15:23:05 +00:00
spark = started_cluster.spark_session
2023-03-24 21:35:12 +00:00
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
2023-03-28 18:51:02 +00:00
TABLE_NAME = "test_multiple_log_files"
2022-09-06 18:05:33 +00:00
2023-03-28 18:51:02 +00:00
write_delta_from_df(
spark, generate_data(spark, 0, 100), f"/{TABLE_NAME}", mode="overwrite"
)
2023-03-24 21:35:12 +00:00
files = upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "")
2023-03-30 16:29:55 +00:00
assert len(files) == 2 # 1 metadata files + 1 data file
2023-03-24 21:35:12 +00:00
s3_objects = list(
Adjust the test_storage_delta::test_multiple_log_files after MinIO upgrade After upgrade MinIO stopped handling of objects with leading slash: (Pdb) p minio_client.fput_object(bucket_name='root', object_name='/test_multiple_log_files/_delta_log/00000000000000000001.json', file_path='/test_multiple_log_files/_delta_log/00000000000000000001.json').http_headers HTTPHeaderDict({'Accept-Ranges': 'bytes', 'Content-Length': '0', 'ETag': '"f1f3276c787b3f6724b9a8edb313c4fa"', 'Server': 'MinIO', 'Strict-Transport-Security': 'max-age=31536000; includeSubDomains', 'Vary': 'Origin, Accept-Encoding', 'X-Amz-Id-2': '7987905dee74cdeb212432486a178e511309594cee7cb75f892cd53e35f09ea4', 'X-Amz-Request-Id': '178B9B0904A9640D', 'X-Content-Type-Options': 'nosniff', 'X-Xss-Protection': '1; mode=block', 'Date': 'Fri, 06 Oct 2023 19:22:27 GMT'}) (Pdb) p list(minio_client.list_objects('root', '/test_multiple_log_files/_delta_log/')) [] (Pdb) p minio_client.get_object(bucket_name='root', object_name='/test_multiple_log_files/_delta_log/00000000000000000001.json').read() b'{"commitInfo":{"timestamp":1696618985577,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numFiles":"1","numOutputRows":"100","numOutputBytes":"2459"},"engineInfo":"Apache-Spark/3.3.2 Delta-Lake/2.3.0","txnId":"2a69a4c9-f72e-4ac9-9374-bdff96707edf"}}\n{"add":{"path":"part-00000-9ff4728b-d5fe-49ac-a773-e2d6e01f4139-c000.parquet","partitionValues":{},"size":2459,"modificationTime":1696618985310,"dataChange":true,"stats":"{\\"numRecords\\":100,\\"minValues\\":{\\"col-ab7641d2-f0a9-4ffb-ae37-27087d13dfc8\\":0,\\"col-6cbfced4-5921-4e3b-9226-558171c11949\\":\\"1\\"},\\"maxValues\\":{\\"col-ab7641d2-f0a9-4ffb-ae37-27087d13dfc8\\":99,\\"col-6cbfced4-5921-4e3b-9226-558171c11949\\":\\"99\\"},\\"nullCount\\":{\\"col-ab7641d2-f0a9-4ffb-ae37-27087d13dfc8\\":0,\\"col-6cbfced4-5921-4e3b-9226-558171c11949\\":0}}"}}\n{"remove":{"path":"part-00000-7c29f5d9-e14d-419a-a273-58d07775e1e9-c000.parquet","deletionTimestamp":1696618985576,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{},"size":2459}}\n' (Pdb) p list(minio_client.list_objects(bucket_name='root', prefix='test_multiple_log_files/', recursive=True)) [<minio.datatypes.Object object at 0x7ffff1ec2fe0>, <minio.datatypes.Object object at 0x7ffff1ec39d0>, <minio.datatypes.Object object at 0x7ffff1ec2590>, <minio.datatypes.Object object at 0x7ffff1ec3dc0>, <minio.datatypes.Object object at 0x7ffff1ec3a00>, <minio.datatypes.Object object at 0x7ffff1ec3fa0>, <minio.datatypes.Object object at 0x7ffff1ec3f40>, <minio.datatypes.Object object at 0x7ffff1ec2770>] Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-10-06 19:48:03 +00:00
minio_client.list_objects(bucket, f"{TABLE_NAME}/_delta_log/", recursive=True)
2023-03-24 21:35:12 +00:00
)
assert len(s3_objects) == 1
create_delta_table(instance, TABLE_NAME)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100
2023-03-28 18:51:02 +00:00
write_delta_from_df(
spark, generate_data(spark, 100, 200), f"/{TABLE_NAME}", mode="append"
)
2023-03-24 21:35:12 +00:00
files = upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "")
2023-03-30 16:29:55 +00:00
assert len(files) == 4 # 2 metadata files + 2 data files
2023-03-24 21:35:12 +00:00
s3_objects = list(
Adjust the test_storage_delta::test_multiple_log_files after MinIO upgrade After upgrade MinIO stopped handling of objects with leading slash: (Pdb) p minio_client.fput_object(bucket_name='root', object_name='/test_multiple_log_files/_delta_log/00000000000000000001.json', file_path='/test_multiple_log_files/_delta_log/00000000000000000001.json').http_headers HTTPHeaderDict({'Accept-Ranges': 'bytes', 'Content-Length': '0', 'ETag': '"f1f3276c787b3f6724b9a8edb313c4fa"', 'Server': 'MinIO', 'Strict-Transport-Security': 'max-age=31536000; includeSubDomains', 'Vary': 'Origin, Accept-Encoding', 'X-Amz-Id-2': '7987905dee74cdeb212432486a178e511309594cee7cb75f892cd53e35f09ea4', 'X-Amz-Request-Id': '178B9B0904A9640D', 'X-Content-Type-Options': 'nosniff', 'X-Xss-Protection': '1; mode=block', 'Date': 'Fri, 06 Oct 2023 19:22:27 GMT'}) (Pdb) p list(minio_client.list_objects('root', '/test_multiple_log_files/_delta_log/')) [] (Pdb) p minio_client.get_object(bucket_name='root', object_name='/test_multiple_log_files/_delta_log/00000000000000000001.json').read() b'{"commitInfo":{"timestamp":1696618985577,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numFiles":"1","numOutputRows":"100","numOutputBytes":"2459"},"engineInfo":"Apache-Spark/3.3.2 Delta-Lake/2.3.0","txnId":"2a69a4c9-f72e-4ac9-9374-bdff96707edf"}}\n{"add":{"path":"part-00000-9ff4728b-d5fe-49ac-a773-e2d6e01f4139-c000.parquet","partitionValues":{},"size":2459,"modificationTime":1696618985310,"dataChange":true,"stats":"{\\"numRecords\\":100,\\"minValues\\":{\\"col-ab7641d2-f0a9-4ffb-ae37-27087d13dfc8\\":0,\\"col-6cbfced4-5921-4e3b-9226-558171c11949\\":\\"1\\"},\\"maxValues\\":{\\"col-ab7641d2-f0a9-4ffb-ae37-27087d13dfc8\\":99,\\"col-6cbfced4-5921-4e3b-9226-558171c11949\\":\\"99\\"},\\"nullCount\\":{\\"col-ab7641d2-f0a9-4ffb-ae37-27087d13dfc8\\":0,\\"col-6cbfced4-5921-4e3b-9226-558171c11949\\":0}}"}}\n{"remove":{"path":"part-00000-7c29f5d9-e14d-419a-a273-58d07775e1e9-c000.parquet","deletionTimestamp":1696618985576,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{},"size":2459}}\n' (Pdb) p list(minio_client.list_objects(bucket_name='root', prefix='test_multiple_log_files/', recursive=True)) [<minio.datatypes.Object object at 0x7ffff1ec2fe0>, <minio.datatypes.Object object at 0x7ffff1ec39d0>, <minio.datatypes.Object object at 0x7ffff1ec2590>, <minio.datatypes.Object object at 0x7ffff1ec3dc0>, <minio.datatypes.Object object at 0x7ffff1ec3a00>, <minio.datatypes.Object object at 0x7ffff1ec3fa0>, <minio.datatypes.Object object at 0x7ffff1ec3f40>, <minio.datatypes.Object object at 0x7ffff1ec2770>] Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-10-06 19:48:03 +00:00
minio_client.list_objects(bucket, f"{TABLE_NAME}/_delta_log/", recursive=True)
2023-03-24 21:35:12 +00:00
)
assert len(s3_objects) == 2
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 200
2023-03-28 18:51:02 +00:00
assert instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY 1") == instance.query(
2023-03-24 21:35:12 +00:00
"SELECT number, toString(number + 1) FROM numbers(200)"
)
def test_metadata(started_cluster):
instance = started_cluster.instances["node1"]
2023-04-11 15:23:05 +00:00
spark = started_cluster.spark_session
minio_client = started_cluster.minio_client
2022-09-02 07:06:24 +00:00
bucket = started_cluster.minio_bucket
2023-03-28 18:51:02 +00:00
TABLE_NAME = "test_metadata"
2023-03-24 21:35:12 +00:00
parquet_data_path = create_initial_data_file(
2023-04-05 18:32:37 +00:00
started_cluster,
instance,
"SELECT number, toString(number) FROM numbers(100)",
TABLE_NAME,
2023-03-24 21:35:12 +00:00
)
write_delta_from_file(spark, parquet_data_path, f"/{TABLE_NAME}")
upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "")
2022-09-06 18:05:33 +00:00
data = get_file_contents(
minio_client,
bucket,
2023-03-24 21:35:12 +00:00
f"/{TABLE_NAME}/_delta_log/00000000000000000000.json",
)
delta_metadata = get_delta_metadata(data)
2022-09-02 07:06:24 +00:00
stats = json.loads(delta_metadata["add"]["stats"])
assert stats["numRecords"] == 100
assert next(iter(stats["minValues"].values())) == 0
assert next(iter(stats["maxValues"].values())) == 99
2022-09-02 07:06:24 +00:00
2023-03-24 21:35:12 +00:00
create_delta_table(instance, TABLE_NAME)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100
2023-03-21 19:19:30 +00:00
def test_types(started_cluster):
2023-03-28 18:51:02 +00:00
TABLE_NAME = "test_types"
2023-04-11 15:23:05 +00:00
spark = started_cluster.spark_session
result_file = f"{TABLE_NAME}_result_2"
2023-03-28 18:51:02 +00:00
delta_table = (
DeltaTable.create(spark)
.tableName(TABLE_NAME)
.location(f"/{result_file}")
.addColumn("a", "INT")
.addColumn("b", "STRING")
.addColumn("c", "DATE")
.addColumn("d", "ARRAY<STRING>")
.addColumn("e", "BOOLEAN")
.execute()
)
data = [
(
123,
"string",
datetime.strptime("2000-01-01", "%Y-%m-%d"),
["str1", "str2"],
True,
)
]
schema = StructType(
[
StructField("a", IntegerType()),
StructField("b", StringType()),
StructField("c", DateType()),
StructField("d", ArrayType(StringType())),
StructField("e", BooleanType()),
]
)
df = spark.createDataFrame(data=data, schema=schema)
df.printSchema()
df.write.mode("append").format("delta").saveAsTable(TABLE_NAME)
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
upload_directory(minio_client, bucket, f"/{result_file}", "")
instance = started_cluster.instances["node1"]
instance.query(
f"""
DROP TABLE IF EXISTS {TABLE_NAME};
CREATE TABLE {TABLE_NAME} ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123')"""
)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 1
assert (
instance.query(f"SELECT * FROM {TABLE_NAME}").strip()
== "123\tstring\t2000-01-01\t['str1','str2']\ttrue"
)
table_function = f"deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123')"
assert (
instance.query(f"SELECT * FROM {table_function}").strip()
== "123\tstring\t2000-01-01\t['str1','str2']\ttrue"
)
assert instance.query(f"DESCRIBE {table_function} FORMAT TSV") == TSV(
[
["a", "Nullable(Int32)"],
["b", "Nullable(String)"],
["c", "Nullable(Date32)"],
["d", "Array(Nullable(String))"],
["e", "Nullable(Bool)"],
]
)