Add tests for deltalake storage

This commit is contained in:
Daniil Rubin 2022-09-02 07:06:24 +00:00 committed by Daniil Rubin
parent c68257d711
commit 4cc397da84
30 changed files with 140 additions and 0 deletions

View File

@ -0,0 +1,118 @@
import logging
import os
import json
import helpers.client
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
def prepare_s3_bucket(started_cluster):
bucket_read_write_policy = {
"Version": "2012-10-17",
"Statement": [
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:GetBucketLocation",
"Resource": "arn:aws:s3:::root",
},
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:ListBucket",
"Resource": "arn:aws:s3:::root",
},
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:GetObject",
"Resource": "arn:aws:s3:::root/*",
},
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:PutObject",
"Resource": "arn:aws:s3:::root/*",
},
],
}
minio_client = started_cluster.minio_client
minio_client.set_bucket_policy(
started_cluster.minio_bucket, json.dumps(bucket_read_write_policy)
)
def upload_test_table(started_cluster):
bucket = started_cluster.minio_bucket
for address, dirs, files in os.walk(SCRIPT_DIR + "/test_table"):
address_without_prefix = address[len(SCRIPT_DIR):]
for name in files:
started_cluster.minio_client.fput_object(bucket, os.path.join(address_without_prefix, name), os.path.join(address, name))
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster = ClickHouseCluster(__file__)
cluster.add_instance(
"main_server",
with_minio=True
)
logging.info("Starting cluster...")
cluster.start()
prepare_s3_bucket(cluster)
logging.info("S3 bucket created")
upload_test_table(cluster)
logging.info("Test table uploaded")
yield cluster
finally:
cluster.shutdown()
def run_query(instance, query, stdin=None, settings=None):
# type: (ClickHouseInstance, str, object, dict) -> str
logging.info("Running query '{}'...".format(query))
result = instance.query(query, stdin=stdin, settings=settings)
logging.info("Query finished")
return result
def test_create_query(started_cluster):
instance = started_cluster.instances["main_server"]
bucket = started_cluster.minio_bucket
create_query = f"""CREATE TABLE deltalake ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')"""
run_query(instance, create_query)
def test_select_query(started_cluster):
instance = started_cluster.instances["main_server"]
bucket = started_cluster.minio_bucket
columns = ['begin_lat',
'begin_lon', 'driver', 'end_lat', 'end_lon',
'fare', 'rider', 'ts', 'uuid']
# create query in case table doesn't exist
create_query = f"""CREATE TABLE IF NOT EXISTS deltalake ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')"""
run_query(instance, create_query)
select_query = "SELECT {} FROM deltalake FORMAT TSV"
for column_name in columns:
result = run_query(instance, select_query.format(column_name)).splitlines()
assert(len(result) > 0)

View File

@ -0,0 +1,9 @@
{"protocol":{"minReaderVersion":1,"minWriterVersion":2}}
{"metaData":{"id":"6eae6736-e014-439d-8301-070bfa5fc358","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"begin_lat\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"begin_lon\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"driver\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"end_lat\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"end_lon\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"fare\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"partitionpath\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"rider\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"ts\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"uuid\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["partitionpath"],"configuration":{},"createdTime":1661963201495}}
{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2795,"modificationTime":1661963202988,"dataChange":true}}
{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2966,"modificationTime":1661963203028,"dataChange":true}}
{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795,"modificationTime":1661963203056,"dataChange":true}}
{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2878,"modificationTime":1661963202988,"dataChange":true}}
{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2878,"modificationTime":1661963203044,"dataChange":true}}
{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795,"modificationTime":1661963203072,"dataChange":true}}
{"commitInfo":{"timestamp":1661963203129,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"partitionpath\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"6","numOutputRows":"10","numOutputBytes":"17107"},"engineInfo":"Apache-Spark/3.2.2 Delta-Lake/1.1.0"}}

View File

@ -0,0 +1,13 @@
{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2795,"modificationTime":1661964654518,"dataChange":true}}
{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2966,"modificationTime":1661964654558,"dataChange":true}}
{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2794,"modificationTime":1661964654586,"dataChange":true}}
{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2878,"modificationTime":1661964654518,"dataChange":true}}
{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2879,"modificationTime":1661964654558,"dataChange":true}}
{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795,"modificationTime":1661964654582,"dataChange":true}}
{"remove":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2795}}
{"remove":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2966}}
{"remove":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2878}}
{"remove":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795}}
{"remove":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2878}}
{"remove":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795}}
{"commitInfo":{"timestamp":1661964655251,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[\"partitionpath\"]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numFiles":"6","numOutputRows":"10","numOutputBytes":"17107"},"engineInfo":"Apache-Spark/3.2.2 Delta-Lake/1.1.0"}}