luoyuxia commented on code in PR #1244: URL: https://github.com/apache/fluss/pull/1244#discussion_r2181425407
########## fluss-client/src/main/java/com/alibaba/fluss/client/admin/FlussAdmin.java: ########## @@ -377,12 +377,12 @@ private ListOffsetsResult listOffsets( Collection<Integer> buckets, OffsetSpec offsetSpec) { Long partitionId = null; - metadataUpdater.checkAndUpdateTableMetadata( + metadataUpdater.updateTableMetadata( Collections.singleton(physicalTablePath.getTablePath())); long tableId = metadataUpdater.getTableId(physicalTablePath.getTablePath()); // if partition name is not null, we need to check and update partition metadata if (physicalTablePath.getPartitionName() != null) { - metadataUpdater.checkAndUpdatePartitionMetadata(physicalTablePath); + metadataUpdater.updatePartitionMetadata(physicalTablePath); Review Comment: nit: you can use `metadataUpdater.updatePhysicalTableMetadata( Collections.singleton(physicalTablePath))` ########## fluss-lake/fluss-lake-paimon/src/test/java/com/alibaba/fluss/lake/paimon/tiering/DropTableAfterTieringTest.java: ########## @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.fluss.lake.paimon.tiering; + +import com.alibaba.fluss.lake.paimon.testutils.FlinkPaimonTieringTestBase; +import com.alibaba.fluss.metadata.TableBucket; +import com.alibaba.fluss.metadata.TablePath; +import com.alibaba.fluss.row.InternalRow; + +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.utils.CloseableIterator; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +import static com.alibaba.fluss.testutils.DataTestUtils.row; +import static org.assertj.core.api.Assertions.assertThat; + +/** A Test case for dropping a pktable after tiering and creating one with the same tablePath. */ +public class DropTableAfterTieringTest extends FlinkPaimonTieringTestBase { Review Comment: nit: ```suggestion class ReCreateSameTableAfterTieringTest extends FlinkPaimonTieringTestBase { ``` ########## fluss-lake/fluss-lake-paimon/src/test/java/com/alibaba/fluss/lake/paimon/tiering/DropTableAfterTieringTest.java: ########## @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.fluss.lake.paimon.tiering; + +import com.alibaba.fluss.lake.paimon.testutils.FlinkPaimonTieringTestBase; +import com.alibaba.fluss.metadata.TableBucket; +import com.alibaba.fluss.metadata.TablePath; +import com.alibaba.fluss.row.InternalRow; + +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.utils.CloseableIterator; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +import static com.alibaba.fluss.testutils.DataTestUtils.row; +import static org.assertj.core.api.Assertions.assertThat; + +/** A Test case for dropping a pktable after tiering and creating one with the same tablePath. */ +public class DropTableAfterTieringTest extends FlinkPaimonTieringTestBase { + protected static final String DEFAULT_DB = "fluss"; + + private static StreamExecutionEnvironment execEnv; + private static Catalog paimonCatalog; + + @BeforeAll + protected static void beforeAll() { + FlinkPaimonTieringTestBase.beforeAll(); + execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + execEnv.setParallelism(2); + execEnv.enableCheckpointing(1000); + paimonCatalog = getPaimonCatalog(); + } + + @Test + void testDropTable() throws Exception { Review Comment: ```suggestion void testReCreateSameTable() throws Exception { ``` ########## fluss-client/src/main/java/com/alibaba/fluss/client/admin/FlussAdmin.java: ########## @@ -377,12 +377,12 @@ private ListOffsetsResult listOffsets( Collection<Integer> buckets, OffsetSpec offsetSpec) { Long partitionId = null; - metadataUpdater.checkAndUpdateTableMetadata( + metadataUpdater.updateTableMetadata( Review Comment: nit: you can use ` metadataUpdater.updateTableOrPartitionMetadata(tablePath, null);` instead of introducing a new method. ########## fluss-lake/fluss-lake-paimon/src/test/java/com/alibaba/fluss/lake/paimon/tiering/DropTableAfterTieringTest.java: ########## @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.fluss.lake.paimon.tiering; + +import com.alibaba.fluss.lake.paimon.testutils.FlinkPaimonTieringTestBase; +import com.alibaba.fluss.metadata.TableBucket; +import com.alibaba.fluss.metadata.TablePath; +import com.alibaba.fluss.row.InternalRow; + +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.utils.CloseableIterator; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +import static com.alibaba.fluss.testutils.DataTestUtils.row; +import static org.assertj.core.api.Assertions.assertThat; + +/** A Test case for dropping a pktable after tiering and creating one with the same tablePath. */ +public class DropTableAfterTieringTest extends FlinkPaimonTieringTestBase { + protected static final String DEFAULT_DB = "fluss"; + + private static StreamExecutionEnvironment execEnv; + private static Catalog paimonCatalog; + + @BeforeAll + protected static void beforeAll() { + FlinkPaimonTieringTestBase.beforeAll(); + execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + execEnv.setParallelism(2); + execEnv.enableCheckpointing(1000); + paimonCatalog = getPaimonCatalog(); + } + + @Test + void testDropTable() throws Exception { + // create a pk table, write some records and wait until snapshot finished + TablePath t1 = TablePath.of(DEFAULT_DB, "pkTable_drop"); + long t1Id = createPkTable(t1); + TableBucket t1Bucket = new TableBucket(t1Id, 0); + // write records + List<InternalRow> rows = Arrays.asList(row(1, "v1"), row(2, "v2"), row(3, "v3")); + writeRows(t1, rows, false); + waitUntilSnapshot(t1Id, 1, 0); + // then start tiering job + JobClient jobClient = buildTieringJob(execEnv); + + // check the status of replica after synced + assertReplicaStatus(t1Bucket, 3); + // check data in paimon + checkDataInPaimonPrimayKeyTable(t1, rows); + + // then drop the table + dropTable(t1); + // and create a new table with the same table path + long t2Id = createPkTable(t1); + TableBucket t2Bucket = new TableBucket(t2Id, 0); + // write some new records + List<InternalRow> newRows = Arrays.asList(row(4, "v4"), row(5, "v5")); + writeRows(t1, newRows, false); + // new table, so the snapshot id should be 0 + waitUntilSnapshot(t2Id, 1, 0); + // check the status of replica after synced + assertReplicaStatus(t2Bucket, 2); + // check data in paimon + checkDataInPaimonPrimayKeyTable(t1, newRows); + + // stop the tiering job + jobClient.cancel().get(); + } + + private void checkDataInPaimonPrimayKeyTable( Review Comment: move `checkDataInPaimonPrimayKeyTable` and `getPaimonRowCloseableIterator` to `FlinkPaimonTieringTestBase` so that both it and `PaimonTieringITCase` can share it. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@fluss.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org