|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | + |
| 20 | +package org.apache.iceberg.spark.sql; |
| 21 | + |
| 22 | +import java.util.List; |
| 23 | +import java.util.Map; |
| 24 | +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; |
| 25 | +import org.apache.iceberg.spark.SparkCatalogTestBase; |
| 26 | +import org.apache.iceberg.spark.source.SimpleRecord; |
| 27 | +import org.apache.spark.sql.Dataset; |
| 28 | +import org.apache.spark.sql.Row; |
| 29 | +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; |
| 30 | +import org.apache.spark.sql.functions; |
| 31 | +import org.junit.After; |
| 32 | +import org.junit.Assert; |
| 33 | +import org.junit.Before; |
| 34 | +import org.junit.Test; |
| 35 | + |
| 36 | +public class TestPartitionedWrites extends SparkCatalogTestBase { |
| 37 | + public TestPartitionedWrites(String catalogName, String implementation, Map<String, String> config) { |
| 38 | + super(catalogName, implementation, config); |
| 39 | + } |
| 40 | + |
| 41 | + @Before |
| 42 | + public void createTables() { |
| 43 | + sql("CREATE TABLE %s (id bigint, data string) USING iceberg PARTITIONED BY (truncate(id, 3))", tableName); |
| 44 | + sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); |
| 45 | + } |
| 46 | + |
| 47 | + @After |
| 48 | + public void removeTables() { |
| 49 | + sql("DROP TABLE IF EXISTS %s", tableName); |
| 50 | + } |
| 51 | + |
| 52 | + @Test |
| 53 | + public void testInsertAppend() { |
| 54 | + Assert.assertEquals("Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", tableName)); |
| 55 | + |
| 56 | + sql("INSERT INTO %s VALUES (4, 'd'), (5, 'e')", tableName); |
| 57 | + |
| 58 | + Assert.assertEquals("Should have 5 rows after insert", 5L, scalarSql("SELECT count(*) FROM %s", tableName)); |
| 59 | + |
| 60 | + List<Object[]> expected = ImmutableList.of( |
| 61 | + new Object[] { 1L, "a" }, |
| 62 | + new Object[] { 2L, "b" }, |
| 63 | + new Object[] { 3L, "c" }, |
| 64 | + new Object[] { 4L, "d" }, |
| 65 | + new Object[] { 5L, "e" } |
| 66 | + ); |
| 67 | + |
| 68 | + assertEquals("Row data should match expected", expected, sql("SELECT * FROM %s ORDER BY id", tableName)); |
| 69 | + } |
| 70 | + |
| 71 | + @Test |
| 72 | + public void testInsertOverwrite() { |
| 73 | + Assert.assertEquals("Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", tableName)); |
| 74 | + |
| 75 | + // 4 and 5 replace 3 in the partition (id - (id % 3)) = 3 |
| 76 | + sql("INSERT OVERWRITE %s VALUES (4, 'd'), (5, 'e')", tableName); |
| 77 | + |
| 78 | + Assert.assertEquals("Should have 4 rows after overwrite", 4L, scalarSql("SELECT count(*) FROM %s", tableName)); |
| 79 | + |
| 80 | + List<Object[]> expected = ImmutableList.of( |
| 81 | + new Object[] { 1L, "a" }, |
| 82 | + new Object[] { 2L, "b" }, |
| 83 | + new Object[] { 4L, "d" }, |
| 84 | + new Object[] { 5L, "e" } |
| 85 | + ); |
| 86 | + |
| 87 | + assertEquals("Row data should match expected", expected, sql("SELECT * FROM %s ORDER BY id", tableName)); |
| 88 | + } |
| 89 | + |
| 90 | + @Test |
| 91 | + public void testDataFrameV2Append() throws NoSuchTableException { |
| 92 | + Assert.assertEquals("Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", tableName)); |
| 93 | + |
| 94 | + List<SimpleRecord> data = ImmutableList.of( |
| 95 | + new SimpleRecord(4, "d"), |
| 96 | + new SimpleRecord(5, "e") |
| 97 | + ); |
| 98 | + Dataset<Row> ds = spark.createDataFrame(data, SimpleRecord.class); |
| 99 | + |
| 100 | + ds.writeTo(tableName).append(); |
| 101 | + |
| 102 | + Assert.assertEquals("Should have 5 rows after insert", 5L, scalarSql("SELECT count(*) FROM %s", tableName)); |
| 103 | + |
| 104 | + List<Object[]> expected = ImmutableList.of( |
| 105 | + new Object[] { 1L, "a" }, |
| 106 | + new Object[] { 2L, "b" }, |
| 107 | + new Object[] { 3L, "c" }, |
| 108 | + new Object[] { 4L, "d" }, |
| 109 | + new Object[] { 5L, "e" } |
| 110 | + ); |
| 111 | + |
| 112 | + assertEquals("Row data should match expected", expected, sql("SELECT * FROM %s ORDER BY id", tableName)); |
| 113 | + } |
| 114 | + |
| 115 | + @Test |
| 116 | + public void testDataFrameV2DynamicOverwrite() throws NoSuchTableException { |
| 117 | + Assert.assertEquals("Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", tableName)); |
| 118 | + |
| 119 | + List<SimpleRecord> data = ImmutableList.of( |
| 120 | + new SimpleRecord(4, "d"), |
| 121 | + new SimpleRecord(5, "e") |
| 122 | + ); |
| 123 | + Dataset<Row> ds = spark.createDataFrame(data, SimpleRecord.class); |
| 124 | + |
| 125 | + ds.writeTo(tableName).overwritePartitions(); |
| 126 | + |
| 127 | + Assert.assertEquals("Should have 4 rows after overwrite", 4L, scalarSql("SELECT count(*) FROM %s", tableName)); |
| 128 | + |
| 129 | + List<Object[]> expected = ImmutableList.of( |
| 130 | + new Object[] { 1L, "a" }, |
| 131 | + new Object[] { 2L, "b" }, |
| 132 | + new Object[] { 4L, "d" }, |
| 133 | + new Object[] { 5L, "e" } |
| 134 | + ); |
| 135 | + |
| 136 | + assertEquals("Row data should match expected", expected, sql("SELECT * FROM %s ORDER BY id", tableName)); |
| 137 | + } |
| 138 | + |
| 139 | + @Test |
| 140 | + public void testDataFrameV2Overwrite() throws NoSuchTableException { |
| 141 | + Assert.assertEquals("Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", tableName)); |
| 142 | + |
| 143 | + List<SimpleRecord> data = ImmutableList.of( |
| 144 | + new SimpleRecord(4, "d"), |
| 145 | + new SimpleRecord(5, "e") |
| 146 | + ); |
| 147 | + Dataset<Row> ds = spark.createDataFrame(data, SimpleRecord.class); |
| 148 | + |
| 149 | + ds.writeTo(tableName).overwrite(functions.col("id").$less(3)); |
| 150 | + |
| 151 | + Assert.assertEquals("Should have 3 rows after overwrite", 3L, scalarSql("SELECT count(*) FROM %s", tableName)); |
| 152 | + |
| 153 | + List<Object[]> expected = ImmutableList.of( |
| 154 | + new Object[] { 3L, "c" }, |
| 155 | + new Object[] { 4L, "d" }, |
| 156 | + new Object[] { 5L, "e" } |
| 157 | + ); |
| 158 | + |
| 159 | + assertEquals("Row data should match expected", expected, sql("SELECT * FROM %s ORDER BY id", tableName)); |
| 160 | + } |
| 161 | +} |
0 commit comments