Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion .github/workflows/core-hadoop3-ci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ jobs:
runs-on: ubuntu-latest
strategy:
matrix:
jdk: [ '11' ]
jdk: [ '11', '17' ]
spark: [ '3.3','3.4', '3.5' ]
name: Build Amoro with JDK ${{ matrix.jdk }} Spark-${{ matrix.spark }}
steps:
Expand Down
2 changes: 1 addition & 1 deletion README.md
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ Amoro contains modules as below:

## Building

Amoro is built using Maven with JDK 8, 11 and 17(required for `amoro-format-mixed/amoro-mixed-trino` module).
Amoro is built using Maven with JDK 11 and 17(required for `amoro-format-mixed/amoro-mixed-trino` module, experimental for other modules).

* Build all modules without `amoro-mixed-trino`: `./mvnw clean package`
* Build and skip tests: `./mvnw clean package -DskipTests`
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,92 +77,102 @@ public static Object[] parameters() {

@Test
public void run() throws Exception {
MixedTable table = getMixedTable();

int partitionCount = 2;
int primaryUpperBound = 30000;

long writeTargetFileSize = 1024 * 12;
long selfTargetFileSize = table.format() == TableFormat.ICEBERG ? 1024 * 384 : 1024 * 128;
int minorTriggerCount = table.format() == TableFormat.ICEBERG ? 3 : 4;
int availableCore = 10;

int cycle = 5;
int recordCountOnceWrite = 2500;

// close full optimize
table.updateProperties().set(SELF_OPTIMIZING_FULL_TRIGGER_INTERVAL, "-1").commit();
// Need move file to hive scene
table.updateProperties().set(SELF_OPTIMIZING_FULL_REWRITE_ALL_FILES, "false").commit();

KeyedTableDataView view =
new KeyedTableDataView(
table,
tableTestHelper().primaryKeySpec().getPkSchema(),
partitionCount,
primaryUpperBound,
writeTargetFileSize,
null);

// init checker
DataConcurrencyChecker dataConcurrencyChecker = new DataConcurrencyChecker(view);
OptimizingCountChecker optimizingCountChecker = new OptimizingCountChecker(0);
FullOptimizingWrite2HiveChecker fullOptimizingWrite2HiveChecker =
new FullOptimizingWrite2HiveChecker(view);
FullOptimizingMove2HiveChecker fullOptimizingMove2HiveChecker =
new FullOptimizingMove2HiveChecker(view);
MinorOptimizingCheck minorOptimizingCheck = new MinorOptimizingCheck();

CompleteOptimizingFlow.Builder builder =
CompleteOptimizingFlow.builder(table, availableCore)
.setTargetSize(selfTargetFileSize)
.setFragmentRatio(null)
.setDuplicateRatio(null)
.setMinorTriggerFileCount(minorTriggerCount)
.addChecker(dataConcurrencyChecker)
.addChecker(optimizingCountChecker)
.addChecker(minorOptimizingCheck);

if (table.format() == TableFormat.MIXED_HIVE) {
builder
.addChecker(fullOptimizingWrite2HiveChecker)
.addChecker(fullOptimizingMove2HiveChecker);
}

CompleteOptimizingFlow optimizingFlow = builder.build();

// full optimizing need move file to hive from change
view.append(recordCountOnceWrite);
mustFullCycle(table, optimizingFlow::optimize);
// Save the original value to restore later
long originalFilterEqDeleteTriggerRecordCount =
CombinedDeleteFilter.FILTER_EQ_DELETE_TRIGGER_RECORD_COUNT;

try {
MixedTable table = getMixedTable();

int partitionCount = 2;
int primaryUpperBound = 30000;

long writeTargetFileSize = 1024 * 12;
long selfTargetFileSize = table.format() == TableFormat.ICEBERG ? 1024 * 384 : 1024 * 128;
int minorTriggerCount = table.format() == TableFormat.ICEBERG ? 3 : 4;
int availableCore = 10;

int cycle = 5;
int recordCountOnceWrite = 2500;

// close full optimize
table.updateProperties().set(SELF_OPTIMIZING_FULL_TRIGGER_INTERVAL, "-1").commit();
// Need move file to hive scene
table.updateProperties().set(SELF_OPTIMIZING_FULL_REWRITE_ALL_FILES, "false").commit();

KeyedTableDataView view =
new KeyedTableDataView(
table,
tableTestHelper().primaryKeySpec().getPkSchema(),
partitionCount,
primaryUpperBound,
writeTargetFileSize,
null);

// init checker
DataConcurrencyChecker dataConcurrencyChecker = new DataConcurrencyChecker(view);
OptimizingCountChecker optimizingCountChecker = new OptimizingCountChecker(0);
FullOptimizingWrite2HiveChecker fullOptimizingWrite2HiveChecker =
new FullOptimizingWrite2HiveChecker(view);
FullOptimizingMove2HiveChecker fullOptimizingMove2HiveChecker =
new FullOptimizingMove2HiveChecker(view);
MinorOptimizingCheck minorOptimizingCheck = new MinorOptimizingCheck();

CompleteOptimizingFlow.Builder builder =
CompleteOptimizingFlow.builder(table, availableCore)
.setTargetSize(selfTargetFileSize)
.setFragmentRatio(null)
.setDuplicateRatio(null)
.setMinorTriggerFileCount(minorTriggerCount)
.addChecker(dataConcurrencyChecker)
.addChecker(optimizingCountChecker)
.addChecker(minorOptimizingCheck);

if (table.format() == TableFormat.MIXED_HIVE) {
builder
.addChecker(fullOptimizingWrite2HiveChecker)
.addChecker(fullOptimizingMove2HiveChecker);
}

view.append(recordCountOnceWrite);
optimizingFlow.optimize();
CompleteOptimizingFlow optimizingFlow = builder.build();

// full optimizing need move file to hive from change and base
view.append(recordCountOnceWrite);
mustFullCycle(table, optimizingFlow::optimize);
// full optimizing need move file to hive from change
view.append(recordCountOnceWrite);
mustFullCycle(table, optimizingFlow::optimize);

while (cycle-- > 0) {
view.onlyDelete(recordCountOnceWrite);
view.append(recordCountOnceWrite);
optimizingFlow.optimize();

view.cdc(recordCountOnceWrite);
optimizingFlow.optimize();
// full optimizing need move file to hive from change and base
view.append(recordCountOnceWrite);
mustFullCycle(table, optimizingFlow::optimize);

view.upsert(recordCountOnceWrite);
if (cycle % 2 == 0) {
// Trigger BloomFilter
CombinedDeleteFilter.FILTER_EQ_DELETE_TRIGGER_RECORD_COUNT = 2499L;
while (cycle-- > 0) {
view.onlyDelete(recordCountOnceWrite);
optimizingFlow.optimize();

mustFullCycle(table, optimizingFlow::optimize);
} else {
view.cdc(recordCountOnceWrite);
optimizingFlow.optimize();

view.upsert(recordCountOnceWrite);
if (cycle % 2 == 0) {
// Trigger BloomFilter
CombinedDeleteFilter.FILTER_EQ_DELETE_TRIGGER_RECORD_COUNT = 2499L;

mustFullCycle(table, optimizingFlow::optimize);
} else {
optimizingFlow.optimize();
}
}
}

List<CompleteOptimizingFlow.Checker> checkers = optimizingFlow.unTriggerChecker();
if (checkers.size() != 0) {
throw new IllegalStateException("Some checkers are not triggered:" + checkers);
List<CompleteOptimizingFlow.Checker> checkers = optimizingFlow.unTriggerChecker();
if (!checkers.isEmpty()) {
throw new IllegalStateException("Some checkers are not triggered:" + checkers);
}
} finally {
// Restore the original value to avoid polluting other tests
CombinedDeleteFilter.FILTER_EQ_DELETE_TRIGGER_RECORD_COUNT =
originalFilterEqDeleteTriggerRecordCount;
}
}

Expand Down
5 changes: 5 additions & 0 deletions amoro-common/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -194,6 +194,11 @@
</java>
</configuration>
</plugin>

<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
</plugin>
</plugins>
<sourceDirectory>${basedir}/src/main/java</sourceDirectory>
</build>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* 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 org.apache.amoro.testutils;

/**
* Marker interface for tests that fail on Java 17.
*
* <p>Tests annotated with this interface will be excluded when running on Java 17. Use this for
* tests that depend on internal JDK APIs that have been removed or restricted in Java 17.
*
* <p>Usage:
*
* <pre>{@code
* @Category(FailsOnJava17.class)
* public class MyTest {
* // test code
* }
* }</pre>
*/
public interface FailsOnJava17 {}
Original file line number Diff line number Diff line change
Expand Up @@ -325,6 +325,14 @@
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.amoro</groupId>
<artifactId>amoro-common</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.amoro</groupId>
<artifactId>amoro-mixed-hive</artifactId>
Expand Down Expand Up @@ -425,7 +433,21 @@
<value>org.apache.amoro.listener.AmoroRunListener</value>
</property>
</properties>
<argLine>-verbose:class</argLine>
<excludedGroups>${surefire.excludedGroups.jdk}</excludedGroups>
<argLine>-verbose:class
--add-opens=java.base/java.lang=ALL-UNNAMED
--add-opens=java.base/java.lang.invoke=ALL-UNNAMED
--add-opens=java.base/java.lang.reflect=ALL-UNNAMED
--add-opens=java.base/java.io=ALL-UNNAMED
--add-opens=java.base/java.net=ALL-UNNAMED
--add-opens=java.base/java.nio=ALL-UNNAMED
--add-opens=java.base/java.util=ALL-UNNAMED
--add-opens=java.base/java.util.concurrent=ALL-UNNAMED
--add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED
--add-opens=java.base/sun.nio.ch=ALL-UNNAMED
--add-opens=java.base/sun.nio.cs=ALL-UNNAMED
--add-opens=java.base/sun.security.action=ALL-UNNAMED
--add-opens=java.base/sun.util.calendar=ALL-UNNAMED</argLine>
</configuration>
</plugin>
<plugin>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.amoro.flink.table.FlinkSource;
import org.apache.amoro.flink.table.MixedFormatTableLoader;
import org.apache.amoro.flink.util.DataUtil;
import org.apache.amoro.testutils.FailsOnJava17;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.streaming.api.CheckpointingMode;
Expand All @@ -49,6 +50,7 @@
import org.apache.iceberg.io.WriteResult;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;

import java.io.IOException;
import java.time.LocalDateTime;
Expand All @@ -63,6 +65,7 @@
import java.util.Optional;
import java.util.Set;

@Category(FailsOnJava17.class)
public class TestFlinkSource extends FlinkTestBase {

protected static final FileFormat FILE_FORMAT =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@
import org.apache.amoro.table.MixedTable;
import org.apache.amoro.table.TableIdentifier;
import org.apache.amoro.table.UnkeyedTable;
import org.apache.amoro.testutils.FailsOnJava17;
import org.apache.amoro.utils.TableFileUtil;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
Expand Down Expand Up @@ -86,6 +87,7 @@
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -104,6 +106,7 @@
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicInteger;

@Category(FailsOnJava17.class)
public class TestMixedFormatSource extends TestRowDataReaderFunction implements Serializable {
private static final Logger LOG = LoggerFactory.getLogger(TestMixedFormatSource.class);
private static final long serialVersionUID = 7418812854449034756L;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import org.apache.amoro.hive.catalog.HiveCatalogTestHelper;
import org.apache.amoro.hive.catalog.HiveTableTestHelper;
import org.apache.amoro.table.TableProperties;
import org.apache.amoro.testutils.FailsOnJava17;
import org.apache.commons.collections.CollectionUtils;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.table.api.ApiExpression;
Expand All @@ -57,6 +58,7 @@
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TemporaryFolder;
import org.junit.rules.TestName;
import org.junit.runner.RunWith;
Expand All @@ -77,6 +79,7 @@
import java.util.Set;

@RunWith(Parameterized.class)
@Category(FailsOnJava17.class)
public class TestKeyed extends FlinkTestBase {

public static final Logger LOG = LoggerFactory.getLogger(TestKeyed.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.amoro.shade.guava32.com.google.common.collect.Lists;
import org.apache.amoro.table.MixedTable;
import org.apache.amoro.table.TableIdentifier;
import org.apache.amoro.testutils.FailsOnJava17;
import org.apache.commons.lang3.ArrayUtils;
import org.apache.flink.table.api.TableResult;
import org.apache.flink.table.data.RowData;
Expand All @@ -39,6 +40,7 @@
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;

import java.io.IOException;
import java.util.ArrayList;
Expand All @@ -48,6 +50,7 @@
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;

@Category(FailsOnJava17.class)
public class TestLookupSecondary extends CatalogITCaseBase implements FlinkTaskWriterBaseTest {
private String db;

Expand Down
Loading
Loading