From 19f4bbba1f6c5377ea591054add25f92dc03fcdd Mon Sep 17 00:00:00 2001 From: hongdd Date: Thu, 25 Mar 2021 15:33:20 +0800 Subject: [PATCH 1/3] [HUDI-1176]upgrade log4j to log4j2 --- hudi-cli/pom.xml | 34 +++++++- .../hudi/cli/commands/CompactionCommand.java | 4 +- .../hudi/cli/commands/ExportCommand.java | 4 +- .../hudi/cli/commands/RepairsCommand.java | 5 +- .../apache/hudi/cli/commands/SparkMain.java | 5 +- .../hudi/cli/utils/SparkTempViewProvider.java | 4 +- .../test/resources/log4j-surefire.properties | 25 ------ .../log4j2-surefire-quiet.properties | 39 +++++++++ .../test/resources/log4j2-surefire.properties | 31 ++++--- hudi-client/hudi-client-common/pom.xml | 24 +++++- .../hudi/async/AsyncCompactService.java | 4 +- .../apache/hudi/async/HoodieAsyncService.java | 4 +- .../HoodieWriteCommitHttpCallbackClient.java | 4 +- .../impl/HoodieWriteCommitHttpCallback.java | 4 +- .../hudi/client/AbstractHoodieClient.java | 4 +- .../client/AbstractHoodieWriteClient.java | 4 +- .../hudi/client/AsyncCleanerService.java | 4 +- .../hudi/client/CompactionAdminClient.java | 4 +- .../hudi/client/ReplaceArchivalHelper.java | 4 +- .../FullRecordBootstrapDataProvider.java | 4 +- .../selector/BootstrapRegexModeSelector.java | 4 +- .../EmbeddedTimelineServerHelper.java | 4 +- .../embedded/EmbeddedTimelineService.java | 4 +- .../hudi/client/heartbeat/HeartbeatUtils.java | 4 +- .../heartbeat/HoodieHeartbeatClient.java | 4 +- .../DefaultHBaseQPSResourceAllocator.java | 4 +- .../apache/hudi/io/HoodieAppendHandle.java | 4 +- .../apache/hudi/io/HoodieCreateHandle.java | 4 +- .../apache/hudi/io/HoodieKeyLookupHandle.java | 4 +- .../org/apache/hudi/io/HoodieMergeHandle.java | 4 +- .../org/apache/hudi/io/HoodieWriteHandle.java | 4 +- .../hudi/io/storage/HoodieConcatHandle.java | 4 +- .../HoodieBackedTableMetadataWriter.java | 4 +- .../hudi/metrics/ConsoleMetricsReporter.java | 4 +- .../apache/hudi/metrics/HoodieMetrics.java | 4 +- .../hudi/metrics/JmxMetricsReporter.java | 5 +- .../java/org/apache/hudi/metrics/Metrics.java | 4 +- .../hudi/metrics/MetricsGraphiteReporter.java | 4 +- .../hudi/metrics/MetricsReporterFactory.java | 4 +- .../metrics/datadog/DatadogHttpClient.java | 4 +- .../hudi/metrics/datadog/DatadogReporter.java | 4 +- .../prometheus/PrometheusReporter.java | 4 +- .../prometheus/PushGatewayReporter.java | 4 +- .../org/apache/hudi/table/HoodieTable.java | 4 +- .../hudi/table/HoodieTimelineArchiveLog.java | 5 +- .../org/apache/hudi/table/MarkerFiles.java | 4 +- .../action/clean/BaseCleanActionExecutor.java | 4 +- .../hudi/table/action/clean/CleanPlanner.java | 4 +- .../strategy/ClusteringExecutionStrategy.java | 4 +- .../strategy/ClusteringPlanStrategy.java | 4 +- .../PartitionAwareClusteringPlanStrategy.java | 4 +- .../commit/BaseCommitActionExecutor.java | 4 +- .../restore/BaseRestoreActionExecutor.java | 4 +- .../AbstractMarkerBasedRollbackStrategy.java | 4 +- ...BaseCopyOnWriteRollbackActionExecutor.java | 4 +- ...BaseMergeOnReadRollbackActionExecutor.java | 4 +- .../rollback/BaseRollbackActionExecutor.java | 5 +- .../table/action/rollback/RollbackUtils.java | 4 +- .../savepoint/SavepointActionExecutor.java | 4 +- .../action/savepoint/SavepointHelpers.java | 4 +- .../upgrade/AbstractUpgradeDowngrade.java | 4 +- .../callback/http/TestCallbackHttpClient.java | 50 +++++++---- .../datadog/TestDatadogHttpClient.java | 54 ++++++++---- .../metrics/datadog/TestDatadogReporter.java | 41 +++++++-- .../testutils/HoodieWriteableTestTable.java | 4 +- .../test/resources/log4j-surefire.properties | 31 ------- .../log4j2-surefire-quiet.properties | 23 +++-- .../test/resources/log4j2-surefire.properties | 51 +++++++++++ hudi-client/hudi-flink-client/pom.xml | 14 +++ .../index/state/FlinkInMemoryStateIndex.java | 4 +- .../org/apache/hudi/io/FlinkCreateHandle.java | 4 +- .../org/apache/hudi/io/FlinkMergeHandle.java | 4 +- .../clean/FlinkCleanActionExecutor.java | 4 +- .../commit/BaseFlinkCommitActionExecutor.java | 4 +- .../rollback/ListingBasedRollbackHelper.java | 4 +- .../HoodieFlinkClientTestHarness.java | 4 +- .../HoodieFlinkWriteableTestTable.java | 4 +- .../test/resources/log4j-surefire.properties | 31 ------- .../log4j2-surefire-quiet.properties | 24 +++--- .../test/resources/log4j2-surefire.properties | 51 +++++++++++ .../action/clean/JavaCleanActionExecutor.java | 4 +- .../commit/BaseJavaCommitActionExecutor.java | 4 +- .../action/commit/JavaUpsertPartitioner.java | 4 +- .../HoodieJavaClientTestHarness.java | 4 +- .../test/resources/log4j-surefire.properties | 31 ------- .../log4j2-surefire-quiet.properties} | 24 +++--- .../test/resources/log4j2-surefire.properties | 51 +++++++++++ hudi-client/hudi-spark-client/pom.xml | 20 +++++ .../hudi/client/HoodieSparkCompactor.java | 4 +- .../hudi/client/SparkRDDWriteClient.java | 4 +- ...SparkRecentDaysClusteringPlanStrategy.java | 4 +- .../SparkSortAndSizeExecutionStrategy.java | 4 +- .../strategy/SparkRejectUpdateStrategy.java | 4 +- .../BucketizedBloomCheckPartitioner.java | 4 +- .../index/bloom/SparkHoodieBloomIndex.java | 4 +- .../index/hbase/SparkHoodieHBaseIndex.java | 4 +- .../apache/hudi/io/HoodieRowCreateHandle.java | 4 +- .../SparkHoodieBackedTableMetadataWriter.java | 4 +- .../table/HoodieSparkCopyOnWriteTable.java | 4 +- .../SparkBootstrapCommitActionExecutor.java | 4 +- .../clean/SparkCleanActionExecutor.java | 4 +- .../SparkClusteringPlanActionExecutor.java | 4 +- ...ExecuteClusteringCommitActionExecutor.java | 4 +- .../commit/BaseSparkCommitActionExecutor.java | 4 +- .../SparkInsertOverwritePartitioner.java | 4 +- .../action/commit/UpsertPartitioner.java | 4 +- .../HoodieSparkMergeOnReadTableCompactor.java | 4 +- ...SparkScheduleCompactionActionExecutor.java | 4 +- ...bstractSparkDeltaCommitActionExecutor.java | 4 +- .../rollback/ListingBasedRollbackHelper.java | 4 +- .../client/TestCompactionAdminClient.java | 4 +- .../TestHoodieClientOnCopyOnWriteStorage.java | 4 +- .../org/apache/hudi/client/TestMultiFS.java | 4 +- .../metadata/TestHoodieBackedMetadata.java | 4 +- .../org/apache/hudi/table/TestCleaner.java | 4 +- .../commit/TestCopyOnWriteActionExecutor.java | 4 +- .../action/commit/TestUpsertPartitioner.java | 4 +- .../hudi/testutils/HoodieClientTestBase.java | 4 +- .../testutils/HoodieClientTestHarness.java | 4 +- .../hudi/testutils/HoodieClientTestUtils.java | 4 +- .../HoodieSparkWriteableTestTable.java | 4 +- .../test/resources/log4j-surefire.properties | 31 ------- ...rties => log4j2-surefire-quiet.properties} | 24 +++--- .../test/resources/log4j2-surefire.properties | 51 +++++++++++ hudi-common/pom.xml | 28 ++++++ .../bootstrap/index/HFileBootstrapIndex.java | 4 +- .../config/DFSPropertiesConfiguration.java | 4 +- .../org/apache/hudi/common/fs/FSUtils.java | 4 +- .../common/fs/FailSafeConsistencyGuard.java | 4 +- .../common/fs/OptimisticConsistencyGuard.java | 4 +- .../common/model/HoodieCommitMetadata.java | 4 +- .../common/model/HoodiePartitionMetadata.java | 4 +- .../model/HoodieReplaceCommitMetadata.java | 4 +- .../model/HoodieRollingStatMetadata.java | 4 +- .../hudi/common/table/HoodieTableConfig.java | 4 +- .../common/table/HoodieTableMetaClient.java | 4 +- .../common/table/TableSchemaResolver.java | 4 +- .../log/AbstractHoodieLogRecordScanner.java | 4 +- .../common/table/log/HoodieLogFileReader.java | 4 +- .../common/table/log/HoodieLogFormat.java | 4 +- .../table/log/HoodieLogFormatReader.java | 4 +- .../table/log/HoodieLogFormatWriter.java | 4 +- .../log/HoodieMergedLogRecordScanner.java | 4 +- .../table/log/block/HoodieHFileDataBlock.java | 4 +- .../table/timeline/HoodieActiveTimeline.java | 4 +- .../timeline/HoodieArchivedTimeline.java | 4 +- .../table/timeline/TimelineDiffHelper.java | 4 +- .../view/AbstractTableFileSystemView.java | 4 +- .../table/view/FileSystemViewManager.java | 4 +- .../table/view/HoodieTableFileSystemView.java | 4 +- ...IncrementalTimelineSyncFileSystemView.java | 4 +- .../view/PriorityBasedFileSystemView.java | 4 +- .../view/RemoteHoodieTableFileSystemView.java | 4 +- .../view/RocksDbBasedFileSystemView.java | 4 +- .../view/SpillableMapBasedFileSystemView.java | 4 +- .../common/util/BufferedRandomAccessFile.java | 5 +- .../hudi/common/util/ClusteringUtils.java | 4 +- .../apache/hudi/common/util/CommitUtils.java | 4 +- .../hudi/common/util/CompactionUtils.java | 4 +- .../util/HoodieRecordSizeEstimator.java | 4 +- .../apache/hudi/common/util/RateLimiter.java | 4 +- .../hudi/common/util/ReflectionUtils.java | 4 +- .../hudi/common/util/TablePathUtils.java | 4 +- .../common/util/collection/DiskBasedMap.java | 4 +- .../util/collection/ExternalSpillableMap.java | 4 +- .../common/util/collection/RocksDBDAO.java | 4 +- .../util/queue/BoundedInMemoryExecutor.java | 4 +- .../util/queue/BoundedInMemoryQueue.java | 4 +- .../queue/FunctionBasedQueueProducer.java | 4 +- .../queue/IteratorBasedQueueProducer.java | 4 +- .../hudi/metadata/BaseTableMetadata.java | 4 +- .../metadata/HoodieBackedTableMetadata.java | 4 +- .../hudi/metadata/HoodieMetadataMetrics.java | 4 +- .../metadata/HoodieTableMetadataUtil.java | 4 +- .../metadata/TimelineMergedTableMetadata.java | 4 +- .../view/TestHoodieTableFileSystemView.java | 4 +- .../table/view/TestIncrementalFSViewSync.java | 4 +- .../testutils/HoodieTestDataGenerator.java | 4 +- .../minicluster/HdfsTestService.java | 4 +- .../minicluster/ZookeeperTestService.java | 4 +- .../test/resources/log4j-surefire.properties | 31 ------- .../log4j2-surefire-quiet.properties | 34 ++++++++ .../test/resources/log4j2-surefire.properties | 51 +++++++++++ hudi-examples/pom.xml | 16 ++++ .../java/HoodieJavaWriteClientExample.java | 4 +- .../spark/HoodieWriteClientExample.java | 4 +- .../hudi/sink/partitioner/BucketAssigner.java | 4 +- .../test/resources/log4j-surefire.properties | 31 ------- .../log4j2-surefire-quiet.properties | 34 ++++++++ .../test/resources/log4j2-surefire.properties | 51 +++++++++++ hudi-hadoop-mr/pom.xml | 22 +++++ .../BootstrapColumnStichingRecordReader.java | 4 +- .../hudi/hadoop/HoodieHFileInputFormat.java | 4 +- .../hudi/hadoop/HoodieParquetInputFormat.java | 4 +- .../hudi/hadoop/HoodieROTablePathFilter.java | 4 +- .../apache/hudi/hadoop/InputPathHandler.java | 4 +- .../hadoop/RecordReaderValueIterator.java | 4 +- .../hive/HoodieCombineHiveInputFormat.java | 4 +- .../AbstractRealtimeRecordReader.java | 4 +- .../HoodieCombineRealtimeRecordReader.java | 4 +- .../HoodieHFileRealtimeInputFormat.java | 4 +- .../HoodieParquetRealtimeInputFormat.java | 4 +- .../realtime/HoodieRealtimeRecordReader.java | 4 +- .../RealtimeCompactedRecordReader.java | 4 +- .../hudi/hadoop/utils/HoodieHiveUtils.java | 4 +- .../hadoop/utils/HoodieInputFormatUtils.java | 4 +- .../utils/HoodieRealtimeInputFormatUtils.java | 4 +- .../HoodieRealtimeRecordReaderUtils.java | 4 +- .../resources/log4j-surefire-quiet.properties | 29 ------- .../test/resources/log4j-surefire.properties | 30 ------- .../log4j2-surefire-quiet.properties | 40 +++++++++ .../test/resources/log4j2-surefire.properties | 45 ++++++++++ hudi-integ-test/pom.xml | 24 +++++- .../integ/testsuite/dag/WriterContext.java | 4 +- .../TestSuiteFileBasedSchemaProvider.java | 4 +- .../org/apache/hudi/integ/ITTestBase.java | 4 +- .../testsuite/job/TestHoodieTestSuiteJob.java | 6 +- .../resources/log4j-surefire-quiet.properties | 30 ------- .../test/resources/log4j-surefire.properties | 30 ------- .../log4j2-surefire-quiet.properties | 40 +++++++++ .../test/resources/log4j2-surefire.properties | 45 ++++++++++ .../java/org/apache/hudi/DataSourceUtils.java | 4 +- .../BulkInsertDataInternalWriterHelper.java | 4 +- .../DataSourceInternalWriterHelper.java | 4 +- .../org/apache/hudi/DataSourceOptions.scala | 3 +- .../log4j2-surefire-quiet.properties | 40 +++++++++ .../java/resources/log4j2-surefire.properties | 45 ++++++++++ hudi-spark-datasource/hudi-spark/pom.xml | 50 +++++++++-- .../hudi/HoodieDatasetBulkInsertHelper.java | 4 +- .../scala/org/apache/hudi/DefaultSource.scala | 2 +- .../apache/hudi/HoodieSparkSqlWriter.scala | 2 +- .../org/apache/hudi/HoodieStreamingSink.scala | 2 +- .../org/apache/hudi/IncrementalRelation.scala | 2 +- .../hudi/MergeOnReadIncrementalRelation.scala | 2 +- .../src/test/java/HoodieJavaApp.java | 4 +- .../src/test/java/HoodieJavaGenerateApp.java | 4 +- .../src/test/java/HoodieJavaStreamingApp.java | 4 +- .../resources/log4j-surefire-quiet.properties | 30 ------- .../test/resources/log4j-surefire.properties | 31 ------- .../log4j2-surefire-quiet.properties | 34 ++++++++ .../test/resources/log4j2-surefire.properties | 51 +++++++++++ .../hudi/functional/TestMORDataSource.scala | 2 +- .../functional/TestStructuredStreaming.scala | 2 +- .../log4j2-surefire-quiet.properties | 40 +++++++++ .../test/resources/log4j2-surefire.properties | 45 ++++++++++ .../log4j2-surefire-quiet.properties | 40 +++++++++ .../test/resources/log4j2-surefire.properties | 45 ++++++++++ hudi-sync/hudi-dla-sync/pom.xml | 10 ++- .../java/org/apache/hudi/dla/DLASyncTool.java | 4 +- .../org/apache/hudi/dla/HoodieDLAClient.java | 4 +- .../resources/log4j-surefire-quiet.properties | 29 ------- .../test/resources/log4j-surefire.properties | 30 ------- .../log4j2-surefire-quiet.properties | 39 +++++++++ .../test/resources/log4j2-surefire.properties | 51 +++++++++++ hudi-sync/hudi-hive-sync/pom.xml | 54 +++++++++++- .../org/apache/hudi/hive/HiveSyncTool.java | 4 +- .../apache/hudi/hive/HoodieHiveClient.java | 4 +- .../apache/hudi/hive/util/HiveSchemaUtil.java | 4 +- .../hudi/hive/testutils/HiveTestService.java | 4 +- .../resources/log4j-surefire-quiet.properties | 29 ------- .../test/resources/log4j-surefire.properties | 30 ------- .../log4j2-surefire-quiet.properties | 39 +++++++++ .../test/resources/log4j2-surefire.properties | 51 +++++++++++ hudi-sync/hudi-sync-common/pom.xml | 6 ++ .../sync/common/AbstractSyncHoodieClient.java | 4 +- hudi-timeline-service/pom.xml | 8 +- .../hudi/timeline/service/RequestHandler.java | 4 +- .../timeline/service/TimelineService.java | 4 +- .../TestRemoteHoodieTableFileSystemView.java | 4 +- .../resources/log4j-surefire-quiet.properties | 29 ------- .../test/resources/log4j-surefire.properties | 30 ------- .../log4j2-surefire-quiet.properties | 39 +++++++++ .../test/resources/log4j2-surefire.properties | 51 +++++++++++ hudi-utilities/pom.xml | 30 ++++++- .../hudi/utilities/HDFSParquetImporter.java | 4 +- .../hudi/utilities/HiveIncrementalPuller.java | 4 +- .../apache/hudi/utilities/HoodieCleaner.java | 4 +- .../hudi/utilities/HoodieClusteringJob.java | 4 +- .../hudi/utilities/HoodieCompactor.java | 4 +- .../hudi/utilities/HoodieSnapshotCopier.java | 4 +- .../utilities/HoodieSnapshotExporter.java | 4 +- .../apache/hudi/utilities/UtilHelpers.java | 4 +- .../kafka/HoodieWriteCommitKafkaCallback.java | 4 +- .../deltastreamer/BootstrapExecutor.java | 4 +- .../utilities/deltastreamer/DeltaSync.java | 4 +- .../deltastreamer/HoodieDeltaStreamer.java | 4 +- .../HoodieMultiTableDeltaStreamer.java | 4 +- .../deltastreamer/SchedulerConfGenerator.java | 4 +- .../utilities/perf/TimelineServerPerf.java | 4 +- .../utilities/sources/AvroKafkaSource.java | 4 +- .../utilities/sources/HiveIncrPullSource.java | 4 +- .../utilities/sources/HoodieIncrSource.java | 4 +- .../utilities/sources/JsonKafkaSource.java | 4 +- .../sources/helpers/DFSPathSelector.java | 4 +- .../helpers/DatePartitionPathSelector.java | 4 +- .../sources/helpers/KafkaOffsetGen.java | 4 +- .../transform/FlatteningTransformer.java | 4 +- .../transform/SqlQueryBasedTransformer.java | 4 +- .../functional/TestHoodieDeltaStreamer.java | 4 +- .../TestHoodieMultiTableDeltaStreamer.java | 4 +- .../TestHoodieSnapshotExporter.java | 4 +- .../TestJdbcbasedSchemaProvider.java | 4 +- .../utilities/sources/TestDataSource.java | 4 +- .../testutils/UtilitiesTestBase.java | 9 +- .../sources/AbstractBaseTestSource.java | 4 +- .../sources/DistributedTestDataSource.java | 4 +- .../resources/log4j-surefire-quiet.properties | 30 ------- .../test/resources/log4j-surefire.properties | 31 ------- .../log4j2-surefire-quiet.properties | 45 ++++++++++ .../test/resources/log4j2-surefire.properties | 51 +++++++++++ packaging/hudi-flink-bundle/pom.xml | 9 ++ packaging/hudi-hadoop-mr-bundle/pom.xml | 7 ++ packaging/hudi-hive-sync-bundle/pom.xml | 3 + packaging/hudi-integ-test-bundle/pom.xml | 59 ++++++++++++- packaging/hudi-presto-bundle/pom.xml | 3 + packaging/hudi-spark-bundle/pom.xml | 33 +++++++ packaging/hudi-timeline-server-bundle/pom.xml | 12 ++- packaging/hudi-utilities-bundle/pom.xml | 33 +++++++ pom.xml | 85 +++++++++++++++++-- 319 files changed, 2500 insertions(+), 1223 deletions(-) delete mode 100644 hudi-cli/src/test/resources/log4j-surefire.properties create mode 100644 hudi-cli/src/test/resources/log4j2-surefire-quiet.properties rename hudi-common/src/test/resources/log4j-surefire-quiet.properties => hudi-cli/src/test/resources/log4j2-surefire.properties (56%) delete mode 100644 hudi-client/hudi-client-common/src/test/resources/log4j-surefire.properties rename hudi-cli/src/test/resources/log4j-surefire-quiet.properties => hudi-client/hudi-client-common/src/test/resources/log4j2-surefire-quiet.properties (62%) create mode 100644 hudi-client/hudi-client-common/src/test/resources/log4j2-surefire.properties delete mode 100644 hudi-client/hudi-flink-client/src/test/resources/log4j-surefire.properties rename hudi-flink/src/test/resources/log4j-surefire-quiet.properties => hudi-client/hudi-flink-client/src/test/resources/log4j2-surefire-quiet.properties (62%) create mode 100644 hudi-client/hudi-flink-client/src/test/resources/log4j2-surefire.properties delete mode 100644 hudi-client/hudi-java-client/src/test/resources/log4j-surefire.properties rename hudi-client/{hudi-client-common/src/test/resources/log4j-surefire-quiet.properties => hudi-java-client/src/test/resources/log4j2-surefire-quiet.properties} (62%) create mode 100644 hudi-client/hudi-java-client/src/test/resources/log4j2-surefire.properties delete mode 100644 hudi-client/hudi-spark-client/src/test/resources/log4j-surefire.properties rename hudi-client/hudi-spark-client/src/test/resources/{log4j-surefire-quiet.properties => log4j2-surefire-quiet.properties} (62%) create mode 100644 hudi-client/hudi-spark-client/src/test/resources/log4j2-surefire.properties delete mode 100644 hudi-common/src/test/resources/log4j-surefire.properties create mode 100644 hudi-common/src/test/resources/log4j2-surefire-quiet.properties create mode 100644 hudi-common/src/test/resources/log4j2-surefire.properties delete mode 100644 hudi-flink/src/test/resources/log4j-surefire.properties create mode 100644 hudi-flink/src/test/resources/log4j2-surefire-quiet.properties create mode 100644 hudi-flink/src/test/resources/log4j2-surefire.properties delete mode 100644 hudi-hadoop-mr/src/test/resources/log4j-surefire-quiet.properties delete mode 100644 hudi-hadoop-mr/src/test/resources/log4j-surefire.properties create mode 100644 hudi-hadoop-mr/src/test/resources/log4j2-surefire-quiet.properties create mode 100644 hudi-hadoop-mr/src/test/resources/log4j2-surefire.properties delete mode 100644 hudi-integ-test/src/test/resources/log4j-surefire-quiet.properties delete mode 100644 hudi-integ-test/src/test/resources/log4j-surefire.properties create mode 100644 hudi-integ-test/src/test/resources/log4j2-surefire-quiet.properties create mode 100644 hudi-integ-test/src/test/resources/log4j2-surefire.properties create mode 100644 hudi-spark-datasource/hudi-spark-common/src/test/java/resources/log4j2-surefire-quiet.properties create mode 100644 hudi-spark-datasource/hudi-spark-common/src/test/java/resources/log4j2-surefire.properties delete mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/log4j-surefire-quiet.properties delete mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/log4j-surefire.properties create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/log4j2-surefire-quiet.properties create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/log4j2-surefire.properties create mode 100644 hudi-spark-datasource/hudi-spark2/src/test/resources/log4j2-surefire-quiet.properties create mode 100644 hudi-spark-datasource/hudi-spark2/src/test/resources/log4j2-surefire.properties create mode 100644 hudi-spark-datasource/hudi-spark3/src/test/resources/log4j2-surefire-quiet.properties create mode 100644 hudi-spark-datasource/hudi-spark3/src/test/resources/log4j2-surefire.properties delete mode 100644 hudi-sync/hudi-dla-sync/src/test/resources/log4j-surefire-quiet.properties delete mode 100644 hudi-sync/hudi-dla-sync/src/test/resources/log4j-surefire.properties create mode 100644 hudi-sync/hudi-dla-sync/src/test/resources/log4j2-surefire-quiet.properties create mode 100644 hudi-sync/hudi-dla-sync/src/test/resources/log4j2-surefire.properties delete mode 100644 hudi-sync/hudi-hive-sync/src/test/resources/log4j-surefire-quiet.properties delete mode 100644 hudi-sync/hudi-hive-sync/src/test/resources/log4j-surefire.properties create mode 100644 hudi-sync/hudi-hive-sync/src/test/resources/log4j2-surefire-quiet.properties create mode 100644 hudi-sync/hudi-hive-sync/src/test/resources/log4j2-surefire.properties delete mode 100644 hudi-timeline-service/src/test/resources/log4j-surefire-quiet.properties delete mode 100644 hudi-timeline-service/src/test/resources/log4j-surefire.properties create mode 100644 hudi-timeline-service/src/test/resources/log4j2-surefire-quiet.properties create mode 100644 hudi-timeline-service/src/test/resources/log4j2-surefire.properties delete mode 100644 hudi-utilities/src/test/resources/log4j-surefire-quiet.properties delete mode 100644 hudi-utilities/src/test/resources/log4j-surefire.properties create mode 100644 hudi-utilities/src/test/resources/log4j2-surefire-quiet.properties create mode 100644 hudi-utilities/src/test/resources/log4j2-surefire.properties diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml index fd82703af3333..1a8c6335153d6 100644 --- a/hudi-cli/pom.xml +++ b/hudi-cli/pom.xml @@ -189,6 +189,18 @@ org.apache.parquet parquet-hadoop-bundle + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-web + + + org.apache.logging.log4j + log4j-slf4j-impl + @@ -200,8 +212,8 @@ - log4j - log4j + org.apache.logging.log4j + log4j-core @@ -219,6 +231,12 @@ org.apache.spark spark-core_${scala.binary.version} + + + log4j + log4j + + org.apache.spark @@ -250,10 +268,22 @@ org.apache.hadoop hadoop-common + + + log4j + log4j + + org.apache.hadoop hadoop-hdfs + + + log4j + log4j + + diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java index 67445ea873f7e..d34dc1eb82dbc 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java @@ -48,8 +48,8 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.launcher.SparkLauncher; import org.apache.spark.util.Utils; import org.springframework.shell.core.CommandMarker; diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java index 8bd842c825659..d9ec2f58c62bd 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java @@ -35,8 +35,8 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.avro.specific.SpecificData; diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java index 40dddfc725488..3a9d576a2bc68 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java @@ -35,7 +35,8 @@ import org.apache.avro.AvroRuntimeException; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.util.StringUtils; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.launcher.SparkLauncher; import org.apache.spark.util.Utils; import org.springframework.shell.core.CommandMarker; @@ -61,7 +62,7 @@ @Component public class RepairsCommand implements CommandMarker { - private static final Logger LOG = Logger.getLogger(RepairsCommand.class); + private static final Logger LOG = LogManager.getLogger(RepairsCommand.class); public static final String DEDUPLICATE_RETURN_PREFIX = "Deduplicated files placed in: "; @CliCommand(value = "repair deduplicate", diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index 7833ee750b5c2..3225cc3f5768d 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -49,7 +49,8 @@ import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; import org.apache.hadoop.fs.Path; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SQLContext; @@ -63,7 +64,7 @@ */ public class SparkMain { - private static final Logger LOG = Logger.getLogger(SparkMain.class); + private static final Logger LOG = LogManager.getLogger(SparkMain.class); /** * Commands. diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkTempViewProvider.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkTempViewProvider.java index 5e029cd050b34..86f318d008428 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkTempViewProvider.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkTempViewProvider.java @@ -20,8 +20,8 @@ import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; diff --git a/hudi-cli/src/test/resources/log4j-surefire.properties b/hudi-cli/src/test/resources/log4j-surefire.properties deleted file mode 100644 index a59d4ebe2b194..0000000000000 --- a/hudi-cli/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,25 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, A1 -log4j.category.org.apache=INFO -log4j.category.org.apache.parquet.hadoop=WARN -# A1 is set to be a ConsoleAppender. -log4j.appender.A1=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.A1.layout=org.apache.log4j.PatternLayout -log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n diff --git a/hudi-cli/src/test/resources/log4j2-surefire-quiet.properties b/hudi-cli/src/test/resources/log4j2-surefire-quiet.properties new file mode 100644 index 0000000000000..6e2ba4b09aa54 --- /dev/null +++ b/hudi-cli/src/test/resources/log4j2-surefire-quiet.properties @@ -0,0 +1,39 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.parquet.name = org.apache.hudi +logger.parquet.level = debug +logger.parquet.additivity = false +logger.parquet.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-common/src/test/resources/log4j-surefire-quiet.properties b/hudi-cli/src/test/resources/log4j2-surefire.properties similarity index 56% rename from hudi-common/src/test/resources/log4j-surefire-quiet.properties rename to hudi-cli/src/test/resources/log4j2-surefire.properties index ca0a50c84270c..617bfab845c31 100644 --- a/hudi-common/src/test/resources/log4j-surefire-quiet.properties +++ b/hudi-cli/src/test/resources/log4j2-surefire.properties @@ -15,16 +15,27 @@ # See the License for the specific language governing permissions and # limitations under the License. ### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR +status = warn +name = PropertiesConfig # CONSOLE is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +appender.console.type = Console +appender.console.name = consoleLogger # CONSOLE uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for parquet.hadoop +logger.parquet.name = org.apache.parquet.hadoop +logger.parquet.level = warn +logger.parquet.additivity = false +logger.parquet.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml index 5978939b0d679..3c7d2ba7a421e 100644 --- a/hudi-client/hudi-client-common/pom.xml +++ b/hudi-client/hudi-client-common/pom.xml @@ -49,8 +49,12 @@ - log4j - log4j + org.apache.logging.log4j + log4j-core + + + org.apache.logging.log4j + log4j-1.2-api @@ -68,6 +72,10 @@ com.rabbitmq * + + org.slf4j + slf4j-api + @@ -124,6 +132,10 @@ javax.servlet * + + log4j + log4j + @@ -143,6 +155,14 @@ javax.servlet * + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java index 1bb5daa528c79..e885543f5edc4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java @@ -24,8 +24,8 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.concurrent.BlockingQueue; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java index 32dd0427255f0..7882980ca479a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java @@ -20,8 +20,8 @@ import org.apache.hudi.common.util.collection.Pair; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.Serializable; import java.util.concurrent.CompletableFuture; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java index 6c41e2f5ead70..40bf66388454a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java @@ -28,8 +28,8 @@ import org.apache.http.impl.client.HttpClientBuilder; import org.apache.hudi.config.HoodieWriteCommitCallbackConfig; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.Closeable; import java.io.IOException; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java index bb60879efbbad..c0631c6528cc8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java @@ -23,8 +23,8 @@ import org.apache.hudi.callback.util.HoodieWriteCommitCallbackUtil; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; /** * A http implementation of {@link HoodieWriteCommitCallback}. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java index 350fe0c9bf7e0..3e2dc05032e89 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java @@ -30,8 +30,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index ed3ba1d220e82..13662c9d11e24 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -70,8 +70,8 @@ import org.apache.hudi.table.MarkerFiles; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.savepoint.SavepointHelpers; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.nio.charset.StandardCharsets; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java index c259acf2df67f..bd5c04a4cb0a1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java @@ -22,8 +22,8 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java index 9ace03a6d50ab..7e1910ee2a0fa 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java @@ -44,8 +44,8 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.action.compact.OperationResult; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.FileNotFoundException; import java.io.IOException; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java index ee03bd7cbf841..22782c1eb9b3b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java @@ -31,8 +31,8 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.TableFileSystemView; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java index 1cf1702717295..a4077abc507bc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java @@ -24,8 +24,8 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.collection.Pair; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.List; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java index 43fae7000f456..6d126962aeba6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java @@ -26,8 +26,8 @@ import org.apache.hudi.client.bootstrap.BootstrapMode; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; public class BootstrapRegexModeSelector extends BootstrapModeSelector { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineServerHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineServerHelper.java index e5a719eb3d8b5..e35aa04583a24 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineServerHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineServerHelper.java @@ -23,8 +23,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java index a2bc7116d32ba..e93e0ee733cde 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java @@ -27,8 +27,8 @@ import org.apache.hudi.common.util.NetworkUtils; import org.apache.hudi.timeline.service.TimelineService; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java index 0c69a047fe167..7b385bcac1acf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java @@ -26,8 +26,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.File; import java.io.IOException; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java index 044e3441d7578..564a8fd90bb7e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java @@ -25,8 +25,8 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieHeartbeatException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import javax.annotation.concurrent.NotThreadSafe; import java.io.File; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/hbase/DefaultHBaseQPSResourceAllocator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/hbase/DefaultHBaseQPSResourceAllocator.java index e3a49041ee45c..25c3c4625da9a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/hbase/DefaultHBaseQPSResourceAllocator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/hbase/DefaultHBaseQPSResourceAllocator.java @@ -20,8 +20,8 @@ import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; public class DefaultHBaseQPSResourceAllocator implements HBaseIndexQPSResourceAllocator { private HoodieWriteConfig hoodieWriteConfig; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index b248a2f68abb7..51d0f2988db90 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -53,8 +53,8 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java index 6fa9b56a63e00..3079ae86e921b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java @@ -40,8 +40,8 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Collections; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java index ad84e3e974af8..9f90298e080aa 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java @@ -30,8 +30,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java index 8579f54b02686..2114a71ad3ca2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java @@ -46,8 +46,8 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Collections; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index df0c2b6e25c31..26cd492e53a8c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -41,8 +41,8 @@ import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Collections; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieConcatHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieConcatHandle.java index ea56689b5364f..603b9e7c5bdc3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieConcatHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieConcatHandle.java @@ -28,8 +28,8 @@ import org.apache.hudi.table.HoodieTable; import org.apache.avro.generic.GenericRecord; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Iterator; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index dfd4c4c0457db..2bcb42fee4dff 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -56,8 +56,8 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Arrays; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/ConsoleMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/ConsoleMetricsReporter.java index b65c4ade88a82..98681a66287ac 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/ConsoleMetricsReporter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/ConsoleMetricsReporter.java @@ -21,8 +21,8 @@ import java.io.Closeable; import java.util.concurrent.TimeUnit; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import com.codahale.metrics.ConsoleReporter; import com.codahale.metrics.MetricFilter; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java index 5db28e2417749..38f78044ab3d6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java @@ -25,8 +25,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import com.codahale.metrics.Timer; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; /** * Wrapper for metrics-related operations. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxMetricsReporter.java index 309981a9d85cf..d21c4995eeed0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxMetricsReporter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxMetricsReporter.java @@ -22,7 +22,8 @@ import org.apache.hudi.exception.HoodieException; import com.codahale.metrics.MetricRegistry; -import org.apache.log4j.LogManager; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import javax.management.MBeanServer; @@ -37,7 +38,7 @@ */ public class JmxMetricsReporter extends MetricsReporter { - private static final org.apache.log4j.Logger LOG = LogManager.getLogger(JmxMetricsReporter.class); + private static final Logger LOG = LogManager.getLogger(JmxMetricsReporter.class); private final MetricRegistry registry; private JmxReporterServer jmxReporterServer; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java index 5667a66a54934..5653daeb28289 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java @@ -24,8 +24,8 @@ import org.apache.hudi.exception.HoodieException; import com.codahale.metrics.MetricRegistry; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.Closeable; import java.util.Map; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsGraphiteReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsGraphiteReporter.java index 9855ac0b0272d..d172105d5d230 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsGraphiteReporter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsGraphiteReporter.java @@ -24,8 +24,8 @@ import com.codahale.metrics.MetricRegistry; import com.codahale.metrics.graphite.Graphite; import com.codahale.metrics.graphite.GraphiteReporter; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.Closeable; import java.net.InetSocketAddress; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java index 66cdeebe97f08..3b60458fae288 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java @@ -28,8 +28,8 @@ import org.apache.hudi.metrics.prometheus.PrometheusReporter; import org.apache.hudi.metrics.prometheus.PushGatewayMetricsReporter; import org.apache.hudi.metrics.userdefined.AbstractUserDefinedMetricsReporter; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.Properties; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogHttpClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogHttpClient.java index b0912aaaabfe2..b3b1bcb8f7866 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogHttpClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogHttpClient.java @@ -32,8 +32,8 @@ import org.apache.http.entity.StringEntity; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClientBuilder; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.Closeable; import java.io.IOException; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogReporter.java index a388aecda0ae4..b67252a729593 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogReporter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogReporter.java @@ -34,8 +34,8 @@ import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.ObjectNode; import com.fasterxml.jackson.databind.node.TextNode; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.List; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java index 81c89b6e1ccc4..9f109150be552 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java @@ -25,8 +25,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.metrics.MetricsReporter; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.Closeable; import java.net.InetSocketAddress; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayReporter.java index 3b1988259a15b..0350778e96060 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayReporter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayReporter.java @@ -29,8 +29,8 @@ import io.prometheus.client.CollectorRegistry; import io.prometheus.client.dropwizard.DropwizardExports; import io.prometheus.client.exporter.PushGateway; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.SortedMap; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index fd5321a9737ef..3d72a5e84972a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -68,8 +68,8 @@ import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java index 09df62c91fd29..b15d4034cb28e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java @@ -49,8 +49,9 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.metadata.HoodieTableMetadata; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.FileNotFoundException; import java.io.IOException; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java index 40be741025e8f..8551be5f7b198 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java @@ -32,8 +32,8 @@ import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java index acc3cdc6793ab..aef8c9b0a2384 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java @@ -35,8 +35,8 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.FileNotFoundException; import java.io.IOException; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java index 80727ffafb31b..a953380a58af4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java @@ -45,8 +45,8 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieSavepointException; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java index dce6eeac3bd0b..14e42a9e6da12 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java @@ -23,8 +23,8 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.Serializable; import java.util.Map; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java index 273ebce761e26..ca3a84622240f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java @@ -32,8 +32,8 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.Serializable; import java.util.Collections; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java index d3b39f59eda0c..bf7287fdf686c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java @@ -29,8 +29,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.List; import java.util.stream.Collectors; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 16638fd95642b..704b91b0d48a9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -43,8 +43,8 @@ import org.apache.hudi.table.action.BaseActionExecutor; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.nio.charset.StandardCharsets; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java index 3b722a7a41ac9..3b9b92bb3acf7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java @@ -31,8 +31,8 @@ import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.LogManager; import java.io.IOException; import java.util.Collections; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java index cc596ba3422b7..4d4a26653ad72 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java @@ -32,8 +32,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Collections; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java index fa74f7f6e86bc..e303358cc352d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java @@ -27,8 +27,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.ArrayList; import java.util.List; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java index 2e751443abc00..b137b4622c2b6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java @@ -25,8 +25,8 @@ import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.ArrayList; import java.util.List; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java index 4792cd7170459..99b5a9a28d9f9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java @@ -37,8 +37,9 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.MarkerFiles; import org.apache.hudi.table.action.BaseActionExecutor; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java index bfbe577f9429f..671a3aea16b10 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java @@ -20,8 +20,8 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.engine.HoodieEngineContext; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java index de1d973a2fa2c..b14029418686c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java @@ -37,8 +37,8 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.List; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java index c12a4f169b9d0..578738c566906 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java @@ -25,8 +25,8 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; public class SavepointHelpers { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java index 67bc1c20986fd..981b16fd69b3a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java @@ -28,8 +28,8 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Date; diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/callback/http/TestCallbackHttpClient.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/callback/http/TestCallbackHttpClient.java index 616dc31734b79..eaf4e53d20e58 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/callback/http/TestCallbackHttpClient.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/callback/http/TestCallbackHttpClient.java @@ -22,10 +22,13 @@ import org.apache.http.client.methods.CloseableHttpResponse; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.hudi.callback.client.http.HoodieWriteCommitHttpCallbackClient; -import org.apache.log4j.AppenderSkeleton; -import org.apache.log4j.Level; -import org.apache.log4j.Logger; -import org.apache.log4j.spi.LoggingEvent; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.core.Appender; +import org.apache.logging.log4j.core.LogEvent; +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.core.Logger; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.ArgumentCaptor; @@ -49,10 +52,10 @@ public class TestCallbackHttpClient { @Mock - AppenderSkeleton appender; + Appender appender; @Captor - ArgumentCaptor logCaptor; + ArgumentCaptor logCaptor; @Mock CloseableHttpClient httpClient; @@ -63,6 +66,8 @@ public class TestCallbackHttpClient { @Mock StatusLine statusLine; + private Logger logger; + private void mockResponse(int statusCode) { when(statusLine.getStatusCode()).thenReturn(statusCode); when(httpResponse.getStatusLine()).thenReturn(statusLine); @@ -73,23 +78,38 @@ private void mockResponse(int statusCode) { } } + @BeforeEach + public void init() { + when(appender.getName()).thenReturn(HoodieWriteCommitHttpCallbackClient.class.getName()); + when(appender.isStarted()).thenReturn(true); + + logger = (Logger)LogManager.getLogger(HoodieWriteCommitHttpCallbackClient.class); + logger.addAppender(appender); + logger.setLevel(Level.INFO); + } + + @AfterEach + public void tearDown() { + // the appender we added will sit in the singleton logger forever + // slowing future things down - so remove it + logger.removeAppender(appender); + } + @Test public void sendPayloadShouldLogWhenRequestFailed() throws IOException { - Logger.getRootLogger().addAppender(appender); when(httpClient.execute(any())).thenThrow(IOException.class); HoodieWriteCommitHttpCallbackClient hoodieWriteCommitCallBackHttpClient = new HoodieWriteCommitHttpCallbackClient("fake_api_key", "fake_url", httpClient); hoodieWriteCommitCallBackHttpClient.send("{}"); - verify(appender).doAppend(logCaptor.capture()); - assertEquals("Failed to send callback.", logCaptor.getValue().getRenderedMessage()); + verify(appender).append(logCaptor.capture()); + assertEquals("Failed to send callback.", logCaptor.getValue().getMessage().getFormattedMessage()); assertEquals(Level.WARN, logCaptor.getValue().getLevel()); } @Test public void sendPayloadShouldLogUnsuccessfulSending() { - Logger.getRootLogger().addAppender(appender); mockResponse(401); when(httpResponse.toString()).thenReturn("unauthorized"); @@ -97,22 +117,22 @@ public void sendPayloadShouldLogUnsuccessfulSending() { new HoodieWriteCommitHttpCallbackClient("fake_api_key", "fake_url", httpClient); hoodieWriteCommitCallBackHttpClient.send("{}"); - verify(appender).doAppend(logCaptor.capture()); - assertEquals("Failed to send callback message. Response was unauthorized", logCaptor.getValue().getRenderedMessage()); + verify(appender).append(logCaptor.capture()); + assertEquals("Failed to send callback message. Response was unauthorized", + logCaptor.getValue().getMessage().getFormattedMessage()); assertEquals(Level.WARN, logCaptor.getValue().getLevel()); } @Test public void sendPayloadShouldLogSuccessfulSending() { - Logger.getRootLogger().addAppender(appender); mockResponse(202); HoodieWriteCommitHttpCallbackClient hoodieWriteCommitCallBackHttpClient = new HoodieWriteCommitHttpCallbackClient("fake_api_key", "fake_url", httpClient); hoodieWriteCommitCallBackHttpClient.send("{}"); - verify(appender).doAppend(logCaptor.capture()); - assertTrue(logCaptor.getValue().getRenderedMessage().startsWith("Sent Callback data")); + verify(appender).append(logCaptor.capture()); + assertTrue(logCaptor.getValue().getMessage().toString().startsWith("Sent Callback data")); assertEquals(Level.INFO, logCaptor.getValue().getLevel()); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogHttpClient.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogHttpClient.java index 5767d189d35dc..9f8f7d262babf 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogHttpClient.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogHttpClient.java @@ -23,10 +23,13 @@ import org.apache.http.StatusLine; import org.apache.http.client.methods.CloseableHttpResponse; import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.log4j.AppenderSkeleton; -import org.apache.log4j.Level; -import org.apache.log4j.Logger; -import org.apache.log4j.spi.LoggingEvent; +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.core.LogEvent; +import org.apache.logging.log4j.core.Appender; +import org.apache.logging.log4j.core.Logger; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; @@ -53,10 +56,10 @@ public class TestDatadogHttpClient { @Mock - AppenderSkeleton appender; + Appender appender; @Captor - ArgumentCaptor logCaptor; + ArgumentCaptor logCaptor; @Mock CloseableHttpClient httpClient; @@ -67,6 +70,24 @@ public class TestDatadogHttpClient { @Mock StatusLine statusLine; + private Logger logger; + + @BeforeEach + public void init() { + when(appender.getName()).thenReturn("Appender"); + + logger = (Logger)LogManager.getLogger(DatadogHttpClient.class); + logger.addAppender(appender); + logger.setLevel(Level.INFO); + } + + @AfterEach + public void tearDown() { + // the appender we added will sit in the singleton logger forever + // slowing future things down - so remove it + logger.removeAppender(appender); + } + private void mockResponse(int statusCode) { when(statusLine.getStatusCode()).thenReturn(statusCode); when(httpResponse.getStatusLine()).thenReturn(statusLine); @@ -99,41 +120,44 @@ public void validateApiKeyShouldThrowExceptionWhenResponseNotSuccessful() { @Test public void sendPayloadShouldLogWhenRequestFailed() throws IOException { - Logger.getRootLogger().addAppender(appender); when(httpClient.execute(any())).thenThrow(IOException.class); + when(appender.isStarted()).thenReturn(true); DatadogHttpClient ddClient = new DatadogHttpClient(ApiSite.US, "foo", true, httpClient); ddClient.send("{}"); - verify(appender).doAppend(logCaptor.capture()); - assertEquals("Failed to send to Datadog.", logCaptor.getValue().getRenderedMessage()); + verify(appender).append(logCaptor.capture()); + assertEquals("Failed to send to Datadog.", logCaptor.getValue().getMessage().getFormattedMessage()); assertEquals(Level.WARN, logCaptor.getValue().getLevel()); } @Test public void sendPayloadShouldLogUnsuccessfulSending() { - Logger.getRootLogger().addAppender(appender); + when(appender.isStarted()).thenReturn(true); mockResponse(401); when(httpResponse.toString()).thenReturn("unauthorized"); DatadogHttpClient ddClient = new DatadogHttpClient(ApiSite.US, "foo", true, httpClient); ddClient.send("{}"); - verify(appender).doAppend(logCaptor.capture()); - assertEquals("Failed to send to Datadog. Response was unauthorized", logCaptor.getValue().getRenderedMessage()); + verify(appender).append(logCaptor.capture()); + assertEquals("Failed to send to Datadog. Response was unauthorized", + logCaptor.getValue().getMessage().getFormattedMessage()); assertEquals(Level.WARN, logCaptor.getValue().getLevel()); } @Test public void sendPayloadShouldLogSuccessfulSending() { - Logger.getRootLogger().addAppender(appender); + when(appender.isStarted()).thenReturn(true); mockResponse(202); + // change log level: DEBUG + logger.setLevel(Level.DEBUG); DatadogHttpClient ddClient = new DatadogHttpClient(ApiSite.US, "foo", true, httpClient); ddClient.send("{}"); - verify(appender).doAppend(logCaptor.capture()); - assertTrue(logCaptor.getValue().getRenderedMessage().startsWith("Sent metrics data")); + verify(appender).append(logCaptor.capture()); + assertTrue(logCaptor.getValue().getMessage().getFormattedMessage().startsWith("Sent metrics data")); assertEquals(Level.DEBUG, logCaptor.getValue().getLevel()); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogReporter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogReporter.java index 1654e1648b053..14f28942b5816 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogReporter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogReporter.java @@ -24,10 +24,13 @@ import com.codahale.metrics.MetricFilter; import com.codahale.metrics.MetricRegistry; -import org.apache.log4j.AppenderSkeleton; -import org.apache.log4j.Level; -import org.apache.log4j.Logger; -import org.apache.log4j.spi.LoggingEvent; +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.core.Appender; +import org.apache.logging.log4j.core.LogEvent; +import org.apache.logging.log4j.core.Logger; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.ArgumentCaptor; @@ -42,15 +45,16 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) public class TestDatadogReporter { @Mock - AppenderSkeleton appender; + Appender appender; @Captor - ArgumentCaptor logCaptor; + ArgumentCaptor logCaptor; @Mock MetricRegistry registry; @@ -58,6 +62,24 @@ public class TestDatadogReporter { @Mock DatadogHttpClient client; + private Logger logger; + + @BeforeEach + public void init() { + when(appender.getName()).thenReturn("Appender"); + + logger = (Logger)LogManager.getLogger(DatadogReporter.class); + logger.addAppender(appender); + logger.setLevel(Level.INFO); + } + + @AfterEach + public void tearDown() { + // the appender we added will sit in the singleton logger forever + // slowing future things down - so remove it + logger.removeAppender(appender); + } + @Test public void stopShouldCloseEnclosedClient() throws IOException { new DatadogReporter(registry, client, "foo", Option.empty(), Option.empty(), @@ -68,14 +90,15 @@ public void stopShouldCloseEnclosedClient() throws IOException { @Test public void stopShouldLogWhenEnclosedClientFailToClose() throws IOException { - Logger.getRootLogger().addAppender(appender); doThrow(IOException.class).when(client).close(); + when(appender.isStarted()).thenReturn(true); new DatadogReporter(registry, client, "foo", Option.empty(), Option.empty(), MetricFilter.ALL, TimeUnit.SECONDS, TimeUnit.SECONDS).stop(); - verify(appender).doAppend(logCaptor.capture()); - assertEquals("Error disconnecting from Datadog.", logCaptor.getValue().getRenderedMessage()); + verify(appender).append(logCaptor.capture()); + assertEquals("Error disconnecting from Datadog.", + logCaptor.getValue().getMessage().getFormattedMessage()); assertEquals(Level.WARN, logCaptor.getValue().getLevel()); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java index 694d44c10e977..b3c3ff5ca04d6 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java @@ -42,8 +42,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.metadata.CompressionCodecName; diff --git a/hudi-client/hudi-client-common/src/test/resources/log4j-surefire.properties b/hudi-client/hudi-client-common/src/test/resources/log4j-surefire.properties deleted file mode 100644 index 32af462093ae5..0000000000000 --- a/hudi-client/hudi-client-common/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,31 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache=INFO -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR - -# A1 is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-cli/src/test/resources/log4j-surefire-quiet.properties b/hudi-client/hudi-client-common/src/test/resources/log4j2-surefire-quiet.properties similarity index 62% rename from hudi-cli/src/test/resources/log4j-surefire-quiet.properties rename to hudi-client/hudi-client-common/src/test/resources/log4j2-surefire-quiet.properties index b21b5d4070c41..bac1b73e0df6b 100644 --- a/hudi-cli/src/test/resources/log4j-surefire-quiet.properties +++ b/hudi-client/hudi-client-common/src/test/resources/log4j2-surefire-quiet.properties @@ -15,15 +15,20 @@ # See the License for the specific language governing permissions and # limitations under the License. ### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache.hudi=DEBUG +status = warn +name = PropertiesConfig # CONSOLE is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +appender.console.type = Console +appender.console.name = consoleLogger # CONSOLE uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set root level +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/test/resources/log4j2-surefire.properties b/hudi-client/hudi-client-common/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..0a089d7486950 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,51 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set log level for hbase +logger.hbase.name = org.apache.hadoop.hbase +logger.hbase.level = error +logger.hbase.additivity = false +logger.hbase.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-client/hudi-flink-client/pom.xml b/hudi-client/hudi-flink-client/pom.xml index 8be65626dc6e2..cc60b3fa29154 100644 --- a/hudi-client/hudi-flink-client/pom.xml +++ b/hudi-client/hudi-flink-client/pom.xml @@ -102,6 +102,10 @@ javax.xml.bind * + + log4j + log4j + @@ -112,6 +116,16 @@ ${hive.version} test ${hive.exec.classifier} + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + ${hive.groupid} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java index 118a0e729b90d..1a6373210ae94 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java @@ -29,8 +29,8 @@ import org.apache.hudi.index.FlinkHoodieIndex; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.List; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java index 2abefa91f1d91..d554827d4d115 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java @@ -29,8 +29,8 @@ import org.apache.hudi.table.HoodieTable; import org.apache.avro.Schema; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Collections; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java index 43930ad3ea3fe..b1f7486c15f73 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java @@ -30,8 +30,8 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkCleanActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkCleanActionExecutor.java index 6ed38a952a63a..2fab3a4b07623 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkCleanActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkCleanActionExecutor.java @@ -35,8 +35,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.HashMap; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java index 5df46a65bc5ee..03a6b47470ee4 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java @@ -43,8 +43,8 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.nio.charset.StandardCharsets; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java index 562acfd2038b0..91b26700f76cd 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java @@ -36,8 +36,8 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.PathFilter; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkClientTestHarness.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkClientTestHarness.java index 171bab9fb2a31..f1e8ef1859220 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkClientTestHarness.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkClientTestHarness.java @@ -40,8 +40,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalFileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestInfo; diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java index 60ae294e6a927..98277016828ec 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java @@ -35,8 +35,8 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.Arrays; import java.util.List; diff --git a/hudi-client/hudi-flink-client/src/test/resources/log4j-surefire.properties b/hudi-client/hudi-flink-client/src/test/resources/log4j-surefire.properties deleted file mode 100644 index 32af462093ae5..0000000000000 --- a/hudi-client/hudi-flink-client/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,31 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache=INFO -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR - -# A1 is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-flink/src/test/resources/log4j-surefire-quiet.properties b/hudi-client/hudi-flink-client/src/test/resources/log4j2-surefire-quiet.properties similarity index 62% rename from hudi-flink/src/test/resources/log4j-surefire-quiet.properties rename to hudi-client/hudi-flink-client/src/test/resources/log4j2-surefire-quiet.properties index 2b94ea2903067..bac1b73e0df6b 100644 --- a/hudi-flink/src/test/resources/log4j-surefire-quiet.properties +++ b/hudi-client/hudi-flink-client/src/test/resources/log4j2-surefire-quiet.properties @@ -15,16 +15,20 @@ # See the License for the specific language governing permissions and # limitations under the License. ### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR +status = warn +name = PropertiesConfig # CONSOLE is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +appender.console.type = Console +appender.console.name = consoleLogger # CONSOLE uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL \ No newline at end of file +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set root level +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-client/hudi-flink-client/src/test/resources/log4j2-surefire.properties b/hudi-client/hudi-flink-client/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..0a089d7486950 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,51 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set log level for hbase +logger.hbase.name = org.apache.hadoop.hbase +logger.hbase.level = error +logger.hbase.additivity = false +logger.hbase.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/clean/JavaCleanActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/clean/JavaCleanActionExecutor.java index 0ca73d40e84dd..ca54f46f30c69 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/clean/JavaCleanActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/clean/JavaCleanActionExecutor.java @@ -36,8 +36,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.HashMap; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java index 765823e77ca08..341871b0f565e 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java @@ -47,8 +47,8 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.nio.charset.StandardCharsets; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java index 6b5cb2997759c..92ebe91fa6654 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java @@ -36,8 +36,8 @@ import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.table.WorkloadStat; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.ArrayList; import java.util.Collections; diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java index 2b4c5d85c8f15..1476dc8896463 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java @@ -36,8 +36,8 @@ import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestInfo; diff --git a/hudi-client/hudi-java-client/src/test/resources/log4j-surefire.properties b/hudi-client/hudi-java-client/src/test/resources/log4j-surefire.properties deleted file mode 100644 index 32af462093ae5..0000000000000 --- a/hudi-client/hudi-java-client/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,31 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache=INFO -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR - -# A1 is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-client/hudi-client-common/src/test/resources/log4j-surefire-quiet.properties b/hudi-client/hudi-java-client/src/test/resources/log4j2-surefire-quiet.properties similarity index 62% rename from hudi-client/hudi-client-common/src/test/resources/log4j-surefire-quiet.properties rename to hudi-client/hudi-java-client/src/test/resources/log4j2-surefire-quiet.properties index 2b94ea2903067..bac1b73e0df6b 100644 --- a/hudi-client/hudi-client-common/src/test/resources/log4j-surefire-quiet.properties +++ b/hudi-client/hudi-java-client/src/test/resources/log4j2-surefire-quiet.properties @@ -15,16 +15,20 @@ # See the License for the specific language governing permissions and # limitations under the License. ### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR +status = warn +name = PropertiesConfig # CONSOLE is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +appender.console.type = Console +appender.console.name = consoleLogger # CONSOLE uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL \ No newline at end of file +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set root level +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-client/hudi-java-client/src/test/resources/log4j2-surefire.properties b/hudi-client/hudi-java-client/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..0a089d7486950 --- /dev/null +++ b/hudi-client/hudi-java-client/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,51 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set log level for hbase +logger.hbase.name = org.apache.hadoop.hbase +logger.hbase.level = error +logger.hbase.additivity = false +logger.hbase.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml index cc1245bae4b10..496b52b5482e4 100644 --- a/hudi-client/hudi-spark-client/pom.xml +++ b/hudi-client/hudi-spark-client/pom.xml @@ -48,6 +48,12 @@ org.apache.spark spark-core_${scala.binary.version} + + + log4j + log4j + + org.apache.spark @@ -108,6 +114,10 @@ javax.xml.bind * + + log4j + log4j + @@ -118,6 +128,16 @@ ${hive.version} test ${hive.exec.classifier} + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + ${hive.groupid} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java index 60a064ab32bf9..60acfe9264463 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java @@ -25,8 +25,8 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import java.io.IOException; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java index 8a0eee59a5486..a48255ad3d200 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -57,8 +57,8 @@ import org.apache.hudi.table.action.compact.SparkCompactHelpers; import org.apache.hudi.table.upgrade.AbstractUpgradeDowngrade; import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java index ed61c0cc7981a..15d7502c9a09c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java @@ -32,8 +32,8 @@ import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; import org.apache.hudi.table.HoodieSparkMergeOnReadTable; import org.apache.hudi.table.action.cluster.strategy.PartitionAwareClusteringPlanStrategy; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import java.util.ArrayList; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java index 07f9bc14f05d6..819164076b35f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java @@ -34,8 +34,8 @@ import org.apache.hudi.table.HoodieSparkMergeOnReadTable; import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import java.util.Map; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java index 134e490246680..59d4bc3e424cd 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java @@ -24,8 +24,8 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.exception.HoodieClusteringUpdateException; import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import java.util.HashSet; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java index 36710dc02bb9b..482eaa702ebb8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java @@ -21,8 +21,8 @@ import org.apache.hudi.common.util.NumericUtils; import org.apache.hudi.common.util.collection.Pair; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.Partitioner; import java.util.ArrayList; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java index 2dd485ebc6a05..5a67a4c97b0cc 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java @@ -34,8 +34,8 @@ import org.apache.hudi.io.HoodieRangeInfoHandle; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java index c55e5e7c94157..2bb9803d933eb 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java @@ -57,8 +57,8 @@ import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.Partitioner; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java index 611e3c8b93c5b..1211456d563ff 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java @@ -37,8 +37,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.StructType; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java index 7c12a9e001024..8a817ca68f626 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -40,8 +40,8 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hadoop.conf.Configuration; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java index 98063f6cf0ccd..ff915ca76753b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java @@ -63,8 +63,8 @@ import org.apache.hudi.table.action.restore.SparkCopyOnWriteRestoreActionExecutor; import org.apache.hudi.table.action.rollback.SparkCopyOnWriteRollbackActionExecutor; import org.apache.hudi.table.action.savepoint.SavepointActionExecutor; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import java.io.IOException; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index 4f204eabd1830..6e15ebc6648d6 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -76,8 +76,8 @@ import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.parquet.avro.AvroParquetReader; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.avro.AvroSchemaConverter; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java index ba2d42f434861..24cadead0c3a0 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java @@ -34,8 +34,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.PairFlatMapFunction; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java index 1f71aa4dfac7c..a9e0294a0f98a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java @@ -30,8 +30,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.cluster.strategy.ClusteringPlanStrategy; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import java.util.Map; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java index 9a9109afb1d62..d4869394f3b26 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java @@ -54,8 +54,8 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index aa199c2b5008e..7f29098621f11 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -53,8 +53,8 @@ import org.apache.hudi.table.WorkloadStat; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java index 75dfbda30b7fb..56f4845c67ea5 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java @@ -22,8 +22,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.Collections; import java.util.List; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java index 9d60cde69e572..d55b94bd1c8e4 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java @@ -35,8 +35,8 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.table.WorkloadStat; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java index 17c19cebee991..472e336489b14 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java @@ -52,8 +52,8 @@ import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.util.AccumulatorV2; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java index 9c44499a8f43e..cce9e2606306c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java @@ -34,8 +34,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCompactionException; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import java.io.IOException; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java index c92cd928474cc..40fb062b5eeae 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java @@ -34,8 +34,8 @@ import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.Partitioner; import java.io.IOException; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java index fcb3882b7bf5e..bf8d9ebe9b435 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java @@ -38,8 +38,8 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.PathFilter; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaSparkContext; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java index 67d82578fccbf..301b39ace015f 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java @@ -35,8 +35,8 @@ import org.apache.hudi.table.action.compact.OperationResult; import org.apache.hudi.testutils.HoodieClientTestBase; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java index 5c2b6152112cf..7dec3634b5ca2 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java @@ -75,8 +75,8 @@ import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.HoodieClientTestUtils; import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java index 83761c985b040..3fec4a7048931 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java @@ -34,8 +34,8 @@ import org.apache.hudi.testutils.HoodieClientTestUtils; import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java index f34f567b19948..30eea0e3deb31 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java @@ -54,8 +54,8 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Disabled; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index 83bb684294024..bee03f2c0a321 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -78,8 +78,8 @@ import org.apache.hudi.index.SparkHoodieIndex; import org.apache.hudi.table.action.clean.CleanPlanner; import org.apache.hudi.testutils.HoodieClientTestBase; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java index 3e03c05366284..b3f24be9c1a5b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java @@ -49,8 +49,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.FileInputFormat; import org.apache.hadoop.mapred.JobConf; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.hadoop.ParquetReader; import org.apache.spark.TaskContext; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java index f40a97c0bbadc..9640fce34403e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java @@ -37,8 +37,8 @@ import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.avro.Schema; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.junit.jupiter.api.Test; import java.io.IOException; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java index 1386edcfee1aa..069146df16ff0 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java @@ -47,8 +47,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java index 951e17ca36c53..2cbb2baf6d698 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java @@ -42,8 +42,8 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.WorkloadStat; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SQLContext; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java index 55c5aa72fa920..8f2b11ac1861a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java @@ -41,8 +41,8 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileScanner; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java index 8e37c92d34a1d..91709f3c49537 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java @@ -29,8 +29,8 @@ import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.Arrays; import java.util.List; diff --git a/hudi-client/hudi-spark-client/src/test/resources/log4j-surefire.properties b/hudi-client/hudi-spark-client/src/test/resources/log4j-surefire.properties deleted file mode 100644 index 32af462093ae5..0000000000000 --- a/hudi-client/hudi-spark-client/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,31 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache=INFO -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR - -# A1 is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-client/hudi-spark-client/src/test/resources/log4j-surefire-quiet.properties b/hudi-client/hudi-spark-client/src/test/resources/log4j2-surefire-quiet.properties similarity index 62% rename from hudi-client/hudi-spark-client/src/test/resources/log4j-surefire-quiet.properties rename to hudi-client/hudi-spark-client/src/test/resources/log4j2-surefire-quiet.properties index 2b94ea2903067..bac1b73e0df6b 100644 --- a/hudi-client/hudi-spark-client/src/test/resources/log4j-surefire-quiet.properties +++ b/hudi-client/hudi-spark-client/src/test/resources/log4j2-surefire-quiet.properties @@ -15,16 +15,20 @@ # See the License for the specific language governing permissions and # limitations under the License. ### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR +status = warn +name = PropertiesConfig # CONSOLE is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +appender.console.type = Console +appender.console.name = consoleLogger # CONSOLE uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL \ No newline at end of file +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set root level +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/test/resources/log4j2-surefire.properties b/hudi-client/hudi-spark-client/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..0a089d7486950 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,51 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set log level for hbase +logger.hbase.name = org.apache.hadoop.hbase +logger.hbase.level = error +logger.hbase.additivity = false +logger.hbase.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index 6ec0d959ea934..a7bf8a1dca1b4 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -151,17 +151,35 @@ hadoop-common tests test + + + log4j + log4j + + org.apache.hadoop hadoop-hdfs provided + + + log4j + log4j + + org.apache.hadoop hadoop-hdfs tests test + + + log4j + log4j + + @@ -238,7 +256,17 @@ tomcat * + + log4j + log4j + + + + + org.apache.logging.log4j + log4j-core + diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java index d4a77b0822847..1d3e244194544 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java @@ -46,8 +46,8 @@ import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java b/hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java index cd4ade1ef31a2..fb37c350177ad 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java @@ -20,8 +20,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.BufferedReader; import java.io.IOException; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index 9b229a37b799f..7605e798a3822 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -41,8 +41,8 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.InvalidHoodiePathException; import org.apache.hudi.metadata.HoodieTableMetadata; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.File; import java.io.FileNotFoundException; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FailSafeConsistencyGuard.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FailSafeConsistencyGuard.java index 2247b92a4cee0..948329b158657 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FailSafeConsistencyGuard.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FailSafeConsistencyGuard.java @@ -23,8 +23,8 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.FileNotFoundException; import java.io.IOException; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/OptimisticConsistencyGuard.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/OptimisticConsistencyGuard.java index ded7ccce90ebe..a2ef0592cd2b6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/OptimisticConsistencyGuard.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/OptimisticConsistencyGuard.java @@ -20,8 +20,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.List; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java index da72b165f2686..d32702c7105cc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java @@ -28,8 +28,8 @@ import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePartitionMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePartitionMetadata.java index faad46653ad25..ac1dd111f91f2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePartitionMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePartitionMetadata.java @@ -24,8 +24,8 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Properties; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java index 7cc9ee3a0c146..6395e5823a05f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java @@ -23,8 +23,8 @@ import com.fasterxml.jackson.annotation.PropertyAccessor; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.nio.charset.StandardCharsets; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRollingStatMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRollingStatMetadata.java index a354092675e4f..3fc5968156063 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRollingStatMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRollingStatMetadata.java @@ -18,8 +18,8 @@ package org.apache.hudi.common.model; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index 0b36e31ee0774..fb3dde3fcee6b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -32,8 +32,8 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index f4edeb81bc4d2..e36f0ecc3f23e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -43,8 +43,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.File; import java.io.IOException; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java index 9467bfd96c340..6cc6896c95b93 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java @@ -41,8 +41,8 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.InvalidTableException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.format.converter.ParquetMetadataConverter; import org.apache.parquet.hadoop.ParquetFileReader; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java index 50a44ad0f3f2f..0a6d3cb436e53 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java @@ -38,8 +38,8 @@ import org.apache.hudi.common.util.SpillableMapUtils; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayDeque; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java index f0f3842e97b36..766d05d7a4494 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java @@ -42,8 +42,8 @@ import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.EOFException; import java.io.IOException; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java index 9b643ec6e16c8..adf57d4412088 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java @@ -27,8 +27,8 @@ import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.Closeable; import java.io.IOException; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java index 72672278b6b65..7e75266798e3c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java @@ -24,8 +24,8 @@ import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatWriter.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatWriter.java index d7e4f7ef108bc..e30cb71fed0be 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatWriter.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatWriter.java @@ -33,8 +33,8 @@ import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException; import org.apache.hadoop.ipc.RemoteException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Collections; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java index 177be8b919a56..db2884575e932 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java @@ -30,8 +30,8 @@ import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Iterator; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java index 6d2682a4ffa09..9b1d1194ae968 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java @@ -25,8 +25,8 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.HoodieHFileReader; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.avro.Schema; import org.apache.avro.Schema.Field; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index 9ff50fe61fd7c..e45a2b4a2ccac 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -28,8 +28,8 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.FileNotFoundException; import java.io.IOException; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java index 9f8c4393b8df8..b079c4d6198e1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java @@ -31,8 +31,8 @@ import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java index 5659dcbdf7fae..205a34ffc4f91 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java @@ -22,8 +22,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.ArrayList; import java.util.List; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index f9a60654c721d..711299e79e006 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -41,8 +41,8 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java index f89c2a670c9f4..170c983608301 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java @@ -29,8 +29,8 @@ import org.apache.hudi.metadata.HoodieMetadataFileSystemView; import org.apache.hudi.metadata.HoodieTableMetadata; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.concurrent.ConcurrentHashMap; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java index f0c095f598b00..864fcc67d1b6e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java @@ -29,8 +29,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java index 9f9fd1f7ce601..f6aa4f022ee5c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java @@ -44,8 +44,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.List; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java index 3783d00b65d2b..be9b3d64b22f6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java @@ -32,8 +32,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.Serializable; import java.util.List; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java index 23b0536c240dc..d43246a55a8fa 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java @@ -43,8 +43,8 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieRemoteException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java index af0dc130162aa..d8a10b8a1d929 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java @@ -35,8 +35,8 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.collection.RocksDBDAO; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.Serializable; import java.util.HashMap; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java index 1dafe33a8d973..fb56b252b4c8b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java @@ -30,8 +30,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.File; import java.io.IOException; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/BufferedRandomAccessFile.java b/hudi-common/src/main/java/org/apache/hudi/common/util/BufferedRandomAccessFile.java index 4eab1905594ae..5f810dbdc7efa 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/BufferedRandomAccessFile.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/BufferedRandomAccessFile.java @@ -18,7 +18,8 @@ package org.apache.hudi.common.util; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.File; import java.io.FileNotFoundException; @@ -57,7 +58,7 @@ * License: http://www.apache.org/licenses/LICENSE-2.0 */ public final class BufferedRandomAccessFile extends RandomAccessFile { - private static final Logger LOG = Logger.getLogger(BufferedRandomAccessFile.class); + private static final Logger LOG = LogManager.getLogger(BufferedRandomAccessFile.class); static final int DEFAULT_BUFFER_SIZE = (1 << 16); // 64K buffer static final int BUFFER_BOUNDARY_MASK = -DEFAULT_BUFFER_SIZE; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java index 1bf97c68eb69a..294e4664c8cfd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java @@ -35,8 +35,8 @@ import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.AbstractMap; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java index ea36f67396f8c..0d453246ddd6f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java @@ -26,8 +26,8 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.HashMap; import java.util.List; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java index f3e4dc62837c1..ccc87cc753eb2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java @@ -34,8 +34,8 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.HashMap; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java index 78f7b1b9bb105..f3fd375ea7181 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java @@ -23,8 +23,8 @@ import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.avro.Schema; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; /** * Size Estimator for Hoodie record payload. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/RateLimiter.java b/hudi-common/src/main/java/org/apache/hudi/common/util/RateLimiter.java index e156ccffdbb97..38264e714cf94 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/RateLimiter.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/RateLimiter.java @@ -18,8 +18,8 @@ package org.apache.hudi.common.util; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java index bb30b2a1f96e1..06e9fb562fadb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java @@ -21,8 +21,8 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.File; import java.io.IOException; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/TablePathUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/TablePathUtils.java index 6982fdbbdf112..6ef4eaeb1d1e3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/TablePathUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/TablePathUtils.java @@ -23,8 +23,8 @@ import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/DiskBasedMap.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/DiskBasedMap.java index fe4666305d3f6..9584ad45ef02f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/DiskBasedMap.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/DiskBasedMap.java @@ -26,8 +26,8 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieNotSupportedException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.File; import java.io.FileOutputStream; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java index 003d525b66d5e..de2c1fd5797e4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java @@ -22,8 +22,8 @@ import org.apache.hudi.common.util.SizeEstimator; import org.apache.hudi.exception.HoodieIOException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/RocksDBDAO.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/RocksDBDAO.java index 3c08460f2339a..f9c6ecc6fd8fc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/RocksDBDAO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/RocksDBDAO.java @@ -25,8 +25,8 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.rocksdb.AbstractImmutableNativeReference; import org.rocksdb.ColumnFamilyDescriptor; import org.rocksdb.ColumnFamilyHandle; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index cec9ab61a9c1b..d2a83ae326b75 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -23,8 +23,8 @@ import org.apache.hudi.common.util.SizeEstimator; import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.Arrays; import java.util.List; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java index f1ebdcd449dec..b6a50e246ace8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java @@ -24,8 +24,8 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.Iterator; import java.util.concurrent.LinkedBlockingQueue; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedQueueProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedQueueProducer.java index 549683754cbd8..6105fba7c8921 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedQueueProducer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedQueueProducer.java @@ -18,8 +18,8 @@ package org.apache.hudi.common.util.queue; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.function.Function; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueProducer.java index 3d11f38e5cf50..2c19d7acf2c48 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueProducer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueProducer.java @@ -18,8 +18,8 @@ package org.apache.hudi.common.util.queue; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.Iterator; diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java index b4143f336039a..0f38ee936070f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java @@ -37,8 +37,8 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index eeede6f411059..f95a897585547 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -46,8 +46,8 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java index 2bd773bc7819f..9fcff6002169b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java @@ -25,8 +25,8 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.exception.HoodieIOException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 594231225a6d7..5c9ce2475083e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -35,8 +35,8 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/TimelineMergedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/TimelineMergedTableMetadata.java index 9ba3f26079d14..0115fc5b341e1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/TimelineMergedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/TimelineMergedTableMetadata.java @@ -25,8 +25,8 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java index e103427d40728..32f98a7a055cd 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java @@ -56,8 +56,8 @@ import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java index 0bcebaf71e9ff..22058f5c38303 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java @@ -53,8 +53,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.exception.HoodieIOException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index 8017bc3d74860..0f7aee7cb4429 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -45,8 +45,8 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.io.Serializable; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/HdfsTestService.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/HdfsTestService.java index 44af4ecea7c71..56a0d2eb723e9 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/HdfsTestService.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/HdfsTestService.java @@ -26,8 +26,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.File; import java.io.IOException; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java index e5c228f40432b..c2ab48e549eaa 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java @@ -20,8 +20,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.zookeeper.server.NIOServerCnxnFactory; import org.apache.zookeeper.server.ZooKeeperServer; import org.apache.zookeeper.server.persistence.FileTxnLog; diff --git a/hudi-common/src/test/resources/log4j-surefire.properties b/hudi-common/src/test/resources/log4j-surefire.properties deleted file mode 100644 index c5bdf75ae2ae3..0000000000000 --- a/hudi-common/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,31 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache=INFO -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR - -# A1 is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL \ No newline at end of file diff --git a/hudi-common/src/test/resources/log4j2-surefire-quiet.properties b/hudi-common/src/test/resources/log4j2-surefire-quiet.properties new file mode 100644 index 0000000000000..bac1b73e0df6b --- /dev/null +++ b/hudi-common/src/test/resources/log4j2-surefire-quiet.properties @@ -0,0 +1,34 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set root level +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-common/src/test/resources/log4j2-surefire.properties b/hudi-common/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..0a089d7486950 --- /dev/null +++ b/hudi-common/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,51 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set log level for hbase +logger.hbase.name = org.apache.hadoop.hbase +logger.hbase.level = error +logger.hbase.additivity = false +logger.hbase.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-examples/pom.xml b/hudi-examples/pom.xml index 09f258621ec82..951fe816b2d93 100644 --- a/hudi-examples/pom.xml +++ b/hudi-examples/pom.xml @@ -173,6 +173,12 @@ org.apache.spark spark-core_${scala.binary.version} + + + log4j + log4j + + org.apache.spark @@ -205,6 +211,16 @@ ${hive.groupid} hive-common + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + ${hive.groupid} diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java index 4d06e4d15fa09..ed64ba28fe9a6 100644 --- a/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java @@ -35,8 +35,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.ArrayList; import java.util.List; diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java index 257519b468fc5..acd93e697aa00 100644 --- a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java @@ -37,8 +37,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigner.java b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigner.java index d89ad83198ba4..16e4db636d284 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigner.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigner.java @@ -34,8 +34,8 @@ import org.apache.hudi.util.StreamerUtil; import org.apache.flink.util.Preconditions; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.ArrayList; import java.util.Collections; diff --git a/hudi-flink/src/test/resources/log4j-surefire.properties b/hudi-flink/src/test/resources/log4j-surefire.properties deleted file mode 100644 index 8dcd17f303f6b..0000000000000 --- a/hudi-flink/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,31 +0,0 @@ -### -# 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. -### -log4j.rootLogger=INFO, CONSOLE -log4j.logger.org.apache=INFO -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR - -# A1 is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=INFO -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-flink/src/test/resources/log4j2-surefire-quiet.properties b/hudi-flink/src/test/resources/log4j2-surefire-quiet.properties new file mode 100644 index 0000000000000..bac1b73e0df6b --- /dev/null +++ b/hudi-flink/src/test/resources/log4j2-surefire-quiet.properties @@ -0,0 +1,34 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set root level +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-flink/src/test/resources/log4j2-surefire.properties b/hudi-flink/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..0a089d7486950 --- /dev/null +++ b/hudi-flink/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,51 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set log level for hbase +logger.hbase.name = org.apache.hadoop.hbase +logger.hbase.level = error +logger.hbase.additivity = false +logger.hbase.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-hadoop-mr/pom.xml b/hudi-hadoop-mr/pom.xml index 07dabb91d9b13..0bc6bed6ac113 100644 --- a/hudi-hadoop-mr/pom.xml +++ b/hudi-hadoop-mr/pom.xml @@ -72,6 +72,16 @@ ${hive.groupid} hive-exec ${hive.exec.classifier} + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + @@ -88,12 +98,24 @@ hadoop-common tests test + + + log4j + log4j + + org.apache.hadoop hadoop-hdfs tests test + + + log4j + log4j + + diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapColumnStichingRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapColumnStichingRecordReader.java index d3a049ae952e3..e75722ba7af96 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapColumnStichingRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapColumnStichingRecordReader.java @@ -27,8 +27,8 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.hudi.common.util.ValidationUtils; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; /** * Stitches 2 record reader returned rows and presents a concatenated view to clients. diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java index e3bac0b4e8f9a..3365a8fd0d3ed 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java @@ -38,8 +38,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java index 0288cbd4ebc26..68b00602c5f22 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java @@ -45,8 +45,8 @@ import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapreduce.Job; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java index d94018b88546f..d672685774d45 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java @@ -36,8 +36,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.Serializable; import java.util.HashMap; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputPathHandler.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputPathHandler.java index 0a5055a056fa2..974f5b56e34ea 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputPathHandler.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputPathHandler.java @@ -24,8 +24,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RecordReaderValueIterator.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RecordReaderValueIterator.java index 0386186e70f9a..37332ba197c7e 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RecordReaderValueIterator.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RecordReaderValueIterator.java @@ -21,8 +21,8 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hadoop.mapred.RecordReader; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Iterator; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java index 356ae96da3c51..35a951fe75136 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java @@ -63,8 +63,8 @@ import org.apache.hadoop.mapred.lib.CombineFileInputFormat; import org.apache.hadoop.mapred.lib.CombineFileSplit; import org.apache.hadoop.mapreduce.JobContext; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.DataInput; import java.io.DataOutput; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java index f378f44135daf..88d569f564f88 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java @@ -31,8 +31,8 @@ import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.JobConf; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieCombineRealtimeRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieCombineRealtimeRecordReader.java index 615d95ee5f8c0..748541426b745 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieCombineRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieCombineRealtimeRecordReader.java @@ -28,8 +28,8 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.lib.CombineFileSplit; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.LinkedList; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieHFileRealtimeInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieHFileRealtimeInputFormat.java index e75cff6416954..4095f268eddcd 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieHFileRealtimeInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieHFileRealtimeInputFormat.java @@ -38,8 +38,8 @@ import org.apache.hudi.hadoop.UseRecordReaderFromInputFormat; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; /** * HoodieRealtimeInputFormat for HUDI datasets which store data in HFile base file format. diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java index d8f0a01a911ed..1fd0f386f0d2b 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java @@ -37,8 +37,8 @@ import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; import org.apache.hudi.hadoop.UseRecordReaderFromInputFormat; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Arrays; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeRecordReader.java index 1e3a25ac78a11..332d03a5f9ab7 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeRecordReader.java @@ -24,8 +24,8 @@ import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java index a98a230102971..f1ef6a008bbe0 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -34,8 +34,8 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Map; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieHiveUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieHiveUtils.java index dae43602b66a3..4256100a6005d 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieHiveUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieHiveUtils.java @@ -25,8 +25,8 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.exception.HoodieIOException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.ArrayList; import java.util.List; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java index f95ca6f4c486d..4cd833f632e45 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java @@ -53,8 +53,8 @@ import org.apache.hadoop.mapred.FileInputFormat; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.Job; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index ce770bad15b78..f63e215f013f0 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -43,8 +43,8 @@ import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.SplitLocationInfo; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java index 0aa74ef154334..34ee0e04c1cbd 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java @@ -46,8 +46,8 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.JobConf; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/hudi-hadoop-mr/src/test/resources/log4j-surefire-quiet.properties b/hudi-hadoop-mr/src/test/resources/log4j-surefire-quiet.properties deleted file mode 100644 index b21b5d4070c41..0000000000000 --- a/hudi-hadoop-mr/src/test/resources/log4j-surefire-quiet.properties +++ /dev/null @@ -1,29 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache.hudi=DEBUG - -# CONSOLE is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# CONSOLE uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-hadoop-mr/src/test/resources/log4j-surefire.properties b/hudi-hadoop-mr/src/test/resources/log4j-surefire.properties deleted file mode 100644 index c03e808cca1f8..0000000000000 --- a/hudi-hadoop-mr/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,30 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache=INFO -log4j.logger.org.apache.hudi=DEBUG - -# A1 is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL \ No newline at end of file diff --git a/hudi-hadoop-mr/src/test/resources/log4j2-surefire-quiet.properties b/hudi-hadoop-mr/src/test/resources/log4j2-surefire-quiet.properties new file mode 100644 index 0000000000000..74bb4ee54b5c9 --- /dev/null +++ b/hudi-hadoop-mr/src/test/resources/log4j2-surefire-quiet.properties @@ -0,0 +1,40 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set root level +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-hadoop-mr/src/test/resources/log4j2-surefire.properties b/hudi-hadoop-mr/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..9e486359b31e2 --- /dev/null +++ b/hudi-hadoop-mr/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,45 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml index 50c9141fe5fc0..dd33903256fd3 100644 --- a/hudi-integ-test/pom.xml +++ b/hudi-integ-test/pom.xml @@ -158,8 +158,8 @@ - log4j - log4j + org.apache.logging.log4j + log4j-core @@ -297,6 +297,10 @@ javax.servlet * + + log4j + log4j + @@ -331,6 +335,14 @@ javax.servlet servlet-api + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + @@ -355,6 +367,14 @@ org.eclipse.jetty * + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + test diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java index 27760f7116ecd..d867c146a2ab6 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java @@ -31,8 +31,8 @@ import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.schema.SchemaProvider; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/TestSuiteFileBasedSchemaProvider.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/TestSuiteFileBasedSchemaProvider.java index e67c5afae80dc..52981feb35d0e 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/TestSuiteFileBasedSchemaProvider.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/TestSuiteFileBasedSchemaProvider.java @@ -25,8 +25,8 @@ import org.apache.avro.Schema; import org.apache.avro.Schema.Field; import org.apache.avro.Schema.Type; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import java.util.ArrayList; diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java index 80ed1d4bf40f1..ccfd92da74166 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java @@ -32,8 +32,8 @@ import com.github.dockerjava.core.DockerClientConfig; import com.github.dockerjava.core.command.ExecStartResultCallback; import com.github.dockerjava.jaxrs.JerseyDockerCmdExecFactory; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.junit.jupiter.api.BeforeEach; import java.io.ByteArrayOutputStream; diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java index ce32ccff49a63..6c08dd2ff5262 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java @@ -42,8 +42,8 @@ import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; import org.apache.hudi.utilities.sources.AvroDFSSource; import org.apache.hudi.utilities.testutils.UtilitiesTestBase; -import org.apache.log4j.Level; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.core.config.Configurator; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; @@ -116,7 +116,7 @@ public static void initClass() throws Exception { UtilitiesTestBase.Helpers.savePropsToDFS(downstreamProps, dfs, dfsBasePath + "/test-downstream-source.properties"); // these tests cause a lot of log verbosity from spark, turning it down - Logger.getLogger("org.apache.spark").setLevel(Level.WARN); + Configurator.setLevel("org.apache.spark", Level.WARN); } @AfterAll diff --git a/hudi-integ-test/src/test/resources/log4j-surefire-quiet.properties b/hudi-integ-test/src/test/resources/log4j-surefire-quiet.properties deleted file mode 100644 index 61fbf78d1dffb..0000000000000 --- a/hudi-integ-test/src/test/resources/log4j-surefire-quiet.properties +++ /dev/null @@ -1,30 +0,0 @@ -### -# 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. -### -log4j.rootLogger=ERROR, CONSOLE -log4j.logger.org.apache.hudi=ERROR -log4j.category.org.apache.spark=ERROR - -# CONSOLE is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# CONSOLE uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=ERROR -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-integ-test/src/test/resources/log4j-surefire.properties b/hudi-integ-test/src/test/resources/log4j-surefire.properties deleted file mode 100644 index c03e808cca1f8..0000000000000 --- a/hudi-integ-test/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,30 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache=INFO -log4j.logger.org.apache.hudi=DEBUG - -# A1 is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL \ No newline at end of file diff --git a/hudi-integ-test/src/test/resources/log4j2-surefire-quiet.properties b/hudi-integ-test/src/test/resources/log4j2-surefire-quiet.properties new file mode 100644 index 0000000000000..74bb4ee54b5c9 --- /dev/null +++ b/hudi-integ-test/src/test/resources/log4j2-surefire-quiet.properties @@ -0,0 +1,40 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set root level +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-integ-test/src/test/resources/log4j2-surefire.properties b/hudi-integ-test/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..9e486359b31e2 --- /dev/null +++ b/hudi-integ-test/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,45 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java index 632a155fafd0f..e662bdbb6dc31 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -50,8 +50,8 @@ import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.parser.AbstractHoodieDateTimeParser; import org.apache.hudi.table.BulkInsertPartitioner; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java index eb26c4f3209c4..e7d054c8bd353 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java @@ -24,8 +24,8 @@ import org.apache.hudi.io.HoodieRowCreateHandle; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.StructType; diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java index e8041636b8fb9..91cef33922c1e 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java @@ -33,8 +33,8 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.hadoop.conf.Configuration; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 4c76f5f380b88..2c93192409482 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -23,10 +23,9 @@ import org.apache.hudi.common.model.WriteOperationType import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.hive.HiveSyncTool import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor -import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.Config import org.apache.hudi.keygen.{CustomKeyGenerator, SimpleKeyGenerator} import org.apache.hudi.keygen.constant.KeyGeneratorOptions -import org.apache.log4j.LogManager +import org.apache.logging.log4j.LogManager import org.apache.spark.sql.execution.datasources.{DataSourceUtils => SparkDataSourceUtils} /** diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/java/resources/log4j2-surefire-quiet.properties b/hudi-spark-datasource/hudi-spark-common/src/test/java/resources/log4j2-surefire-quiet.properties new file mode 100644 index 0000000000000..74bb4ee54b5c9 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/test/java/resources/log4j2-surefire-quiet.properties @@ -0,0 +1,40 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set root level +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/java/resources/log4j2-surefire.properties b/hudi-spark-datasource/hudi-spark-common/src/test/java/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..9e486359b31e2 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/test/java/resources/log4j2-surefire.properties @@ -0,0 +1,45 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark/pom.xml b/hudi-spark-datasource/hudi-spark/pom.xml index e6e9daf6bf814..60c482424d3e8 100644 --- a/hudi-spark-datasource/hudi-spark/pom.xml +++ b/hudi-spark-datasource/hudi-spark/pom.xml @@ -223,8 +223,8 @@ - log4j - log4j + org.apache.logging.log4j + log4j-core @@ -261,10 +261,14 @@ org.apache.spark spark-core_${scala.binary.version} - - javax.servlet - * - + + javax.servlet + * + + + log4j + log4j + @@ -283,6 +287,12 @@ spark-core_${scala.binary.version} tests test + + + log4j + log4j + + org.apache.spark @@ -322,6 +332,10 @@ javax.servlet.jsp * + + log4j + log4j + provided @@ -341,6 +355,14 @@ org.eclipse.jetty.aggregate * + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + @@ -356,6 +378,14 @@ javax.servlet.jsp * + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + @@ -382,6 +412,14 @@ org.eclipse.jetty.orbit javax.servlet + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java index c820ebef43a5a..7d9000043cbd6 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java @@ -30,8 +30,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.keygen.BuiltinKeyGenerator; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala index 0b8234d7bcd2e..392a71c3a3c2d 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -26,7 +26,7 @@ import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_REA import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.exception.HoodieException import org.apache.hudi.hadoop.HoodieROTablePathFilter -import org.apache.log4j.LogManager +import org.apache.logging.log4j.LogManager import org.apache.spark.sql.execution.datasources.{DataSource, FileStatusCache, HadoopFsRelation} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.streaming.{Sink, Source} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 5b87278713650..53996b9ef23f0 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -40,7 +40,7 @@ import org.apache.hudi.exception.HoodieException import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool} import org.apache.hudi.internal.DataSourceInternalWriterHelper import org.apache.hudi.sync.common.AbstractSyncTool -import org.apache.log4j.LogManager +import org.apache.logging.log4j.LogManager import org.apache.spark.SPARK_VERSION import org.apache.spark.SparkContext import org.apache.spark.api.java.JavaSparkContext diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala index f9a799e6a6dfd..eea33abceca33 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala @@ -28,7 +28,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant.State import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} import org.apache.hudi.common.util.CompactionUtils import org.apache.hudi.exception.HoodieCorruptedDataException -import org.apache.log4j.LogManager +import org.apache.logging.log4j.LogManager import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.execution.streaming.Sink import org.apache.spark.sql.streaming.OutputMode diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala index 5c20656c3ecd5..1214524d88c8d 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala @@ -25,7 +25,7 @@ import org.apache.hudi.exception.HoodieException import org.apache.hadoop.fs.GlobPattern import org.apache.hudi.client.common.HoodieSparkEngineContext import org.apache.hudi.table.HoodieSparkTable -import org.apache.log4j.LogManager +import org.apache.logging.log4j.LogManager import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.sources.{BaseRelation, TableScan} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala index 4c2d332ad4532..4d23c03e55e09 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala @@ -26,7 +26,7 @@ import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.listAffectedFilesForC import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes import org.apache.hadoop.fs.{FileStatus, GlobPattern, Path} import org.apache.hadoop.mapred.JobConf -import org.apache.log4j.LogManager +import org.apache.logging.log4j.LogManager import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.PartitionedFile diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java index 2ee6cae6248b1..dd519e7b960b0 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java @@ -33,8 +33,8 @@ import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.DataFrameWriter; import org.apache.spark.sql.Dataset; diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java index 012134fdb7e6d..435359f0e6a82 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java @@ -31,8 +31,8 @@ import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.DataFrameWriter; import org.apache.spark.sql.Dataset; diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java index 3b55434f3617e..a12d1901c06a7 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java @@ -34,8 +34,8 @@ import com.beust.jcommander.Parameter; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/log4j-surefire-quiet.properties b/hudi-spark-datasource/hudi-spark/src/test/resources/log4j-surefire-quiet.properties deleted file mode 100644 index ca0a50c84270c..0000000000000 --- a/hudi-spark-datasource/hudi-spark/src/test/resources/log4j-surefire-quiet.properties +++ /dev/null @@ -1,30 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR - -# CONSOLE is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# CONSOLE uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/log4j-surefire.properties b/hudi-spark-datasource/hudi-spark/src/test/resources/log4j-surefire.properties deleted file mode 100644 index 32af462093ae5..0000000000000 --- a/hudi-spark-datasource/hudi-spark/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,31 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache=INFO -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR - -# A1 is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/log4j2-surefire-quiet.properties b/hudi-spark-datasource/hudi-spark/src/test/resources/log4j2-surefire-quiet.properties new file mode 100644 index 0000000000000..bac1b73e0df6b --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/log4j2-surefire-quiet.properties @@ -0,0 +1,34 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set root level +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/log4j2-surefire.properties b/hudi-spark-datasource/hudi-spark/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..0a089d7486950 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,51 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set log level for hbase +logger.hbase.name = org.apache.hadoop.hbase +logger.hbase.level = error +logger.hbase.additivity = false +logger.hbase.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala index 00c40abf1c3c7..b1c50a15a3874 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala @@ -27,7 +27,7 @@ import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDat import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.keygen.NonpartitionedKeyGenerator import org.apache.hudi.testutils.HoodieClientTestBase -import org.apache.log4j.LogManager +import org.apache.logging.log4j.LogManager import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala index 49dff4d0d544b..1a2bcd2e33826 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala @@ -26,7 +26,7 @@ import org.apache.hudi.config.{HoodieClusteringConfig, HoodieStorageConfig, Hood import org.apache.hudi.exception.TableNotFoundException import org.apache.hudi.testutils.HoodieClientTestBase import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} -import org.apache.log4j.LogManager +import org.apache.logging.log4j.LogManager import org.apache.spark.sql._ import org.apache.spark.sql.streaming.{OutputMode, Trigger} import org.apache.spark.sql.types.StructType diff --git a/hudi-spark-datasource/hudi-spark2/src/test/resources/log4j2-surefire-quiet.properties b/hudi-spark-datasource/hudi-spark2/src/test/resources/log4j2-surefire-quiet.properties new file mode 100644 index 0000000000000..74bb4ee54b5c9 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark2/src/test/resources/log4j2-surefire-quiet.properties @@ -0,0 +1,40 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set root level +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark2/src/test/resources/log4j2-surefire.properties b/hudi-spark-datasource/hudi-spark2/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..9e486359b31e2 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark2/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,45 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark3/src/test/resources/log4j2-surefire-quiet.properties b/hudi-spark-datasource/hudi-spark3/src/test/resources/log4j2-surefire-quiet.properties new file mode 100644 index 0000000000000..74bb4ee54b5c9 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/test/resources/log4j2-surefire-quiet.properties @@ -0,0 +1,40 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set root level +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark3/src/test/resources/log4j2-surefire.properties b/hudi-spark-datasource/hudi-spark3/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..9e486359b31e2 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,45 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-sync/hudi-dla-sync/pom.xml b/hudi-sync/hudi-dla-sync/pom.xml index 769d12fa880a5..5fe69282d918e 100644 --- a/hudi-sync/hudi-dla-sync/pom.xml +++ b/hudi-sync/hudi-dla-sync/pom.xml @@ -98,8 +98,8 @@ - log4j - log4j + org.apache.logging.log4j + log4j-core @@ -111,6 +111,12 @@ org.apache.hadoop hadoop-common + + + log4j + log4j + + org.apache.hive diff --git a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java index 140938f7046a1..700224008ff82 100644 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java +++ b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java @@ -34,8 +34,8 @@ import org.apache.hudi.hive.util.HiveSchemaUtil; import org.apache.hudi.sync.common.AbstractSyncHoodieClient; import org.apache.hudi.sync.common.AbstractSyncTool; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.parquet.schema.MessageType; import java.util.List; diff --git a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java index 02c07d6e5861f..f7ebc50cfbce5 100644 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java +++ b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java @@ -31,8 +31,8 @@ import org.apache.hudi.hive.SchemaDifference; import org.apache.hudi.hive.util.HiveSchemaUtil; import org.apache.hudi.sync.common.AbstractSyncHoodieClient; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.parquet.schema.MessageType; import java.io.IOException; diff --git a/hudi-sync/hudi-dla-sync/src/test/resources/log4j-surefire-quiet.properties b/hudi-sync/hudi-dla-sync/src/test/resources/log4j-surefire-quiet.properties deleted file mode 100644 index b21b5d4070c41..0000000000000 --- a/hudi-sync/hudi-dla-sync/src/test/resources/log4j-surefire-quiet.properties +++ /dev/null @@ -1,29 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache.hudi=DEBUG - -# CONSOLE is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# CONSOLE uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-sync/hudi-dla-sync/src/test/resources/log4j-surefire.properties b/hudi-sync/hudi-dla-sync/src/test/resources/log4j-surefire.properties deleted file mode 100644 index c03e808cca1f8..0000000000000 --- a/hudi-sync/hudi-dla-sync/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,30 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache=INFO -log4j.logger.org.apache.hudi=DEBUG - -# A1 is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL \ No newline at end of file diff --git a/hudi-sync/hudi-dla-sync/src/test/resources/log4j2-surefire-quiet.properties b/hudi-sync/hudi-dla-sync/src/test/resources/log4j2-surefire-quiet.properties new file mode 100644 index 0000000000000..a25ec2ecbbefd --- /dev/null +++ b/hudi-sync/hudi-dla-sync/src/test/resources/log4j2-surefire-quiet.properties @@ -0,0 +1,39 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-sync/hudi-dla-sync/src/test/resources/log4j2-surefire.properties b/hudi-sync/hudi-dla-sync/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..0a089d7486950 --- /dev/null +++ b/hudi-sync/hudi-dla-sync/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,51 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set log level for hbase +logger.hbase.name = org.apache.hadoop.hbase +logger.hbase.level = error +logger.hbase.additivity = false +logger.hbase.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-sync/hudi-hive-sync/pom.xml b/hudi-sync/hudi-hive-sync/pom.xml index 44e5f9eeedcdc..2faadb8e42a84 100644 --- a/hudi-sync/hudi-hive-sync/pom.xml +++ b/hudi-sync/hudi-hive-sync/pom.xml @@ -53,8 +53,8 @@ - log4j - log4j + org.apache.logging.log4j + log4j-core @@ -76,6 +76,12 @@ org.apache.hadoop hadoop-common + + + log4j + log4j + + org.apache.hadoop @@ -84,20 +90,44 @@ org.apache.hadoop hadoop-hdfs + + + log4j + log4j + + org.apache.hadoop hadoop-auth + + + log4j + log4j + + org.apache.hadoop hadoop-common tests + + + log4j + log4j + + org.apache.hadoop hadoop-hdfs tests + + + log4j + log4j + + @@ -120,6 +150,16 @@ ${hive.groupid} hive-jdbc ${hive.version} + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + ${hive.groupid} @@ -130,6 +170,16 @@ ${hive.groupid} hive-common ${hive.version} + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java index bbda97efd10aa..1eb1652cf1ee1 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java @@ -34,8 +34,8 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hudi.sync.common.AbstractSyncTool; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.parquet.schema.MessageType; import java.util.ArrayList; diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java index b6211671b5119..a31293c9b32f5 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java @@ -42,8 +42,8 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hudi.sync.common.AbstractSyncHoodieClient; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.parquet.schema.MessageType; import org.apache.thrift.TException; diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java index 6a209bef7c3ec..64a78d211ce37 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java @@ -22,8 +22,8 @@ import org.apache.hudi.hive.HoodieHiveSyncException; import org.apache.hudi.hive.SchemaDifference; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.parquet.schema.DecimalMetadata; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java index ca7aa7a02ce49..bfc4e94296ef9 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java @@ -33,8 +33,8 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.thrift.TUGIContainingTransport; import org.apache.hive.service.server.HiveServer2; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.thrift.TProcessor; import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.server.TServer; diff --git a/hudi-sync/hudi-hive-sync/src/test/resources/log4j-surefire-quiet.properties b/hudi-sync/hudi-hive-sync/src/test/resources/log4j-surefire-quiet.properties deleted file mode 100644 index b21b5d4070c41..0000000000000 --- a/hudi-sync/hudi-hive-sync/src/test/resources/log4j-surefire-quiet.properties +++ /dev/null @@ -1,29 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache.hudi=DEBUG - -# CONSOLE is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# CONSOLE uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-sync/hudi-hive-sync/src/test/resources/log4j-surefire.properties b/hudi-sync/hudi-hive-sync/src/test/resources/log4j-surefire.properties deleted file mode 100644 index c03e808cca1f8..0000000000000 --- a/hudi-sync/hudi-hive-sync/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,30 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache=INFO -log4j.logger.org.apache.hudi=DEBUG - -# A1 is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL \ No newline at end of file diff --git a/hudi-sync/hudi-hive-sync/src/test/resources/log4j2-surefire-quiet.properties b/hudi-sync/hudi-hive-sync/src/test/resources/log4j2-surefire-quiet.properties new file mode 100644 index 0000000000000..a25ec2ecbbefd --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/test/resources/log4j2-surefire-quiet.properties @@ -0,0 +1,39 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-sync/hudi-hive-sync/src/test/resources/log4j2-surefire.properties b/hudi-sync/hudi-hive-sync/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..0a089d7486950 --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,51 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set log level for hbase +logger.hbase.name = org.apache.hadoop.hbase +logger.hbase.level = error +logger.hbase.additivity = false +logger.hbase.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-sync/hudi-sync-common/pom.xml b/hudi-sync/hudi-sync-common/pom.xml index a17739f7b0229..21c727bedbcf5 100644 --- a/hudi-sync/hudi-sync-common/pom.xml +++ b/hudi-sync/hudi-sync-common/pom.xml @@ -44,6 +44,12 @@ org.apache.hadoop hadoop-common + + + log4j + log4j + + diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java index 8477ed6cec222..eb4ba90b62867 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java @@ -29,8 +29,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.parquet.schema.MessageType; import java.sql.ResultSet; diff --git a/hudi-timeline-service/pom.xml b/hudi-timeline-service/pom.xml index e19871bbc8876..01296da7773e5 100644 --- a/hudi-timeline-service/pom.xml +++ b/hudi-timeline-service/pom.xml @@ -77,8 +77,8 @@ - log4j - log4j + org.apache.logging.log4j + log4j-core @@ -156,6 +156,10 @@ javax.servlet * + + log4j + log4j + diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java index 09ebeb5cce3b1..952aa875bf013 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java @@ -43,8 +43,8 @@ import io.javalin.Handler; import io.javalin.Javalin; import org.apache.hadoop.conf.Configuration; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.jetbrains.annotations.NotNull; import java.io.IOException; diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java index 3cae8896613b9..86ef0c1728b71 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java @@ -33,8 +33,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.util.thread.QueuedThreadPool; diff --git a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/functional/TestRemoteHoodieTableFileSystemView.java b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/functional/TestRemoteHoodieTableFileSystemView.java index 0865585deff21..9ef83fea793d8 100644 --- a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/functional/TestRemoteHoodieTableFileSystemView.java +++ b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/functional/TestRemoteHoodieTableFileSystemView.java @@ -29,8 +29,8 @@ import org.apache.hudi.common.table.view.TestHoodieTableFileSystemView; import org.apache.hudi.timeline.service.TimelineService; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; /** * Bring up a remote Timeline Server and run all test-cases of TestHoodieTableFileSystemView against it. diff --git a/hudi-timeline-service/src/test/resources/log4j-surefire-quiet.properties b/hudi-timeline-service/src/test/resources/log4j-surefire-quiet.properties deleted file mode 100644 index b21b5d4070c41..0000000000000 --- a/hudi-timeline-service/src/test/resources/log4j-surefire-quiet.properties +++ /dev/null @@ -1,29 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache.hudi=DEBUG - -# CONSOLE is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# CONSOLE uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-timeline-service/src/test/resources/log4j-surefire.properties b/hudi-timeline-service/src/test/resources/log4j-surefire.properties deleted file mode 100644 index c03e808cca1f8..0000000000000 --- a/hudi-timeline-service/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,30 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache=INFO -log4j.logger.org.apache.hudi=DEBUG - -# A1 is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL \ No newline at end of file diff --git a/hudi-timeline-service/src/test/resources/log4j2-surefire-quiet.properties b/hudi-timeline-service/src/test/resources/log4j2-surefire-quiet.properties new file mode 100644 index 0000000000000..a25ec2ecbbefd --- /dev/null +++ b/hudi-timeline-service/src/test/resources/log4j2-surefire-quiet.properties @@ -0,0 +1,39 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-timeline-service/src/test/resources/log4j2-surefire.properties b/hudi-timeline-service/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..0a089d7486950 --- /dev/null +++ b/hudi-timeline-service/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,51 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set log level for hbase +logger.hbase.name = org.apache.hadoop.hbase +logger.hbase.level = error +logger.hbase.additivity = false +logger.hbase.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml index 70878e792e738..bbc227d3617dd 100644 --- a/hudi-utilities/pom.xml +++ b/hudi-utilities/pom.xml @@ -154,8 +154,8 @@ - log4j - log4j + org.apache.logging.log4j + log4j-core @@ -200,6 +200,10 @@ org.slf4j slf4j-api + + log4j + log4j + @@ -335,6 +339,10 @@ javax.servlet * + + log4j + log4j + @@ -356,6 +364,14 @@ org.eclipse.jetty.orbit javax.servlet + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + @@ -413,6 +429,16 @@ ${hive.version} test ${hive.exec.classifier} + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java index 1864795411c9b..f32966c80103f 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java @@ -42,8 +42,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.hadoop.ParquetInputFormat; import org.apache.spark.api.java.JavaRDD; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java index a3570b1f75dfe..ccb84967e6bc3 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java @@ -33,8 +33,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.stringtemplate.v4.ST; import java.io.File; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java index 24e2828a5e2f5..5f54822f76cda 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java @@ -28,8 +28,8 @@ import com.beust.jcommander.Parameter; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import java.io.Serializable; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java index 44328d3d65975..e3586917a9902 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java @@ -31,8 +31,8 @@ import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.jetbrains.annotations.TestOnly; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java index c1493e6a4940f..12bfd2500be23 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java @@ -28,8 +28,8 @@ import com.beust.jcommander.Parameter; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java index 6378318b96ded..8c06a264eab30 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java @@ -39,8 +39,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java index ee31d4a600a22..8db3b25d48377 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java @@ -45,8 +45,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Column; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java index 3904765036f0c..8fb63e09020e0 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java @@ -51,8 +51,8 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/kafka/HoodieWriteCommitKafkaCallback.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/kafka/HoodieWriteCommitKafkaCallback.java index 9133e502d951d..a1808c7b413db 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/kafka/HoodieWriteCommitKafkaCallback.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/kafka/HoodieWriteCommitKafkaCallback.java @@ -29,8 +29,8 @@ import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.Properties; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java index bcf40259ffc6e..dcf879e036e6f 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java @@ -41,8 +41,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 01a374d7ed8f7..916de5a761fd0 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -66,8 +66,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java index 6e3a024d0816e..d5c92949daee0 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -55,8 +55,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java index be2fe54256a08..01df2e5a1a7a4 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java @@ -37,8 +37,8 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; import org.apache.hudi.utilities.sources.JsonDFSSource; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SchedulerConfGenerator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SchedulerConfGenerator.java index f5f1f384765ab..404bd73d94156 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SchedulerConfGenerator.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SchedulerConfGenerator.java @@ -22,8 +22,8 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.util.Option; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.SparkConf; import java.io.BufferedWriter; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java index 184d7c7e4586f..c73ab4a4034cc 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java @@ -40,8 +40,8 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java index 511a72c280e8a..f7fa6f13d8a80 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java @@ -31,8 +31,8 @@ import io.confluent.kafka.serializers.KafkaAvroDeserializer; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HiveIncrPullSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HiveIncrPullSource.java index 58531908258c9..a1b134c161a9e 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HiveIncrPullSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HiveIncrPullSource.java @@ -33,8 +33,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.NullWritable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java index 2842c1dd50995..3986fdc799072 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java @@ -28,8 +28,8 @@ import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.sources.helpers.IncrSourceHelper; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.DataFrameReader; import org.apache.spark.sql.Dataset; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java index cedaba48c2e3d..ae4ed7fc4bc7f 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java @@ -26,8 +26,8 @@ import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java index d9d3444bf0e97..5ec6a310e91ae 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java @@ -32,8 +32,8 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DatePartitionPathSelector.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DatePartitionPathSelector.java index 97106ded9bcb8..60eb202386d8a 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DatePartitionPathSelector.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DatePartitionPathSelector.java @@ -30,8 +30,8 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import java.time.LocalDate; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java index 789daf1885578..b0b528c91e9bf 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java @@ -29,8 +29,8 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.streaming.kafka010.OffsetRange; import java.util.Arrays; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/FlatteningTransformer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/FlatteningTransformer.java index a85fb1a5fc575..93a8b6fa4200b 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/FlatteningTransformer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/FlatteningTransformer.java @@ -20,8 +20,8 @@ import org.apache.hudi.common.config.TypedProperties; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/SqlQueryBasedTransformer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/SqlQueryBasedTransformer.java index 9429c89d0ebf5..634699853f64a 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/SqlQueryBasedTransformer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/SqlQueryBasedTransformer.java @@ -20,8 +20,8 @@ import org.apache.hudi.common.config.TypedProperties; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 8d837fd0972be..a0d3b9c8c5425 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -66,8 +66,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.kafka.common.errors.TopicExistsException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.AnalysisException; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieMultiTableDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieMultiTableDeltaStreamer.java index 7b5ce9d74a566..bd433c6626008 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieMultiTableDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieMultiTableDeltaStreamer.java @@ -30,8 +30,8 @@ import org.apache.hudi.utilities.sources.TestDataSource; import org.apache.hudi.utilities.testutils.UtilitiesTestBase; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java index aefa49fa90922..59cb89ec4f08c 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java @@ -38,8 +38,8 @@ import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Column; import org.apache.spark.sql.DataFrameWriter; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestJdbcbasedSchemaProvider.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestJdbcbasedSchemaProvider.java index fde26f5376343..f7e3ceda9ee9e 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestJdbcbasedSchemaProvider.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestJdbcbasedSchemaProvider.java @@ -26,8 +26,8 @@ import org.apache.hudi.utilities.testutils.UtilitiesTestBase; import org.apache.avro.Schema; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestDataSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestDataSource.java index 1806d5c48b06d..9c82cdeae078b 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestDataSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestDataSource.java @@ -24,8 +24,8 @@ import org.apache.hudi.utilities.testutils.sources.AbstractBaseTestSource; import org.apache.avro.generic.GenericRecord; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java index 0adef5210cf61..709fba1294aa0 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java @@ -56,8 +56,8 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hive.service.server.HiveServer2; -import org.apache.log4j.Level; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.core.config.Configurator; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetFileWriter.Mode; import org.apache.parquet.hadoop.ParquetWriter; @@ -99,9 +99,8 @@ public class UtilitiesTestBase { @BeforeAll public static void initClass() throws Exception { // Set log level to WARN for spark logs to avoid exceeding log limit in travis - Logger rootLogger = Logger.getRootLogger(); - rootLogger.setLevel(Level.ERROR); - Logger.getLogger("org.apache.spark").setLevel(Level.WARN); + Configurator.setRootLevel(Level.ERROR); + Configurator.setLevel("org.apache.spark", Level.WARN); initClass(true); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java index 524591dd7bca5..36cc2cdb0ab9d 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java @@ -29,8 +29,8 @@ import org.apache.hudi.utilities.testutils.sources.config.SourceConfigs; import org.apache.avro.generic.GenericRecord; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/DistributedTestDataSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/DistributedTestDataSource.java index 0ccddd582a06f..9450f62959c8c 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/DistributedTestDataSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/DistributedTestDataSource.java @@ -25,8 +25,8 @@ import org.apache.hudi.utilities.testutils.sources.config.SourceConfigs; import org.apache.avro.generic.GenericRecord; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; diff --git a/hudi-utilities/src/test/resources/log4j-surefire-quiet.properties b/hudi-utilities/src/test/resources/log4j-surefire-quiet.properties deleted file mode 100644 index ca0a50c84270c..0000000000000 --- a/hudi-utilities/src/test/resources/log4j-surefire-quiet.properties +++ /dev/null @@ -1,30 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR - -# CONSOLE is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# CONSOLE uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-utilities/src/test/resources/log4j-surefire.properties b/hudi-utilities/src/test/resources/log4j-surefire.properties deleted file mode 100644 index c5bdf75ae2ae3..0000000000000 --- a/hudi-utilities/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,31 +0,0 @@ -### -# 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. -### -log4j.rootLogger=WARN, CONSOLE -log4j.logger.org.apache=INFO -log4j.logger.org.apache.hudi=DEBUG -log4j.logger.org.apache.hadoop.hbase=ERROR - -# A1 is set to be a ConsoleAppender. -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -# A1 uses PatternLayout. -log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n -log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter -log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true -log4j.appender.CONSOLE.filter.a.LevelMin=WARN -log4j.appender.CONSOLE.filter.a.LevelMax=FATAL \ No newline at end of file diff --git a/hudi-utilities/src/test/resources/log4j2-surefire-quiet.properties b/hudi-utilities/src/test/resources/log4j2-surefire-quiet.properties new file mode 100644 index 0000000000000..63d88ff478e1c --- /dev/null +++ b/hudi-utilities/src/test/resources/log4j2-surefire-quiet.properties @@ -0,0 +1,45 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5p] %d %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set log level for hbase +logger.hbase.name = org.apache.hadoop.hbase +logger.hbase.level = error +logger.hbase.additivity = false +logger.hbase.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/hudi-utilities/src/test/resources/log4j2-surefire.properties b/hudi-utilities/src/test/resources/log4j2-surefire.properties new file mode 100644 index 0000000000000..0a089d7486950 --- /dev/null +++ b/hudi-utilities/src/test/resources/log4j2-surefire.properties @@ -0,0 +1,51 @@ +### +# 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. +### +status = warn +name = PropertiesConfig + +# CONSOLE is set to be a ConsoleAppender. +appender.console.type = Console +appender.console.name = consoleLogger +# CONSOLE uses PatternLayout. +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +appender.console.filter.threshold.type = LevelRangeFilter +appender.console.filter.threshold.minLevel = warn +appender.console.filter.threshold.maxLevel = fatal +appender.console.filter.threshold.onMatch = accept + +# set log level for apache +logger.apache.name = org.apache +logger.apache.level = info +logger.apache.additivity = false +logger.apache.appenderRef.console.ref = consoleLogger + +# set log level for hudi +logger.hudi.name = org.apache.hudi +logger.hudi.level = debug +logger.hudi.additivity = false +logger.hudi.appenderRef.console.ref = consoleLogger + +# set log level for hbase +logger.hbase.name = org.apache.hadoop.hbase +logger.hbase.level = error +logger.hbase.additivity = false +logger.hbase.appenderRef.console.ref = consoleLogger + +rootLogger.level = warn +rootLogger.appenderRef.console.ref = consoleLogger \ No newline at end of file diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index 966df9dfaf10d..a1887340ae182 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -134,6 +134,9 @@ org.apache.hbase:hbase-common commons-codec:commons-codec + + org.apache.logging.log4j:log4j-core + org.apache.logging.log4j:log4j-api @@ -460,6 +463,12 @@ org.apache.hbase hbase-common ${hbase.version} + + + log4j + log4j + + diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index 5d02bcc8e7d36..e45db6c6c40ed 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -79,6 +79,9 @@ org.apache.htrace:htrace-core com.yammer.metrics:metrics-core com.google.guava:guava + + org.apache.logging.log4j:log4j-core + org.apache.logging.log4j:log4j-api @@ -190,6 +193,10 @@ tomcat * + + log4j + log4j + diff --git a/packaging/hudi-hive-sync-bundle/pom.xml b/packaging/hudi-hive-sync-bundle/pom.xml index a8dbc1e915172..a940eeb9e9af1 100644 --- a/packaging/hudi-hive-sync-bundle/pom.xml +++ b/packaging/hudi-hive-sync-bundle/pom.xml @@ -75,6 +75,9 @@ com.esotericsoftware:kryo-shaded org.objenesis:objenesis com.esotericsoftware:minlog + + org.apache.logging.log4j:log4j-core + org.apache.logging.log4j:log4j-api diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index 60d9cc61ab996..6f38c63fbf5bd 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -156,6 +156,9 @@ org.apache.curator:curator-framework org.apache.curator:curator-client org.apache.curator:curator-recipes + + org.apache.logging.log4j:log4j-core + org.apache.logging.log4j:log4j-api @@ -360,6 +363,12 @@ org.apache.hadoop hadoop-hdfs tests + + + log4j + log4j + + org.apache.hadoop @@ -378,6 +387,10 @@ javax.servlet * + + log4j + log4j + @@ -386,6 +399,16 @@ hive-exec ${hive.version} compile + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + @@ -409,6 +432,14 @@ javax.servlet servlet-api + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + @@ -425,6 +456,14 @@ javax.servlet servlet-api + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + @@ -433,6 +472,16 @@ hive-common ${hive.version} compile + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + @@ -512,8 +561,8 @@ - log4j - log4j + org.apache.logging.log4j + log4j-core org.slf4j @@ -540,6 +589,12 @@ org.apache.spark spark-core_${scala.binary.version} + + + log4j + log4j + + diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index a40e414066e12..6f6cef260835d 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -82,6 +82,9 @@ com.google.guava:guava commons-lang:commons-lang com.google.protobuf:protobuf-java + + org.apache.logging.log4j:log4j-core + org.apache.logging.log4j:log4j-api diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index 0ee483d2224db..3f2f5e6c0303b 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -116,6 +116,9 @@ org.apache.curator:curator-client org.apache.curator:curator-recipes commons-codec:commons-codec + + org.apache.logging.log4j:log4j-core + org.apache.logging.log4j:log4j-api @@ -291,6 +294,16 @@ hive-jdbc ${hive.version} ${spark.bundle.hive.scope} + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + @@ -305,6 +318,16 @@ hive-common ${hive.version} ${spark.bundle.hive.scope} + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + @@ -319,6 +342,12 @@ org.apache.hbase hbase-common ${hbase.version} + + + log4j + log4j + + org.apache.hbase @@ -342,6 +371,10 @@ tomcat * + + log4j + log4j + diff --git a/packaging/hudi-timeline-server-bundle/pom.xml b/packaging/hudi-timeline-server-bundle/pom.xml index 53173b9726352..d49ff3578c15c 100644 --- a/packaging/hudi-timeline-server-bundle/pom.xml +++ b/packaging/hudi-timeline-server-bundle/pom.xml @@ -44,8 +44,8 @@ - log4j - log4j + org.apache.logging.log4j + log4j-core @@ -102,6 +102,10 @@ javax.servlet * + + log4j + log4j + @@ -120,6 +124,10 @@ javax.servlet * + + log4j + log4j + diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 7a5f63ec7daa0..048f49295e270 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -127,6 +127,9 @@ org.apache.curator:curator-client org.apache.curator:curator-recipes commons-codec:commons-codec + + org.apache.logging.log4j:log4j-core + org.apache.logging.log4j:log4j-api @@ -287,6 +290,16 @@ hive-jdbc ${hive.version} ${utilities.bundle.hive.scope} + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + @@ -301,6 +314,16 @@ hive-common ${hive.version} ${utilities.bundle.hive.scope} + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + @@ -315,6 +338,12 @@ org.apache.hbase hbase-common ${hbase.version} + + + log4j + log4j + + org.apache.hbase @@ -338,6 +367,10 @@ tomcat * + + log4j + log4j + diff --git a/pom.xml b/pom.xml index 7f371ddf27caf..127fd70d274fd 100644 --- a/pom.xml +++ b/pom.xml @@ -95,7 +95,7 @@ 5.7.0-M1 1.7.0-M1 3.3.3 - 1.2.17 + 2.11.0 1.7.15 2.9.9 2.7.3 @@ -120,7 +120,7 @@ 0.12 3.3.1 3.0.1 - file://${project.basedir}/src/test/resources/log4j-surefire.properties + ${project.basedir}/src/test/resources/log4j2-surefire.properties 0.12.0 9.4.15.v20190215 3.1.0-incubating @@ -301,9 +301,9 @@ @{argLine} - - ${surefire-log4j.file} - + + ${surefire-log4j2.file} + @@ -455,8 +455,13 @@ - log4j - log4j + org.apache.logging.log4j + log4j-core + ${log4j.version} + + + org.apache.logging.log4j + log4j-1.2-api ${log4j.version} @@ -526,6 +531,12 @@ spark-core_${scala.binary.version} ${spark.version} provided + + + log4j + log4j + + org.apache.spark @@ -546,6 +557,12 @@ tests ${spark.version} test + + + log4j + log4j + + org.apache.spark @@ -721,6 +738,10 @@ javax.xml.bind jaxb-api + + log4j + log4j + @@ -728,12 +749,24 @@ hadoop-hdfs ${hadoop.version} provided + + + log4j + log4j + + org.apache.hadoop hadoop-auth ${hadoop.version} provided + + + log4j + log4j + + org.apache.hadoop @@ -764,6 +797,12 @@ hadoop-hdfs tests ${hadoop.version} + + + log4j + log4j + + org.apache.hadoop @@ -779,6 +818,10 @@ javax.xml.bind jaxb-api + + log4j + log4j + @@ -833,6 +876,14 @@ org.eclipse.jetty.aggregate * + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + @@ -885,6 +936,14 @@ org.eclipse.jetty.aggregate * + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + @@ -902,6 +961,14 @@ org.eclipse.jetty.aggregate * + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + @@ -1098,8 +1165,8 @@ - file://${project.basedir}/src/test/resources/log4j-surefire-quiet.properties - + ${project.basedir}/src/test/resources/log4j2-surefire-quiet.properties + From 77d3697ca05e1845483d1b16a64dba0301ef0080 Mon Sep 17 00:00:00 2001 From: hongdd Date: Tue, 6 Apr 2021 17:56:06 +0800 Subject: [PATCH 2/3] Remove all dependency of log4j:log4j --- hudi-cli/pom.xml | 4 ++ hudi-client/hudi-flink-client/pom.xml | 4 ++ hudi-client/hudi-java-client/pom.xml | 12 ++++++ hudi-common/pom.xml | 30 ++++++++++++++- .../common/table/timeline/TimelineUtils.java | 4 +- hudi-examples/pom.xml | 4 ++ hudi-flink/pom.xml | 8 ++++ hudi-hadoop-mr/pom.xml | 30 +++++++++++++++ hudi-integ-test/pom.xml | 14 +++++++ hudi-spark-datasource/hudi-spark/pom.xml | 26 +++++++++++++ hudi-spark-datasource/hudi-spark3/pom.xml | 6 +++ hudi-sync/hudi-dla-sync/pom.xml | 4 ++ hudi-sync/hudi-hive-sync/pom.xml | 38 +++++++++++++++++++ hudi-sync/hudi-sync-common/pom.xml | 8 ++++ hudi-timeline-service/pom.xml | 8 ++++ hudi-utilities/pom.xml | 30 +++++++++++++++ packaging/hudi-flink-bundle/pom.xml | 16 ++++++++ packaging/hudi-integ-test-bundle/pom.xml | 32 ++++++++++++++++ packaging/hudi-spark-bundle/pom.xml | 32 ++++++++++++++++ packaging/hudi-timeline-server-bundle/pom.xml | 4 ++ packaging/hudi-utilities-bundle/pom.xml | 38 +++++++++++++++++++ pom.xml | 10 +++++ 22 files changed, 358 insertions(+), 4 deletions(-) diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml index 1a8c6335153d6..1003b74451b43 100644 --- a/hudi-cli/pom.xml +++ b/hudi-cli/pom.xml @@ -201,6 +201,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + diff --git a/hudi-client/hudi-flink-client/pom.xml b/hudi-client/hudi-flink-client/pom.xml index cc60b3fa29154..240aaf2d30d0e 100644 --- a/hudi-client/hudi-flink-client/pom.xml +++ b/hudi-client/hudi-flink-client/pom.xml @@ -125,6 +125,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + diff --git a/hudi-client/hudi-java-client/pom.xml b/hudi-client/hudi-java-client/pom.xml index c73f8aadaa6fa..96aefd3e8892b 100644 --- a/hudi-client/hudi-java-client/pom.xml +++ b/hudi-client/hudi-java-client/pom.xml @@ -72,12 +72,24 @@ ${hive.version} test ${hive.exec.classifier} + + + log4j + log4j + + ${hive.groupid} hive-metastore ${hive.version} test + + + log4j + log4j + + diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index a7bf8a1dca1b4..5891be6c303ee 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -143,6 +143,14 @@ javax.servlet * + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + provided @@ -153,8 +161,8 @@ test - log4j - log4j + org.slf4j + slf4j-log4j12 @@ -231,6 +239,12 @@ hbase-client ${hbase.version} test + + + log4j + log4j + + @@ -268,5 +282,17 @@ org.apache.logging.log4j log4j-core + + org.apache.logging.log4j + log4j-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + + org.apache.logging.log4j + log4j-1.2-api + diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java index de8c5821c1118..c0f9c170975d4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java @@ -26,8 +26,8 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.Collection; diff --git a/hudi-examples/pom.xml b/hudi-examples/pom.xml index 951fe816b2d93..081db656f19dc 100644 --- a/hudi-examples/pom.xml +++ b/hudi-examples/pom.xml @@ -220,6 +220,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + diff --git a/hudi-flink/pom.xml b/hudi-flink/pom.xml index a95404cba1516..7f7aa545cf364 100644 --- a/hudi-flink/pom.xml +++ b/hudi-flink/pom.xml @@ -220,6 +220,10 @@ org.slf4j slf4j-log4j12 + + log4j + log4j + @@ -253,6 +257,10 @@ org.eclipse.jetty.aggregate * + + log4j + log4j + diff --git a/hudi-hadoop-mr/pom.xml b/hudi-hadoop-mr/pom.xml index 0bc6bed6ac113..500f63b64c54a 100644 --- a/hudi-hadoop-mr/pom.xml +++ b/hudi-hadoop-mr/pom.xml @@ -57,16 +57,42 @@ org.apache.hadoop hadoop-mapreduce-client-core + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + org.apache.hadoop hadoop-mapreduce-client-common + + + log4j + log4j + + ${hive.groupid} hive-jdbc + + + log4j + log4j + + + org.apache.logging.log4j + log4j-web + + ${hive.groupid} @@ -81,6 +107,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml index dd33903256fd3..280c26ea742c1 100644 --- a/hudi-integ-test/pom.xml +++ b/hudi-integ-test/pom.xml @@ -144,6 +144,12 @@ org.apache.curator curator-framework ${zk-curator.version} + + + log4j + log4j + + org.apache.curator @@ -343,6 +349,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + @@ -375,6 +385,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + test diff --git a/hudi-spark-datasource/hudi-spark/pom.xml b/hudi-spark-datasource/hudi-spark/pom.xml index 60c482424d3e8..5148b57c1a270 100644 --- a/hudi-spark-datasource/hudi-spark/pom.xml +++ b/hudi-spark-datasource/hudi-spark/pom.xml @@ -317,6 +317,10 @@ javax.servlet * + + log4j + log4j + provided @@ -363,6 +367,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + @@ -386,6 +394,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + @@ -401,6 +413,10 @@ javax.servlet.jsp * + + log4j + log4j + @@ -420,6 +436,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + @@ -427,6 +447,12 @@ org.apache.curator curator-framework ${zk-curator.version} + + + log4j + log4j + + diff --git a/hudi-spark-datasource/hudi-spark3/pom.xml b/hudi-spark-datasource/hudi-spark3/pom.xml index f3c25a8a19f76..b79e0d425b634 100644 --- a/hudi-spark-datasource/hudi-spark3/pom.xml +++ b/hudi-spark-datasource/hudi-spark3/pom.xml @@ -158,6 +158,12 @@ spark-sql_2.12 ${spark3.version} true + + + log4j + log4j + + diff --git a/hudi-sync/hudi-dla-sync/pom.xml b/hudi-sync/hudi-dla-sync/pom.xml index 5fe69282d918e..36afb9541ebcb 100644 --- a/hudi-sync/hudi-dla-sync/pom.xml +++ b/hudi-sync/hudi-dla-sync/pom.xml @@ -135,6 +135,10 @@ org.apache.zookeeper zookeeper + + log4j + log4j + diff --git a/hudi-sync/hudi-hive-sync/pom.xml b/hudi-sync/hudi-hive-sync/pom.xml index 2faadb8e42a84..1115da17612de 100644 --- a/hudi-sync/hudi-hive-sync/pom.xml +++ b/hudi-sync/hudi-hive-sync/pom.xml @@ -81,6 +81,14 @@ log4j log4j + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + @@ -144,6 +152,10 @@ org.slf4j slf4j-log4j12 + + log4j + log4j + @@ -159,12 +171,22 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + ${hive.groupid} hive-metastore ${hive.version} + + + log4j + log4j + + ${hive.groupid} @@ -179,6 +201,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + @@ -233,12 +259,24 @@ org.apache.hadoop hadoop-mapreduce-client-common test + + + log4j + log4j + + org.apache.hadoop hadoop-mapreduce-client-core test + + + log4j + log4j + + diff --git a/hudi-sync/hudi-sync-common/pom.xml b/hudi-sync/hudi-sync-common/pom.xml index 21c727bedbcf5..cbcd933c22c81 100644 --- a/hudi-sync/hudi-sync-common/pom.xml +++ b/hudi-sync/hudi-sync-common/pom.xml @@ -49,6 +49,14 @@ log4j log4j + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + diff --git a/hudi-timeline-service/pom.xml b/hudi-timeline-service/pom.xml index 01296da7773e5..1615aa09107fe 100644 --- a/hudi-timeline-service/pom.xml +++ b/hudi-timeline-service/pom.xml @@ -160,6 +160,10 @@ log4j log4j + + org.slf4j + slf4j-log4j12 + @@ -170,6 +174,10 @@ javax.servlet * + + log4j + log4j + diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml index bbc227d3617dd..763aa42ddfdcc 100644 --- a/hudi-utilities/pom.xml +++ b/hudi-utilities/pom.xml @@ -280,6 +280,12 @@ io.confluent common-utils 3.0.0 + + + log4j + log4j + + io.confluent @@ -304,6 +310,12 @@ org.apache.hadoop hadoop-client + + + log4j + log4j + + org.apache.hadoop @@ -313,6 +325,10 @@ javax.servlet servlet-api + + log4j + log4j + @@ -372,12 +388,22 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + ${hive.groupid} hive-service ${hive.version} + + + log4j + log4j + + @@ -438,6 +464,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index a1887340ae182..ac1ca0acb2a9e 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -388,6 +388,10 @@ org.eclipse.jetty.aggregate * + + log4j + log4j + @@ -404,6 +408,10 @@ javax.servlet.jsp * + + log4j + log4j + @@ -420,6 +428,10 @@ javax.servlet.jsp * + + log4j + log4j + @@ -432,6 +444,10 @@ org.eclipse.jetty.orbit javax.servlet + + log4j + log4j + diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index 6f38c63fbf5bd..b1b69005e29b1 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -408,6 +408,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + @@ -416,6 +420,12 @@ hive-metastore ${hive.version} provided + + + log4j + log4j + + @@ -440,6 +450,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + @@ -464,6 +478,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + @@ -481,6 +499,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + @@ -578,6 +600,10 @@ javax.servlet servlet-api + + log4j + log4j + @@ -680,6 +706,12 @@ io.confluent common-utils 3.0.0 + + + log4j + log4j + + diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index 3f2f5e6c0303b..ce3ad4a4fb727 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -280,6 +280,12 @@ hive-service ${hive.version} ${spark.bundle.hive.scope} + + + log4j + log4j + + @@ -303,6 +309,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + @@ -311,6 +321,12 @@ hive-metastore ${hive.version} ${spark.bundle.hive.scope} + + + log4j + log4j + + @@ -327,6 +343,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + @@ -381,11 +401,23 @@ org.apache.hbase hbase-client ${hbase.version} + + + log4j + log4j + + org.apache.hbase hbase-protocol ${hbase.version} + + + log4j + log4j + + diff --git a/packaging/hudi-timeline-server-bundle/pom.xml b/packaging/hudi-timeline-server-bundle/pom.xml index d49ff3578c15c..9b7bffb0e349a 100644 --- a/packaging/hudi-timeline-server-bundle/pom.xml +++ b/packaging/hudi-timeline-server-bundle/pom.xml @@ -138,6 +138,10 @@ javax.servlet * + + log4j + log4j + diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 048f49295e270..ab3ccd9939c50 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -276,6 +276,12 @@ hive-service ${hive.version} ${utilities.bundle.hive.scope} + + + log4j + log4j + + @@ -283,6 +289,12 @@ hive-service-rpc ${hive.version} ${utilities.bundle.hive.scope} + + + log4j + log4j + + @@ -299,6 +311,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + @@ -307,6 +323,12 @@ hive-metastore ${hive.version} ${utilities.bundle.hive.scope} + + + log4j + log4j + + @@ -323,6 +345,10 @@ org.apache.logging.log4j log4j-slf4j-impl + + log4j + log4j + @@ -382,6 +408,12 @@ org.apache.hbase hbase-protocol ${hbase.version} + + + log4j + log4j + + @@ -389,6 +421,12 @@ org.apache.curator curator-framework ${zk-curator.version} + + + log4j + log4j + + diff --git a/pom.xml b/pom.xml index 127fd70d274fd..59b61d2c681ea 100644 --- a/pom.xml +++ b/pom.xml @@ -459,11 +459,21 @@ log4j-core ${log4j.version} + + org.apache.logging.log4j + log4j-api + ${log4j.version} + org.apache.logging.log4j log4j-1.2-api ${log4j.version} + + org.apache.logging.log4j + log4j-slf4j-impl + ${log4j.version} + From 8b343f593ce18c95ea1c4cc87683a2381d000ba6 Mon Sep 17 00:00:00 2001 From: hongdd Date: Wed, 7 Apr 2021 17:39:24 +0800 Subject: [PATCH 3/3] upgrade log4j to 2.13.3 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 59b61d2c681ea..8b7ab756030d4 100644 --- a/pom.xml +++ b/pom.xml @@ -95,7 +95,7 @@ 5.7.0-M1 1.7.0-M1 3.3.3 - 2.11.0 + 2.13.3 1.7.15 2.9.9 2.7.3