Skip to content

Commit

Permalink
[AMORO-1812] Support spark-based external optimizer (#2421)
Browse files Browse the repository at this point in the history
* [AMORO-1812] support spark-based external optimizer

* resolve code style error

* [AMORO-1951] Support parallelized planning in one optimizer group (#2282)

* [AMORO-1951] Support parallelized planning in one optimizer group

* [AMORO-1951] add unit test for OptimizingQueue and DefaultOptimizingService

* [AMORO-1951] optimize default parameters

* fix bugs

* fix warnings and spotless issues

* merge from #2290

* add apache license and fix spotless

* fix config error

* Update ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java

Co-authored-by: ZhouJinsong <zhoujinsong0505@163.com>

* add annotations

* fix compile errors

* fix import problem

* remove isDebugEnabled()

* spotless apply

* Update ArcticManagementConf.java

* fix reboot bug and supply document content

* use MoreObjects.toStringHelper for OptimizerThread.java

* Merged from [AMORO-2376] Print right log info after calculating and sorting tables

* fix import problem

* remove unused codes

* spotless

* remove incorrect comments

* add max-planning-parallelism to config

---------

Co-authored-by: majin1102 <majin1102@163.com>
Co-authored-by: ZhouJinsong <zhoujinsong0505@163.com>

* [AMORO-2378] The optimizer based on Flink on YARN should prioritize loading the optimizer-job.jar (#2379)

* load optimizer jar first

* fix code style

* change config name

* add config taskmanager.memory.managed.fraction

* fix

* [AMORO-2222] [Improvement]: Skip cleaning up dangling delete files for Iceberg V1 table (#2361)

* [AMORO-2222] [Improvement]: Skip cleaning up dangling delete files for Iceberg V1 table

* Update IcebergTableMaintainer.java

The `total-delete-files` could be 0.

---------

Co-authored-by: wangtaohz <103108928+wangtaohz@users.noreply.github.com>

* [AMORO-2404] fix Mixed Hive table mistakenly deletes hive files during expiring snapshots (#2405)

get hive locations return the uri path

* [AMORO-2407] Fix access data file from dashboard of non-partitioned table (#2408)

* fix null partition

* fix listing files of non-partitioned iceberg table

* [AMORO-2383] Add serialVersionUID to RewriteFilesInput (#2384)

* add serialVersionUID

* fix comment

* [AMORO-1720] Fix Mixed Format KeyedTable expiring all the snapshots with optimized sequence (#2394)

* should not expire the latest snapshot contains optimized sequence

* add visible for testing

* add fetchLatestNonOptimizedSnapshotTime for base store

* get hive locations return the uri path

* refactor codes and fix comments

* improve for exclude files is empty for expring snapshots

---------

Co-authored-by: ZhouJinsong <zhoujinsong0505@163.com>

* [AMORO-2386][AMS] Configure `iceberg.worker.num-threads` in the config.yaml (#2393)

* [AMORO-2386][AMS] Configure `iceberg.worker.num-threads` in the config.yaml

* Fix

* [AMORO-2386][AMS] reuse config `table-manifest-io.thread-count` and reuse thread pool

* Add comment

* [AMORO-1716] [Improvement]: sort the table list returned by server (#2362)

* improve: sort the table list returned by server

* optimize: sort tables by format

* optimize: optimiz tables sorting

* style: udpate comment

---------

Co-authored-by: chenyuzhi <chenyuzhi@corp.netease.com>
Co-authored-by: ZhouJinsong <zhoujinsong0505@163.com>

* [HotFix] Re-add table-filter to Server ExternalCatalog (#2310)

* re add table filter

* implement in external catalog

* add ut case

* fix comment

* fix comment

* fix comment

* fix ut

* fix update properties

* roll back the engine side's filter

* resolve conflicts

* add ut

---------

Co-authored-by: baiyangtx <xiangnebula@163.com>
Co-authored-by: ZhouJinsong <zhoujinsong0505@163.com>

* [AMORO-2299]: Cancel the running optimizing process from ams web console (#2297)

* cancel the running opimizing process from ams web console

* refact code to avoid NPE

* add o comment for com.netease.arctic.server.table.TableService#getServerTableIdentifier

* change the cancel post api to be more restful style

* [AMORO-2415] Print GC date stamps  (#2416)

add gc timestamp

* Update wrong comments in SnapshotsExpiringExecutor.java (#2422)

* [AMORO-2276]: UnifiiedCatalog for Spark Engine (#2269)

* Add UnifiedSparkCatalog under spark common module
* Extract MixedSparkCatalogBase and MixedSparkSessionCatalogBase to spark common module
* Refactor spark unit test framework to adapt unifed catalog tests and mixed format tests.

* [AMORO-2261] Extract the deleting dangling files from the cleaning orphan files (#2403)

* [Improvement]: Extract the deleting dangling files from the cleaning orphan files

* [Improvement]: Extract the deleting dangling files from the cleaning orphan files

* [Improvement]: Extract the deleting dangling files from the cleaning orphan files

* [AMORO-1341] [Flink]: Support UnifiedCatalog to contain Mixed format table in Flink Engine (#2370)

* [AMORO-1341] [Flink]: Support UnifiedCatalog to contain Mixed format table in Flink Engine

* [AMORO-2413] Need to select the first db after switching to another Catalog (#2419)

* fix: If the current catalog is not the one in the query, the first db is selected by default.

* build dashboard frontend

---------

Co-authored-by: wangtao <wangtao3@corp.netease.com>

* [HotFix] Fix loading the optimizing snapshot id of change store for Mixed Format KeyedTable (#2430)

fix load target change snapshot id

* [AMORO-2260] Show the format version of iceberg table (#2425)

[AMORO-2260] Show the format version of Iceberg Table

Signed-off-by: tcodehuber <tcodehuber@gmail.com>

* [AMORO-2115] Support displaying Optimizing tasks (#2322)

* dashboard: rename optimized to optimizing

* dashboard: support optimizing taskes

* add optimizer token

* dashboard: modify column width

* dashboard: build

* sort the metrics field and change record cnt to long

* modify MetricsSummary Compatibility

* dashbard: build

* Update ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java

Co-authored-by: Qishang Zhong <zhongqishang@gmail.com>

* fix

* support input metrics and output metrics for optimizing process

* dashboard: support optimizing metrics

* dashbard: build

* dashboard:rebuild

* support MetricsSummary to map

* optimizing task supports input output

* dashboard: optimizing tasks support input and output

* dashboard: not display seconds when longer than 1 hour

* dashboard: optimizing process show summary

* remove useless import

* dashboard: build

* as head

* dashbard: build

* change process status to CLOSED after cancel process

* remove useless log

* dashboard: refresh after cancelled

* support cancel optimizing tasks

* dashboard: handle exception when can't cancel optimizing process

* throw exception when can't cancel optimizing process

* dashboard: build

* dashboard: refresh optimizing process when exist optimizing detail page

* dashboard: build

* fix cost time is 0ms

* change metrics name

* fix task startTime and endTime

* fix costTime

* using Preconditions.checkArgument

* fix task reset

* add comments

* cancel tasks before closing optimizing process

* fix unit test

* fix cancel task

* as head

* Revert "as head"

This reverts commit e469e71.

* dashboard: build

---------

Co-authored-by: Qishang Zhong <zhongqishang@gmail.com>

* [AMORO-2385] Make the maximum input file size for per optimize thread configurable (#2387)

* add config self-optimizing.max-input-file-size-per-thread

* add doc

* add resource group property max-input-file-size-per-thread

* add doc

* fix compile

* [Hotfix] Add database filter to Server ExternalCatalog (#2414)

* [Hotfix] Add database filter to Server ExternalCatalog

* [Hotfix] Add database filter to Server ExternalCatalog

* Rename config database.filter-regular-expression to database-filter

---------

Co-authored-by: baiyangtx <xiangnebula@163.com>

* [AMORO-2423] [Flink]: Using 'mixed_iceberg' and 'mixed_hive' indentifier to CREATE CATALOG and deprecate 'arctic' identifier (#2424)

* [AMORO-2423] [Flink]: Using 'mixed_iceberg' and 'mixed_hive' identifiers to CREATE CATALOG and deprecate 'arctic' identifier

* [AMORO-2316] The Files page supports filtering by partition name and sorting by dictionary value. (#2420)

* AMORO-2316: The Files page supports filtering by partition name and sorting by dictionary value.

* build dashboard frontend

* dashboard: build

---------

Co-authored-by: wangtao <wangtao3@corp.netease.com>

* [AMORO-1892] Improve the SQL Shortcuts in Terminal Web UI (#2434)

* [AMORO-1892] Improve the SQL Shortcuts in Terminal Web UI

* refactor some code

---------

Signed-off-by: tcodehuber <tcodehuber@gmail.com>

* [AMORO-2440] Fix the batch deletion of Change Store files for Mixed Format Table (#2439)

fix remove change files

* [AMORO-2418] Exclude kryo dependency from flink-optimizer (#2437)

exclude kryo

* [AMORO-2441] Fix `TableEntriesScan` when no file format is specified in the file name (#2442)

* fix TableEntriesScan without format suffix

* using the file format from entries

* [Hotfix] Fix fetchLatestNonOptimizedSnapshotTime (#2396)

* fix fetchLatestNonOptimizedSnapshotTime

* fix

* spotless

* fix ut

* rename data-expire.since

* using UTC zone for snapshot timestamp

* resolve conflict

* rerview

* spotless

* review

* review

* [AMORO-2344] [Flink]: Support UnifiedCatalog to contain Iceberg format table in Flink Engine (#2427)

* [AMORO-2330] Improve major plan (#2332)

* [AMORO-2330][AMS] Improve major plan

* [AMORO-2330][AMS] remove filter dataFileWith1Pos

* Fix comments

* Fix

* Move the rollback logic of undersized segment to the split task stage

* Rename

* Fix

* Rollback method name `fileShouldRewrite`

* Rollback mixed table format full trigger condition & reuse `isUndersizedSegmentFile`

* Rollback testSegmentFilesBase()

* TreeNodeTaskSplitter logic keep same with bin-pack

* Improve code duplicate

---------

Co-authored-by: ZhouJinsong <zhoujinsong0505@163.com>

* [AMORO-1810] Check the validity of the heatbeat interval when an opti… (#2432)

* [AMORO-1810] Check the validity of the heatbeat interval when an optimizer start

* adjust the import way

* resolve some logic code

* refactor code

* refactor code

* fix ut error

* resolve ut error

* fix ut error

* fix ut error

* fix ut error

* fix ci error

* refactor code

* refactor code

* refactor code

* [AMORO-1812] support spark-based external optimizer

* resolve code style error

* refactor code

* refactor code

* bugfix

* refactor code

* refactor code

* code style

* bugfix

* bugfix

---------

Signed-off-by: tcodehuber <tcodehuber@gmail.com>
Co-authored-by: JinMat <majin1102@gmail.com>
Co-authored-by: majin1102 <majin1102@163.com>
Co-authored-by: ZhouJinsong <zhoujinsong0505@163.com>
Co-authored-by: wangzeyu <hameizi369@gmail.com>
Co-authored-by: ConradJam <jam.gzczy@gmail.com>
Co-authored-by: wangtaohz <103108928+wangtaohz@users.noreply.github.com>
Co-authored-by: yeatsliao <liaoyt66066@gmail.com>
Co-authored-by: Qishang Zhong <zhongqishang@gmail.com>
Co-authored-by: chenyuzhi459 <553673833@qq.com>
Co-authored-by: chenyuzhi <chenyuzhi@corp.netease.com>
Co-authored-by: HuangFru <68625618+HuangFru@users.noreply.github.com>
Co-authored-by: baiyangtx <xiangnebula@163.com>
Co-authored-by: xujiangfeng001 <104614523+xujiangfeng001@users.noreply.github.com>
Co-authored-by: Xianxun Ye <yesorno828423@gmail.com>
Co-authored-by: liuweimin <minteliu.l@gmail.com>
Co-authored-by: wangtao <wangtao3@corp.netease.com>
Co-authored-by: Xavier Bai <xuba@cisco.com>
  • Loading branch information
18 people committed Feb 22, 2024
1 parent f94a1db commit 0330b53
Show file tree
Hide file tree
Showing 11 changed files with 473 additions and 44 deletions.
6 changes: 6 additions & 0 deletions ams/dist/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,12 @@
<version>${project.version}</version>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>com.netease.amoro</groupId>
<artifactId>spark-optimizer</artifactId>
<version>${project.version}</version>
<scope>compile</scope>
</dependency>
</dependencies>

<build>
Expand Down
10 changes: 9 additions & 1 deletion ams/dist/src/main/assemblies/bin.xml
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,14 @@
<destName>optimizer-job.jar</destName>
<fileMode>0644</fileMode>
</file>
<file>
<source>
../optimizer/spark-optimizer/target/spark-optimizer-${project.version}.jar
</source>
<outputDirectory>plugin/optimizer/spark</outputDirectory>
<destName>optimizer-job.jar</destName>
<fileMode>0644</fileMode>
</file>
<file>
<source>../server/target/amoro-ams-server-${project.version}.jar</source>
<outputDirectory>lib/</outputDirectory>
Expand Down Expand Up @@ -81,4 +89,4 @@
<fileMode>0644</fileMode>
</fileSet>
</fileSets>
</assembly>
</assembly>
Original file line number Diff line number Diff line change
Expand Up @@ -163,11 +163,11 @@ protected void waitAShortTime(long waitTime) {
}
}

interface AmsCallOperation<T> {
protected interface AmsCallOperation<T> {
T call(OptimizingService.Iface client) throws TException;
}

interface AmsAuthenticatedCallOperation<T> {
protected interface AmsAuthenticatedCallOperation<T> {
T call(OptimizingService.Iface client, String token) throws TException;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@
import org.slf4j.LoggerFactory;

import java.util.Arrays;
import java.util.function.IntFunction;
import java.util.function.Supplier;
import java.util.stream.IntStream;

public class Optimizer {
Expand All @@ -33,11 +35,18 @@ public class Optimizer {
private final OptimizerExecutor[] executors;

public Optimizer(OptimizerConfig config) {
this(config, () -> new OptimizerToucher(config), (i) -> new OptimizerExecutor(config, i));
}

protected Optimizer(
OptimizerConfig config,
Supplier<OptimizerToucher> toucherFactory,
IntFunction<OptimizerExecutor> executorFactory) {
this.config = config;
this.toucher = new OptimizerToucher(config);
this.toucher = toucherFactory.get();
this.executors = new OptimizerExecutor[config.getExecutionParallel()];
IntStream.range(0, config.getExecutionParallel())
.forEach(i -> executors[i] = new OptimizerExecutor(config, i));
.forEach(i -> executors[i] = executorFactory.apply(i));
if (config.getResourceId() != null) {
toucher.withRegisterProperty(OptimizerProperties.RESOURCE_ID, config.getResourceId());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,24 +99,50 @@ private boolean ackTask(OptimizingTask task) {
}
}

protected OptimizingTaskResult executeTask(OptimizingTask task) {
return executeTask(getConfig(), getThreadId(), task, LOG);
}

protected void completeTask(OptimizingTaskResult optimizingTaskResult) {
try {
callAuthenticatedAms(
(client, token) -> {
client.completeTask(token, optimizingTaskResult);
return null;
});
LOG.info(
"Optimizer executor[{}] completed task[{}] to ams",
threadId,
optimizingTaskResult.getTaskId());
} catch (TException exception) {
LOG.error(
"Optimizer executor[{}] completed task[{}] failed",
threadId,
optimizingTaskResult.getTaskId(),
exception);
}
}

@SuppressWarnings({"rawtypes", "unchecked"})
private OptimizingTaskResult executeTask(OptimizingTask task) {
public static OptimizingTaskResult executeTask(
OptimizerConfig config, int threadId, OptimizingTask task, Logger logger) {
long startTime = System.currentTimeMillis();
TableOptimizing.OptimizingInput input = null;
try {
OptimizingInputProperties properties = OptimizingInputProperties.parse(task.getProperties());
input = SerializationUtil.simpleDeserialize(task.getTaskInput());
String executorFactoryImpl = properties.getExecutorFactoryImpl();
TableOptimizing.OptimizingInput input =
SerializationUtil.simpleDeserialize(task.getTaskInput());
DynConstructors.Ctor<OptimizingExecutorFactory> ctor =
DynConstructors.builder(OptimizingExecutorFactory.class)
.impl(executorFactoryImpl)
.buildChecked();
OptimizingExecutorFactory factory = ctor.newInstance();

if (getConfig().isExtendDiskStorage()) {
if (config.isExtendDiskStorage()) {
properties.enableSpillMap();
}
properties.setMaxSizeInMemory(getConfig().getMemoryStorageSize() * 1024 * 1024);
properties.setSpillMapPath(getConfig().getDiskStoragePath());
properties.setMaxSizeInMemory(config.getMemoryStorageSize() * 1024 * 1024);
properties.setSpillMapPath(config.getDiskStoragePath());
factory.initialize(properties.getProperties());

OptimizingExecutor executor = factory.createExecutor(input);
Expand All @@ -125,33 +151,24 @@ private OptimizingTaskResult executeTask(OptimizingTask task) {
OptimizingTaskResult result = new OptimizingTaskResult(task.getTaskId(), threadId);
result.setTaskOutput(outputByteBuffer);
result.setSummary(output.summary());
LOG.info("Optimizer executor[{}] executed task[{}]", threadId, task.getTaskId());
logger.info(
"Optimizer executor[{}] executed task[{}]({}) and cost {}",
threadId,
task.getTaskId(),
input,
System.currentTimeMillis() - startTime);
return result;
} catch (Throwable t) {
LOG.error("Optimizer executor[{}] executed task[{}] failed", threadId, task.getTaskId(), t);
logger.error(
"Optimizer executor[{}] executed task[{}]({}) failed and cost {}",
threadId,
task.getTaskId(),
input,
System.currentTimeMillis() - startTime,
t);
OptimizingTaskResult errorResult = new OptimizingTaskResult(task.getTaskId(), threadId);
errorResult.setErrorMessage(ExceptionUtil.getErrorMessage(t, 4000));
return errorResult;
}
}

private void completeTask(OptimizingTaskResult optimizingTaskResult) {
try {
callAuthenticatedAms(
(client, token) -> {
client.completeTask(token, optimizingTaskResult);
return null;
});
LOG.info(
"Optimizer executor[{}] completed task[{}] to ams",
threadId,
optimizingTaskResult.getTaskId());
} catch (TException exception) {
LOG.error(
"Optimizer executor[{}] completed task[{}] failed",
threadId,
optimizingTaskResult.getTaskId(),
exception);
}
}
}
1 change: 1 addition & 0 deletions ams/optimizer/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@

<modules>
<module>flink-optimizer</module>
<module>spark-optimizer</module>
<module>standalone-optimizer</module>
<module>common</module>
</modules>
Expand Down
186 changes: 186 additions & 0 deletions ams/optimizer/spark-optimizer/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,186 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
-->

<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>amoro-optimizer</artifactId>
<groupId>com.netease.amoro</groupId>
<version>0.7.0-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<modelVersion>4.0.0</modelVersion>

<artifactId>spark-optimizer</artifactId>
<name>Amoro Project AMS Spark Optimizer</name>
<url>https://amoro.netease.com</url>

<properties>
<spark-optimizer.spark-version>3.3.2</spark-optimizer.spark-version>
<spark-optimizer.scala-version>2.12</spark-optimizer.scala-version>
</properties>

<dependencies>
<dependency>
<groupId>com.netease.amoro</groupId>
<artifactId>optimizer-common</artifactId>
<version>${project.version}</version>
</dependency>

<!-- spark dependencies begin -->
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${spark-optimizer.scala-version}</artifactId>
<version>${spark-optimizer.spark-version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<artifactId>slf4j-api</artifactId>
<groupId>org.slf4j</groupId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${spark-optimizer.scala-version}</artifactId>
<version>${spark-optimizer.spark-version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-column</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-hadoop</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-hadoop</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.arrow</groupId>
<artifactId>arrow-memory-core</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.arrow</groupId>
<artifactId>arrow-memory-netty</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.arrow</groupId>
<artifactId>arrow-vector</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.orc</groupId>
<artifactId>orc-core</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.orc</groupId>
<artifactId>orc-mapreduce</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<createDependencyReducedPom>false</createDependencyReducedPom>
<filters>
<filter>
<artifact>*:*</artifact>
<excludes>
<exclude>META-INF/*.SF</exclude>
<exclude>META-INF/*.DSA</exclude>
<exclude>META-INF/*.RSA</exclude>
</excludes>
</filter>
</filters>
<transformers>
<transformer
implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
</transformers>
<relocations>
<relocation>
<pattern>org.apache.parquet</pattern>
<shadedPattern>com.netease.arctic.shade.org.apache.parquet</shadedPattern>
</relocation>
</relocations>
<finalName>${project.artifactId}-${project.version}</finalName>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-deploy-plugin</artifactId>
<configuration>
<skip>true</skip>
</configuration>
</plugin>
</plugins>
</build>

<profiles>
<profile>
<id>optimizer.spark3.3</id>
<activation>
<property>
<name>!optimizer.spark</name>
</property>
</activation>
<properties>
<spark-optimizer.spark-version>3.3.2</spark-optimizer.spark-version>
<spark-optimizer.scala-version>2.12</spark-optimizer.scala-version>
</properties>
</profile>
<profile>
<id>optimizer.spark3.2</id>
<activation>
<property>
<name>optimizer.spark</name>
<value>3.2</value>
</property>
</activation>
<properties>
<spark-optimizer.spark-version>3.2.2</spark-optimizer.spark-version>
<spark-optimizer.scala-version>2.12</spark-optimizer.scala-version>
</properties>
</profile>
</profiles>
</project>
Loading

0 comments on commit 0330b53

Please sign in to comment.