Skip to content

Commit

Permalink
Tests are passing now.
Browse files Browse the repository at this point in the history
  • Loading branch information
jsimsa committed Sep 15, 2015
1 parent 2b2b110 commit 3d1cef9
Show file tree
Hide file tree
Showing 10 changed files with 208 additions and 9 deletions.
2 changes: 1 addition & 1 deletion common/src/main/java/tachyon/client/BlockMasterClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@
* Since thrift clients are not thread safe, this class is a wrapper to provide thread safety, and
* to provide retries.
*/
public final class BlockMasterClient extends MasterClientBase {
public class BlockMasterClient extends MasterClientBase {
private static final Logger LOG = LoggerFactory.getLogger(Constants.LOGGER_TYPE);

private BlockMasterService.Client mClient = null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@
*/
// TODO: split out worker-specific calls to a fs master client for workers.
// TODO: figure out a retry utility to make all the retry logic in this file better.
public final class FileSystemMasterClient extends MasterClientBase {
public class FileSystemMasterClient extends MasterClientBase {
private static final Logger LOG = LoggerFactory.getLogger(Constants.LOGGER_TYPE);

private FileSystemMasterService.Client mClient = null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,8 +86,8 @@ public BlockDataManager(WorkerSource workerSource, BlockMasterClient blockMaster
mTachyonConf = WorkerContext.getConf();
mHeartbeatReporter = new BlockHeartbeatReporter();
mBlockStore = new TieredBlockStore();
mMetricsReporter = new BlockMetricsReporter(mWorkerSource);
mWorkerSource = workerSource;
mMetricsReporter = new BlockMetricsReporter(mWorkerSource);
mBlockMasterClient = blockMasterClient;
mFileSystemMasterClient = fileSystemMasterClient;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
* through {@link tachyon.worker.block.BlockDataManager#commitBlock(long, long)}. This class is
* thread safe.
*/
public final class BlockHeartbeatReporter extends BlockStoreEventListenerBase {
public class BlockHeartbeatReporter extends BlockStoreEventListenerBase {
/** Lock for operations on the removed and added block collections */
private final Object mLock;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
/**
* This class listens on block events and increases the metrics counters.
*/
public final class BlockMetricsReporter extends BlockStoreEventListenerBase {
public class BlockMetricsReporter extends BlockStoreEventListenerBase {
private final WorkerSource mWorkerSource;

public BlockMetricsReporter(WorkerSource workerSource) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
* Where to store a block within a block store. Currently, this is a wrapper on an integer
* representing the tier to put this block.
*/
public final class BlockStoreLocation {
public class BlockStoreLocation {
/** Special value to indicate any tier */
private static final int ANY_TIER = -1;
/** Special value to indicate any dir */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
* <p>
* TODO: use proto buf to represent this information
*/
public final class BlockStoreMeta {
public class BlockStoreMeta {
// TODO: the following two fields don't need to be computed on the creation of each
// {@link BlockStoreMeta} instance.
/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
/**
* Represents the metadata of a block in Tachyon managed storage.
*/
public final class BlockMeta extends BlockMetaBase {
public class BlockMeta extends BlockMetaBase {
private final long mBlockSize;

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@
* <p>
* This class does not guarantee thread safety.
*/
public final class StorageDir {
public class StorageDir {
private static final Logger LOG = LoggerFactory.getLogger(Constants.LOGGER_TYPE);
private final long mCapacityBytes;
/** A map from block ID to block meta data */
Expand Down
199 changes: 199 additions & 0 deletions servers/src/test/java/tachyon/worker/block/BlockDataManagerTest.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,199 @@
/*
* Licensed to the University of California, Berkeley 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 tachyon.worker.block;


import java.io.IOException;
import java.util.LinkedList;
import java.util.Random;

import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;

import tachyon.Constants;
import tachyon.Sessions;
import tachyon.client.BlockMasterClient;
import tachyon.client.FileSystemMasterClient;
import tachyon.conf.TachyonConf;
import tachyon.test.Tester;
import tachyon.thrift.FileInfo;
import tachyon.underfs.UnderFileSystem;
import tachyon.util.io.PathUtils;
import tachyon.worker.WorkerSource;
import tachyon.worker.block.meta.BlockMeta;
import tachyon.worker.block.meta.StorageDir;
import tachyon.worker.block.meta.TempBlockMeta;

public class BlockDataManagerTest {
class BlockDataManagerTester implements Tester<BlockDataManager> {
BlockDataManager.PrivateAccess mPrivateAccess;

public void receiveAccess(Object access) {
mPrivateAccess = (BlockDataManager.PrivateAccess) access;
}
}

class TestHarness {
BlockMasterClient mBlockMasterClient;
BlockStore mBlockStore;
FileSystemMasterClient mFileSystemMasterClient;
BlockHeartbeatReporter mHeartbeatReporter;
BlockDataManager mManager;
BlockMetricsReporter mMetricsReporter;
Random mRandom;
Sessions mSessions;
TachyonConf mTachyonConf;
BlockDataManagerTester mTester;
UnderFileSystem mUfs;
long mWorkerId;
WorkerSource mWorkerSource;

public TestHarness() throws IOException {
mRandom = new Random();

mBlockMasterClient = Mockito.mock(BlockMasterClient.class);
mBlockStore = Mockito.mock(BlockStore.class);
mFileSystemMasterClient = Mockito.mock(FileSystemMasterClient.class);
mHeartbeatReporter = Mockito.mock(BlockHeartbeatReporter.class);
mMetricsReporter = Mockito.mock(BlockMetricsReporter.class);
mSessions = Mockito.mock(Sessions.class);
mTachyonConf = Mockito.mock(TachyonConf.class);
mUfs = Mockito.mock(UnderFileSystem.class);
mWorkerId = mRandom.nextLong();
mWorkerSource = Mockito.mock(WorkerSource.class);

mManager = new BlockDataManager(mWorkerSource, mBlockMasterClient, mFileSystemMasterClient);
mManager.setSessions(mSessions);
mManager.setWorkerId(mWorkerId);

mTester = new BlockDataManagerTester();
mManager.grantAccess(mTester);
mTester.mPrivateAccess.setBlockStore(mBlockStore);
mTester.mPrivateAccess.setHeartbeatReporter(mHeartbeatReporter);
mTester.mPrivateAccess.setMetricsReporter(mMetricsReporter);
mTester.mPrivateAccess.setTachyonConf(mTachyonConf);
mTester.mPrivateAccess.setUnderFileSystem(mUfs);
}
}

@Test
public void AbortBlockTest() throws Exception {
TestHarness harness = new TestHarness();
long blockId = harness.mRandom.nextLong();
long sessionId = harness.mRandom.nextLong();
harness.mManager.abortBlock(sessionId, blockId);
Mockito.verify(harness.mBlockStore).abortBlock(sessionId, blockId);
}

@Test
public void AccessBlockTest() throws Exception {
TestHarness harness = new TestHarness();
long blockId = harness.mRandom.nextLong();
long sessionId = harness.mRandom.nextLong();
harness.mManager.accessBlock(sessionId, blockId);
Mockito.verify(harness.mBlockStore).accessBlock(sessionId, blockId);
}

@Test
public void AddCheckpointTest() throws Exception {
TestHarness harness = new TestHarness();
long fileId = harness.mRandom.nextLong();
long fileSize = harness.mRandom.nextLong();
long sessionId = harness.mRandom.nextLong();
FileInfo fileInfo = new FileInfo();
fileInfo.setPath("/foo/bar");
String srcPath = "/tmp/" + fileId;
String parentPath = "/tmp/foo";
String dstPath = "/tmp/foo/bar";

// TODO(jsimsa): Add test cases for error cases.
Mockito.when(harness.mTachyonConf
.get(Constants.UNDERFS_DATA_FOLDER, Constants.DEFAULT_DATA_FOLDER)).thenReturn("/tmp");
Mockito.when(harness.mSessions.getSessionUfsTempFolder(sessionId)).thenReturn("/tmp");
Mockito.when(harness.mFileSystemMasterClient.getFileInfo(fileId)).thenReturn(fileInfo);
Mockito.when(harness.mUfs.exists(parentPath)).thenReturn(true);
Mockito.when(harness.mUfs.mkdirs(parentPath, true)).thenReturn(true);
Mockito.when(harness.mUfs.rename(srcPath, dstPath)).thenReturn(true);
Mockito.when(harness.mUfs.getFileSize(dstPath)).thenReturn(fileSize);
harness.mManager.addCheckpoint(sessionId, fileId);
Mockito.verify(harness.mFileSystemMasterClient)
.addCheckpoint(harness.mWorkerId, fileId, fileSize, dstPath);
}

@Test
public void CleanupSessionsTest() throws Exception {
TestHarness harness = new TestHarness();
long sessionId = 1;
LinkedList<Long> sessions = new LinkedList<Long>();
sessions.add(sessionId);

Mockito.when(harness.mSessions.getTimedOutSessions()).thenReturn(sessions);
harness.mManager.cleanupSessions();
Mockito.verify(harness.mSessions).removeSession(sessionId);
Mockito.verify(harness.mBlockStore).cleanupSession(sessionId);
}

@Test
public void CommitBlockTest() throws Exception {
TestHarness harness = new TestHarness();
long blockId = harness.mRandom.nextLong();
long length = harness.mRandom.nextLong();
long lockId = harness.mRandom.nextLong();
long sessionId = harness.mRandom.nextLong();
long usedBytes = harness.mRandom.nextLong();
int tierAlias = 1;
LinkedList<Long> usedBytesOnTiers = new LinkedList<Long>();
usedBytesOnTiers.add(usedBytes);
BlockMeta blockMeta = Mockito.mock(BlockMeta.class);
BlockStoreLocation blockStoreLocation = Mockito.mock(BlockStoreLocation.class);
BlockStoreMeta blockStoreMeta = Mockito.mock(BlockStoreMeta.class);

Mockito.when(harness.mBlockStore.lockBlock(sessionId, blockId)).thenReturn(lockId);
Mockito.when(harness.mBlockStore.getBlockMeta(sessionId, blockId, lockId))
.thenReturn(blockMeta);
Mockito.when(harness.mBlockStore.getBlockStoreMeta()).thenReturn(blockStoreMeta);
Mockito.when(blockMeta.getBlockLocation()).thenReturn(blockStoreLocation);
Mockito.when(blockStoreLocation.tierAlias()).thenReturn(tierAlias);
Mockito.when(blockMeta.getBlockSize()).thenReturn(length);
Mockito.when(blockStoreMeta.getUsedBytesOnTiers()).thenReturn(usedBytesOnTiers);

harness.mManager.commitBlock(sessionId, blockId);
Mockito.verify(harness.mBlockMasterClient)
.workerCommitBlock(harness.mWorkerId, usedBytes, tierAlias, blockId, length);
Mockito.verify(harness.mBlockStore).unlockBlock(lockId);
}

@Test
public void CreateBlockTest() throws Exception {
TestHarness harness = new TestHarness();
long blockId = harness.mRandom.nextLong();
long initialBytes = harness.mRandom.nextLong();
long sessionId = harness.mRandom.nextLong();
int tierAlias = 1;
BlockStoreLocation location = BlockStoreLocation.anyDirInTier(tierAlias);
StorageDir storageDir = Mockito.mock(StorageDir.class);
TempBlockMeta meta = new TempBlockMeta(sessionId, blockId, initialBytes, storageDir);

Mockito.when(harness.mBlockStore
.createBlockMeta(sessionId, blockId, location, initialBytes)).thenReturn(meta);
Mockito.when(storageDir.getDirPath()).thenReturn("/tmp");
Assert.assertEquals(PathUtils.concatPath("/tmp", sessionId, blockId),
harness.mManager.createBlock(sessionId, blockId, tierAlias, initialBytes));
}

// TODO(jsimsa): Write unit tests for untested public methods.
}

0 comments on commit 3d1cef9

Please sign in to comment.