|
1 | 1 | package com.linkedin.openhouse.spark.catalogtest;
|
2 | 2 |
|
3 | 3 | import com.google.common.collect.Sets;
|
| 4 | +import com.google.gson.Gson; |
| 5 | +import com.google.gson.GsonBuilder; |
| 6 | +import com.linkedin.openhouse.gen.tables.client.model.Policies; |
4 | 7 | import com.linkedin.openhouse.tablestest.OpenHouseSparkITest;
|
5 | 8 | import java.util.HashMap;
|
| 9 | +import java.util.List; |
6 | 10 | import java.util.Map;
|
7 | 11 | import java.util.concurrent.atomic.AtomicReference;
|
| 12 | +import java.util.stream.Collectors; |
8 | 13 | import org.apache.iceberg.CatalogUtil;
|
9 | 14 | import org.apache.iceberg.DataFile;
|
10 | 15 | import org.apache.iceberg.DataFiles;
|
|
15 | 20 | import org.apache.iceberg.catalog.Catalog;
|
16 | 21 | import org.apache.iceberg.catalog.TableIdentifier;
|
17 | 22 | import org.apache.iceberg.types.Types;
|
| 23 | +import org.apache.spark.sql.Row; |
18 | 24 | import org.apache.spark.sql.SparkSession;
|
19 | 25 | import org.apache.spark.sql.types.DateType;
|
20 | 26 | import org.apache.spark.sql.types.StructField;
|
@@ -127,6 +133,58 @@ public void testCreateReplicaSkipFieldIdReassignmentUnPartitionedTable() throws
|
127 | 133 | }
|
128 | 134 | }
|
129 | 135 |
|
| 136 | + @Test |
| 137 | + public void testAlterTableUnsetReplicationPolicy() throws Exception { |
| 138 | + try (SparkSession spark = getSparkSession()) { |
| 139 | + spark.sql("CREATE TABLE openhouse.d1.`ttt1` (name string)"); |
| 140 | + spark.sql("INSERT INTO openhouse.d1.ttt1 VALUES ('foo')"); |
| 141 | + spark.sql( |
| 142 | + "ALTER TABLE openhouse.d1.ttt1 SET POLICY (REPLICATION=({destination:'WAR', interval:12h}))"); |
| 143 | + spark.sql( |
| 144 | + "ALTER TABLE openhouse.d1.ttt1 SET POLICY (RETENTION= 30d on column name where pattern='yyyy-MM-dd')"); |
| 145 | + Policies policies = getPoliciesObj("openhouse.d1.ttt1", spark); |
| 146 | + Assertions.assertNotNull(policies); |
| 147 | + Assertions.assertEquals( |
| 148 | + "'WAR'", policies.getReplication().getConfig().get(0).getDestination()); |
| 149 | + Assertions.assertNotNull(policies.getRetention()); |
| 150 | + Assertions.assertEquals( |
| 151 | + "'yyyy-MM-dd'", policies.getRetention().getColumnPattern().getPattern()); |
| 152 | + |
| 153 | + // unset replication policy |
| 154 | + spark.sql("ALTER TABLE openhouse.d1.ttt1 UNSET POLICY (REPLICATION)"); |
| 155 | + Policies updatedPolicy = getPoliciesObj("openhouse.d1.ttt1", spark); |
| 156 | + Assertions.assertEquals(updatedPolicy.getReplication().getConfig().size(), 0); |
| 157 | + // assert that other policies, retention is not modified after unsetting replication |
| 158 | + Assertions.assertNotNull(updatedPolicy.getRetention()); |
| 159 | + Assertions.assertEquals( |
| 160 | + "'yyyy-MM-dd'", updatedPolicy.getRetention().getColumnPattern().getPattern()); |
| 161 | + |
| 162 | + // assert retention can be set after unsetting replication |
| 163 | + spark.sql( |
| 164 | + "ALTER TABLE openhouse.d1.ttt1 SET POLICY (RETENTION = 30D on COLUMN name WHERE pattern = 'yyyy')"); |
| 165 | + Policies policyWithRetention = getPoliciesObj("openhouse.d1.ttt1", spark); |
| 166 | + Assertions.assertNotNull(policyWithRetention); |
| 167 | + Assertions.assertEquals( |
| 168 | + "'yyyy'", policyWithRetention.getRetention().getColumnPattern().getPattern()); |
| 169 | + Assertions.assertEquals(0, policyWithRetention.getReplication().getConfig().size()); |
| 170 | + |
| 171 | + // assert replication can be set again after retention policy |
| 172 | + spark.sql( |
| 173 | + "ALTER TABLE openhouse.d1.ttt1 SET POLICY (REPLICATION=({destination:'WAR', interval:12h}))"); |
| 174 | + Policies policyWithReplication = getPoliciesObj("openhouse.d1.ttt1", spark); |
| 175 | + Assertions.assertNotNull(policyWithReplication); |
| 176 | + Assertions.assertEquals( |
| 177 | + "'WAR'", policyWithReplication.getReplication().getConfig().get(0).getDestination()); |
| 178 | + |
| 179 | + // UNSET policy for table without replication |
| 180 | + spark.sql("CREATE TABLE openhouse.d1.`tttest1` (name string)"); |
| 181 | + spark.sql("INSERT INTO openhouse.d1.tttest1 VALUES ('foo')"); |
| 182 | + spark.sql("ALTER TABLE openhouse.d1.tttest1 UNSET POLICY (REPLICATION)"); |
| 183 | + Policies policytttest1 = getPoliciesObj("openhouse.d1.tttest1", spark); |
| 184 | + Assertions.assertEquals(0, policytttest1.getReplication().getConfig().size()); |
| 185 | + } |
| 186 | + } |
| 187 | + |
130 | 188 | @Test
|
131 | 189 | public void testCreateReplicaSkipFieldIdReassignmentPartitionedTable() throws Exception {
|
132 | 190 | try (SparkSession spark = getSparkSession()) {
|
@@ -185,4 +243,13 @@ private Catalog getOpenHouseCatalog(SparkSession spark) {
|
185 | 243 | catalogProperties,
|
186 | 244 | spark.sparkContext().hadoopConfiguration());
|
187 | 245 | }
|
| 246 | + |
| 247 | + private Policies getPoliciesObj(String tableName, SparkSession spark) { |
| 248 | + List<Row> props = spark.sql(String.format("show tblProperties %s", tableName)).collectAsList(); |
| 249 | + Map<String, String> collect = |
| 250 | + props.stream().collect(Collectors.toMap(r -> r.getString(0), r -> r.getString(1))); |
| 251 | + String policiesStr = String.valueOf(collect.get("policies")); |
| 252 | + Gson gson = new GsonBuilder().setPrettyPrinting().create(); |
| 253 | + return gson.fromJson(policiesStr, Policies.class); |
| 254 | + } |
188 | 255 | }
|
0 commit comments