|
| 1 | +# Licensed to the Apache Software Foundation (ASF) under one |
| 2 | +# or more contributor license agreements. See the NOTICE file |
| 3 | +# distributed with this work for additional information |
| 4 | +# regarding copyright ownership. The ASF licenses this file |
| 5 | +# to you under the Apache License, Version 2.0 (the |
| 6 | +# "License"); you may not use this file except in compliance |
| 7 | +# with the License. You may obtain a copy of the License at |
| 8 | +# |
| 9 | +# http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | +# |
| 11 | +# Unless required by applicable law or agreed to in writing, |
| 12 | +# software distributed under the License is distributed on an |
| 13 | +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 14 | +# KIND, either express or implied. See the License for the |
| 15 | +# specific language governing permissions and limitations |
| 16 | +# under the License. |
| 17 | +import time |
| 18 | + |
| 19 | +import pytest |
| 20 | +from pyspark.sql import SparkSession |
| 21 | + |
| 22 | +from pyiceberg.catalog import Catalog |
| 23 | + |
| 24 | + |
| 25 | +@pytest.mark.integration |
| 26 | +def test_migrate_table( |
| 27 | + session_catalog_hive: Catalog, |
| 28 | + spark: SparkSession, |
| 29 | +) -> None: |
| 30 | + """ |
| 31 | + Imported tables are an edge case since the partition column is not stored |
| 32 | + in the Parquet files: |
| 33 | +
|
| 34 | + test_migrate_table_hive_1754486926/dt=2022-01-01/part-00000-30a9798b-7597-4027-86d9-79d7c529bc87.c000.snappy.parquet |
| 35 | + { |
| 36 | + "type" : "record", |
| 37 | + "name" : "spark_schema", |
| 38 | + "fields" : [ { |
| 39 | + "name" : "number", |
| 40 | + "type" : "int" |
| 41 | + } ] |
| 42 | + } |
| 43 | +
|
| 44 | + PyIceberg will project this column when the table is being read |
| 45 | + """ |
| 46 | + # Create new tables to avoid complex cleanup |
| 47 | + src_table_identifier = f"spark_catalog.default.test_migrate_table_hive_{int(time.time())}" |
| 48 | + dst_table_identifier = f"default.test_migrate_table_{int(time.time())}" |
| 49 | + |
| 50 | + spark.sql(f""" |
| 51 | + CREATE TABLE {src_table_identifier} ( |
| 52 | + number INTEGER |
| 53 | + ) |
| 54 | + PARTITIONED BY (dt date) |
| 55 | + STORED AS parquet |
| 56 | + """) |
| 57 | + |
| 58 | + spark.sql(f""" |
| 59 | + INSERT OVERWRITE TABLE {src_table_identifier} |
| 60 | + PARTITION (dt='2022-01-01') |
| 61 | + VALUES (1), (2), (3) |
| 62 | + """) |
| 63 | + |
| 64 | + spark.sql(f""" |
| 65 | + INSERT OVERWRITE TABLE {src_table_identifier} |
| 66 | + PARTITION (dt='2023-01-01') |
| 67 | + VALUES (4), (5), (6) |
| 68 | + """) |
| 69 | + |
| 70 | + # Docs: https://iceberg.apache.org/docs/latest/hive-migration/#snapshot-hive-table-to-iceberg |
| 71 | + spark.sql(f""" |
| 72 | + CALL hive.system.snapshot('{src_table_identifier}', 'hive.{dst_table_identifier}') |
| 73 | + """) |
| 74 | + |
| 75 | + tbl = session_catalog_hive.load_table(dst_table_identifier) |
| 76 | + assert tbl.schema().column_names == ["number", "dt"] |
| 77 | + |
| 78 | + # TODO: Returns the primitive type (int), rather than the logical type |
| 79 | + # assert set(tbl.scan().to_arrow().column(1).combine_chunks().tolist()) == {'2022-01-01', '2023-01-01'} |
| 80 | + |
| 81 | + assert tbl.scan(row_filter="number > 3").to_arrow().column(0).combine_chunks().tolist() == [4, 5, 6] |
| 82 | + |
| 83 | + assert tbl.scan(row_filter="dt == '2023-01-01'").to_arrow().column(0).combine_chunks().tolist() == [4, 5, 6] |
| 84 | + |
| 85 | + # TODO: Issue with filtering the projected column |
| 86 | + # assert tbl.scan(row_filter="dt == '2022-01-01'").to_arrow().column(0).combine_chunks().tolist() == [1, 2, 3] |
0 commit comments