From 61e804f3a0368fc8301bab16404c248f8fa87112 Mon Sep 17 00:00:00 2001 From: zhangbutao Date: Tue, 28 May 2024 09:38:09 +0800 Subject: [PATCH] use iceberg's conf name --- .../src/main/java/org/apache/iceberg/mr/InputFormatConfig.java | 1 - .../org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java | 3 ++- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java index 4d935302437d..36f409432a58 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java @@ -87,7 +87,6 @@ private InputFormatConfig() { public static final String CATALOG_DEFAULT_CONFIG_PREFIX = "iceberg.catalog-default."; public static final String QUERY_FILTERS = "iceberg.query.filters"; - public static final String TABLE_PLAN_WORKER_POOL_SIZE = "iceberg.worker.num-threads"; public enum InMemoryDataModel { PIG, diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java index bc30a89207bb..5a38e98ad11a 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java @@ -63,6 +63,7 @@ import org.apache.iceberg.SchemaParser; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.StructLike; +import org.apache.iceberg.SystemConfigs; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableScan; @@ -211,7 +212,7 @@ public List getSplits(JobContext context) { }); final ExecutorService workerPool = ThreadPools.newWorkerPool("iceberg-plan-worker-pool", - conf.getInt(InputFormatConfig.TABLE_PLAN_WORKER_POOL_SIZE, ThreadPools.WORKER_THREAD_POOL_SIZE)); + conf.getInt(SystemConfigs.WORKER_THREAD_POOL_SIZE.propertyKey(), ThreadPools.WORKER_THREAD_POOL_SIZE)); try { return planInputSplits(table, conf, workerPool); } finally {