Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
23 commits
Select commit Hold shift + click to select a range
5a70ca7
HDDS-12185. Enhance FileSizeCountTask for Faster Processing.
ArafatKhan2198 Feb 3, 2025
7aca9f2
Made code changes
ArafatKhan2198 Feb 3, 2025
4b1da9f
Also fixed the testcases
ArafatKhan2198 Feb 5, 2025
c8b2e47
Made final review changes
ArafatKhan2198 Feb 7, 2025
68b8948
Passing the executor from the injector
ArafatKhan2198 Feb 7, 2025
9e00401
Removed unwanted changes
ArafatKhan2198 Feb 7, 2025
991c7d5
Removed unwanted import
ArafatKhan2198 Feb 7, 2025
1166bce
Fixd checkstyle and using injectors
ArafatKhan2198 Feb 10, 2025
678ddb2
more changes
ArafatKhan2198 Feb 10, 2025
978c160
Merge branch 'master' into HDDS-12185
ArafatKhan2198 Feb 10, 2025
4719546
Changed log levels
ArafatKhan2198 Feb 10, 2025
ce7e523
Split the FilesizecountTask into two separate tasks
ArafatKhan2198 Feb 13, 2025
0a845bc
Fixed checkstyle
ArafatKhan2198 Feb 13, 2025
1608468
Refactored code to a helper class
ArafatKhan2198 Feb 17, 2025
b7883ec
Fixed checkstyle issues
ArafatKhan2198 Feb 17, 2025
a426eb3
Fixed unit tests
ArafatKhan2198 Feb 17, 2025
46a0b26
Fixed doc
ArafatKhan2198 Feb 17, 2025
dbd54c2
Merge remote-tracking branch 'origin/master' into HDDS-12185
adoroszlai Feb 17, 2025
d31e450
Ensure shared synchronization for table truncation in FileSizeCount t…
ArafatKhan2198 Feb 18, 2025
3484f2a
Changed parameter name
ArafatKhan2198 Feb 18, 2025
e80362c
Ensure proper error handling in table truncation and reprocess by res…
ArafatKhan2198 Feb 19, 2025
ad3e6c1
Added the excpetion propagation test case
ArafatKhan2198 Feb 20, 2025
5e6fdc3
Fixed checkstyle
ArafatKhan2198 Feb 20, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.hadoop.ozone.recon;

import java.util.concurrent.atomic.AtomicBoolean;

/**
* Recon Server constants file.
*/
Expand Down Expand Up @@ -89,4 +91,15 @@ private ReconConstants() {
(double) MAX_CONTAINER_SIZE_UPPER_BOUND /
MIN_CONTAINER_SIZE_UPPER_BOUND) /
Math.log(2)) + 1;

// For file-size count reprocessing: ensure only one task truncates the table.
public static final AtomicBoolean FILE_SIZE_COUNT_TABLE_TRUNCATED = new AtomicBoolean(false);

/**
* Resets the table-truncated flag for the given tables. This should be called once per reprocess cycle,
* for example by the OM task controller, before the tasks run.
*/
public static void resetTableTruncatedFlags() {
FILE_SIZE_COUNT_TABLE_TRUNCATED.set(false);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
Expand Down Expand Up @@ -57,7 +59,8 @@
import org.apache.hadoop.ozone.recon.spi.impl.ReconNamespaceSummaryManagerImpl;
import org.apache.hadoop.ozone.recon.spi.impl.StorageContainerServiceProviderImpl;
import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTask;
import org.apache.hadoop.ozone.recon.tasks.FileSizeCountTask;
import org.apache.hadoop.ozone.recon.tasks.FileSizeCountTaskFSO;
import org.apache.hadoop.ozone.recon.tasks.FileSizeCountTaskOBS;
import org.apache.hadoop.ozone.recon.tasks.NSSummaryTask;
import org.apache.hadoop.ozone.recon.tasks.OmTableInsightTask;
import org.apache.hadoop.ozone.recon.tasks.ReconOmTask;
Expand Down Expand Up @@ -129,12 +132,19 @@ protected void configure() {
Multibinder<ReconOmTask> taskBinder =
Multibinder.newSetBinder(binder(), ReconOmTask.class);
taskBinder.addBinding().to(ContainerKeyMapperTask.class);
taskBinder.addBinding().to(FileSizeCountTask.class);
taskBinder.addBinding().to(FileSizeCountTaskFSO.class);
taskBinder.addBinding().to(FileSizeCountTaskOBS.class);
taskBinder.addBinding().to(OmTableInsightTask.class);
taskBinder.addBinding().to(NSSummaryTask.class);
}
}

@Provides
@Singleton
public ExecutorService provideReconExecutorService() {
return Executors.newFixedThreadPool(5);
}

/**
* Class that has all the DAO bindings in Recon.
*/
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.hadoop.ozone.recon.tasks;

import com.google.inject.Inject;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.ozone.om.OMMetadataManager;
import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
import org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition;
import org.hadoop.ozone.recon.schema.tables.daos.FileCountBySizeDao;
import org.jooq.DSLContext;

/**
* Task for FileSystemOptimized (FSO) which processes the FILE_TABLE.
*/
public class FileSizeCountTaskFSO implements ReconOmTask {

private final FileCountBySizeDao fileCountBySizeDao;
private final DSLContext dslContext;

@Inject
public FileSizeCountTaskFSO(FileCountBySizeDao fileCountBySizeDao,
UtilizationSchemaDefinition utilizationSchemaDefinition) {
this.fileCountBySizeDao = fileCountBySizeDao;
this.dslContext = utilizationSchemaDefinition.getDSLContext();
}

@Override
public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
return FileSizeCountTaskHelper.reprocess(
omMetadataManager,
dslContext,
fileCountBySizeDao,
BucketLayout.FILE_SYSTEM_OPTIMIZED,
getTaskName()
);
}

@Override
public Pair<String, Boolean> process(OMUpdateEventBatch events) {
// This task listens only on the FILE_TABLE.
return FileSizeCountTaskHelper.processEvents(
events,
OmMetadataManagerImpl.FILE_TABLE,
dslContext,
fileCountBySizeDao,
getTaskName());
}

@Override
public String getTaskName() {
return "FileSizeCountTaskFSO";
}
}
Loading