Skip to content

Commit

Permalink
Move safe mode manager to abstract master
Browse files Browse the repository at this point in the history
  • Loading branch information
bf8086 committed Dec 21, 2017
1 parent 2983b46 commit 6caa8fb
Show file tree
Hide file tree
Showing 13 changed files with 107 additions and 52 deletions.
Expand Up @@ -16,7 +16,6 @@
import alluxio.clock.Clock; import alluxio.clock.Clock;
import alluxio.master.journal.Journal; import alluxio.master.journal.Journal;
import alluxio.master.journal.JournalContext; import alluxio.master.journal.JournalContext;
import alluxio.master.journal.JournalSystem;
import alluxio.util.executor.ExecutorServiceFactory; import alluxio.util.executor.ExecutorServiceFactory;


import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
Expand Down Expand Up @@ -53,18 +52,22 @@ public abstract class AbstractMaster implements Master {
/** The clock to use for determining the time. */ /** The clock to use for determining the time. */
protected final Clock mClock; protected final Clock mClock;


/** The manager for safe mode state */
protected final SafeModeManager mSafeModeManager;

/** /**
* @param journalSystem the journal system to use for tracking master operations * @param masterContext the context for Alluxio master
* @param clock the Clock to use for determining the time * @param clock the Clock to use for determining the time
* @param executorServiceFactory a factory for creating the executor service to use for * @param executorServiceFactory a factory for creating the executor service to use for
* running maintenance threads * running maintenance threads
*/ */
protected AbstractMaster(JournalSystem journalSystem, Clock clock, protected AbstractMaster(MasterContext masterContext, Clock clock,
ExecutorServiceFactory executorServiceFactory) { ExecutorServiceFactory executorServiceFactory) {
mJournal = journalSystem.createJournal(this); Preconditions.checkNotNull(masterContext, "masterContext");
mClock = Preconditions.checkNotNull(clock, "clock"); mJournal = masterContext.getJournalSystem().createJournal(this);
mExecutorServiceFactory = mSafeModeManager = masterContext.getmSafeModeManager();
Preconditions.checkNotNull(executorServiceFactory, "executorServiceFactory"); mClock = clock;
mExecutorServiceFactory = executorServiceFactory;
} }


@Override @Override
Expand Down
47 changes: 47 additions & 0 deletions core/server/common/src/main/java/alluxio/master/MasterContext.java
@@ -0,0 +1,47 @@
/*
* The Alluxio Open Foundation licenses this work under the Apache License, version 2.0
* (the "License"). You may not use this work except in compliance with the License, which is
* available at www.apache.org/licenses/LICENSE-2.0
*
* This software is distributed on an "AS IS" basis, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
* either express or implied, as more fully set forth in the License.
*
* See the NOTICE file distributed with this work for information regarding copyright ownership.
*/

package alluxio.master;

import alluxio.master.journal.JournalSystem;

import com.google.common.base.Preconditions;

/**
* Stores context information for Alluxio masters.
*/
public final class MasterContext {
private final JournalSystem mJournalSystem;
private final SafeModeManager mSafeModeManager;

/**
* @param journalSystem the journal system to use for tracking master operations
* @param safeModeManager the manager for master safe mode
*/
public MasterContext(JournalSystem journalSystem, SafeModeManager safeModeManager) {
mJournalSystem = Preconditions.checkNotNull(journalSystem, "journalSystem");
mSafeModeManager = Preconditions.checkNotNull(safeModeManager, "safeModeManager");
}

/**
* @return the journal system to use for tracking master operations
*/
public JournalSystem getJournalSystem() {
return mJournalSystem;
}

/**
* @return the manager for master safe mode
*/
public SafeModeManager getmSafeModeManager() {
return mSafeModeManager;
}
}
Expand Up @@ -12,6 +12,7 @@
package alluxio.master.block; package alluxio.master.block;


import alluxio.Constants; import alluxio.Constants;
import alluxio.master.MasterContext;
import alluxio.master.MasterFactory; import alluxio.master.MasterFactory;
import alluxio.master.MasterRegistry; import alluxio.master.MasterRegistry;
import alluxio.master.SafeModeManager; import alluxio.master.SafeModeManager;
Expand Down Expand Up @@ -50,7 +51,7 @@ public BlockMaster create(MasterRegistry registry, JournalSystem journalFactory,
SafeModeManager safeModeManager) { SafeModeManager safeModeManager) {
Preconditions.checkArgument(journalFactory != null, "journal"); Preconditions.checkArgument(journalFactory != null, "journal");
LOG.info("Creating {} ", BlockMaster.class.getName()); LOG.info("Creating {} ", BlockMaster.class.getName());
BlockMaster master = new DefaultBlockMaster(journalFactory, safeModeManager); BlockMaster master = new DefaultBlockMaster(new MasterContext(journalFactory, safeModeManager));
registry.add(BlockMaster.class, master); registry.add(BlockMaster.class, master);
return master; return master;
} }
Expand Down
Expand Up @@ -29,12 +29,11 @@
import alluxio.heartbeat.HeartbeatExecutor; import alluxio.heartbeat.HeartbeatExecutor;
import alluxio.heartbeat.HeartbeatThread; import alluxio.heartbeat.HeartbeatThread;
import alluxio.master.AbstractMaster; import alluxio.master.AbstractMaster;
import alluxio.master.SafeModeManager; import alluxio.master.MasterContext;
import alluxio.master.block.meta.MasterBlockInfo; import alluxio.master.block.meta.MasterBlockInfo;
import alluxio.master.block.meta.MasterBlockLocation; import alluxio.master.block.meta.MasterBlockLocation;
import alluxio.master.block.meta.MasterWorkerInfo; import alluxio.master.block.meta.MasterWorkerInfo;
import alluxio.master.journal.JournalContext; import alluxio.master.journal.JournalContext;
import alluxio.master.journal.JournalSystem;
import alluxio.metrics.MetricsSystem; import alluxio.metrics.MetricsSystem;
import alluxio.proto.journal.Block.BlockContainerIdGeneratorEntry; import alluxio.proto.journal.Block.BlockContainerIdGeneratorEntry;
import alluxio.proto.journal.Block.BlockInfoEntry; import alluxio.proto.journal.Block.BlockInfoEntry;
Expand Down Expand Up @@ -165,30 +164,28 @@ public Object getFieldValue(MasterWorkerInfo o) {
/** The value of the 'next container id' last journaled. */ /** The value of the 'next container id' last journaled. */
@GuardedBy("mBlockContainerIdGenerator") @GuardedBy("mBlockContainerIdGenerator")
private long mJournaledNextContainerId = 0; private long mJournaledNextContainerId = 0;
private SafeModeManager mSafeModeManager;


/** /**
* Creates a new instance of {@link DefaultBlockMaster}. * Creates a new instance of {@link DefaultBlockMaster}.
* *
* @param journalSystem the journal system to use for tracking master operations * @param masterContext the context for Alluxio master
*/ */
DefaultBlockMaster(JournalSystem journalSystem, SafeModeManager safeModeManager) { DefaultBlockMaster(MasterContext masterContext) {
this(journalSystem, new SystemClock(), safeModeManager, ExecutorServiceFactories this(masterContext, new SystemClock(), ExecutorServiceFactories
.fixedThreadPoolExecutorServiceFactory(Constants.BLOCK_MASTER_NAME, 2)); .fixedThreadPoolExecutorServiceFactory(Constants.BLOCK_MASTER_NAME, 2));
} }


/** /**
* Creates a new instance of {@link DefaultBlockMaster}. * Creates a new instance of {@link DefaultBlockMaster}.
* *
* @param journalSystem the journal system to use for tracking master operations * @param masterContext the context for Alluxio master
* @param clock the clock to use for determining the time * @param clock the clock to use for determining the time
* @param executorServiceFactory a factory for creating the executor service to use for running * @param executorServiceFactory a factory for creating the executor service to use for running
* maintenance threads * maintenance threads
*/ */
DefaultBlockMaster(JournalSystem journalSystem, Clock clock, SafeModeManager safeModeManager, DefaultBlockMaster(MasterContext masterContext, Clock clock,
ExecutorServiceFactory executorServiceFactory) { ExecutorServiceFactory executorServiceFactory) {
super(journalSystem, clock, executorServiceFactory); super(masterContext, clock, executorServiceFactory);
mSafeModeManager = safeModeManager;
Metrics.registerGauges(this); Metrics.registerGauges(this);
} }


Expand Down
Expand Up @@ -38,8 +38,8 @@
import alluxio.heartbeat.HeartbeatContext; import alluxio.heartbeat.HeartbeatContext;
import alluxio.heartbeat.HeartbeatThread; import alluxio.heartbeat.HeartbeatThread;
import alluxio.master.AbstractMaster; import alluxio.master.AbstractMaster;
import alluxio.master.MasterContext;
import alluxio.master.ProtobufUtils; import alluxio.master.ProtobufUtils;
import alluxio.master.SafeModeManager;
import alluxio.master.audit.AsyncUserAccessAuditLogWriter; import alluxio.master.audit.AsyncUserAccessAuditLogWriter;
import alluxio.master.audit.AuditContext; import alluxio.master.audit.AuditContext;
import alluxio.master.block.BlockId; import alluxio.master.block.BlockId;
Expand Down Expand Up @@ -75,7 +75,6 @@
import alluxio.master.file.options.RenameOptions; import alluxio.master.file.options.RenameOptions;
import alluxio.master.file.options.SetAttributeOptions; import alluxio.master.file.options.SetAttributeOptions;
import alluxio.master.journal.JournalContext; import alluxio.master.journal.JournalContext;
import alluxio.master.journal.JournalSystem;
import alluxio.master.journal.NoopJournalContext; import alluxio.master.journal.NoopJournalContext;
import alluxio.metrics.MetricsSystem; import alluxio.metrics.MetricsSystem;
import alluxio.proto.journal.File.AddMountPointEntry; import alluxio.proto.journal.File.AddMountPointEntry;
Expand Down Expand Up @@ -307,10 +306,6 @@ public final class DefaultFileSystemMaster extends AbstractMaster implements Fil
/** This caches block locations in the UFS. */ /** This caches block locations in the UFS. */
private final UfsBlockLocationCache mUfsBlockLocationCache; private final UfsBlockLocationCache mUfsBlockLocationCache;


/** The manager for master safe mode state. */
@SuppressFBWarnings("URF_UNREAD_FIELD")
private final SafeModeManager mSafeModeManager;

/** /**
* The service that checks for inode files with ttl set. We store it here so that it can be * The service that checks for inode files with ttl set. We store it here so that it can be
* accessed from tests. * accessed from tests.
Expand All @@ -335,32 +330,30 @@ public final class DefaultFileSystemMaster extends AbstractMaster implements Fil
* Creates a new instance of {@link DefaultFileSystemMaster}. * Creates a new instance of {@link DefaultFileSystemMaster}.
* *
* @param blockMaster a block master handle * @param blockMaster a block master handle
* @param journalSystem the journal system to use for tracking master operations * @param masterContext the context for Alluxio master
*/ */
DefaultFileSystemMaster(BlockMaster blockMaster, JournalSystem journalSystem, DefaultFileSystemMaster(BlockMaster blockMaster, MasterContext masterContext) {
SafeModeManager safeModeManager) { this(blockMaster, masterContext, ExecutorServiceFactories
this(blockMaster, journalSystem, safeModeManager, ExecutorServiceFactories
.fixedThreadPoolExecutorServiceFactory(Constants.FILE_SYSTEM_MASTER_NAME, 3)); .fixedThreadPoolExecutorServiceFactory(Constants.FILE_SYSTEM_MASTER_NAME, 3));
} }


/** /**
* Creates a new instance of {@link DefaultFileSystemMaster}. * Creates a new instance of {@link DefaultFileSystemMaster}.
* *
* @param blockMaster a block master handle * @param blockMaster a block master handle
* @param journalSystem the journal system to use for tracking master operations * @param masterContext the context for Alluxio master
* @param executorServiceFactory a factory for creating the executor service to use for running * @param executorServiceFactory a factory for creating the executor service to use for running
* maintenance threads * maintenance threads
*/ */
DefaultFileSystemMaster(BlockMaster blockMaster, JournalSystem journalSystem, DefaultFileSystemMaster(BlockMaster blockMaster, MasterContext masterContext,
SafeModeManager safeModeManager, ExecutorServiceFactory executorServiceFactory) { ExecutorServiceFactory executorServiceFactory) {
super(journalSystem, new SystemClock(), executorServiceFactory); super(masterContext, new SystemClock(), executorServiceFactory);


mBlockMaster = blockMaster; mBlockMaster = blockMaster;
mDirectoryIdGenerator = new InodeDirectoryIdGenerator(mBlockMaster); mDirectoryIdGenerator = new InodeDirectoryIdGenerator(mBlockMaster);
mUfsManager = new MasterUfsManager(); mUfsManager = new MasterUfsManager();
mMountTable = new MountTable(mUfsManager); mMountTable = new MountTable(mUfsManager);
mInodeTree = new InodeTree(mBlockMaster, mDirectoryIdGenerator, mMountTable); mInodeTree = new InodeTree(mBlockMaster, mDirectoryIdGenerator, mMountTable);
mSafeModeManager = safeModeManager;


// TODO(gene): Handle default config value for whitelist. // TODO(gene): Handle default config value for whitelist.
mWhitelist = new PrefixList(Configuration.getList(PropertyKey.MASTER_WHITELIST, ",")); mWhitelist = new PrefixList(Configuration.getList(PropertyKey.MASTER_WHITELIST, ","));
Expand Down
Expand Up @@ -12,6 +12,7 @@
package alluxio.master.file; package alluxio.master.file;


import alluxio.Constants; import alluxio.Constants;
import alluxio.master.MasterContext;
import alluxio.master.MasterFactory; import alluxio.master.MasterFactory;
import alluxio.master.MasterRegistry; import alluxio.master.MasterRegistry;
import alluxio.master.SafeModeManager; import alluxio.master.SafeModeManager;
Expand Down Expand Up @@ -52,8 +53,8 @@ public FileSystemMaster create(MasterRegistry registry, JournalSystem journalFac
Preconditions.checkArgument(journalFactory != null, "journal factory may not be null"); Preconditions.checkArgument(journalFactory != null, "journal factory may not be null");
LOG.info("Creating {} ", FileSystemMaster.class.getName()); LOG.info("Creating {} ", FileSystemMaster.class.getName());
BlockMaster blockMaster = registry.get(BlockMaster.class); BlockMaster blockMaster = registry.get(BlockMaster.class);
FileSystemMaster fileSystemMaster = new DefaultFileSystemMaster(blockMaster, journalFactory, FileSystemMaster fileSystemMaster = new DefaultFileSystemMaster(blockMaster,
safeModeManager); new MasterContext(journalFactory, safeModeManager));
registry.add(FileSystemMaster.class, fileSystemMaster); registry.add(FileSystemMaster.class, fileSystemMaster);
return fileSystemMaster; return fileSystemMaster;
} }
Expand Down
Expand Up @@ -32,10 +32,10 @@
import alluxio.job.CommandLineJob; import alluxio.job.CommandLineJob;
import alluxio.job.Job; import alluxio.job.Job;
import alluxio.master.AbstractMaster; import alluxio.master.AbstractMaster;
import alluxio.master.MasterContext;
import alluxio.master.file.FileSystemMaster; import alluxio.master.file.FileSystemMaster;
import alluxio.master.file.options.CreateFileOptions; import alluxio.master.file.options.CreateFileOptions;
import alluxio.master.journal.JournalContext; import alluxio.master.journal.JournalContext;
import alluxio.master.journal.JournalSystem;
import alluxio.master.lineage.checkpoint.CheckpointPlan; import alluxio.master.lineage.checkpoint.CheckpointPlan;
import alluxio.master.lineage.checkpoint.CheckpointSchedulingExecutor; import alluxio.master.lineage.checkpoint.CheckpointSchedulingExecutor;
import alluxio.master.lineage.meta.Lineage; import alluxio.master.lineage.meta.Lineage;
Expand Down Expand Up @@ -89,24 +89,24 @@ public final class DefaultLineageMaster extends AbstractMaster implements Lineag
* Creates a new instance of {@link LineageMaster}. * Creates a new instance of {@link LineageMaster}.
* *
* @param fileSystemMaster the file system master handle * @param fileSystemMaster the file system master handle
* @param journalSystem the journal system to use for tracking master operations * @param masterContext the context for Alluxio master
*/ */
DefaultLineageMaster(FileSystemMaster fileSystemMaster, JournalSystem journalSystem) { DefaultLineageMaster(FileSystemMaster fileSystemMaster, MasterContext masterContext) {
this(fileSystemMaster, journalSystem, ExecutorServiceFactories this(fileSystemMaster, masterContext, ExecutorServiceFactories
.fixedThreadPoolExecutorServiceFactory(Constants.LINEAGE_MASTER_NAME, 2)); .fixedThreadPoolExecutorServiceFactory(Constants.LINEAGE_MASTER_NAME, 2));
} }


/** /**
* Creates a new instance of {@link LineageMaster}. * Creates a new instance of {@link LineageMaster}.
* *
* @param fileSystemMaster the file system master handle * @param fileSystemMaster the file system master handle
* @param journalSystem the journal system to use for tracking master operations * @param masterContext the context for Alluxio master
* @param executorServiceFactory a factory for creating the executor service to use for running * @param executorServiceFactory a factory for creating the executor service to use for running
* maintenance threads * maintenance threads
*/ */
DefaultLineageMaster(FileSystemMaster fileSystemMaster, JournalSystem journalSystem, DefaultLineageMaster(FileSystemMaster fileSystemMaster, MasterContext masterContext,
ExecutorServiceFactory executorServiceFactory) { ExecutorServiceFactory executorServiceFactory) {
super(journalSystem, new SystemClock(), executorServiceFactory); super(masterContext, new SystemClock(), executorServiceFactory);
mLineageIdGenerator = new LineageIdGenerator(); mLineageIdGenerator = new LineageIdGenerator();
mLineageStore = new LineageStore(mLineageIdGenerator); mLineageStore = new LineageStore(mLineageIdGenerator);
mFileSystemMaster = fileSystemMaster; mFileSystemMaster = fileSystemMaster;
Expand Down
Expand Up @@ -14,6 +14,7 @@
import alluxio.Configuration; import alluxio.Configuration;
import alluxio.Constants; import alluxio.Constants;
import alluxio.PropertyKey; import alluxio.PropertyKey;
import alluxio.master.MasterContext;
import alluxio.master.MasterFactory; import alluxio.master.MasterFactory;
import alluxio.master.MasterRegistry; import alluxio.master.MasterRegistry;
import alluxio.master.SafeModeManager; import alluxio.master.SafeModeManager;
Expand Down Expand Up @@ -54,7 +55,8 @@ public LineageMaster create(MasterRegistry registry, JournalSystem journalSystem
Preconditions.checkArgument(journalSystem != null, "journal system may not be null"); Preconditions.checkArgument(journalSystem != null, "journal system may not be null");
LOG.info("Creating {} ", LineageMaster.class.getName()); LOG.info("Creating {} ", LineageMaster.class.getName());
FileSystemMaster fileSystemMaster = registry.get(FileSystemMaster.class); FileSystemMaster fileSystemMaster = registry.get(FileSystemMaster.class);
LineageMaster lineageMaster = new DefaultLineageMaster(fileSystemMaster, journalSystem); LineageMaster lineageMaster = new DefaultLineageMaster(fileSystemMaster,
new MasterContext(journalSystem, safeModeManager));
registry.add(LineageMaster.class, lineageMaster); registry.add(LineageMaster.class, lineageMaster);
return lineageMaster; return lineageMaster;
} }
Expand Down
Expand Up @@ -20,6 +20,7 @@
import alluxio.heartbeat.HeartbeatScheduler; import alluxio.heartbeat.HeartbeatScheduler;
import alluxio.heartbeat.ManuallyScheduleHeartbeat; import alluxio.heartbeat.ManuallyScheduleHeartbeat;
import alluxio.master.DefaultSafeModeManager; import alluxio.master.DefaultSafeModeManager;
import alluxio.master.MasterContext;
import alluxio.master.MasterRegistry; import alluxio.master.MasterRegistry;
import alluxio.master.SafeModeManager; import alluxio.master.SafeModeManager;
import alluxio.master.journal.JournalSystem; import alluxio.master.journal.JournalSystem;
Expand Down Expand Up @@ -91,8 +92,8 @@ public void before() throws Exception {
mClock = new ManualClock(); mClock = new ManualClock();
mExecutorService = mExecutorService =
Executors.newFixedThreadPool(2, ThreadFactoryUtils.build("TestBlockMaster-%d", true)); Executors.newFixedThreadPool(2, ThreadFactoryUtils.build("TestBlockMaster-%d", true));
mBlockMaster = new DefaultBlockMaster(journalSystem, mClock, mSafeModeManager, mBlockMaster = new DefaultBlockMaster(new MasterContext(journalSystem, mSafeModeManager),
ExecutorServiceFactories.constantExecutorServiceFactory(mExecutorService)); mClock, ExecutorServiceFactories.constantExecutorServiceFactory(mExecutorService));
mRegistry.add(BlockMaster.class, mBlockMaster); mRegistry.add(BlockMaster.class, mBlockMaster);
mRegistry.start(true); mRegistry.start(true);
} }
Expand Down
Expand Up @@ -36,6 +36,7 @@
import alluxio.heartbeat.HeartbeatScheduler; import alluxio.heartbeat.HeartbeatScheduler;
import alluxio.heartbeat.ManuallyScheduleHeartbeat; import alluxio.heartbeat.ManuallyScheduleHeartbeat;
import alluxio.master.DefaultSafeModeManager; import alluxio.master.DefaultSafeModeManager;
import alluxio.master.MasterContext;
import alluxio.master.MasterRegistry; import alluxio.master.MasterRegistry;
import alluxio.master.SafeModeManager; import alluxio.master.SafeModeManager;
import alluxio.master.block.BlockMaster; import alluxio.master.block.BlockMaster;
Expand Down Expand Up @@ -1902,7 +1903,8 @@ private void startServices() throws Exception {
mBlockMaster = new BlockMasterFactory().create(mRegistry, mJournalSystem, mSafeModeManager); mBlockMaster = new BlockMasterFactory().create(mRegistry, mJournalSystem, mSafeModeManager);
mExecutorService = Executors mExecutorService = Executors
.newFixedThreadPool(2, ThreadFactoryUtils.build("DefaultFileSystemMasterTest-%d", true)); .newFixedThreadPool(2, ThreadFactoryUtils.build("DefaultFileSystemMasterTest-%d", true));
mFileSystemMaster = new DefaultFileSystemMaster(mBlockMaster, mJournalSystem, mSafeModeManager, mFileSystemMaster = new DefaultFileSystemMaster(mBlockMaster,
new MasterContext(mJournalSystem, mSafeModeManager),
ExecutorServiceFactories.constantExecutorServiceFactory(mExecutorService)); ExecutorServiceFactories.constantExecutorServiceFactory(mExecutorService));
mRegistry.add(FileSystemMaster.class, mFileSystemMaster); mRegistry.add(FileSystemMaster.class, mFileSystemMaster);
mJournalSystem.start(); mJournalSystem.start();
Expand Down
Expand Up @@ -19,7 +19,10 @@
import alluxio.job.CommandLineJob; import alluxio.job.CommandLineJob;
import alluxio.job.Job; import alluxio.job.Job;
import alluxio.job.JobConf; import alluxio.job.JobConf;
import alluxio.master.DefaultSafeModeManager;
import alluxio.master.MasterContext;
import alluxio.master.MasterRegistry; import alluxio.master.MasterRegistry;
import alluxio.master.SafeModeManager;
import alluxio.master.file.FileSystemMaster; import alluxio.master.file.FileSystemMaster;
import alluxio.master.file.options.CompleteFileOptions; import alluxio.master.file.options.CompleteFileOptions;
import alluxio.master.journal.JournalSystem; import alluxio.master.journal.JournalSystem;
Expand Down Expand Up @@ -55,6 +58,7 @@ public final class LineageMasterTest {
private FileSystemMaster mFileSystemMaster; private FileSystemMaster mFileSystemMaster;
private Job mJob; private Job mJob;
private MasterRegistry mRegistry; private MasterRegistry mRegistry;
private SafeModeManager mSafeModeManager;


/** Rule to create a new temporary folder during each test. */ /** Rule to create a new temporary folder during each test. */
@Rule @Rule
Expand All @@ -69,9 +73,11 @@ public void before() throws Exception {
JournalSystem journalSystem = new NoopJournalSystem(); JournalSystem journalSystem = new NoopJournalSystem();
mFileSystemMaster = Mockito.mock(FileSystemMaster.class); mFileSystemMaster = Mockito.mock(FileSystemMaster.class);
mRegistry.add(FileSystemMaster.class, mFileSystemMaster); mRegistry.add(FileSystemMaster.class, mFileSystemMaster);
mSafeModeManager = new DefaultSafeModeManager();
ThreadFactory threadPool = ThreadFactoryUtils.build("LineageMasterTest-%d", true); ThreadFactory threadPool = ThreadFactoryUtils.build("LineageMasterTest-%d", true);
mExecutorService = Executors.newFixedThreadPool(2, threadPool); mExecutorService = Executors.newFixedThreadPool(2, threadPool);
mLineageMaster = new DefaultLineageMaster(mFileSystemMaster, journalSystem, mLineageMaster = new DefaultLineageMaster(mFileSystemMaster,
new MasterContext(journalSystem, mSafeModeManager),
ExecutorServiceFactories.constantExecutorServiceFactory(mExecutorService)); ExecutorServiceFactories.constantExecutorServiceFactory(mExecutorService));
mRegistry.add(LineageMaster.class, mLineageMaster); mRegistry.add(LineageMaster.class, mLineageMaster);
mJob = new CommandLineJob("test", new JobConf("output")); mJob = new CommandLineJob("test", new JobConf("output"));
Expand Down
Expand Up @@ -22,12 +22,12 @@
import alluxio.exception.FileDoesNotExistException; import alluxio.exception.FileDoesNotExistException;
import alluxio.exception.InvalidPathException; import alluxio.exception.InvalidPathException;
import alluxio.master.AbstractMaster; import alluxio.master.AbstractMaster;
import alluxio.master.MasterContext;
import alluxio.master.file.FileSystemMaster; import alluxio.master.file.FileSystemMaster;
import alluxio.master.file.options.CreateDirectoryOptions; import alluxio.master.file.options.CreateDirectoryOptions;
import alluxio.master.file.options.DeleteOptions; import alluxio.master.file.options.DeleteOptions;
import alluxio.master.file.options.RenameOptions; import alluxio.master.file.options.RenameOptions;
import alluxio.master.journal.JournalContext; import alluxio.master.journal.JournalContext;
import alluxio.master.journal.JournalSystem;
import alluxio.proto.journal.Journal.JournalEntry; import alluxio.proto.journal.Journal.JournalEntry;
import alluxio.proto.journal.KeyValue; import alluxio.proto.journal.KeyValue;
import alluxio.thrift.KeyValueMasterClientService; import alluxio.thrift.KeyValueMasterClientService;
Expand Down Expand Up @@ -75,10 +75,10 @@ public class DefaultKeyValueMaster extends AbstractMaster implements KeyValueMas


/** /**
* @param fileSystemMaster the file system master handle * @param fileSystemMaster the file system master handle
* @param journalSystem the journal system to use for tracking master operations * @param masterContext the context for Alluxio master
*/ */
DefaultKeyValueMaster(FileSystemMaster fileSystemMaster, JournalSystem journalSystem) { DefaultKeyValueMaster(FileSystemMaster fileSystemMaster, MasterContext masterContext) {
super(journalSystem, new SystemClock(), ExecutorServiceFactories super(masterContext, new SystemClock(), ExecutorServiceFactories
.fixedThreadPoolExecutorServiceFactory(Constants.KEY_VALUE_MASTER_NAME, 2)); .fixedThreadPoolExecutorServiceFactory(Constants.KEY_VALUE_MASTER_NAME, 2));
mFileSystemMaster = fileSystemMaster; mFileSystemMaster = fileSystemMaster;
mCompleteStoreToPartitions = new HashMap<>(); mCompleteStoreToPartitions = new HashMap<>();
Expand Down

0 comments on commit 6caa8fb

Please sign in to comment.