def test_table_schema(): table_path = "../rust/tests/data/simple_table" dt = DeltaTable(table_path) schema = dt.schema() assert schema.json() == { "fields": [{ "metadata": {}, "name": "id", "nullable": True, "type": "long" }], "type": "struct", } assert len(schema.fields) == 1 field = schema.fields[0] assert field.name == "id" assert field.type == DataType("long") assert field.nullable is True assert field.metadata == {} json = '{"type":"struct","fields":[{"name":"x","type":{"type":"array","elementType":"long","containsNull":true},"nullable":true,"metadata":{}}]}' schema = Schema.from_json(json) assert schema.fields[0] == Field("x", ArrayType(DataType("long"), True), True, {})
def setUpClass(self): self.container = str(uuid.uuid4()) self.path = f"{self.container}/tests/table1" self.spark = ( pyspark.sql.SparkSession.builder.appName("deltalake").config( "spark.jars.packages", "io.delta:delta-core_2.12:0.7.0").config( "spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension").config( "spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog", ).getOrCreate()) df = (self.spark.range(0, 1000).withColumn("number", rand()).withColumn( "number2", when(col("id") < 500, 0).otherwise(1))) for i in range(12): df.write.partitionBy("number2").format("delta").mode( "append").save(self.path) self.fs = AzureBlobFileSystem(account_name=AZURE_ACCOUNT_NAME, account_key=AZURE_ACCOUNT_KEY) self.fs.mkdir(self.container) self.fs.upload(self.path, self.path, recursive=True) self.table = DeltaTable(self.path, file_system=self.fs)
def test_write_recordbatchreader(tmp_path: pathlib.Path, existing_table: DeltaTable, sample_data: pa.Table): batches = existing_table.to_pyarrow_dataset().to_batches() reader = RecordBatchReader.from_batches(sample_data.schema, batches) write_deltalake(str(tmp_path), reader, mode="overwrite") assert DeltaTable(str(tmp_path)).to_pyarrow_table() == sample_data
def test_write_pandas(tmp_path: pathlib.Path, sample_data: pa.Table): # When timestamp is converted to Pandas, it gets casted to ns resolution, # but Delta Lake schemas only support us resolution. sample_pandas = sample_data.to_pandas().drop(["timestamp"], axis=1) write_deltalake(str(tmp_path), sample_pandas) delta_table = DeltaTable(str(tmp_path)) df = delta_table.to_pandas() assert_frame_equal(df, sample_pandas)
def test_read_table_with_column_subset(): table_path = "../rust/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) expected = { "value": ["1", "2", "3", "6", "7", "5", "4"], "day": ["1", "3", "5", "20", "20", "4", "5"], } assert (dt.to_pyarrow_dataset().to_table( columns=["value", "day"]).to_pydict() == expected)
def test_roundtrip_multi_partitioned(tmp_path: pathlib.Path, sample_data: pa.Table): write_deltalake(str(tmp_path), sample_data, partition_by=["int32", "bool"]) delta_table = DeltaTable(str(tmp_path)) assert delta_table.pyarrow_schema() == sample_data.schema table = delta_table.to_pyarrow_table() table = table.take(pc.sort_indices(table["int64"])) assert table == sample_data
def test_table_schema_pyarrow_020(): table_path = "../rust/tests/data/delta-0.2.0" dt = DeltaTable(table_path) schema = dt.pyarrow_schema() field = schema.field(0) assert len(schema.types) == 1 assert field.name == "value" assert field.type == pyarrow.int32() assert field.nullable is True assert field.metadata is None
def test_read_table_with_edge_timestamps(): table_path = "../rust/tests/data/table_with_edge_timestamps" dt = DeltaTable(table_path) assert dt.to_pyarrow_dataset( parquet_read_options=ParquetReadOptions(coerce_int96_timestamp_unit="ms") ).to_table().to_pydict() == { "BIG_DATE": [datetime(9999, 12, 31, 0, 0, 0), datetime(9999, 12, 30, 0, 0, 0)], "NORMAL_DATE": [datetime(2022, 1, 1, 0, 0, 0), datetime(2022, 2, 1, 0, 0, 0)], "SOME_VALUE": [1, 2], }
def read_table(): b.wait() t = DeltaTable("s3://deltars/simple") assert t.files() == [ "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", ]
def test_roundtrip_basic(tmp_path: pathlib.Path, sample_data: pa.Table): write_deltalake(str(tmp_path), sample_data) assert ("0" * 20 + ".json") in os.listdir(tmp_path / "_delta_log") delta_table = DeltaTable(str(tmp_path)) assert delta_table.pyarrow_schema() == sample_data.schema table = delta_table.to_pyarrow_table() assert table == sample_data
def test_read_partitioned_table_metadata(): table_path = "../rust/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) metadata = dt.metadata() assert metadata.id == "fe5a3c11-30d4-4dd7-b115-a1c121e66a4e" assert metadata.name is None assert metadata.description is None assert metadata.partition_columns == ["year", "month", "day"] assert metadata.created_time == 1615555644515 assert metadata.configuration == {}
def test_read_partitioned_table_to_dict(): table_path = "../rust/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) expected = { "value": ["1", "2", "3", "6", "7", "5", "4"], "year": ["2020", "2020", "2020", "2021", "2021", "2021", "2021"], "month": ["1", "2", "2", "12", "12", "12", "4"], "day": ["1", "3", "5", "20", "20", "4", "5"], } assert dt.to_pyarrow_dataset().to_table().to_pydict() == expected
def test_table_schema_pyarrow_simple(): table_path = "../rust/tests/data/simple_table" dt = DeltaTable(table_path) schema = dt.pyarrow_schema() field = schema.field(0) assert len(schema.types) == 1 assert field.name == "id" assert field.type == pyarrow.int64() assert field.nullable is True assert field.metadata is None
def test_read_files(s3_localstack): table_path = "s3://deltars/simple" handler = DeltaStorageHandler(table_path) dt = DeltaTable(table_path) files = dt.file_uris() assert len(files) > 0 for file in files: with handler.open_input_file(file) as f_: table = pq.read_table(f_) assert isinstance(table, pa.Table) assert table.shape > (0, 0)
def test_write_iterator(tmp_path: pathlib.Path, existing_table: DeltaTable, sample_data: pa.Table): batches = existing_table.to_pyarrow_dataset().to_batches() with pytest.raises(ValueError): write_deltalake(str(tmp_path), batches, mode="overwrite") write_deltalake(str(tmp_path), batches, schema=sample_data.schema, mode="overwrite") assert DeltaTable(str(tmp_path)).to_pyarrow_table() == sample_data
def test_read_multiple_tables_from_s3(s3cred): """ Should be able to create multiple cloud storage based DeltaTable instances without blocking on async rust function calls. """ for path in ["s3://deltars/simple", "s3://deltars/simple"]: t = DeltaTable(path) assert t.files() == [ "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", ]
def test_read_table_with_filter(): table_path = "../rust/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) expected = { "value": ["6", "7", "5"], "year": ["2021", "2021", "2021"], "month": ["12", "12", "12"], "day": ["20", "20", "4"], } filter_expr = (ds.field("year") == "2021") & (ds.field("month") == "12") dataset = dt.to_pyarrow_dataset() assert len(list(dataset.get_fragments(filter=filter_expr))) == 2 assert dataset.to_table(filter=filter_expr).to_pydict() == expected
def test_writer_with_max_rows(tmp_path: pathlib.Path, row_count: int, rows_per_file: int, expected_files: int): def get_multifile_stats(table: DeltaTable) -> Iterable[Dict]: log_path = get_log_path(table) # Should only have single add entry for line in open(log_path, "r").readlines(): log_entry = json.loads(line) if "add" in log_entry: yield json.loads(log_entry["add"]["stats"]) data = pa.table({ "colA": pa.array(range(0, row_count), pa.int32()), "colB": pa.array([i * random.random() for i in range(0, row_count)], pa.float64()), }) path = str(tmp_path) write_deltalake( path, data, file_options=ParquetFileFormat().make_write_options(), max_rows_per_file=rows_per_file, max_rows_per_group=rows_per_file, ) table = DeltaTable(path) stats = get_multifile_stats(table) files_written = [f for f in os.listdir(path) if f != "_delta_log"] assert sum([stat_entry["numRecords"] for stat_entry in stats]) == row_count assert len(files_written) == expected_files
def test_read_table_with_stats(): table_path = "../rust/tests/data/COVID-19_NYT" dt = DeltaTable(table_path) dataset = dt.to_pyarrow_dataset() filter_expr = ds.field("date") > "2021-02-20" assert len(list(dataset.get_fragments(filter=filter_expr))) == 2 data = dataset.to_table(filter=filter_expr) assert data.num_rows < 147181 + 47559 filter_expr = ds.field("cases") < 0 assert len(list(dataset.get_fragments(filter=filter_expr))) == 0 data = dataset.to_table(filter=filter_expr) assert data.num_rows == 0
def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): write_deltalake( str(tmp_path), sample_data, name="test_name", description="test_desc", configuration={"configTest": "foobar"}, ) delta_table = DeltaTable(str(tmp_path)) metadata = delta_table.metadata() assert metadata.name == "test_name" assert metadata.description == "test_desc" assert metadata.configuration == {"configTest": "foobar"}
def test_vacuum_dry_run_simple_table(): table_path = "../rust/tests/data/delta-0.2.0" dt = DeltaTable(table_path) retention_periods = 169 tombstones = dt.vacuum(retention_periods) tombstones.sort() assert tombstones == [ "../rust/tests/data/delta-0.2.0/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet", "../rust/tests/data/delta-0.2.0/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet", "../rust/tests/data/delta-0.2.0/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet", "../rust/tests/data/delta-0.2.0/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet", ] retention_periods = -1 with pytest.raises(Exception) as exception: dt.vacuum(retention_periods) assert str(exception.value) == "The retention periods should be positive." retention_periods = 167 with pytest.raises(Exception) as exception: dt.vacuum(retention_periods) assert ( str(exception.value) == "Invalid retention period, retention for Vacuum must be greater than 1 week (168 hours)" )
def test_write_modes(tmp_path: pathlib.Path, sample_data: pa.Table): path = str(tmp_path) write_deltalake(path, sample_data) assert DeltaTable(path).to_pyarrow_table() == sample_data with pytest.raises(AssertionError): write_deltalake(path, sample_data, mode="error") write_deltalake(path, sample_data, mode="ignore") assert ("0" * 19 + "1.json") not in os.listdir(tmp_path / "_delta_log") write_deltalake(path, sample_data, mode="append") expected = pa.concat_tables([sample_data, sample_data]) assert DeltaTable(path).to_pyarrow_table() == expected write_deltalake(path, sample_data, mode="overwrite") assert DeltaTable(path).to_pyarrow_table() == sample_data
def test_writer_partitioning(tmp_path: pathlib.Path): test_strings = ["a=b", "hello world", "hello%20world"] data = pa.table({ "p": pa.array(test_strings), "x": pa.array(range(len(test_strings))) }) write_deltalake(str(tmp_path), data) assert DeltaTable(str(tmp_path)).to_pyarrow_table() == data
def setUpClass(self): self.path = f"tests/{str(uuid.uuid4())}/table1" self.spark = ( pyspark.sql.SparkSession.builder.appName("deltalake").config( "spark.jars.packages", "io.delta:delta-core_2.12:0.7.0").config( "spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension").config( "spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog", ).getOrCreate()) df = (self.spark.range(0, 1000).withColumn("number", rand()).withColumn( "number2", when(col("id") < 500, 0).otherwise(1))) for i in range(12): df.write.partitionBy("number2").format("delta").mode( "append").save(self.path) self.table = DeltaTable(self.path)
def test_history_partitioned_table_metadata(): table_path = "../rust/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) history = dt.history() commit_info = history[0] assert len(history) == 1 assert commit_info == { "timestamp": 1615555646188, "operation": "WRITE", "operationParameters": { "mode": "ErrorIfExists", "partitionBy": '["year","month","day"]', }, "isBlindAppend": True, "operationMetrics": { "numFiles": "6", "numOutputBytes": "2477", "numOutputRows": "7", }, }
def test_read_simple_table_update_incremental(): table_path = "../rust/tests/data/simple_table" dt = DeltaTable(table_path, version=0) assert dt.to_pyarrow_dataset().to_table().to_pydict() == { "id": [0, 1, 2, 3, 4] } dt.update_incremental() assert dt.to_pyarrow_dataset().to_table().to_pydict() == {"id": [5, 7, 9]}
def test_writer_null_stats(tmp_path: pathlib.Path): data = pa.table({ "int32": pa.array([1, None, 2, None], pa.int32()), "float64": pa.array([1.0, None, None, None], pa.float64()), "str": pa.array([None] * 4, pa.string()), }) path = str(tmp_path) write_deltalake(path, data) table = DeltaTable(path) stats = get_stats(table) expected_nulls = {"int32": 2, "float64": 3, "str": 4} assert stats["nullCount"] == expected_nulls
def test_load_with_datetime_bad_format(): table_path = "../rust/tests/data/simple_table" dt = DeltaTable(table_path) with pytest.raises(Exception) as exception: dt.load_with_datetime("2020-05-01T00:47:31") assert (str(exception.value) == "Parse date and time string failed: premature end of input") with pytest.raises(Exception) as exception: dt.load_with_datetime("2020-05-01 00:47:31") assert ( str(exception.value) == "Parse date and time string failed: input contains invalid characters") with pytest.raises(Exception) as exception: dt.load_with_datetime("2020-05-01T00:47:31+08") assert (str(exception.value) == "Parse date and time string failed: premature end of input")
def test_writer_with_options(tmp_path: pathlib.Path): column_values = [ datetime(year_, 1, 1, 0, 0, 0) for year_ in range(9000, 9010) ] data = pa.table({"colA": pa.array(column_values, pa.timestamp("us"))}) path = str(tmp_path) opts = (ParquetFileFormat().make_write_options().update( compression="GZIP", coerce_timestamps="us")) write_deltalake(path, data, file_options=opts) table = (DeltaTable(path).to_pyarrow_dataset( parquet_read_options=ParquetReadOptions( coerce_int96_timestamp_unit="us")).to_table()) assert table == data
class DeltaReaderAppendTest(TestCase): @classmethod def setUpClass(self): self.path = f"tests/{str(uuid.uuid4())}/table1" self.spark = ( pyspark.sql.SparkSession.builder.appName("deltalake").config( "spark.jars.packages", "io.delta:delta-core_2.12:0.7.0").config( "spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension").config( "spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog", ).getOrCreate()) df = (self.spark.range(0, 1000).withColumn("number", rand()).withColumn( "number2", when(col("id") < 500, 0).otherwise(1))) for i in range(12): df.write.partitionBy("number2").format("delta").mode( "append").save(self.path) self.table = DeltaTable(self.path) @classmethod def tearDownClass(self): # remove folder when we are done with the test shutil.rmtree(self.path) def test_paths(self): assert self.table.path == self.path assert self.table.log_path == f"{self.path}/_delta_log" def test_versions(self): assert self.table.checkpoint == 10 assert self.table.version == 11 def test_data(self): # read the parquet files using pandas df_pandas = self.table.to_pandas() # read the table using spark df_spark = self.spark.read.format("delta").load( self.table.path).toPandas() # compare dataframes. The index may not be the same order, so we ignore it assert_frame_equal( df_pandas.sort_values("id").reset_index(drop=True), df_spark.sort_values("id").reset_index(drop=True), ) def test_version(self): # read the parquet files using pandas df_pandas = self.table.as_version(5, inplace=False).to_pandas() # read the table using spark df_spark = (self.spark.read.format("delta").option( "versionAsOf", 5).load(self.table.path).toPandas()) # compare dataframes. The index may not be the same order, so we ignore it assert_frame_equal( df_pandas.sort_values("id").reset_index(drop=True), df_spark.sort_values("id").reset_index(drop=True), ) def test_partitioning(self): # Partition pruning should half number of rows assert self.table.to_table( filter=ds.field("number2") == 0).num_rows == 6000 def test_predicate_pushdown(self): # number is random 0-1, so we should have fewer than 12000 rows no matter what assert self.table.to_table( filter=ds.field("number") < 0.5).num_rows < 12000 def test_column_pruning(self): t = self.table.to_table(columns=["number", "number2"]) assert t.column_names == ["number", "number2"]